@Override public Instant maxTimestamp() { return GlobalWindow.INSTANCE.maxTimestamp(); }
@Override public Collection<IntervalWindow> assignWindows(AssignContext c) { // Assign each element into a window from its timestamp until the end of // the global window. I'd rather assign to the end of time, but it appears // that any window that exceeds the global window is dropped from the output // received by PAssert. return Arrays.asList(new IntervalWindow(c.timestamp(), GlobalWindow.INSTANCE.maxTimestamp())); }
/** * Return when {@code window} should be garbage collected. If the window's expiration time is on * or after the end of the global window, it will be truncated to the end of the global window. */ public static Instant garbageCollectionTime(BoundedWindow window, Duration allowedLateness) { // If the end of the window + allowed lateness is beyond the "end of time" aka the end of the // global window, then we truncate it. The conditional is phrased like it is because the // addition of EOW + allowed lateness might even overflow the maximum allowed Instant if (GlobalWindow.INSTANCE .maxTimestamp() .minus(allowedLateness) .isBefore(window.maxTimestamp())) { return GlobalWindow.INSTANCE.maxTimestamp(); } else { return window.maxTimestamp().plus(allowedLateness); } }
public static String formatTime(Instant timestamp) { if (timestamp == null) { return "null"; } else if (timestamp.equals(BoundedWindow.TIMESTAMP_MIN_VALUE)) { return "TIMESTAMP_MIN_VALUE"; } else if (timestamp.equals(BoundedWindow.TIMESTAMP_MAX_VALUE)) { return "TIMESTAMP_MAX_VALUE"; } else if (timestamp.equals(GlobalWindow.INSTANCE.maxTimestamp())) { return "END_OF_GLOBAL_WINDOW"; } else { return timestamp.toString(TIME_FMT); } }
/** * Formats a {@link Instant} timestamp with additional Beam-specific metadata, such as indicating * whether the timestamp is the end of the global window or one of the distinguished values {@link * #TIMESTAMP_MIN_VALUE} or {@link #TIMESTAMP_MIN_VALUE}. */ public static String formatTimestamp(Instant timestamp) { if (timestamp.equals(TIMESTAMP_MIN_VALUE)) { return timestamp.toString() + " (TIMESTAMP_MIN_VALUE)"; } else if (timestamp.equals(TIMESTAMP_MAX_VALUE)) { return timestamp.toString() + " (TIMESTAMP_MAX_VALUE)"; } else if (timestamp.equals(GlobalWindow.INSTANCE.maxTimestamp())) { return timestamp.toString() + " (end of global window)"; } else { return timestamp.toString(); } }
@Test public void garbageCollectionTimeAfterEndOfGlobalWindow() { FixedWindows windowFn = FixedWindows.of(Duration.standardMinutes(5)); WindowingStrategy<?, ?> strategy = WindowingStrategy.globalDefault().withWindowFn(windowFn); IntervalWindow window = windowFn.assignWindow(new Instant(BoundedWindow.TIMESTAMP_MAX_VALUE)); assertThat(window.maxTimestamp(), equalTo(GlobalWindow.INSTANCE.maxTimestamp())); assertThat( LateDataUtils.garbageCollectionTime(window, strategy), equalTo(GlobalWindow.INSTANCE.maxTimestamp())); }
@Override public IntervalWindow assignWindow(Instant timestamp) { Instant start = new Instant( timestamp.getMillis() - timestamp.plus(size).minus(offset).getMillis() % size.getMillis()); // The global window is inclusive of max timestamp, while interval window excludes its // upper bound Instant endOfGlobalWindow = GlobalWindow.INSTANCE.maxTimestamp().plus(1); // The end of the window is either start + size if that is within the allowable range, otherwise // the end of the global window. Truncating the window drives many other // areas of this system in the appropriate way automatically. // // Though it is curious that the very last representable fixed window is shorter than the rest, // when we are processing data in the year 294247, we'll probably have technology that can // account for this. Instant end = start.isAfter(endOfGlobalWindow.minus(size)) ? endOfGlobalWindow : start.plus(size); return new IntervalWindow(start, end); }
/** Tests that the last hour of the universe in fact ends at the end of time. */ @Test public void testEndOfTime() { Instant endOfGlobalWindow = GlobalWindow.INSTANCE.maxTimestamp(); FixedWindows windowFn = FixedWindows.of(Duration.standardHours(1)); IntervalWindow truncatedWindow = windowFn.assignWindow(endOfGlobalWindow.minus(1)); assertThat(truncatedWindow.maxTimestamp(), equalTo(endOfGlobalWindow)); }
@Test public void garbageCollectionTimeAfterEndOfGlobalWindowWithLateness() { FixedWindows windowFn = FixedWindows.of(Duration.standardMinutes(5)); Duration allowedLateness = Duration.millis(Long.MAX_VALUE); WindowingStrategy<?, ?> strategy = WindowingStrategy.globalDefault() .withWindowFn(windowFn) .withAllowedLateness(allowedLateness); IntervalWindow window = windowFn.assignWindow(new Instant(-100)); assertThat( window.maxTimestamp().plus(allowedLateness), Matchers.greaterThan(GlobalWindow.INSTANCE.maxTimestamp())); assertThat( LateDataUtils.garbageCollectionTime(window, strategy), equalTo(GlobalWindow.INSTANCE.maxTimestamp())); } }
0L, BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis(), GlobalWindow.INSTANCE.maxTimestamp().getMillis()))) .apply("AssignTimestampToValue", ParDo.of(new TestOutputTimestampDoFn<>())) .apply( anyOf( equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()), equalTo(GlobalWindow.INSTANCE.maxTimestamp().getMillis()), equalTo(0L)));
@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)); }
GlobalWindow.INSTANCE.maxTimestamp().minus(allowedLateness).plus(1); assertTrue(window.maxTimestamp().isBefore(GlobalWindow.INSTANCE.maxTimestamp())); assertTrue( window.maxTimestamp().plus(allowedLateness).isAfter(GlobalWindow.INSTANCE.maxTimestamp())); tester.advanceInputWatermark(GlobalWindow.INSTANCE.maxTimestamp()); assertThat(tester.extractOutput(), contains(isWindowedValue(equalTo(55))));
TimestampedValue.of("foo", new Instant(0)), TimestampedValue.of("bar", new Instant(33)), TimestampedValue.of("bar", GlobalWindow.INSTANCE.maxTimestamp())) .withCoder(StringUtf8Coder.of())) .apply(
@Test @Category({NeedsRunner.class, UsesTestStream.class}) public void testElementsAtAlmostPositiveInfinity() { Instant endOfGlobalWindow = GlobalWindow.INSTANCE.maxTimestamp(); TestStream<String> stream = TestStream.create(StringUtf8Coder.of()) .addElements( TimestampedValue.of("foo", endOfGlobalWindow), TimestampedValue.of("bar", endOfGlobalWindow)) .advanceWatermarkToInfinity(); FixedWindows windows = FixedWindows.of(Duration.standardHours(6)); PCollection<String> windowedValues = p.apply(stream) .apply(Window.into(windows)) .apply(WithKeys.of(1)) .apply(GroupByKey.create()) .apply(Values.create()) .apply(Flatten.iterables()); PAssert.that(windowedValues) .inWindow(windows.assignWindow(endOfGlobalWindow)) .containsInAnyOrder("foo", "bar"); p.run(); }