@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))); }
public void dontSetHoldIfTooLateForEndOfWindowTimer() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(Duration.millis(10)), mockTriggerStateMachine, Duration.millis(10), ClosingBehavior.FIRE_ALWAYS); tester.setAutoAdvanceOutputWatermark(false); tester.advanceInputWatermark(new Instant(15)); tester.advanceOutputWatermark(new Instant(11)); assertEquals(new Instant(14), tester.getWatermarkHold()); tester.advanceInputWatermark(new Instant(20)); tester.fireTimer(expectedWindow, expectedWindow.maxTimestamp(), TimeDomain.EVENT_TIME); assertEquals(new Instant(29), tester.getWatermarkHold()); assertEquals(new Instant(29), tester.getNextTimer(TimeDomain.EVENT_TIME)); assertEquals(new Instant(29), tester.getWatermarkHold()); assertEquals(new Instant(29), tester.getNextTimer(TimeDomain.EVENT_TIME)); tester.advanceInputWatermark(new Instant(30)); assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(10), new Instant(20)))); tester.assertHasOnlyGlobalAndFinishedSetsFor();
Duration allowedLateness = Duration.standardDays(1); ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) .withTrigger( tester.advanceInputWatermark(new Instant(0)); tester.advanceProcessingTime(new Instant(0)); tester.injectElements(TimestampedValue.of(1, new Instant(1))); assertThat(tester.getWatermarkHold(), equalTo(expectedWindow.maxTimestamp())); tester.advanceProcessingTime(new Instant(6000)); assertThat(tester.getOutputSize(), equalTo(1)); assertThat(tester.getWatermarkHold(), nullValue()); tester.advanceInputWatermark( new Instant(expectedWindow.maxTimestamp().plus(Duration.standardHours(1)))); tester.injectElements(TimestampedValue.of(3, new Instant(3))); tester.getWatermarkHold(), equalTo(expectedWindow.maxTimestamp().plus(allowedLateness)));
/** * 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)))); }
@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()); }
public void setGarbageCollectionHoldOnLateElements() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) .withTrigger( .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))); Instant hold = tester.getWatermarkHold(); assertEquals(new Instant(109), hold); tester.advanceInputWatermark(new Instant(110)); tester.advanceOutputWatermark(new Instant(110)); List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput(); assertEquals(2, output.size());
/** * 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)))); }
ReduceFnTester.nonCombining( FixedWindows.of(Duration.millis(10)), mockTriggerStateMachine, assertEquals(null, tester.getWatermarkHold()); assertEquals(null, tester.getOutputWatermark()); injectElement(tester, 1); injectElement(tester, 3); assertEquals(new Instant(1), tester.getWatermarkHold()); when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); injectElement(tester, 2); List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput(); assertThat( output, assertThat(tester.getWatermarkHold(), nullValue()); tester.advanceInputWatermark(new Instant(4)); assertEquals(new Instant(4), tester.getOutputWatermark()); tester.advanceInputWatermark(new Instant(4)); injectElement(tester, 2); injectElement(tester, 3); tester.getWatermarkHold(), equalTo(expectedWindow.maxTimestamp().plus(allowedLateness))); assertEquals(new Instant(5), tester.getWatermarkHold());
ReduceFnTester.nonCombining( FixedWindows.of(Duration.millis(10)), mockTriggerStateMachine, tester.advanceInputWatermark(new Instant(12)); tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME); List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput(); assertThat(output.size(), equalTo(1)); assertThat(output.get(0), isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)); tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME); assertThat(tester.extractOutput().size(), equalTo(0)); triggerShouldFinish(mockTriggerStateMachine); injectElement(tester, 3); output = tester.extractOutput(); assertThat(output.size(), equalTo(1)); output.get(0).getPane(), equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1))); assertTrue(tester.isMarkedFinished(firstWindow)); tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
Duration allowedLateness = Duration.millis(50); ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(Duration.millis(10)), mockTriggerStateMachine, tester.injectElements(TimestampedValue.of(1, new Instant(1))); // in [1, 11), gc at 21. tester.advanceInputWatermark(new Instant(15)); tester.fireTimer(mergedWindow, mergedWindow.maxTimestamp(), TimeDomain.EVENT_TIME); tester.injectElements(TimestampedValue.of(1, new Instant(1))); // in [1, 11), gc at 21. assertTrue(tester.hasNoActiveWindows()); tester.advanceInputWatermark(new Instant(100)); List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput(); assertThat(output.size(), equalTo(1)); assertThat(
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.advanceInputWatermarkNoTimers(new Instant(100)); tester.advanceProcessingTimeNoTimers(new Instant(5010)); tester.fireTimers( new IntervalWindow(new Instant(0), new Instant(100)), TimestampedValue.of(TimeDomain.EVENT_TIME, new Instant(100)), tester.extractOutput(), contains( isSingleWindowedValue(
/** 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 testOnElementBufferingAccumulating() throws Exception { // Test basic execution of a trigger using a non-combining window set and accumulating mode. ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(Duration.millis(10)), mockTriggerStateMachine, AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(100), ClosingBehavior.FIRE_IF_NON_EMPTY); injectElement(tester, 1); // Fires {1, 2} when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); injectElement(tester, 2); // Fires {1, 2, 3} because we are in accumulating mode when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); triggerShouldFinish(mockTriggerStateMachine); injectElement(tester, 3); // This element shouldn't be seen, because the trigger has finished injectElement(tester, 4); assertThat( tester.extractOutput(), contains( isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10), isSingleWindowedValue(containsInAnyOrder(1, 2, 3), 3, 0, 10))); assertTrue(tester.isMarkedFinished(firstWindow)); tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow); }
@Test public void testOnElementCombiningDiscarding() throws Exception { // Test basic execution of a trigger using a non-combining window set and discarding mode. WindowingStrategy<?, IntervalWindow> strategy = WindowingStrategy.of((WindowFn<?, IntervalWindow>) FixedWindows.of(Duration.millis(10))) .withTimestampCombiner(TimestampCombiner.EARLIEST) .withMode(AccumulationMode.DISCARDING_FIRED_PANES) .withAllowedLateness(Duration.millis(100)); ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining( strategy, mockTriggerStateMachine, Sum.ofIntegers(), VarIntCoder.of()); injectElement(tester, 2); when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); injectElement(tester, 3); when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); triggerShouldFinish(mockTriggerStateMachine); injectElement(tester, 4); // This element shouldn't be seen, because the trigger has finished injectElement(tester, 6); assertThat( tester.extractOutput(), contains( isSingleWindowedValue(equalTo(5), 2, 0, 10), isSingleWindowedValue(equalTo(4), 4, 0, 10))); assertTrue(tester.isMarkedFinished(firstWindow)); tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow); }
/** * Tests that when a processing time timer comes in after a window is expired it is just ignored. */ @Test public void testLateProcessingTimeTimer() 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); // After this advancement, the window is expired and only the GC process // should be allowed to touch it tester.advanceInputWatermarkNoTimers(new Instant(100)); // This should not output tester.advanceProcessingTime(new Instant(6000)); assertThat(tester.extractOutput(), emptyIterable()); }
/** Tests that a processing time timer does not cause window GC. */ @Test public void testProcessingTimeTimerDoesNotGc() 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.advanceProcessingTime(new Instant(10000)); tester.assertHasOnlyGlobalAndStateFor(new IntervalWindow(new Instant(0), new Instant(100))); assertThat( tester.extractOutput(), contains( isSingleWindowedValue( equalTo(7), 2, 0, 100, PaneInfo.createPane(true, false, Timing.EARLY, 0, 0)))); }
@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)))); }
/** * 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)))); }
MetricsEnvironment.setCurrentContainer(container); ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining( WindowingStrategy.of(SlidingWindows.of(Duration.millis(100)).every(Duration.millis(30))) .withTrigger(AfterWatermark.pastEndOfWindow()) VarIntCoder.of()); tester.injectElements( assertEquals(0, droppedElements); tester.advanceInputWatermark(new Instant(70)); tester.injectElements( assertEquals(1, droppedElements); tester.advanceInputWatermark(new Instant(130)); tester.injectElements(TimestampedValue.of(16, new Instant(40)));
/** * Advance the input watermark to the specified time, firing any timers that should fire. Then * advance the output watermark as far as possible. */ public void advanceInputWatermark(Instant newInputWatermark) throws Exception { timerInternals.advanceInputWatermark(newInputWatermark); ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner(); while (true) { TimerData timer; List<TimerInternals.TimerData> timers = new ArrayList<>(); while ((timer = timerInternals.removeNextEventTimer()) != null) { timers.add(timer); } if (timers.isEmpty()) { break; } runner.onTimers(timers); } if (autoAdvanceOutputWatermark) { Instant hold = stateInternals.earliestWatermarkHold(); if (hold == null) { WindowTracing.trace( "TestInMemoryTimerInternals.advanceInputWatermark: no holds, " + "so output watermark = input watermark"); hold = timerInternals.currentInputWatermarkTime(); } advanceOutputWatermark(hold); } runner.persist(); }