@Test public void testFireDeadline() throws Exception { assertEquals( new Instant(9), DefaultTrigger.of() .getWatermarkThatGuaranteesFiring(new IntervalWindow(new Instant(0), new Instant(10)))); assertEquals( GlobalWindow.INSTANCE.maxTimestamp(), DefaultTrigger.of().getWatermarkThatGuaranteesFiring(GlobalWindow.INSTANCE)); }
@Test public void testDefaultTriggerWithSessionWindow() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(Duration.millis(10)), DefaultTrigger.<IntervalWindow>of(), AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); tester.injectElements( TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(9))); // no output, because we merged into the [9-19) session tester.advanceInputWatermark(new Instant(10)); assertThat(tester.extractOutput(), Matchers.emptyIterable()); tester.injectElements( TimestampedValue.of(3, new Instant(15)), TimestampedValue.of(4, new Instant(30))); tester.advanceInputWatermark(new Instant(100)); assertThat( tester.extractOutput(), Matchers.contains( isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2, 3), 1, 1, 25), isSingleWindowedValue(Matchers.contains(4), 30, 30, 40))); assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(1), new Instant(25)))); assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(30), new Instant(40)))); tester.assertHasOnlyGlobalAndPaneInfoFor( new IntervalWindow(new Instant(1), new Instant(25)), new IntervalWindow(new Instant(30), new Instant(40))); }
@Test public void testDefaultTriggerWithFixedWindow() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( FixedWindows.of(Duration.millis(10)), DefaultTrigger.<IntervalWindow>of(), AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); tester.injectElements( TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(9)), TimestampedValue.of(3, new Instant(15)), TimestampedValue.of(4, new Instant(19)), TimestampedValue.of(5, new Instant(30))); // Advance the watermark almost to the end of the first window. tester.advanceProcessingTime(new Instant(500)); tester.advanceInputWatermark(new Instant(8)); assertThat(tester.extractOutput(), Matchers.emptyIterable()); // Advance watermark to 10 (past end of the window), which causes the first fixed window to // be emitted tester.advanceInputWatermark(new Instant(10)); assertThat( tester.extractOutput(), Matchers.contains(isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, 0, 10))); // Advance watermark to 100, which causes the remaining two windows to be emitted. // Since their timers were at different timestamps, they should fire in order. tester.advanceInputWatermark(new Instant(100)); assertThat( tester.extractOutput(), Matchers.contains( isSingleWindowedValue(Matchers.containsInAnyOrder(3, 4), 15, 10, 20), isSingleWindowedValue(Matchers.contains(5), 30, 30, 40))); assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(30), new Instant(40)))); tester.assertHasOnlyGlobalAndPaneInfoFor( new IntervalWindow(new Instant(0), new Instant(10)), new IntervalWindow(new Instant(10), new Instant(20)), new IntervalWindow(new Instant(30), new Instant(40))); }
@Test public void testDefaultTriggerWithContainedSessionWindow() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( Sessions.withGapDuration(Duration.millis(10)), DefaultTrigger.<IntervalWindow>of(), AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); tester.injectElements( TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(9)), TimestampedValue.of(3, new Instant(7))); tester.advanceInputWatermark(new Instant(20)); Iterable<WindowedValue<Iterable<Integer>>> extractOutput = tester.extractOutput(); assertThat( extractOutput, Matchers.contains(isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2, 3), 1, 1, 19))); tester.assertHasOnlyGlobalAndPaneInfoFor(new IntervalWindow(new Instant(1), new Instant(19))); }
@Test public void testDefaultTriggerWithSlidingWindow() throws Exception { ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining( SlidingWindows.of(Duration.millis(10)).every(Duration.millis(5)), DefaultTrigger.<IntervalWindow>of(), AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100)); tester.injectElements( TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(4)), TimestampedValue.of(3, new Instant(9))); tester.advanceInputWatermark(new Instant(100)); assertThat( tester.extractOutput(), Matchers.contains( isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2), 1, -5, 5), isSingleWindowedValue(Matchers.containsInAnyOrder(1, 2, 3), 5, 0, 10), isSingleWindowedValue(Matchers.containsInAnyOrder(3), 10, 5, 15))); // This data is too late to hold the output watermark, either to the element // or the end of window. tester.injectElements(TimestampedValue.of(4, new Instant(8))); tester.advanceInputWatermark(new Instant(120)); List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput(); assertThat( output, Matchers.contains( isSingleWindowedValue(Matchers.contains(4), 9, 0, 10), isSingleWindowedValue(Matchers.contains(4), 14, 5, 15))); assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(1), new Instant(10)))); assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(5), new Instant(15)))); tester.assertHasOnlyGlobalState(); }
@Test public void testContinuation() throws Exception { assertEquals(DefaultTrigger.of(), DefaultTrigger.of().getContinuationTrigger()); }