/** * When the watermark passes the end-of-window and window expiration time in a single update, this * tests that it does not crash. */ @Test public void testFixedWindowsEowAndGcTogether() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(Duration.millis(10)), DefaultTriggerStateMachine.of(), AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(50), ClosingBehavior.FIRE_ALWAYS); tester.setAutoAdvanceOutputWatermark(true); tester.advanceInputWatermark(new Instant(0)); injectElement(tester, 1); tester.advanceInputWatermark(new Instant(100)); assertThat( tester.extractOutput(), contains( isSingleWindowedValue( contains(1), 1, 0, 10, PaneInfo.createPane(true, true, Timing.ON_TIME)))); }
/** * When the watermark passes the end-of-window and window expiration time in a single update, this * tests that it does not crash. */ @Test public void testFixedWindowsEowAndGcTogetherFireIfNonEmpty() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(Duration.millis(10)), DefaultTriggerStateMachine.of(), AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(50), ClosingBehavior.FIRE_IF_NON_EMPTY); tester.setAutoAdvanceOutputWatermark(true); tester.advanceInputWatermark(new Instant(0)); injectElement(tester, 1); tester.advanceInputWatermark(new Instant(100)); List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput(); assertThat( output, contains( isSingleWindowedValue( contains(1), 1, 0, 10, PaneInfo.createPane(true, true, Timing.ON_TIME)))); }
tester.advanceInputWatermark(new Instant(30)); tester.advanceInputWatermark(new Instant(100));
/** * When the watermark passes the end-of-window and window expiration time in a single update, this * tests that it does not crash. */ @Test public void testSessionEowAndGcTogether() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(Duration.millis(10)), DefaultTriggerStateMachine.of(), AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(50), ClosingBehavior.FIRE_ALWAYS); tester.setAutoAdvanceOutputWatermark(true); tester.advanceInputWatermark(new Instant(0)); injectElement(tester, 1); tester.advanceInputWatermark(new Instant(100)); assertThat( tester.extractOutput(), contains( isSingleWindowedValue( contains(1), 1, 1, 11, PaneInfo.createPane(true, true, Timing.ON_TIME)))); }
@Test public void testPaneInfoSkipToFinish() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(Duration.millis(10)), mockTriggerStateMachine, AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100), ClosingBehavior.FIRE_IF_NON_EMPTY); tester.advanceInputWatermark(new Instant(0)); when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); triggerShouldFinish(mockTriggerStateMachine); injectElement(tester, 1); assertThat( tester.extractOutput(), contains(WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.EARLY)))); }
@Test public void testPaneInfoSkipToNonSpeculativeAndFinish() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(Duration.millis(10)), mockTriggerStateMachine, AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100), ClosingBehavior.FIRE_IF_NON_EMPTY); tester.advanceInputWatermark(new Instant(15)); when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); triggerShouldFinish(mockTriggerStateMachine); injectElement(tester, 1); assertThat( tester.extractOutput(), contains(WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.LATE)))); }
@Test public void testMergeBeforeFinalizing() throws Exception { // Verify that we merge windows before producing output so users don't see undesired // unmerged windows. ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(Duration.millis(10)), mockTriggerStateMachine, AccumulationMode.DISCARDING_FIRED_PANES, Duration.ZERO, ClosingBehavior.FIRE_IF_NON_EMPTY); // All on time data, verify watermark hold. // These two windows should pre-merge immediately to [1, 20) tester.injectElements( TimestampedValue.of(1, new Instant(1)), // in [1, 11) TimestampedValue.of(10, new Instant(10))); // in [10, 20) // And this should fire the end-of-window timer tester.advanceInputWatermark(new Instant(100)); List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput(); assertThat(output.size(), equalTo(1)); assertThat( output.get(0), isSingleWindowedValue( containsInAnyOrder(1, 10), 1, // timestamp 1, // window start 20)); // window end assertThat( output.get(0).getPane(), equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0))); }
/** * We should fire a non-empty ON_TIME pane in the GlobalWindow when the watermark moves to * end-of-time. */ @Test public void fireNonEmptyOnDrainInGlobalWindow() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, GlobalWindow> tester = ReduceFnTester.nonCombining( WindowingStrategy.of(new GlobalWindows()) .withTrigger(Repeatedly.forever(AfterPane.elementCountAtLeast(3))) .withMode(AccumulationMode.DISCARDING_FIRED_PANES)); tester.advanceInputWatermark(new Instant(0)); final int n = 20; for (int i = 0; i < n; i++) { tester.injectElements(TimestampedValue.of(i, new Instant(i))); } List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput(); assertEquals(n / 3, output.size()); for (int i = 0; i < output.size(); i++) { assertEquals(Timing.EARLY, output.get(i).getPane().getTiming()); assertEquals(i, output.get(i).getPane().getIndex()); assertEquals(3, Iterables.size(output.get(i).getValue())); } tester.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE); output = tester.extractOutput(); assertEquals(1, output.size()); assertEquals(Timing.ON_TIME, output.get(0).getPane().getTiming()); assertEquals(n / 3, output.get(0).getPane().getIndex()); assertEquals(n - ((n / 3) * 3), Iterables.size(output.get(0).getValue())); }
@Test public void noEmptyPanesFinalAlways() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) .withTrigger( Repeatedly.forever( AfterFirst.of( AfterPane.elementCountAtLeast(2), AfterWatermark.pastEndOfWindow()))) .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES) .withAllowedLateness(Duration.millis(100)) .withTimestampCombiner(TimestampCombiner.EARLIEST) .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS)); tester.advanceInputWatermark(new Instant(0)); tester.injectElements( TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); tester.advanceInputWatermark(new Instant(20)); tester.advanceInputWatermark(new Instant(250)); List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput(); assertThat( output, contains( // Trigger with 2 elements isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10), // Trigger for the empty on time pane isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10), // Trigger for the final pane isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10))); }
@Test public void noEmptyPanesFinalIfNonEmpty() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) .withTrigger( Repeatedly.forever( AfterFirst.of( AfterPane.elementCountAtLeast(2), AfterWatermark.pastEndOfWindow()))) .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES) .withAllowedLateness(Duration.millis(100)) .withTimestampCombiner(TimestampCombiner.EARLIEST) .withClosingBehavior(ClosingBehavior.FIRE_IF_NON_EMPTY)); tester.advanceInputWatermark(new Instant(0)); tester.injectElements( TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); tester.advanceInputWatermark(new Instant(20)); tester.advanceInputWatermark(new Instant(250)); List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput(); assertThat( output, contains( // Trigger with 2 elements isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10), // Trigger for the empty on time pane isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10))); }
tester.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE);
ReduceFnTester.combining(strategy, Sum.ofIntegers(), VarIntCoder.of()); tester.advanceInputWatermark(new Instant(0)); tester.advanceProcessingTime(new Instant(0)); tester.advanceInputWatermark(new Instant(11));
.withClosingBehavior(ClosingBehavior.FIRE_IF_NON_EMPTY)); tester.advanceInputWatermark(new Instant(0)); tester.advanceOutputWatermark(new Instant(0)); tester.injectElements(TimestampedValue.of(1, new Instant(1))); tester.advanceInputWatermark(new Instant(109)); tester.advanceOutputWatermark(new Instant(109)); tester.injectElements(TimestampedValue.of(2, new Instant(2))); assertEquals(new Instant(109), hold); tester.advanceInputWatermark(new Instant(110)); tester.advanceOutputWatermark(new Instant(110));
@Test public void testPaneInfoFinalAndOnTime() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) .withTrigger( Repeatedly.forever(AfterPane.elementCountAtLeast(2)) .orFinally(AfterWatermark.pastEndOfWindow())) .withMode(AccumulationMode.DISCARDING_FIRED_PANES) .withAllowedLateness(Duration.millis(100)) .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS)); tester.advanceInputWatermark(new Instant(0)); // Should trigger due to element count tester.injectElements( TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); assertThat( tester.extractOutput(), contains( WindowMatchers.valueWithPaneInfo( PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)))); tester.advanceInputWatermark(new Instant(150)); assertThat( tester.extractOutput(), contains( WindowMatchers.valueWithPaneInfo( PaneInfo.createPane(false, true, Timing.ON_TIME, 1, 0)))); }
/** * Tests that if end-of-window and GC timers come in together, that the pane is correctly marked * as final. */ @Test public void testCombiningAccumulatingEventTime() throws Exception { WindowingStrategy<?, IntervalWindow> strategy = WindowingStrategy.of((WindowFn<?, IntervalWindow>) FixedWindows.of(Duration.millis(100))) .withTimestampCombiner(TimestampCombiner.EARLIEST) .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES) .withAllowedLateness(Duration.millis(1)) .withTrigger(Repeatedly.forever(AfterWatermark.pastEndOfWindow())); ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining(strategy, Sum.ofIntegers(), VarIntCoder.of()); injectElement(tester, 2); // processing timer @ 5000 + 10; EOW timer @ 100 injectElement(tester, 5); tester.advanceInputWatermark(new Instant(1000)); assertThat( tester.extractOutput(), contains( isSingleWindowedValue( equalTo(7), 2, 0, 100, PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)))); }
assertEquals(null, tester.getOutputWatermark()); tester.advanceInputWatermark(new Instant(20)); tester.advanceInputWatermark(new Instant(1000)); assertEquals(expectedGcHold, tester.getWatermarkHold());
tester.advanceInputWatermark(new Instant(15)); tester.advanceOutputWatermark(new Instant(11)); tester.advanceInputWatermark(new Instant(20)); tester.fireTimer(expectedWindow, expectedWindow.maxTimestamp(), TimeDomain.EVENT_TIME); tester.advanceInputWatermark(new Instant(30));
/** Ensure a closed trigger has its state recorded in the merge result window. */ @Test public void testMergingWithCloseTrigger() throws Exception { Duration allowedLateness = Duration.millis(50); ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(Duration.millis(10)), mockTriggerStateMachine, AccumulationMode.DISCARDING_FIRED_PANES, allowedLateness, ClosingBehavior.FIRE_IF_NON_EMPTY); // Create a new merged session window. IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(12)); tester.injectElements( TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); // Force the trigger to be closed for the merged window. when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); triggerShouldFinish(mockTriggerStateMachine); // Fire and end-of-window timer as though the trigger set it tester.advanceInputWatermark(new Instant(13)); tester.fireTimer(mergedWindow, mergedWindow.maxTimestamp(), TimeDomain.EVENT_TIME); // Trigger is now closed. assertTrue(tester.isMarkedFinished(mergedWindow)); when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(false); // Revisit the same session window. tester.injectElements( TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); // Trigger is still closed. assertTrue(tester.isMarkedFinished(mergedWindow)); }
@Test public void testWatermarkHoldForLateNewWindow() throws Exception { Duration allowedLateness = Duration.standardMinutes(1); Duration gapDuration = Duration.millis(10); ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( WindowingStrategy.of(Sessions.withGapDuration(gapDuration)) .withMode(AccumulationMode.DISCARDING_FIRED_PANES) .withTrigger( Repeatedly.forever( AfterWatermark.pastEndOfWindow() .withLateFirings(AfterPane.elementCountAtLeast(1)))) .withAllowedLateness(allowedLateness)); tester.setAutoAdvanceOutputWatermark(false); assertEquals(null, tester.getWatermarkHold()); assertEquals(null, tester.getOutputWatermark()); tester.advanceInputWatermark(new Instant(40)); injectElements(tester, 1); assertThat(tester.getWatermarkHold(), nullValue()); injectElements(tester, 10); assertThat(tester.getWatermarkHold(), nullValue()); }
/** * Tests that when a processing time timers comes in after a window is expired and GC'd it does * not cause a spurious output. */ @Test public void testCombiningAccumulatingProcessingTimeSeparateBundles() throws Exception { WindowingStrategy<?, IntervalWindow> strategy = WindowingStrategy.of((WindowFn<?, IntervalWindow>) FixedWindows.of(Duration.millis(100))) .withTimestampCombiner(TimestampCombiner.EARLIEST) .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES) .withAllowedLateness(Duration.ZERO) .withTrigger( Repeatedly.forever( AfterProcessingTime.pastFirstElementInPane().plusDelayOf(Duration.millis(10)))); ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining(strategy, Sum.ofIntegers(), VarIntCoder.of()); tester.advanceProcessingTime(new Instant(5000)); injectElement(tester, 2); // processing timer @ 5000 + 10; EOW timer @ 100 injectElement(tester, 5); tester.advanceInputWatermark(new Instant(100)); tester.advanceProcessingTime(new Instant(5011)); assertThat( tester.extractOutput(), contains( isSingleWindowedValue( equalTo(7), 2, 0, 100, PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)))); }