public PCollectionContentsAssert( PCollection<T> actual, AssertionWindows rewindowingStrategy, SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> paneExtractor, PAssertionSite site) { this.actual = actual; this.rewindowingStrategy = rewindowingStrategy; this.paneExtractor = paneExtractor; this.site = site; }
@Override public PCollectionContentsAssert<T> inFinalPane(BoundedWindow window) { return withPane(window, PaneExtractors.finalPane()); }
@Override public PCollectionContentsAssert<T> inCombinedNonLatePanes(BoundedWindow window) { return withPane(window, PaneExtractors.nonLatePanes()); }
@Override public PCollectionSingletonIterableAssert<T> inOnTimePane(BoundedWindow window) { return withPanes(window, PaneExtractors.onTimePane()); }
@Override public PCollectionViewAssert<ElemT, ViewT> inOnlyPane(BoundedWindow window) { return inPane(window, PaneExtractors.onlyPane(site)); }
@Override public PCollectionContentsAssert<T> inEarlyPane(BoundedWindow window) { return withPane(window, PaneExtractors.earlyPanes()); }
@Override public PCollectionContentsAssert<T> inOnTimePane(BoundedWindow window) { return withPane(window, PaneExtractors.onTimePane()); }
@Test public void onlyPaneNoFiring() { SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor = PaneExtractors.onlyPane(PAssert.PAssertionSite.capture("")); Iterable<ValueInSingleWindow<Integer>> noFiring = ImmutableList.of( ValueInSingleWindow.of( 9, BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), ValueInSingleWindow.of( 19, BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); assertThat(extractor.apply(noFiring), containsInAnyOrder(9, 19)); }
@Override public IterableAssert<T> inEarlyGlobalWindowPanes() { return withPane(GlobalWindow.INSTANCE, PaneExtractors.earlyPanes()); }
@Override public PCollectionSingletonIterableAssert<T> inWindow(BoundedWindow window) { return withPanes(window, PaneExtractors.allPanes()); }
@Override public PCollectionSingletonIterableAssert<T> inCombinedNonLatePanes(BoundedWindow window) { return withPanes(window, PaneExtractors.nonLatePanes()); }
@Override public PCollectionViewAssert<ElemT, ViewT> inOnTimePane(BoundedWindow window) { return inPane(window, PaneExtractors.onTimePane()); }
@Override public PCollectionSingletonIterableAssert<T> inFinalPane(BoundedWindow window) { return withPanes(window, PaneExtractors.finalPane()); }
@Test public void onlyPaneOnlyOneFiring() { SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor = PaneExtractors.onlyPane(PAssert.PAssertionSite.capture("")); Iterable<ValueInSingleWindow<Integer>> onlyFiring = ImmutableList.of( ValueInSingleWindow.of( 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), ValueInSingleWindow.of( 1, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); assertThat(extractor.apply(onlyFiring), containsInAnyOrder(2, 1)); }
@Override public IterableAssert<T> inEarlyGlobalWindowPanes() { return withPanes(GlobalWindow.INSTANCE, PaneExtractors.earlyPanes()); }
@Override public PCollectionContentsAssert<T> inWindow(BoundedWindow window) { return withPane(window, PaneExtractors.allPanes()); }
@Test public void nonLatePanesSingleOnTime() { SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor = PaneExtractors.nonLatePanes(); Iterable<ValueInSingleWindow<Integer>> onlyOnTime = ImmutableList.of( ValueInSingleWindow.of( 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), ValueInSingleWindow.of( 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); assertThat(extractor.apply(onlyOnTime), containsInAnyOrder(2, 4, 8)); }
@Test public void onTimePane() { SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor = PaneExtractors.onTimePane(); Iterable<ValueInSingleWindow<Integer>> onlyOnTime = ImmutableList.of( ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), ValueInSingleWindow.of( 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L))); assertThat(extractor.apply(onlyOnTime), containsInAnyOrder(2, 4)); }
@Override public PCollectionViewAssert<ElemT, ViewT> inFinalPane(BoundedWindow window) { return inPane(window, PaneExtractors.finalPane()); }
@Test public void onlyPaneMultiplePanesFails() { SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor = PaneExtractors.onlyPane(PAssert.PAssertionSite.capture("")); Iterable<ValueInSingleWindow<Integer>> multipleFiring = ImmutableList.of( ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY)), ValueInSingleWindow.of( 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), ValueInSingleWindow.of( 1, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L))); thrown.expectMessage("trigger that fires at most once"); extractor.apply(multipleFiring); }