scwhittle commented on code in PR #37648:
URL: https://github.com/apache/beam/pull/37648#discussion_r2829079768


##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaExactlyOnceSink.java:
##########
@@ -176,13 +180,27 @@ public PCollection<Void> 
expand(PCollection<ProducerRecord<K, V>> input) {
     return input
         .apply(
             Window.<ProducerRecord<K, V>>into(new GlobalWindows()) // 
Everything into global window.
-                
.triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1)))
+                .triggering(
+                    Repeatedly.forever(
+                        AfterFirst.of(
+                            AfterPane.elementCountAtLeast(numElements),
+                            
AfterProcessingTime.pastFirstElementInPane().plusDelayOf(timeout))))
                 .discardingFiredPanes())
         .apply(
             String.format("Shuffle across %d shards", numShards),
             ParDo.of(new Reshard<>(numShards)))
         .apply("Persist sharding", GroupByKey.create())
         .apply("Assign sequential ids", ParDo.of(new Sequencer<>()))
+        .apply(

Review Comment:
   is this needed or is the the windowing inherited from above?



##########
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaExactlyOnceSink.java:
##########
@@ -176,13 +180,27 @@ public PCollection<Void> 
expand(PCollection<ProducerRecord<K, V>> input) {
     return input
         .apply(
             Window.<ProducerRecord<K, V>>into(new GlobalWindows()) // 
Everything into global window.
-                
.triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1)))
+                .triggering(

Review Comment:
   this is going to break update compatability, I think that either:
   1. have the defaults result in the same windowing
   2. or add support for the compatabilityversion option to be used to ignore 
the options and use the old windowing



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to