@Override public PCollectionSingletonIterableAssert<T> inCombinedNonLatePanes(BoundedWindow window) { return withPanes(window, PaneExtractors.nonLatePanes()); }
@Override public PCollectionContentsAssert<T> inCombinedNonLatePanes(BoundedWindow window) { return withPane(window, PaneExtractors.nonLatePanes()); }
@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 allPanesSingleLate() { SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor = PaneExtractors.nonLatePanes(); Iterable<ValueInSingleWindow<Integer>> onlyOnTime = ImmutableList.of( ValueInSingleWindow.of( 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.LATE, 0L, 0L))); assertThat(extractor.apply(onlyOnTime), emptyIterable()); }
@Test public void nonLatePanesSingleEarly() { SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor = PaneExtractors.nonLatePanes(); Iterable<ValueInSingleWindow<Integer>> onlyOnTime = ImmutableList.of( ValueInSingleWindow.of( 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY)), ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY))); assertThat(extractor.apply(onlyOnTime), containsInAnyOrder(4, 8)); }
@Test public void nonLatePanesMultiplePanes() { SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor = PaneExtractors.nonLatePanes(); Iterable<ValueInSingleWindow<Integer>> onlyOnTime = ImmutableList.of( ValueInSingleWindow.of( 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L)), ValueInSingleWindow.of(7, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), ValueInSingleWindow.of( 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), ValueInSingleWindow.of( 1, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY))); assertThat(extractor.apply(onlyOnTime), containsInAnyOrder(4, 1, 7)); }