private <W extends BoundedWindow> WindowingStrategy<?, W> getOutputWindowing( WindowingStrategy<?, W> inputStrategy) { if (inputStrategy.getWindowFn() instanceof InvalidWindows) { @SuppressWarnings("unchecked") InvalidWindows<W> invalidWindows = (InvalidWindows<W>) inputStrategy.getWindowFn(); return inputStrategy.withWindowFn(invalidWindows.getOriginalWindowFn()); } else { return inputStrategy; } } }
public WindowingStrategy<?, ?> updateWindowingStrategy(WindowingStrategy<?, ?> inputStrategy) { WindowFn<?, ?> inputWindowFn = inputStrategy.getWindowFn(); if (!inputWindowFn.isNonMerging()) { // Prevent merging windows again, without explicit user // involvement, e.g., by Window.into() or Window.remerge(). inputWindowFn = new InvalidWindows<>( "WindowFn has already been consumed by previous GroupByKey", inputWindowFn); } // We also switch to the continuation trigger associated with the current trigger. return inputStrategy .withWindowFn(inputWindowFn) .withTrigger(inputStrategy.getTrigger().getContinuationTrigger()); }
@Test public void testAssignDisplayDataUnchanged() { FixedWindows windowFn = FixedWindows.of(Duration.standardHours(5)); Window<Object> original = Window.into(windowFn); WindowingStrategy<?, ?> updated = WindowingStrategy.globalDefault().withWindowFn(windowFn); DisplayData displayData = DisplayData.from(new Window.Assign<>(original, updated)); assertThat(displayData, hasDisplayItem("windowFn", windowFn.getClass())); assertThat(displayData, includesDisplayDataFor("windowFn", windowFn)); assertThat(displayData, not(hasDisplayItem("trigger"))); assertThat(displayData, not(hasDisplayItem("accumulationMode"))); assertThat(displayData, not(hasDisplayItem("allowedLateness"))); assertThat(displayData, not(hasDisplayItem("closingBehavior"))); assertThat(displayData, not(hasDisplayItem("timestampCombiner"))); }
@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())); }
@Test public void beforeEndOfGlobalWindowSame() { FixedWindows windowFn = FixedWindows.of(Duration.standardMinutes(5)); Duration allowedLateness = Duration.standardMinutes(2); WindowingStrategy<?, ?> strategy = WindowingStrategy.globalDefault() .withWindowFn(windowFn) .withAllowedLateness(allowedLateness); IntervalWindow window = windowFn.assignWindow(new Instant(10)); assertThat( LateDataUtils.garbageCollectionTime(window, strategy), equalTo(window.maxTimestamp().plus(allowedLateness))); }
@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())); } }
/** Get the output strategy of this {@link Window Window PTransform}. For internal use only. */ public WindowingStrategy<?, ?> getOutputStrategyInternal(WindowingStrategy<?, ?> inputStrategy) { WindowingStrategy<?, ?> result = inputStrategy; if (getWindowFn() != null) { result = result.withWindowFn(getWindowFn()); } if (getTrigger() != null) { result = result.withTrigger(getTrigger()); } if (getAccumulationMode() != null) { result = result.withMode(getAccumulationMode()); } if (getAllowedLateness() != null) { result = result.withAllowedLateness( Ordering.natural().max(getAllowedLateness(), inputStrategy.getAllowedLateness())); } if (getClosingBehavior() != null) { result = result.withClosingBehavior(getClosingBehavior()); } if (getOnTimeBehavior() != null) { result = result.withOnTimeBehavior(getOnTimeBehavior()); } if (getTimestampCombiner() != null) { result = result.withTimestampCombiner(getTimestampCombiner()); } return result; }