4848import org .apache .beam .sdk .transforms .windowing .WindowFn ;
4949import org .apache .beam .sdk .util .WindowTracing ;
5050import org .apache .beam .sdk .util .WindowedValueReceiver ;
51+ import org .apache .beam .sdk .values .CausedByDrain ;
5152import org .apache .beam .sdk .values .KV ;
5253import org .apache .beam .sdk .values .PCollection ;
5354import org .apache .beam .sdk .values .WindowedValue ;
@@ -374,7 +375,8 @@ public void processElements(Iterable<WindowedValue<InputT>> values) throws Excep
374375 for (W window : windowsToFire ) {
375376 emit (
376377 contextFactory .base (window , StateStyle .DIRECT ),
377- contextFactory .base (window , StateStyle .RENAMED ));
378+ contextFactory .base (window , StateStyle .RENAMED ),
379+ null );
378380 }
379381
380382 // We're all done with merging and emitting elements so can compress the activeWindow state.
@@ -583,7 +585,11 @@ private void processElement(Map<W, W> windowToMergeResult, WindowedValue<InputT>
583585 for (W window : windows ) {
584586 ReduceFn <K , InputT , OutputT , W >.ProcessValueContext directContext =
585587 contextFactory .forValue (
586- window , value .getValue (), value .getTimestamp (), StateStyle .DIRECT );
588+ window ,
589+ value .getValue (),
590+ value .getTimestamp (),
591+ StateStyle .DIRECT ,
592+ value .causedByDrain ());
587593 if (triggerRunner .isClosed (directContext .state ())) {
588594 // This window has already been closed.
589595 droppedDueToClosedWindow .inc ();
@@ -601,7 +607,11 @@ private void processElement(Map<W, W> windowToMergeResult, WindowedValue<InputT>
601607 activeWindows .ensureWindowIsActive (window );
602608 ReduceFn <K , InputT , OutputT , W >.ProcessValueContext renamedContext =
603609 contextFactory .forValue (
604- window , value .getValue (), value .getTimestamp (), StateStyle .RENAMED );
610+ window ,
611+ value .getValue (),
612+ value .getTimestamp (),
613+ StateStyle .RENAMED ,
614+ value .causedByDrain ());
605615
606616 nonEmptyPanes .recordContent (renamedContext .state ());
607617 scheduleGarbageCollectionTimer (directContext );
@@ -639,12 +649,15 @@ private class WindowActivation {
639649 // garbage collect the window. We'll consider any timer at or after the
640650 // end-of-window time to be a signal to garbage collect.
641651 public final boolean isGarbageCollection ;
652+ public final CausedByDrain causedByDrain ;
642653
643654 WindowActivation (
644655 ReduceFn <K , InputT , OutputT , W >.Context directContext ,
645- ReduceFn <K , InputT , OutputT , W >.Context renamedContext ) {
656+ ReduceFn <K , InputT , OutputT , W >.Context renamedContext ,
657+ CausedByDrain causedByDrain ) {
646658 this .directContext = directContext ;
647659 this .renamedContext = renamedContext ;
660+ this .causedByDrain = causedByDrain ;
648661 W window = directContext .window ();
649662
650663 // The output watermark is before the end of the window if it is either unknown
@@ -701,12 +714,13 @@ public void onTimers(Iterable<TimerData> timers) throws Exception {
701714
702715 WindowTracing .debug (
703716 "ReduceFnRunner: Received timer key:{}; window:{}; data:{} with "
704- + "inputWatermark:{}; outputWatermark:{}" ,
717+ + "inputWatermark:{}; outputWatermark:{}; draining:{} " ,
705718 key ,
706719 window ,
707720 timer ,
708721 timerInternals .currentInputWatermarkTime (),
709- timerInternals .currentOutputWatermarkTime ());
722+ timerInternals .currentOutputWatermarkTime (),
723+ timer .causedByDrain ());
710724
711725 // Processing time timers for an expired window are ignored, just like elements
712726 // that show up too late. Window GC is management by an event time timer
@@ -722,11 +736,13 @@ public void onTimers(Iterable<TimerData> timers) throws Exception {
722736 continue ;
723737 }
724738
739+ // pass draining bit
725740 ReduceFn <K , InputT , OutputT , W >.Context directContext =
726741 contextFactory .base (window , StateStyle .DIRECT );
727742 ReduceFn <K , InputT , OutputT , W >.Context renamedContext =
728743 contextFactory .base (window , StateStyle .RENAMED );
729- WindowActivation windowActivation = new WindowActivation (directContext , renamedContext );
744+ WindowActivation windowActivation =
745+ new WindowActivation (directContext , renamedContext , timer .causedByDrain ());
730746 windowActivations .put (window , windowActivation );
731747
732748 // Perform prefetching of state to determine if the trigger should fire.
@@ -757,11 +773,12 @@ public void onTimers(Iterable<TimerData> timers) throws Exception {
757773
758774 if (windowActivation .isGarbageCollection ) {
759775 WindowTracing .debug (
760- "ReduceFnRunner: Cleaning up for key:{}; window:{} with inputWatermark:{}; outputWatermark:{}" ,
776+ "ReduceFnRunner: Cleaning up for key:{}; window:{} with inputWatermark:{}; outputWatermark:{}; draining:{} " ,
761777 key ,
762778 directContext .window (),
763779 timerInternals .currentInputWatermarkTime (),
764- timerInternals .currentOutputWatermarkTime ());
780+ timerInternals .currentOutputWatermarkTime (),
781+ windowActivation .causedByDrain );
765782
766783 boolean windowIsActiveAndOpen = windowActivation .windowIsActiveAndOpen ();
767784 if (windowIsActiveAndOpen ) {
@@ -774,7 +791,8 @@ public void onTimers(Iterable<TimerData> timers) throws Exception {
774791 directContext ,
775792 renamedContext ,
776793 true /* isFinished */ ,
777- windowActivation .isEndOfWindow );
794+ windowActivation .isEndOfWindow ,
795+ windowActivation .causedByDrain );
778796 checkState (newHold == null , "Hold placed at %s despite isFinished being true." , newHold );
779797 }
780798
@@ -792,7 +810,7 @@ public void onTimers(Iterable<TimerData> timers) throws Exception {
792810 if (windowActivation .windowIsActiveAndOpen ()
793811 && triggerRunner .shouldFire (
794812 directContext .window (), directContext .timers (), directContext .state ())) {
795- emit (directContext , renamedContext );
813+ emit (directContext , renamedContext , windowActivation . causedByDrain );
796814 }
797815
798816 if (windowActivation .isEndOfWindow ) {
@@ -915,7 +933,8 @@ private void prefetchEmit(
915933 /** Emit if a trigger is ready to fire or timers require it, and cleanup state. */
916934 private void emit (
917935 ReduceFn <K , InputT , OutputT , W >.Context directContext ,
918- ReduceFn <K , InputT , OutputT , W >.Context renamedContext )
936+ ReduceFn <K , InputT , OutputT , W >.Context renamedContext ,
937+ CausedByDrain causedByDrain )
919938 throws Exception {
920939 checkState (
921940 triggerRunner .shouldFire (
@@ -931,7 +950,7 @@ private void emit(
931950 // Run onTrigger to produce the actual pane contents.
932951 // As a side effect it will clear all element holds, but not necessarily any
933952 // end-of-window or garbage collection holds.
934- onTrigger (directContext , renamedContext , isFinished , false /*isEndOfWindow*/ );
953+ onTrigger (directContext , renamedContext , isFinished , false /*isEndOfWindow*/ , causedByDrain );
935954
936955 // Now that we've triggered, the pane is empty.
937956 nonEmptyPanes .clearPane (renamedContext .state ());
@@ -989,7 +1008,8 @@ private void prefetchOnTrigger(
9891008 final ReduceFn <K , InputT , OutputT , W >.Context directContext ,
9901009 ReduceFn <K , InputT , OutputT , W >.Context renamedContext ,
9911010 final boolean isFinished ,
992- boolean isEndOfWindow )
1011+ boolean isEndOfWindow ,
1012+ CausedByDrain causedByDrain )
9931013 throws Exception {
9941014 // Extract the window hold, and as a side effect clear it.
9951015 final WatermarkHold .OldAndNewHolds pair =
@@ -1061,10 +1081,12 @@ private void prefetchOnTrigger(
10611081 .setValue (KV .of (key , toOutput ))
10621082 .setTimestamp (outputTimestamp )
10631083 .setWindows (windows )
1084+ .setCausedByDrain (causedByDrain )
10641085 .setPaneInfo (paneInfo )
10651086 .setReceiver (outputter )
10661087 .output ();
1067- });
1088+ },
1089+ causedByDrain );
10681090
10691091 reduceFn .onTrigger (renamedTriggerContext );
10701092 }
0 commit comments