3939import org .apache .beam .sdk .metrics .Metrics ;
4040import org .apache .beam .sdk .options .PipelineOptions ;
4141import org .apache .beam .sdk .state .TimeDomain ;
42+ import org .apache .beam .sdk .state .ValueState ;
4243import org .apache .beam .sdk .transforms .windowing .BoundedWindow ;
4344import org .apache .beam .sdk .transforms .windowing .PaneInfo ;
4445import org .apache .beam .sdk .transforms .windowing .PaneInfo .Timing ;
5152import org .apache .beam .sdk .values .CausedByDrain ;
5253import org .apache .beam .sdk .values .KV ;
5354import org .apache .beam .sdk .values .PCollection ;
55+ import org .apache .beam .sdk .values .PipelineMetadata ;
5456import org .apache .beam .sdk .values .WindowedValue ;
5557import org .apache .beam .sdk .values .WindowedValues ;
5658import org .apache .beam .sdk .values .WindowingStrategy ;
@@ -107,6 +109,10 @@ public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
107109 * <li>It uses discarding or accumulation mode according to the {@link WindowingStrategy}.
108110 * </ul>
109111 */
112+ static final StateTag <ValueState <PipelineMetadata >> METADATA_TAG =
113+ StateTags .makeSystemTagInternal (
114+ StateTags .value ("pipelineMetadata" , PipelineMetadata .Coder .of ()));
115+
110116 private final WindowingStrategy <Object , W > windowingStrategy ;
111117
112118 private final WindowedValueReceiver <KV <K , OutputT >> outputter ;
@@ -376,7 +382,7 @@ public void processElements(Iterable<WindowedValue<InputT>> values) throws Excep
376382 emit (
377383 contextFactory .base (window , StateStyle .DIRECT ),
378384 contextFactory .base (window , StateStyle .RENAMED ),
379- CausedByDrain . NORMAL );
385+ PipelineMetadata . createDefault () );
380386 }
381387
382388 // We're all done with merging and emitting elements so can compress the activeWindow state.
@@ -590,6 +596,17 @@ private void processElement(Map<W, W> windowToMergeResult, WindowedValue<InputT>
590596 value .getTimestamp (),
591597 StateStyle .DIRECT ,
592598 value .causedByDrain ());
599+
600+ ValueState <PipelineMetadata > metadataState = directContext .state ().access (METADATA_TAG );
601+ PipelineMetadata currentMetadata = metadataState .read ();
602+ if (currentMetadata == null ) {
603+ currentMetadata = PipelineMetadata .createDefault ();
604+ }
605+ PipelineMetadata inputMetadata = PipelineMetadata .create (value .causedByDrain ());
606+ PipelineMetadata newMetadata =
607+ PipelineMetadataCombiner .of ().addInput (currentMetadata , inputMetadata );
608+ metadataState .write (newMetadata );
609+
593610 if (triggerRunner .isClosed (directContext .state ())) {
594611 // This window has already been closed.
595612 droppedDueToClosedWindow .inc ();
@@ -792,7 +809,7 @@ public void onTimers(Iterable<TimerData> timers) throws Exception {
792809 renamedContext ,
793810 true /* isFinished */ ,
794811 windowActivation .isEndOfWindow ,
795- windowActivation .causedByDrain );
812+ PipelineMetadata . create ( windowActivation .causedByDrain ) );
796813 checkState (newHold == null , "Hold placed at %s despite isFinished being true." , newHold );
797814 }
798815
@@ -810,7 +827,10 @@ public void onTimers(Iterable<TimerData> timers) throws Exception {
810827 if (windowActivation .windowIsActiveAndOpen ()
811828 && triggerRunner .shouldFire (
812829 directContext .window (), directContext .timers (), directContext .state ())) {
813- emit (directContext , renamedContext , windowActivation .causedByDrain );
830+ emit (
831+ directContext ,
832+ renamedContext ,
833+ PipelineMetadata .create (windowActivation .causedByDrain ));
814834 }
815835
816836 if (windowActivation .isEndOfWindow ) {
@@ -874,6 +894,7 @@ private void clearAllState(
874894 triggerRunner .clearState (
875895 directContext .window (), directContext .timers (), directContext .state ());
876896 paneInfoTracker .clear (directContext .state ());
897+ directContext .state ().access (METADATA_TAG ).clear ();
877898 } else {
878899 // If !windowIsActiveAndOpen then !activeWindows.isActive (1) or triggerRunner.isClosed (2).
879900 // For (1), if !activeWindows.isActive then the window must be merging and has been
@@ -934,8 +955,9 @@ private void prefetchEmit(
934955 private void emit (
935956 ReduceFn <K , InputT , OutputT , W >.Context directContext ,
936957 ReduceFn <K , InputT , OutputT , W >.Context renamedContext ,
937- CausedByDrain causedByDrain )
958+ PipelineMetadata metadata )
938959 throws Exception {
960+
939961 checkState (
940962 triggerRunner .shouldFire (
941963 directContext .window (), directContext .timers (), directContext .state ()));
@@ -950,13 +972,14 @@ private void emit(
950972 // Run onTrigger to produce the actual pane contents.
951973 // As a side effect it will clear all element holds, but not necessarily any
952974 // end-of-window or garbage collection holds.
953- onTrigger (directContext , renamedContext , isFinished , false /*isEndOfWindow*/ , causedByDrain );
975+ onTrigger (directContext , renamedContext , isFinished , false /*isEndOfWindow*/ , metadata );
954976
955977 // Now that we've triggered, the pane is empty.
956978 nonEmptyPanes .clearPane (renamedContext .state ());
957979
958980 // Cleanup buffered data if appropriate
959981 if (shouldDiscard ) {
982+ directContext .state ().access (METADATA_TAG ).clear ();
960983 // Cleanup flavor C: The user does not want any buffered data to persist between panes.
961984 reduceFn .clearState (renamedContext );
962985 }
@@ -1009,8 +1032,19 @@ private void prefetchOnTrigger(
10091032 ReduceFn <K , InputT , OutputT , W >.Context renamedContext ,
10101033 final boolean isFinished ,
10111034 boolean isEndOfWindow ,
1012- CausedByDrain causedByDrain )
1035+ PipelineMetadata metadata )
10131036 throws Exception {
1037+ ValueState <PipelineMetadata > metadataState = directContext .state ().access (METADATA_TAG );
1038+ PipelineMetadata aggregatedMetadata = metadataState .read ();
1039+ if (aggregatedMetadata == null ) {
1040+ aggregatedMetadata = PipelineMetadata .createDefault ();
1041+ }
1042+ PipelineMetadata fullyAggregatedMetadata =
1043+ PipelineMetadataCombiner .of ().addInput (aggregatedMetadata , metadata );
1044+ final CausedByDrain aggregatedCausedByDrain = fullyAggregatedMetadata .causedByDrain ();
1045+ if (isFinished ) {
1046+ metadataState .clear ();
1047+ }
10141048 // Extract the window hold, and as a side effect clear it.
10151049 final WatermarkHold .OldAndNewHolds pair =
10161050 watermarkHold .extractAndRelease (renamedContext , isFinished ).read ();
@@ -1081,12 +1115,12 @@ private void prefetchOnTrigger(
10811115 .setValue (KV .of (key , toOutput ))
10821116 .setTimestamp (outputTimestamp )
10831117 .setWindows (windows )
1084- .setCausedByDrain (causedByDrain )
1118+ .setCausedByDrain (aggregatedCausedByDrain )
10851119 .setPaneInfo (paneInfo )
10861120 .setReceiver (outputter )
10871121 .output ();
10881122 },
1089- causedByDrain );
1123+ aggregatedCausedByDrain );
10901124
10911125 reduceFn .onTrigger (renamedTriggerContext );
10921126 }
0 commit comments