@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()); }
@Override public PCollection<T> expand(PBegin input) { return PCollection.createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED, coder); }
public static <W extends BoundedWindow> ReduceFnTester<Integer, Iterable<Integer>, W> nonCombining( WindowFn<?, W> windowFn, TriggerStateMachine triggerStateMachine, AccumulationMode mode, Duration allowedDataLateness, ClosingBehavior closingBehavior) throws Exception { WindowingStrategy<?, W> strategy = WindowingStrategy.of(windowFn) .withTimestampCombiner(TimestampCombiner.EARLIEST) .withMode(mode) .withAllowedLateness(allowedDataLateness) .withClosingBehavior(closingBehavior); return nonCombining(strategy, triggerStateMachine); }
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 <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; } } }
static void verifyStateSupportForWindowingStrategy(WindowingStrategy strategy) { // https://issues.apache.org/jira/browse/BEAM-2507 if (!strategy.getWindowFn().isNonMerging()) { throw new UnsupportedOperationException( String.format( "%s does not currently support state or timers with merging windows", DataflowRunner.class.getSimpleName())); } } }
/** * 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<KeyedWorkItem<KeyT, InputT>> expand(PCollection<KV<KeyT, InputT>> input) { KvCoder<KeyT, InputT> kvCoder = (KvCoder<KeyT, InputT>) input.getCoder(); return PCollection.createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), input.isBounded(), KeyedWorkItemCoder.of( kvCoder.getKeyCoder(), kvCoder.getValueCoder(), input.getWindowingStrategy().getWindowFn().windowCoder())); }
public static void applicableTo(PCollection<?> input) { WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy(); // Verify that the input PCollection is bounded, or that there is windowing/triggering being // used. Without this, the watermark (at end of global window) will never be reached. if (windowingStrategy.getWindowFn() instanceof GlobalWindows && windowingStrategy.getTrigger() instanceof DefaultTrigger && input.isBounded() != IsBounded.BOUNDED) { throw new IllegalStateException( "GroupByKey cannot be applied to non-bounded PCollection in " + "the GlobalWindow without a trigger. Use a Window.into or Window.triggering transform " + "prior to GroupByKey."); } // Validate the window merge function. if (windowingStrategy.getWindowFn() instanceof InvalidWindows) { String cause = ((InvalidWindows<?>) windowingStrategy.getWindowFn()).getCause(); throw new IllegalStateException( "GroupByKey must have a valid Window merge function. " + "Invalid because: " + cause); } }
@Test public void testWindowIntoSetWindowfn() { WindowingStrategy<?, ?> strategy = pipeline .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of())) .apply(Window.into(FixedWindows.of(Duration.standardMinutes(10)))) .getWindowingStrategy(); assertTrue(strategy.getWindowFn() instanceof FixedWindows); assertTrue(strategy.getTrigger() instanceof DefaultTrigger); assertEquals(AccumulationMode.DISCARDING_FIRED_PANES, strategy.getMode()); }
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()); }
private void applicableTo(PCollection<?> input) { WindowingStrategy<?, ?> outputStrategy = getOutputStrategyInternal(input.getWindowingStrategy()); // Make sure that the windowing strategy is complete & valid. if (outputStrategy.isTriggerSpecified() && !(outputStrategy.getTrigger() instanceof DefaultTrigger) && !(outputStrategy.getWindowFn() instanceof GlobalWindows) && !outputStrategy.isAllowedLatenessSpecified()) { throw new IllegalArgumentException( "Except when using GlobalWindows," + " calling .triggering() to specify a trigger requires that the allowed lateness" + " be specified using .withAllowedLateness() to set the upper bound on how late" + " data can arrive before being dropped. See Javadoc for more details."); } if (!outputStrategy.isModeSpecified() && canProduceMultiplePanes(outputStrategy)) { throw new IllegalArgumentException( "Calling .triggering() to specify a trigger or calling .withAllowedLateness() to" + " specify an allowed lateness greater than zero requires that the accumulation" + " mode be specified using .discardingFiredPanes() or .accumulatingFiredPanes()." + " See Javadoc for more details."); } }
@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()); }
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<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()); } }
/** * Return {@code timestamp}, possibly shifted forward in time according to the window strategy's * output time function. */ private Instant shift(Instant timestamp, W window) { Instant shifted = windowingStrategy .getTimestampCombiner() .assign(window, windowingStrategy.getWindowFn().getOutputTime(timestamp, window)); // Don't call checkState(), to avoid calling BoundedWindow.formatTimestamp() every time if (shifted.isBefore(timestamp)) { throw new IllegalStateException( String.format( "TimestampCombiner moved element from %s to earlier time %s for window %s", BoundedWindow.formatTimestamp(timestamp), BoundedWindow.formatTimestamp(shifted), window)); } checkState( timestamp.isAfter(window.maxTimestamp()) || !shifted.isAfter(window.maxTimestamp()), "TimestampCombiner moved element from %s to %s which is beyond end of " + "window %s", timestamp, shifted, window); return shifted; }
/** 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; }
@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))); }
/** * 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()); }
public static <W extends BoundedWindow> WatermarkCallback onGuaranteedFiring( BoundedWindow window, WindowingStrategy<?, W> strategy, Runnable callback) { @SuppressWarnings("unchecked") Instant firingAfter = strategy.getTrigger().getWatermarkThatGuaranteesFiring((W) window); return new WatermarkCallback(firingAfter, callback); }