/** * 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, WindowingStrategy windowingStrategy) { return garbageCollectionTime(window, windowingStrategy.getAllowedLateness()); }
private boolean canProduceMultiplePanes(WindowingStrategy<?, ?> strategy) { // The default trigger is Repeatedly.forever(AfterWatermark.pastEndOfWindow()); This fires // for every late-arriving element if allowed lateness is nonzero, and thus we must have // an accumulating mode specified boolean dataCanArriveLate = !(strategy.getWindowFn() instanceof GlobalWindows) && strategy.getAllowedLateness().getMillis() > 0; boolean hasCustomTrigger = !(strategy.getTrigger() instanceof DefaultTrigger); return dataCanArriveLate || hasCustomTrigger; }
private static <T, W extends BoundedWindow> void validateWindowStrategy( WindowingStrategy<T, W> strategy) { if (!strategy.getWindowFn().isNonMerging() && (!strategy.getTrigger().getClass().equals(DefaultTrigger.class) || strategy.getAllowedLateness().isLongerThan(Duration.ZERO))) { throw new UnsupportedOperationException( String.format( "%s does not support merging windowing strategies, except when using the default " + "trigger and zero allowed lateness.", Distinct.class.getSimpleName())); } }
private boolean triggersOncePerWindow(WindowingStrategy windowingStrategy) { Trigger trigger = windowingStrategy.getTrigger(); return !(windowingStrategy.getWindowFn() instanceof GlobalWindows) && trigger instanceof DefaultTrigger && ZERO.equals(windowingStrategy.getAllowedLateness()); }
public static <W extends BoundedWindow> WatermarkCallback afterWindowExpiration( BoundedWindow window, WindowingStrategy<?, W> strategy, Runnable callback) { // Fire one milli past the end of the window. This ensures that all window expiration // timers are delivered first Instant firingAfter = window.maxTimestamp().plus(strategy.getAllowedLateness()).plus(1L); return new WatermarkCallback(firingAfter, callback); }
private boolean windowIsExpired(BoundedWindow w) { return timerInternals .currentInputWatermarkTime() .isAfter(w.maxTimestamp().plus(windowingStrategy.getAllowedLateness())); } }
public static <W extends BoundedWindow> WatermarkCallback afterWindowExpiration( BoundedWindow window, WindowingStrategy<?, W> strategy, Runnable callback) { // Fire one milli past the end of the window. This ensures that all window expiration // timers are delivered first Instant firingAfter = window.maxTimestamp().plus(strategy.getAllowedLateness()).plus(1L); return new WatermarkCallback(firingAfter, callback); }
window .maxTimestamp() .plus(windowingStrategy.getAllowedLateness()) .isBefore(timerInternals.currentInputWatermarkTime()); if (expired) {
window .maxTimestamp() .plus(windowingStrategy.getAllowedLateness()) .isBefore(timerInternals.currentInputWatermarkTime()); if (expired) {
private void verifyInputWindowing(PCollection<KV<KeyT, ValueT>> input) { if (input.isBounded().equals(PCollection.IsBounded.UNBOUNDED)) { checkArgument( !input.getWindowingStrategy().equals(WindowingStrategy.globalDefault()), "Cannot work with %s and GLOBAL %s", PCollection.IsBounded.UNBOUNDED, WindowingStrategy.class.getSimpleName()); checkArgument( input.getWindowingStrategy().getTrigger().getClass().equals(DefaultTrigger.class), "Cannot work with %s trigger. Write works correctly only with %s", input.getWindowingStrategy().getTrigger().getClass().getSimpleName(), DefaultTrigger.class.getSimpleName()); checkArgument( input.getWindowingStrategy().getAllowedLateness().equals(Duration.ZERO), "Write does not allow late data."); } }
@Override public PCollection<KV<K, Iterable<InputT>>> expand(PCollection<KV<K, InputT>> input) { Duration allowedLateness = input.getWindowingStrategy().getAllowedLateness(); checkArgument( input.getCoder() instanceof KvCoder, "coder specified in the input PCollection is not a KvCoder"); KvCoder inputCoder = (KvCoder) input.getCoder(); Coder<K> keyCoder = (Coder<K>) inputCoder.getCoderArguments().get(0); Coder<InputT> valueCoder = (Coder<InputT>) inputCoder.getCoderArguments().get(1); return input.apply( ParDo.of(new GroupIntoBatchesDoFn<>(batchSize, allowedLateness, keyCoder, valueCoder))); }
@Override public boolean equals(Object object) { if (!(object instanceof WindowingStrategy)) { return false; } WindowingStrategy<?, ?> other = (WindowingStrategy<?, ?>) object; return isAllowedLatenessSpecified() == other.isAllowedLatenessSpecified() && isModeSpecified() == other.isModeSpecified() && isTimestampCombinerSpecified() == other.isTimestampCombinerSpecified() && getMode().equals(other.getMode()) && getAllowedLateness().equals(other.getAllowedLateness()) && getClosingBehavior().equals(other.getClosingBehavior()) && getOnTimeBehavior().equals(other.getOnTimeBehavior()) && getTrigger().equals(other.getTrigger()) && getTimestampCombiner().equals(other.getTimestampCombiner()) && getWindowFn().equals(other.getWindowFn()); }
@Test public void testWindowIntoAccumulatingLatenessNoTrigger() { FixedWindows fixed = FixedWindows.of(Duration.standardMinutes(10)); WindowingStrategy<?, ?> strategy = pipeline .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of())) .apply( "Lateness", Window.<String>into(fixed) .withAllowedLateness(Duration.standardDays(1)) .accumulatingFiredPanes()) .getWindowingStrategy(); assertThat(strategy.isTriggerSpecified(), is(false)); assertThat(strategy.isModeSpecified(), is(true)); assertThat(strategy.isAllowedLatenessSpecified(), is(true)); assertThat(strategy.getMode(), equalTo(AccumulationMode.ACCUMULATING_FIRED_PANES)); assertThat(strategy.getAllowedLateness(), equalTo(Duration.standardDays(1))); }
/** * Converts a {@link WindowingStrategy} into a {@link RunnerApi.WindowingStrategy}, registering * any components in the provided {@link SdkComponents}. */ public static RunnerApi.WindowingStrategy toProto( WindowingStrategy<?, ?> windowingStrategy, SdkComponents components) throws IOException { SdkFunctionSpec windowFnSpec = toProto(windowingStrategy.getWindowFn(), components); RunnerApi.WindowingStrategy.Builder windowingStrategyProto = RunnerApi.WindowingStrategy.newBuilder() .setOutputTime(toProto(windowingStrategy.getTimestampCombiner())) .setAccumulationMode(toProto(windowingStrategy.getMode())) .setClosingBehavior(toProto(windowingStrategy.getClosingBehavior())) .setAllowedLateness(windowingStrategy.getAllowedLateness().getMillis()) .setTrigger(TriggerTranslation.toProto(windowingStrategy.getTrigger())) .setWindowFn(windowFnSpec) .setAssignsToOneWindow(windowingStrategy.getWindowFn().assignsToOneWindow()) .setOnTimeBehavior(toProto(windowingStrategy.getOnTimeBehavior())) .setWindowCoderId( components.registerCoder(windowingStrategy.getWindowFn().windowCoder())); return windowingStrategyProto.build(); }
@Override public PCollection<Iterable<ValueInSingleWindow<T>>> expand(PCollection<T> input) { WindowFn<?, ?> originalWindowFn = input.getWindowingStrategy().getWindowFn(); return input .apply(Reify.windows()) .apply( WithKeys.<Integer, ValueInSingleWindow<T>>of(0) .withKeyType(new TypeDescriptor<Integer>() {})) .apply( Window.into( new IdentityWindowFn<KV<Integer, ValueInSingleWindow<T>>>( originalWindowFn.windowCoder())) .triggering(Never.ever()) .withAllowedLateness(input.getWindowingStrategy().getAllowedLateness()) .discardingFiredPanes()) // all values have the same key so they all appear as a single output element .apply(GroupByKey.create()) .apply(Values.create()) .setWindowingStrategyInternal(input.getWindowingStrategy()); } }
.triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1))) .discardingFiredPanes() .withAllowedLateness(inputWindowingStrategy.getAllowedLateness()) .withTimestampCombiner(TimestampCombiner.EARLIEST))
/** 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; }
Window.<String>into(fixed10).withAllowedLateness(Duration.standardDays(2))); assertEquals(Duration.standardDays(2), notChanged.getWindowingStrategy().getAllowedLateness()); .accumulatingFiredPanes()); assertEquals(Duration.standardDays(1), longWindow.getWindowingStrategy().getAllowedLateness()); Duration.standardDays(1), autoCorrectedWindow.getWindowingStrategy().getAllowedLateness());
@Test public void testWindowIntoPropagatesLateness() { FixedWindows fixed10 = FixedWindows.of(Duration.standardMinutes(10)); FixedWindows fixed25 = FixedWindows.of(Duration.standardMinutes(25)); WindowingStrategy<?, ?> strategy = pipeline .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of())) .apply( "WindowInto10", Window.<String>into(fixed10) .withAllowedLateness(Duration.standardDays(1)) .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(5))) .accumulatingFiredPanes()) .apply("WindowInto25", Window.into(fixed25)) .getWindowingStrategy(); assertEquals(Duration.standardDays(1), strategy.getAllowedLateness()); assertEquals(fixed25, strategy.getWindowFn()); }
@Test public void testWindowPropagatesEachPart() { FixedWindows fixed10 = FixedWindows.of(Duration.standardMinutes(10)); Repeatedly trigger = Repeatedly.forever(AfterPane.elementCountAtLeast(5)); WindowingStrategy<?, ?> strategy = pipeline .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of())) .apply("Mode", Window.<String>configure().accumulatingFiredPanes()) .apply( "Lateness", Window.<String>configure().withAllowedLateness(Duration.standardDays(1))) .apply("Trigger", Window.<String>configure().triggering(trigger)) .apply("Window", Window.into(fixed10)) .getWindowingStrategy(); assertEquals(fixed10, strategy.getWindowFn()); assertEquals(trigger, strategy.getTrigger()); assertEquals(AccumulationMode.ACCUMULATING_FIRED_PANES, strategy.getMode()); assertEquals(Duration.standardDays(1), strategy.getAllowedLateness()); }