public void testRunningAsTrigger(OnceTrigger<IntervalWindow> mockTrigger, IntervalWindow window) throws Exception { // Don't fire due to mock saying no when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false); assertFalse(tester.shouldFire(window)); // not ready // Fire due to mock trigger; early trigger is required to be a OnceTrigger when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true); assertTrue(tester.shouldFire(window)); // ready tester.fireIfShouldFire(window); assertFalse(tester.isMarkedFinished(window)); }
private void injectElements(int... elements) throws Exception { for (int element : elements) { doNothing().when(mockEarly).onElement(anyElementContext()); doNothing().when(mockLate).onElement(anyElementContext()); tester.injectElements(element); } }
@Test public void testAtWatermarkAndLate() throws Exception { tester = TriggerTester.forTrigger( AfterWatermark.<IntervalWindow>pastEndOfWindow().withLateFirings(mockLate), FixedWindows.of(Duration.millis(100))); injectElements(1); IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100)); // No early firing, just double checking when(mockEarly.shouldFire(anyTriggerContext())).thenReturn(true); assertFalse(tester.shouldFire(window)); tester.fireIfShouldFire(window); assertFalse(tester.isMarkedFinished(window)); // Fire due to watermark when(mockEarly.shouldFire(anyTriggerContext())).thenReturn(false); tester.advanceInputWatermark(new Instant(100)); assertTrue(tester.shouldFire(window)); tester.fireIfShouldFire(window); assertFalse(tester.isMarkedFinished(window)); testRunningAsTrigger(mockLate, window); }
/** * Tests that the trigger rewinds to be non-finished in the merged window. * * <p>Because windows are discarded when a trigger finishes, we need to embed this in a sequence * in order to check that it is re-activated. So this test is potentially sensitive to other * triggers' correctness. */ @Test public void testEarlyAndLateOnMergeRewinds() throws Exception { tester = TriggerTester.forTrigger( AfterWatermark.<IntervalWindow>pastEndOfWindow() .withEarlyFirings(AfterPane.<IntervalWindow>elementCountAtLeast(100)) .withLateFirings(AfterPane.<IntervalWindow>elementCountAtLeast(1)), Sessions.withGapDuration(Duration.millis(10))); tester.injectElements(1); tester.injectElements(5); IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11)); IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15)); IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15)); // Finish the AfterWatermark.pastEndOfWindow() bit of the trigger in only the first window tester.advanceInputWatermark(new Instant(11)); assertTrue(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); tester.fireIfShouldFire(firstWindow); // Confirm that we are on the late trigger by probing assertFalse(tester.shouldFire(firstWindow)); tester.injectElements(1); assertTrue(tester.shouldFire(firstWindow)); tester.fireIfShouldFire(firstWindow); // Merging should re-activate the early trigger in the merged window tester.mergeWindows(); // Confirm that we are not on the second trigger by probing assertFalse(tester.shouldFire(mergedWindow)); tester.injectElements(1); assertFalse(tester.shouldFire(mergedWindow)); // And confirm that advancing the watermark fires again tester.advanceInputWatermark(new Instant(15)); assertTrue(tester.shouldFire(mergedWindow)); }
/** * Tests that if the EOW is finished in both as well as the merged window, then it is finished in * the merged result. * * <p>Because windows are discarded when a trigger finishes, we need to embed this in a sequence * in order to check that it is re-activated. So this test is potentially sensitive to other * triggers' correctness. */ @Test public void testOnMergeAlreadyFinished() throws Exception { tester = TriggerTester.forTrigger( AfterEach.inOrder( AfterWatermark.<IntervalWindow>pastEndOfWindow(), Repeatedly.forever(AfterPane.<IntervalWindow>elementCountAtLeast(1))), Sessions.withGapDuration(Duration.millis(10))); tester.injectElements(1); tester.injectElements(5); IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11)); IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15)); IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15)); // Finish the AfterWatermark.pastEndOfWindow() trigger in both windows tester.advanceInputWatermark(new Instant(15)); assertTrue(tester.shouldFire(firstWindow)); assertTrue(tester.shouldFire(secondWindow)); tester.fireIfShouldFire(firstWindow); tester.fireIfShouldFire(secondWindow); // Confirm that we are on the second trigger by probing assertFalse(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); tester.injectElements(1); tester.injectElements(5); assertTrue(tester.shouldFire(firstWindow)); assertTrue(tester.shouldFire(secondWindow)); tester.fireIfShouldFire(firstWindow); tester.fireIfShouldFire(secondWindow); // Merging should leave it finished tester.mergeWindows(); // Confirm that we are on the second trigger by probing assertFalse(tester.shouldFire(mergedWindow)); tester.injectElements(1); assertTrue(tester.shouldFire(mergedWindow)); }