You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ke...@apache.org on 2016/06/22 02:22:15 UTC
[01/12] incubator-beam git commit: Move some easy stuff into
runners/core-java
Repository: incubator-beam
Updated Branches:
refs/heads/master 73862b422 -> e255cd6be
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
deleted file mode 100644
index 9916c5c..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
+++ /dev/null
@@ -1,784 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
-import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.transforms.windowing.TriggerBuilder;
-import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
-import org.apache.beam.sdk.util.state.InMemoryStateInternals;
-import org.apache.beam.sdk.util.state.State;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.util.state.StateNamespace;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.apache.beam.sdk.values.TupleTag;
-
-import com.google.common.base.Function;
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.PriorityQueue;
-import java.util.Set;
-
-import javax.annotation.Nullable;
-
-/**
- * Test utility that runs a {@link ReduceFn}, {@link WindowFn}, {@link Trigger} using in-memory stub
- * implementations to provide the {@link TimerInternals} and {@link WindowingInternals} needed to
- * run {@code Trigger}s and {@code ReduceFn}s.
- *
- * @param <InputT> The element types.
- * @param <OutputT> The final type for elements in the window (for instance,
- * {@code Iterable<InputT>})
- * @param <W> The type of windows being used.
- */
-public class ReduceFnTester<InputT, OutputT, W extends BoundedWindow> {
- private static final String KEY = "TEST_KEY";
-
- private final TestInMemoryStateInternals<String> stateInternals =
- new TestInMemoryStateInternals<>(KEY);
- private final TestTimerInternals timerInternals = new TestTimerInternals();
-
- private final WindowFn<Object, W> windowFn;
- private final TestWindowingInternals windowingInternals;
- private final Coder<OutputT> outputCoder;
- private final WindowingStrategy<Object, W> objectStrategy;
- private final ReduceFn<String, InputT, OutputT, W> reduceFn;
- private final PipelineOptions options;
-
- /**
- * If true, the output watermark is automatically advanced to the latest possible
- * point when the input watermark is advanced. This is the default for most tests.
- * If false, the output watermark must be explicitly advanced by the test, which can
- * be used to exercise some of the more subtle behavior of WatermarkHold.
- */
- private boolean autoAdvanceOutputWatermark;
-
- private ExecutableTrigger executableTrigger;
-
- private final InMemoryLongSumAggregator droppedDueToClosedWindow =
- new InMemoryLongSumAggregator(GroupAlsoByWindowsDoFn.DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER);
-
- public static <W extends BoundedWindow> ReduceFnTester<Integer, Iterable<Integer>, W>
- nonCombining(WindowingStrategy<?, W> windowingStrategy) throws Exception {
- return new ReduceFnTester<Integer, Iterable<Integer>, W>(
- windowingStrategy,
- SystemReduceFn.<String, Integer, W>buffering(VarIntCoder.of()),
- IterableCoder.of(VarIntCoder.of()),
- PipelineOptionsFactory.create(),
- NullSideInputReader.empty());
- }
-
- public static <W extends BoundedWindow> ReduceFnTester<Integer, Iterable<Integer>, W>
- nonCombining(WindowFn<?, W> windowFn, TriggerBuilder trigger, AccumulationMode mode,
- Duration allowedDataLateness, ClosingBehavior closingBehavior) throws Exception {
- WindowingStrategy<?, W> strategy =
- WindowingStrategy.of(windowFn)
- .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
- .withTrigger(trigger.buildTrigger())
- .withMode(mode)
- .withAllowedLateness(allowedDataLateness)
- .withClosingBehavior(closingBehavior);
- return nonCombining(strategy);
- }
-
- public static <W extends BoundedWindow, AccumT, OutputT> ReduceFnTester<Integer, OutputT, W>
- combining(WindowingStrategy<?, W> strategy,
- KeyedCombineFn<String, Integer, AccumT, OutputT> combineFn,
- Coder<OutputT> outputCoder) throws Exception {
-
- CoderRegistry registry = new CoderRegistry();
- registry.registerStandardCoders();
- AppliedCombineFn<String, Integer, AccumT, OutputT> fn =
- AppliedCombineFn.<String, Integer, AccumT, OutputT>withInputCoder(
- combineFn, registry, KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()));
-
- return new ReduceFnTester<Integer, OutputT, W>(
- strategy,
- SystemReduceFn.<String, Integer, AccumT, OutputT, W>combining(StringUtf8Coder.of(), fn),
- outputCoder,
- PipelineOptionsFactory.create(),
- NullSideInputReader.empty());
- }
-
- public static <W extends BoundedWindow, AccumT, OutputT> ReduceFnTester<Integer, OutputT, W>
- combining(WindowingStrategy<?, W> strategy,
- KeyedCombineFnWithContext<String, Integer, AccumT, OutputT> combineFn,
- Coder<OutputT> outputCoder,
- PipelineOptions options,
- SideInputReader sideInputReader) throws Exception {
- CoderRegistry registry = new CoderRegistry();
- registry.registerStandardCoders();
- AppliedCombineFn<String, Integer, AccumT, OutputT> fn =
- AppliedCombineFn.<String, Integer, AccumT, OutputT>withInputCoder(
- combineFn, registry, KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()));
-
- return new ReduceFnTester<Integer, OutputT, W>(
- strategy,
- SystemReduceFn.<String, Integer, AccumT, OutputT, W>combining(StringUtf8Coder.of(), fn),
- outputCoder,
- options,
- sideInputReader);
- }
- public static <W extends BoundedWindow, AccumT, OutputT> ReduceFnTester<Integer, OutputT, W>
- combining(WindowFn<?, W> windowFn, Trigger trigger, AccumulationMode mode,
- KeyedCombineFn<String, Integer, AccumT, OutputT> combineFn, Coder<OutputT> outputCoder,
- Duration allowedDataLateness) throws Exception {
-
- WindowingStrategy<?, W> strategy =
- WindowingStrategy.of(windowFn)
- .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
- .withTrigger(trigger)
- .withMode(mode)
- .withAllowedLateness(allowedDataLateness);
-
- return combining(strategy, combineFn, outputCoder);
- }
-
- private ReduceFnTester(WindowingStrategy<?, W> wildcardStrategy,
- ReduceFn<String, InputT, OutputT, W> reduceFn, Coder<OutputT> outputCoder,
- PipelineOptions options, SideInputReader sideInputReader) throws Exception {
- @SuppressWarnings("unchecked")
- WindowingStrategy<Object, W> objectStrategy = (WindowingStrategy<Object, W>) wildcardStrategy;
-
- this.objectStrategy = objectStrategy;
- this.reduceFn = reduceFn;
- this.windowFn = objectStrategy.getWindowFn();
- this.windowingInternals = new TestWindowingInternals(sideInputReader);
- this.outputCoder = outputCoder;
- this.autoAdvanceOutputWatermark = true;
- this.executableTrigger = wildcardStrategy.getTrigger();
- this.options = options;
- }
-
- public void setAutoAdvanceOutputWatermark(boolean autoAdvanceOutputWatermark) {
- this.autoAdvanceOutputWatermark = autoAdvanceOutputWatermark;
- }
-
- @Nullable
- public Instant getNextTimer(TimeDomain domain) {
- return timerInternals.getNextTimer(domain);
- }
-
- ReduceFnRunner<String, InputT, OutputT, W> createRunner() {
- return new ReduceFnRunner<>(
- KEY,
- objectStrategy,
- stateInternals,
- timerInternals,
- windowingInternals,
- droppedDueToClosedWindow,
- reduceFn,
- options);
- }
-
- public ExecutableTrigger getTrigger() {
- return executableTrigger;
- }
-
- public boolean isMarkedFinished(W window) {
- return createRunner().isFinished(window);
- }
-
- public boolean hasNoActiveWindows() {
- return createRunner().hasNoActiveWindows();
- }
-
- @SafeVarargs
- public final void assertHasOnlyGlobalAndFinishedSetsFor(W... expectedWindows) {
- assertHasOnlyGlobalAndAllowedTags(
- ImmutableSet.copyOf(expectedWindows),
- ImmutableSet.<StateTag<? super String, ?>>of(TriggerRunner.FINISHED_BITS_TAG));
- }
-
- @SafeVarargs
- public final void assertHasOnlyGlobalAndFinishedSetsAndPaneInfoFor(W... expectedWindows) {
- assertHasOnlyGlobalAndAllowedTags(
- ImmutableSet.copyOf(expectedWindows),
- ImmutableSet.<StateTag<? super String, ?>>of(
- TriggerRunner.FINISHED_BITS_TAG, PaneInfoTracker.PANE_INFO_TAG,
- WatermarkHold.watermarkHoldTagForOutputTimeFn(objectStrategy.getOutputTimeFn()),
- WatermarkHold.EXTRA_HOLD_TAG));
- }
-
- public final void assertHasOnlyGlobalState() {
- assertHasOnlyGlobalAndAllowedTags(
- Collections.<W>emptySet(), Collections.<StateTag<? super String, ?>>emptySet());
- }
-
- @SafeVarargs
- public final void assertHasOnlyGlobalAndPaneInfoFor(W... expectedWindows) {
- assertHasOnlyGlobalAndAllowedTags(
- ImmutableSet.copyOf(expectedWindows),
- ImmutableSet.<StateTag<? super String, ?>>of(
- PaneInfoTracker.PANE_INFO_TAG,
- WatermarkHold.watermarkHoldTagForOutputTimeFn(objectStrategy.getOutputTimeFn()),
- WatermarkHold.EXTRA_HOLD_TAG));
- }
-
- /**
- * Verifies that the the set of windows that have any state stored is exactly
- * {@code expectedWindows} and that each of these windows has only tags from {@code allowedTags}.
- */
- private void assertHasOnlyGlobalAndAllowedTags(
- Set<W> expectedWindows, Set<StateTag<? super String, ?>> allowedTags) {
- Set<StateNamespace> expectedWindowsSet = new HashSet<>();
- for (W expectedWindow : expectedWindows) {
- expectedWindowsSet.add(windowNamespace(expectedWindow));
- }
- Map<StateNamespace, Set<StateTag<? super String, ?>>> actualWindows = new HashMap<>();
-
- for (StateNamespace namespace : stateInternals.getNamespacesInUse()) {
- if (namespace instanceof StateNamespaces.GlobalNamespace) {
- continue;
- } else if (namespace instanceof StateNamespaces.WindowNamespace) {
- Set<StateTag<? super String, ?>> tagsInUse = stateInternals.getTagsInUse(namespace);
- if (tagsInUse.isEmpty()) {
- continue;
- }
- actualWindows.put(namespace, tagsInUse);
- Set<StateTag<? super String, ?>> unexpected = Sets.difference(tagsInUse, allowedTags);
- if (unexpected.isEmpty()) {
- continue;
- } else {
- fail(namespace + " has unexpected states: " + tagsInUse);
- }
- } else if (namespace instanceof StateNamespaces.WindowAndTriggerNamespace) {
- Set<StateTag<? super String, ?>> tagsInUse = stateInternals.getTagsInUse(namespace);
- assertTrue(namespace + " contains " + tagsInUse, tagsInUse.isEmpty());
- } else {
- fail("Unrecognized namespace " + namespace);
- }
- }
-
- assertEquals("Still in use: " + actualWindows.toString(), expectedWindowsSet,
- actualWindows.keySet());
- }
-
- private StateNamespace windowNamespace(W window) {
- return StateNamespaces.window(windowFn.windowCoder(), window);
- }
-
- public Instant getWatermarkHold() {
- return stateInternals.earliestWatermarkHold();
- }
-
- public Instant getOutputWatermark() {
- return timerInternals.currentOutputWatermarkTime();
- }
-
- public long getElementsDroppedDueToClosedWindow() {
- return droppedDueToClosedWindow.getSum();
- }
-
- /**
- * How many panes do we have in the output?
- */
- public int getOutputSize() {
- return windowingInternals.outputs.size();
- }
-
- /**
- * Retrieve the values that have been output to this time, and clear out the output accumulator.
- */
- public List<WindowedValue<OutputT>> extractOutput() {
- ImmutableList<WindowedValue<OutputT>> result =
- FluentIterable.from(windowingInternals.outputs)
- .transform(new Function<WindowedValue<KV<String, OutputT>>, WindowedValue<OutputT>>() {
- @Override
- public WindowedValue<OutputT> apply(WindowedValue<KV<String, OutputT>> input) {
- return input.withValue(input.getValue().getValue());
- }
- })
- .toList();
- windowingInternals.outputs.clear();
- return result;
- }
-
- /**
- * Advance the input watermark to the specified time, firing any timers that should
- * fire. Then advance the output watermark as far as possible.
- */
- public void advanceInputWatermark(Instant newInputWatermark) throws Exception {
- ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
- timerInternals.advanceInputWatermark(runner, newInputWatermark);
- runner.persist();
- }
-
- /**
- * If {@link #autoAdvanceOutputWatermark} is {@literal false}, advance the output watermark
- * to the given value. Otherwise throw.
- */
- public void advanceOutputWatermark(Instant newOutputWatermark) throws Exception {
- timerInternals.advanceOutputWatermark(newOutputWatermark);
- }
-
- /** Advance the processing time to the specified time, firing any timers that should fire. */
- public void advanceProcessingTime(Instant newProcessingTime) throws Exception {
- ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
- timerInternals.advanceProcessingTime(runner, newProcessingTime);
- runner.persist();
- }
-
- /**
- * Advance the synchronized processing time to the specified time,
- * firing any timers that should fire.
- */
- public void advanceSynchronizedProcessingTime(Instant newProcessingTime) throws Exception {
- ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
- timerInternals.advanceSynchronizedProcessingTime(runner, newProcessingTime);
- runner.persist();
- }
-
- /**
- * Inject all the timestamped values (after passing through the window function) as if they
- * arrived in a single chunk of a bundle (or work-unit).
- */
- @SafeVarargs
- public final void injectElements(TimestampedValue<InputT>... values) throws Exception {
- for (TimestampedValue<InputT> value : values) {
- WindowTracing.trace("TriggerTester.injectElements: {}", value);
- }
- ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
- runner.processElements(Iterables.transform(
- Arrays.asList(values), new Function<TimestampedValue<InputT>, WindowedValue<InputT>>() {
- @Override
- public WindowedValue<InputT> apply(TimestampedValue<InputT> input) {
- try {
- InputT value = input.getValue();
- Instant timestamp = input.getTimestamp();
- Collection<W> windows = windowFn.assignWindows(new TestAssignContext<W>(
- windowFn, value, timestamp, Arrays.asList(GlobalWindow.INSTANCE)));
- return WindowedValue.of(value, timestamp, windows, PaneInfo.NO_FIRING);
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
- }));
-
- // Persist after each bundle.
- runner.persist();
- }
-
- public void fireTimer(W window, Instant timestamp, TimeDomain domain) throws Exception {
- ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
- runner.onTimer(
- TimerData.of(StateNamespaces.window(windowFn.windowCoder(), window), timestamp, domain));
- runner.persist();
- }
-
- /**
- * Simulate state.
- */
- private static class TestInMemoryStateInternals<K> extends InMemoryStateInternals<K> {
-
- public TestInMemoryStateInternals(K key) {
- super(key);
- }
-
- public Set<StateTag<? super K, ?>> getTagsInUse(StateNamespace namespace) {
- Set<StateTag<? super K, ?>> inUse = new HashSet<>();
- for (Entry<StateTag<? super K, ?>, State> entry :
- inMemoryState.getTagsInUse(namespace).entrySet()) {
- if (!isEmptyForTesting(entry.getValue())) {
- inUse.add(entry.getKey());
- }
- }
- return inUse;
- }
-
- public Set<StateNamespace> getNamespacesInUse() {
- return inMemoryState.getNamespacesInUse();
- }
-
- /** Return the earliest output watermark hold in state, or null if none. */
- public Instant earliestWatermarkHold() {
- Instant minimum = null;
- for (State storage : inMemoryState.values()) {
- if (storage instanceof WatermarkHoldState) {
- Instant hold = ((WatermarkHoldState<?>) storage).read();
- if (minimum == null || (hold != null && hold.isBefore(minimum))) {
- minimum = hold;
- }
- }
- }
- return minimum;
- }
- }
-
- /**
- * Convey the simulated state and implement {@link #outputWindowedValue} to capture all output
- * elements.
- */
- private class TestWindowingInternals implements WindowingInternals<InputT, KV<String, OutputT>> {
- private List<WindowedValue<KV<String, OutputT>>> outputs = new ArrayList<>();
- private SideInputReader sideInputReader;
-
- private TestWindowingInternals(SideInputReader sideInputReader) {
- this.sideInputReader = sideInputReader;
- }
-
- @Override
- public void outputWindowedValue(KV<String, OutputT> output, Instant timestamp,
- Collection<? extends BoundedWindow> windows, PaneInfo pane) {
- // Copy the output value (using coders) before capturing it.
- KV<String, OutputT> copy = SerializableUtils.<KV<String, OutputT>>ensureSerializableByCoder(
- KvCoder.of(StringUtf8Coder.of(), outputCoder), output, "outputForWindow");
- WindowedValue<KV<String, OutputT>> value = WindowedValue.of(copy, timestamp, windows, pane);
- outputs.add(value);
- }
-
- @Override
- public TimerInternals timerInternals() {
- throw new UnsupportedOperationException(
- "Testing triggers should not use timers from WindowingInternals.");
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- throw new UnsupportedOperationException(
- "Testing triggers should not use windows from WindowingInternals.");
- }
-
- @Override
- public PaneInfo pane() {
- throw new UnsupportedOperationException(
- "Testing triggers should not use pane from WindowingInternals.");
- }
-
- @Override
- public <T> void writePCollectionViewData(
- TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
- throw new UnsupportedOperationException(
- "Testing triggers should not use writePCollectionViewData from WindowingInternals.");
- }
-
- @Override
- public StateInternals<Object> stateInternals() {
- // Safe for testing only
- @SuppressWarnings({"unchecked", "rawtypes"})
- TestInMemoryStateInternals<Object> untypedStateInternals =
- (TestInMemoryStateInternals) stateInternals;
- return untypedStateInternals;
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
- if (!sideInputReader.contains(view)) {
- throw new IllegalArgumentException("calling sideInput() with unknown view");
- }
- BoundedWindow sideInputWindow =
- view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow);
- return sideInputReader.get(view, sideInputWindow);
- }
- }
-
- private static class TestAssignContext<W extends BoundedWindow>
- extends WindowFn<Object, W>.AssignContext {
- private Object element;
- private Instant timestamp;
- private Collection<? extends BoundedWindow> windows;
-
- public TestAssignContext(WindowFn<Object, W> windowFn, Object element, Instant timestamp,
- Collection<? extends BoundedWindow> windows) {
- windowFn.super();
- this.element = element;
- this.timestamp = timestamp;
- this.windows = windows;
- }
-
- @Override
- public Object element() {
- return element;
- }
-
- @Override
- public Instant timestamp() {
- return timestamp;
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- return windows;
- }
- }
-
- private static class InMemoryLongSumAggregator implements Aggregator<Long, Long> {
- private final String name;
- private long sum = 0;
-
- public InMemoryLongSumAggregator(String name) {
- this.name = name;
- }
-
- @Override
- public void addValue(Long value) {
- sum += value;
- }
-
- @Override
- public String getName() {
- return name;
- }
-
- @Override
- public CombineFn<Long, ?, Long> getCombineFn() {
- return new Sum.SumLongFn();
- }
-
- public long getSum() {
- return sum;
- }
- }
-
- /**
- * Simulate the firing of timers and progression of input and output watermarks for a
- * single computation and key in a Windmill-like streaming environment. Similar to
- * {@link BatchTimerInternals}, but also tracks the output watermark.
- */
- private class TestTimerInternals implements TimerInternals {
- /** At most one timer per timestamp is kept. */
- private Set<TimerData> existingTimers = new HashSet<>();
-
- /** Pending input watermark timers, in timestamp order. */
- private PriorityQueue<TimerData> watermarkTimers = new PriorityQueue<>(11);
-
- /** Pending processing time timers, in timestamp order. */
- private PriorityQueue<TimerData> processingTimers = new PriorityQueue<>(11);
-
- /** Current input watermark. */
- @Nullable
- private Instant inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
-
- /** Current output watermark. */
- @Nullable
- private Instant outputWatermarkTime = null;
-
- /** Current processing time. */
- private Instant processingTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
-
- /** Current synchronized processing time. */
- @Nullable
- private Instant synchronizedProcessingTime = null;
-
- @Nullable
- public Instant getNextTimer(TimeDomain domain) {
- TimerData data = null;
- switch (domain) {
- case EVENT_TIME:
- data = watermarkTimers.peek();
- break;
- case PROCESSING_TIME:
- case SYNCHRONIZED_PROCESSING_TIME:
- data = processingTimers.peek();
- break;
- }
- Preconditions.checkNotNull(data); // cases exhaustive
- return data == null ? null : data.getTimestamp();
- }
-
- private PriorityQueue<TimerData> queue(TimeDomain domain) {
- switch (domain) {
- case EVENT_TIME:
- return watermarkTimers;
- case PROCESSING_TIME:
- case SYNCHRONIZED_PROCESSING_TIME:
- return processingTimers;
- }
- throw new RuntimeException(); // cases exhaustive
- }
-
- @Override
- public void setTimer(TimerData timer) {
- WindowTracing.trace("TestTimerInternals.setTimer: {}", timer);
- if (existingTimers.add(timer)) {
- queue(timer.getDomain()).add(timer);
- }
- }
-
- @Override
- public void deleteTimer(TimerData timer) {
- WindowTracing.trace("TestTimerInternals.deleteTimer: {}", timer);
- existingTimers.remove(timer);
- queue(timer.getDomain()).remove(timer);
- }
-
- @Override
- public Instant currentProcessingTime() {
- return processingTime;
- }
-
- @Override
- @Nullable
- public Instant currentSynchronizedProcessingTime() {
- return synchronizedProcessingTime;
- }
-
- @Override
- public Instant currentInputWatermarkTime() {
- return Preconditions.checkNotNull(inputWatermarkTime);
- }
-
- @Override
- @Nullable
- public Instant currentOutputWatermarkTime() {
- return outputWatermarkTime;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("watermarkTimers", watermarkTimers)
- .add("processingTimers", processingTimers)
- .add("inputWatermarkTime", inputWatermarkTime)
- .add("outputWatermarkTime", outputWatermarkTime)
- .add("processingTime", processingTime)
- .toString();
- }
-
- public void advanceInputWatermark(
- ReduceFnRunner<?, ?, ?, ?> runner, Instant newInputWatermark) throws Exception {
- Preconditions.checkNotNull(newInputWatermark);
- Preconditions.checkState(
- !newInputWatermark.isBefore(inputWatermarkTime),
- "Cannot move input watermark time backwards from %s to %s", inputWatermarkTime,
- newInputWatermark);
- WindowTracing.trace("TestTimerInternals.advanceInputWatermark: from {} to {}",
- inputWatermarkTime, newInputWatermark);
- inputWatermarkTime = newInputWatermark;
- advanceAndFire(runner, newInputWatermark, TimeDomain.EVENT_TIME);
-
- Instant hold = stateInternals.earliestWatermarkHold();
- if (hold == null) {
- WindowTracing.trace("TestTimerInternals.advanceInputWatermark: no holds, "
- + "so output watermark = input watermark");
- hold = inputWatermarkTime;
- }
- if (autoAdvanceOutputWatermark) {
- advanceOutputWatermark(hold);
- }
- }
-
- public void advanceOutputWatermark(Instant newOutputWatermark) {
- Preconditions.checkNotNull(newOutputWatermark);
- if (newOutputWatermark.isAfter(inputWatermarkTime)) {
- WindowTracing.trace(
- "TestTimerInternals.advanceOutputWatermark: clipping output watermark from {} to {}",
- newOutputWatermark, inputWatermarkTime);
- newOutputWatermark = inputWatermarkTime;
- }
- Preconditions.checkState(
- outputWatermarkTime == null || !newOutputWatermark.isBefore(outputWatermarkTime),
- "Cannot move output watermark time backwards from %s to %s", outputWatermarkTime,
- newOutputWatermark);
- WindowTracing.trace("TestTimerInternals.advanceOutputWatermark: from {} to {}",
- outputWatermarkTime, newOutputWatermark);
- outputWatermarkTime = newOutputWatermark;
- }
-
- public void advanceProcessingTime(
- ReduceFnRunner<?, ?, ?, ?> runner, Instant newProcessingTime) throws Exception {
- Preconditions.checkState(!newProcessingTime.isBefore(processingTime),
- "Cannot move processing time backwards from %s to %s", processingTime, newProcessingTime);
- WindowTracing.trace("TestTimerInternals.advanceProcessingTime: from {} to {}", processingTime,
- newProcessingTime);
- processingTime = newProcessingTime;
- advanceAndFire(runner, newProcessingTime, TimeDomain.PROCESSING_TIME);
- }
-
- public void advanceSynchronizedProcessingTime(
- ReduceFnRunner<?, ?, ?, ?> runner, Instant newSynchronizedProcessingTime) throws Exception {
- Preconditions.checkState(!newSynchronizedProcessingTime.isBefore(synchronizedProcessingTime),
- "Cannot move processing time backwards from %s to %s", processingTime,
- newSynchronizedProcessingTime);
- WindowTracing.trace("TestTimerInternals.advanceProcessingTime: from {} to {}",
- synchronizedProcessingTime, newSynchronizedProcessingTime);
- synchronizedProcessingTime = newSynchronizedProcessingTime;
- advanceAndFire(
- runner, newSynchronizedProcessingTime, TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
- }
-
- private void advanceAndFire(
- ReduceFnRunner<?, ?, ?, ?> runner, Instant currentTime, TimeDomain domain)
- throws Exception {
- PriorityQueue<TimerData> queue = queue(domain);
- boolean shouldFire = false;
-
- do {
- TimerData timer = queue.peek();
- // Timers fire when the current time progresses past the timer time.
- shouldFire = timer != null && currentTime.isAfter(timer.getTimestamp());
- if (shouldFire) {
- WindowTracing.trace(
- "TestTimerInternals.advanceAndFire: firing {} at {}", timer, currentTime);
- // Remove before firing, so that if the trigger adds another identical
- // timer we don't remove it.
- queue.remove();
-
- runner.onTimer(timer);
- }
- } while (shouldFire);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
deleted file mode 100644
index fb74fc6..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static org.hamcrest.Matchers.is;
-import static org.mockito.Mockito.mock;
-
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.util.BaseExecutionContext.StepContext;
-import org.apache.beam.sdk.values.TupleTag;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.Arrays;
-import java.util.List;
-
-/**
- * Tests for base {@link DoFnRunnerBase} functionality.
- */
-@RunWith(JUnit4.class)
-public class SimpleDoFnRunnerTest {
- @Rule
- public ExpectedException thrown = ExpectedException.none();
-
- @Test
- public void testExceptionsWrappedAsUserCodeException() {
- ThrowingDoFn fn = new ThrowingDoFn();
- DoFnRunner<String, String> runner = createRunner(fn);
-
- thrown.expect(UserCodeException.class);
- thrown.expectCause(is(fn.exceptionToThrow));
-
- runner.processElement(WindowedValue.valueInGlobalWindow("anyValue"));
- }
-
- @Test
- public void testSystemDoFnInternalExceptionsNotWrapped() {
- ThrowingSystemDoFn fn = new ThrowingSystemDoFn();
- DoFnRunner<String, String> runner = createRunner(fn);
-
- thrown.expect(is(fn.exceptionToThrow));
-
- runner.processElement(WindowedValue.valueInGlobalWindow("anyValue"));
- }
-
- private DoFnRunner<String, String> createRunner(DoFn<String, String> fn) {
- // Pass in only necessary parameters for the test
- List<TupleTag<?>> sideOutputTags = Arrays.asList();
- StepContext context = mock(StepContext.class);
- return new SimpleDoFnRunner<>(
- null, fn, null, null, null, sideOutputTags, context, null, null);
- }
-
- static class ThrowingDoFn extends DoFn<String, String> {
- final Exception exceptionToThrow =
- new UnsupportedOperationException("Expected exception");
-
- @Override
- public void processElement(ProcessContext c) throws Exception {
- throw exceptionToThrow;
- }
- }
-
- @SystemDoFnInternal
- static class ThrowingSystemDoFn extends ThrowingDoFn {
- }
-}
[11/12] incubator-beam git commit: Directly instantiate
SimpleDoFnRunner in its test suite
Posted by ke...@apache.org.
Directly instantiate SimpleDoFnRunner in its test suite
Previously, it went through a static utility class, creating
quasi-circular dependency.
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/efaad322
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/efaad322
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/efaad322
Branch: refs/heads/master
Commit: efaad322f1d3c044b7689764c0eba4f58ded718a
Parents: 3001804
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Jun 21 10:06:36 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Jun 21 14:09:49 2016 -0700
----------------------------------------------------------------------
.../test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/efaad322/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
index 680422b..fb74fc6 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
@@ -66,7 +66,7 @@ public class SimpleDoFnRunnerTest {
// Pass in only necessary parameters for the test
List<TupleTag<?>> sideOutputTags = Arrays.asList();
StepContext context = mock(StepContext.class);
- return DoFnRunners.simpleRunner(
+ return new SimpleDoFnRunner<>(
null, fn, null, null, null, sideOutputTags, context, null, null);
}
[08/12] incubator-beam git commit: Move some easy stuff into
runners/core-java
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
new file mode 100644
index 0000000..f104f6a
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
@@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.ValueState;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+
+import org.joda.time.Instant;
+
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ * Executes a trigger while managing persistence of information about which subtriggers are
+ * finished. Subtriggers include all recursive trigger expressions as well as the entire trigger.
+ *
+ * <p>Specifically, the responsibilities are:
+ *
+ * <ul>
+ * <li>Invoking the trigger's methods via its {@link ExecutableTrigger} wrapper by
+ * constructing the appropriate trigger contexts.</li>
+ * <li>Committing a record of which subtriggers are finished to persistent state.</li>
+ * <li>Restoring the record of which subtriggers are finished from persistent state.</li>
+ * <li>Clearing out the persisted finished set when a caller indicates
+ * (via {#link #clearFinished}) that it is no longer needed.</li>
+ * </ul>
+ *
+ * <p>These responsibilities are intertwined: trigger contexts include mutable information about
+ * which subtriggers are finished. This class provides the information when building the contexts
+ * and commits the information when the method of the {@link ExecutableTrigger} returns.
+ *
+ * @param <W> The kind of windows being processed.
+ */
+public class TriggerRunner<W extends BoundedWindow> {
+ @VisibleForTesting
+ static final StateTag<Object, ValueState<BitSet>> FINISHED_BITS_TAG =
+ StateTags.makeSystemTagInternal(StateTags.value("closed", BitSetCoder.of()));
+
+ private final ExecutableTrigger rootTrigger;
+ private final TriggerContextFactory<W> contextFactory;
+
+ public TriggerRunner(ExecutableTrigger rootTrigger, TriggerContextFactory<W> contextFactory) {
+ Preconditions.checkState(rootTrigger.getTriggerIndex() == 0);
+ this.rootTrigger = rootTrigger;
+ this.contextFactory = contextFactory;
+ }
+
+ private FinishedTriggersBitSet readFinishedBits(ValueState<BitSet> state) {
+ if (!isFinishedSetNeeded()) {
+ // If no trigger in the tree will ever have finished bits, then we don't need to read them.
+ // So that the code can be agnostic to that fact, we create a BitSet that is all 0 (not
+ // finished) for each trigger in the tree.
+ return FinishedTriggersBitSet.emptyWithCapacity(rootTrigger.getFirstIndexAfterSubtree());
+ }
+
+ BitSet bitSet = state.read();
+ return bitSet == null
+ ? FinishedTriggersBitSet.emptyWithCapacity(rootTrigger.getFirstIndexAfterSubtree())
+ : FinishedTriggersBitSet.fromBitSet(bitSet);
+ }
+
+
+ private void clearFinishedBits(ValueState<BitSet> state) {
+ if (!isFinishedSetNeeded()) {
+ // Nothing to clear.
+ return;
+ }
+ state.clear();
+ }
+
+ /** Return true if the trigger is closed in the window corresponding to the specified state. */
+ public boolean isClosed(StateAccessor<?> state) {
+ return readFinishedBits(state.access(FINISHED_BITS_TAG)).isFinished(rootTrigger);
+ }
+
+ public void prefetchForValue(W window, StateAccessor<?> state) {
+ if (isFinishedSetNeeded()) {
+ state.access(FINISHED_BITS_TAG).readLater();
+ }
+ rootTrigger.getSpec().prefetchOnElement(
+ contextFactory.createStateAccessor(window, rootTrigger));
+ }
+
+ public void prefetchOnFire(W window, StateAccessor<?> state) {
+ if (isFinishedSetNeeded()) {
+ state.access(FINISHED_BITS_TAG).readLater();
+ }
+ rootTrigger.getSpec().prefetchOnFire(contextFactory.createStateAccessor(window, rootTrigger));
+ }
+
+ public void prefetchShouldFire(W window, StateAccessor<?> state) {
+ if (isFinishedSetNeeded()) {
+ state.access(FINISHED_BITS_TAG).readLater();
+ }
+ rootTrigger.getSpec().prefetchShouldFire(
+ contextFactory.createStateAccessor(window, rootTrigger));
+ }
+
+ /**
+ * Run the trigger logic to deal with a new value.
+ */
+ public void processValue(W window, Instant timestamp, Timers timers, StateAccessor<?> state)
+ throws Exception {
+ // Clone so that we can detect changes and so that changes here don't pollute merging.
+ FinishedTriggersBitSet finishedSet =
+ readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+ Trigger.OnElementContext triggerContext = contextFactory.createOnElementContext(
+ window, timers, timestamp, rootTrigger, finishedSet);
+ rootTrigger.invokeOnElement(triggerContext);
+ persistFinishedSet(state, finishedSet);
+ }
+
+ public void prefetchForMerge(
+ W window, Collection<W> mergingWindows, MergingStateAccessor<?, W> state) {
+ if (isFinishedSetNeeded()) {
+ for (ValueState<?> value : state.accessInEachMergingWindow(FINISHED_BITS_TAG).values()) {
+ value.readLater();
+ }
+ }
+ rootTrigger.getSpec().prefetchOnMerge(contextFactory.createMergingStateAccessor(
+ window, mergingWindows, rootTrigger));
+ }
+
+ /**
+ * Run the trigger merging logic as part of executing the specified merge.
+ */
+ public void onMerge(W window, Timers timers, MergingStateAccessor<?, W> state) throws Exception {
+ // Clone so that we can detect changes and so that changes here don't pollute merging.
+ FinishedTriggersBitSet finishedSet =
+ readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+
+ // And read the finished bits in each merging window.
+ ImmutableMap.Builder<W, FinishedTriggers> builder = ImmutableMap.builder();
+ for (Map.Entry<W, ValueState<BitSet>> entry :
+ state.accessInEachMergingWindow(FINISHED_BITS_TAG).entrySet()) {
+ // Don't need to clone these, since the trigger context doesn't allow modification
+ builder.put(entry.getKey(), readFinishedBits(entry.getValue()));
+ // Clear the underlying finished bits.
+ clearFinishedBits(entry.getValue());
+ }
+ ImmutableMap<W, FinishedTriggers> mergingFinishedSets = builder.build();
+
+ Trigger.OnMergeContext mergeContext = contextFactory.createOnMergeContext(
+ window, timers, rootTrigger, finishedSet, mergingFinishedSets);
+
+ // Run the merge from the trigger
+ rootTrigger.invokeOnMerge(mergeContext);
+
+ persistFinishedSet(state, finishedSet);
+ }
+
+ public boolean shouldFire(W window, Timers timers, StateAccessor<?> state) throws Exception {
+ FinishedTriggers finishedSet = readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+ Trigger.TriggerContext context = contextFactory.base(window, timers,
+ rootTrigger, finishedSet);
+ return rootTrigger.invokeShouldFire(context);
+ }
+
+ public void onFire(W window, Timers timers, StateAccessor<?> state) throws Exception {
+ // shouldFire should be false.
+ // However it is too expensive to assert.
+ FinishedTriggersBitSet finishedSet =
+ readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
+ Trigger.TriggerContext context = contextFactory.base(window, timers,
+ rootTrigger, finishedSet);
+ rootTrigger.invokeOnFire(context);
+ persistFinishedSet(state, finishedSet);
+ }
+
+ private void persistFinishedSet(
+ StateAccessor<?> state, FinishedTriggersBitSet modifiedFinishedSet) {
+ if (!isFinishedSetNeeded()) {
+ return;
+ }
+
+ ValueState<BitSet> finishedSetState = state.access(FINISHED_BITS_TAG);
+ if (!readFinishedBits(finishedSetState).equals(modifiedFinishedSet)) {
+ if (modifiedFinishedSet.getBitSet().isEmpty()) {
+ finishedSetState.clear();
+ } else {
+ finishedSetState.write(modifiedFinishedSet.getBitSet());
+ }
+ }
+ }
+
+ /**
+ * Clear the finished bits.
+ */
+ public void clearFinished(StateAccessor<?> state) {
+ clearFinishedBits(state.access(FINISHED_BITS_TAG));
+ }
+
+ /**
+ * Clear the state used for executing triggers, but leave the finished set to indicate
+ * the window is closed.
+ */
+ public void clearState(W window, Timers timers, StateAccessor<?> state) throws Exception {
+ // Don't need to clone, because we'll be clearing the finished bits anyways.
+ FinishedTriggers finishedSet = readFinishedBits(state.access(FINISHED_BITS_TAG));
+ rootTrigger.invokeClear(contextFactory.base(window, timers, rootTrigger, finishedSet));
+ }
+
+ private boolean isFinishedSetNeeded() {
+ // TODO: If we know that no trigger in the tree will ever finish, we don't need to do the
+ // lookup. Right now, we special case this for the DefaultTrigger.
+ return !(rootTrigger.getSpec() instanceof DefaultTrigger);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
new file mode 100644
index 0000000..14ec082
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
@@ -0,0 +1,536 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateMerging;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.io.Serializable;
+
+import javax.annotation.Nullable;
+
+/**
+ * Implements the logic to hold the output watermark for a computation back
+ * until it has seen all the elements it needs based on the input watermark for the
+ * computation.
+ *
+ * <p>The backend ensures the output watermark can never progress beyond the
+ * input watermark for a computation. GroupAlsoByWindows computations may add a 'hold'
+ * to the output watermark in order to prevent it progressing beyond a time within a window.
+ * The hold will be 'cleared' when the associated pane is emitted.
+ *
+ * <p>This class is only intended for use by {@link ReduceFnRunner}. The two evolve together and
+ * will likely break any other uses.
+ *
+ * @param <W> The kind of {@link BoundedWindow} the hold is for.
+ */
+class WatermarkHold<W extends BoundedWindow> implements Serializable {
+ /**
+ * Return tag for state containing the output watermark hold
+ * used for elements.
+ */
+ public static <W extends BoundedWindow>
+ StateTag<Object, WatermarkHoldState<W>> watermarkHoldTagForOutputTimeFn(
+ OutputTimeFn<? super W> outputTimeFn) {
+ return StateTags.<Object, WatermarkHoldState<W>>makeSystemTagInternal(
+ StateTags.<W>watermarkStateInternal("hold", outputTimeFn));
+ }
+
+ /**
+ * Tag for state containing end-of-window and garbage collection output watermark holds.
+ * (We can't piggy-back on the data hold state since the outputTimeFn may be
+ * {@link OutputTimeFns#outputAtLatestInputTimestamp()}, in which case every pane will
+ * would take the end-of-window time as its element time.)
+ */
+ @VisibleForTesting
+ public static final StateTag<Object, WatermarkHoldState<BoundedWindow>> EXTRA_HOLD_TAG =
+ StateTags.makeSystemTagInternal(StateTags.watermarkStateInternal(
+ "extra", OutputTimeFns.outputAtEarliestInputTimestamp()));
+
+ private final TimerInternals timerInternals;
+ private final WindowingStrategy<?, W> windowingStrategy;
+ private final StateTag<Object, WatermarkHoldState<W>> elementHoldTag;
+
+ public WatermarkHold(TimerInternals timerInternals, WindowingStrategy<?, W> windowingStrategy) {
+ this.timerInternals = timerInternals;
+ this.windowingStrategy = windowingStrategy;
+ this.elementHoldTag = watermarkHoldTagForOutputTimeFn(windowingStrategy.getOutputTimeFn());
+ }
+
+ /**
+ * Add a hold to prevent the output watermark progressing beyond the (possibly adjusted) timestamp
+ * of the element in {@code context}. We allow the actual hold time to be shifted later by
+ * {@link OutputTimeFn#assignOutputTime}, but no further than the end of the window. The hold will
+ * remain until cleared by {@link #extractAndRelease}. Return the timestamp at which the hold
+ * was placed, or {@literal null} if no hold was placed.
+ *
+ * <p>In the following we'll write {@code E} to represent an element's timestamp after passing
+ * through the window strategy's output time function, {@code IWM} for the local input watermark,
+ * {@code OWM} for the local output watermark, and {@code GCWM} for the garbage collection
+ * watermark (which is at {@code IWM - getAllowedLateness}). Time progresses from left to right,
+ * and we write {@code [ ... ]} to denote a bounded window with implied lower bound.
+ *
+ * <p>Note that the GCWM will be the same as the IWM if {@code getAllowedLateness}
+ * is {@code ZERO}.
+ *
+ * <p>Here are the cases we need to handle. They are conceptually considered in the
+ * sequence written since if getAllowedLateness is ZERO the GCWM is the same as the IWM.
+ * <ol>
+ * <li>(Normal)
+ * <pre>
+ * |
+ * [ | E ]
+ * |
+ * IWM
+ * </pre>
+ * This is, hopefully, the common and happy case. The element is locally on-time and can
+ * definitely make it to an {@code ON_TIME} pane which we can still set an end-of-window timer
+ * for. We place an element hold at E, which may contribute to the {@code ON_TIME} pane's
+ * timestamp (depending on the output time function). Thus the OWM will not proceed past E
+ * until the next pane fires.
+ *
+ * <li>(Discard - no target window)
+ * <pre>
+ * | |
+ * [ E ] | |
+ * | |
+ * GCWM <-getAllowedLateness-> IWM
+ * </pre>
+ * The element is very locally late. The window has been garbage collected, thus there
+ * is no target pane E could be assigned to. We discard E.
+ *
+ * <li>(Unobservably late)
+ * <pre>
+ * | |
+ * [ | E | ]
+ * | |
+ * OWM IWM
+ * </pre>
+ * The element is locally late, however we can still treat this case as for 'Normal' above
+ * since the IWM has not yet passed the end of the window and the element is ahead of the
+ * OWM. In effect, we get to 'launder' the locally late element and consider it as locally
+ * on-time because no downstream computation can observe the difference.
+ *
+ * <li>(Maybe late 1)
+ * <pre>
+ * | |
+ * [ | E ] |
+ * | |
+ * OWM IWM
+ * </pre>
+ * The end-of-window timer may have already fired for this window, and thus an {@code ON_TIME}
+ * pane may have already been emitted. However, if timer firings have been delayed then it
+ * is possible the {@code ON_TIME} pane has not yet been emitted. We can't place an element
+ * hold since we can't be sure if it will be cleared promptly. Thus this element *may* find
+ * its way into an {@code ON_TIME} pane, but if so it will *not* contribute to that pane's
+ * timestamp. We may however set a garbage collection hold if required.
+ *
+ * <li>(Maybe late 2)
+ * <pre>
+ * | |
+ * [ E | | ]
+ * | |
+ * OWM IWM
+ * </pre>
+ * The end-of-window timer has not yet fired, so this element may still appear in an
+ * {@code ON_TIME} pane. However the element is too late to contribute to the output
+ * watermark hold, and thus won't contribute to the pane's timestamp. We can still place an
+ * end-of-window hold.
+ *
+ * <li>(Maybe late 3)
+ * <pre>
+ * | |
+ * [ E | ] |
+ * | |
+ * OWM IWM
+ * </pre>
+ * As for the (Maybe late 2) case, however we don't even know if the end-of-window timer
+ * has already fired, or it is about to fire. We can place only the garbage collection hold,
+ * if required.
+ *
+ * <li>(Definitely late)
+ * <pre>
+ * | |
+ * [ E ] | |
+ * | |
+ * OWM IWM
+ * </pre>
+ * The element is definitely too late to make an {@code ON_TIME} pane. We are too late to
+ * place an end-of-window hold. We can still place a garbage collection hold if required.
+ *
+ * </ol>
+ */
+ @Nullable
+ public Instant addHolds(ReduceFn<?, ?, ?, W>.ProcessValueContext context) {
+ Instant hold = addElementHold(context);
+ if (hold == null) {
+ hold = addEndOfWindowOrGarbageCollectionHolds(context, false/*paneIsEmpty*/);
+ }
+ return hold;
+ }
+
+ /**
+ * 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.getOutputTimeFn().assignOutputTime(timestamp, window);
+ Preconditions.checkState(!shifted.isBefore(timestamp),
+ "OutputTimeFn moved element from %s to earlier time %s for window %s",
+ timestamp, shifted, window);
+ Preconditions.checkState(timestamp.isAfter(window.maxTimestamp())
+ || !shifted.isAfter(window.maxTimestamp()),
+ "OutputTimeFn moved element from %s to %s which is beyond end of "
+ + "window %s",
+ timestamp, shifted, window);
+
+ return shifted;
+ }
+
+ /**
+ * Attempt to add an 'element hold'. Return the {@link Instant} at which the hold was
+ * added (ie the element timestamp plus any forward shift requested by the
+ * {@link WindowingStrategy#getOutputTimeFn}), or {@literal null} if no hold was added.
+ * The hold is only added if both:
+ * <ol>
+ * <li>The backend will be able to respect it. In other words the output watermark cannot
+ * be ahead of the proposed hold time.
+ * <li>A timer will be set (by {@link ReduceFnRunner}) to clear the hold by the end of the
+ * window. In other words the input watermark cannot be ahead of the end of the window.
+ * </ol>
+ * The hold ensures the pane which incorporates the element is will not be considered late by
+ * any downstream computation when it is eventually emitted.
+ */
+ @Nullable
+ private Instant addElementHold(ReduceFn<?, ?, ?, W>.ProcessValueContext context) {
+ // Give the window function a chance to move the hold timestamp forward to encourage progress.
+ // (A later hold implies less impediment to the output watermark making progress, which in
+ // turn encourages end-of-window triggers to fire earlier in following computations.)
+ Instant elementHold = shift(context.timestamp(), context.window());
+
+ Instant outputWM = timerInternals.currentOutputWatermarkTime();
+ Instant inputWM = timerInternals.currentInputWatermarkTime();
+
+ String which;
+ boolean tooLate;
+ // TODO: These case labels could be tightened.
+ // See the case analysis in addHolds above for the motivation.
+ if (outputWM != null && elementHold.isBefore(outputWM)) {
+ which = "too late to effect output watermark";
+ tooLate = true;
+ } else if (context.window().maxTimestamp().isBefore(inputWM)) {
+ which = "too late for end-of-window timer";
+ tooLate = true;
+ } else {
+ which = "on time";
+ tooLate = false;
+ Preconditions.checkState(!elementHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+ "Element hold %s is beyond end-of-time", elementHold);
+ context.state().access(elementHoldTag).add(elementHold);
+ }
+ WindowTracing.trace(
+ "WatermarkHold.addHolds: element hold at {} is {} for "
+ + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
+ elementHold, which, context.key(), context.window(), inputWM,
+ outputWM);
+
+ return tooLate ? null : elementHold;
+ }
+
+ /**
+ * Add an end-of-window hold or, if too late for that, a garbage collection hold (if required).
+ * Return the {@link Instant} at which hold was added, or {@literal null} if no hold was added.
+ */
+ @Nullable
+ private Instant addEndOfWindowOrGarbageCollectionHolds(
+ ReduceFn<?, ?, ?, W>.Context context, boolean paneIsEmpty) {
+ Instant hold = addEndOfWindowHold(context, paneIsEmpty);
+ if (hold == null) {
+ hold = addGarbageCollectionHold(context, paneIsEmpty);
+ }
+ return hold;
+ }
+
+ /**
+ * Attempt to add an 'end-of-window hold'. Return the {@link Instant} at which the hold was added
+ * (ie the end of window time), or {@literal null} if no end of window hold is possible and we
+ * should fallback to a garbage collection hold.
+ *
+ * <p>We only add the hold if we can be sure a timer will be set (by {@link ReduceFnRunner})
+ * to clear it. In other words, the input watermark cannot be ahead of the end of window time.
+ *
+ * <p>An end-of-window hold is added in two situations:
+ * <ol>
+ * <li>An incoming element came in behind the output watermark (so we are too late for placing
+ * the usual element hold), but it may still be possible to include the element in an
+ * {@link Timing#ON_TIME} pane. We place the end of window hold to ensure that pane will
+ * not be considered late by any downstream computation.
+ * <li>We guarantee an {@link Timing#ON_TIME} pane will be emitted for all windows which saw at
+ * least one element, even if that {@link Timing#ON_TIME} pane is empty. Thus when elements in
+ * a pane are processed due to a fired trigger we must set both an end of window timer and an end
+ * of window hold. Again, the hold ensures the {@link Timing#ON_TIME} pane will not be considered
+ * late by any downstream computation.
+ * </ol>
+ */
+ @Nullable
+ private Instant addEndOfWindowHold(ReduceFn<?, ?, ?, W>.Context context, boolean paneIsEmpty) {
+ Instant outputWM = timerInternals.currentOutputWatermarkTime();
+ Instant inputWM = timerInternals.currentInputWatermarkTime();
+ Instant eowHold = context.window().maxTimestamp();
+
+ if (eowHold.isBefore(inputWM)) {
+ WindowTracing.trace(
+ "WatermarkHold.addEndOfWindowHold: end-of-window hold at {} is too late for "
+ + "end-of-window timer for key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
+ eowHold, context.key(), context.window(), inputWM, outputWM);
+ return null;
+ }
+
+ Preconditions.checkState(outputWM == null || !eowHold.isBefore(outputWM),
+ "End-of-window hold %s cannot be before output watermark %s",
+ eowHold, outputWM);
+ Preconditions.checkState(!eowHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+ "End-of-window hold %s is beyond end-of-time", eowHold);
+ // If paneIsEmpty then this hold is just for empty ON_TIME panes, so we want to keep
+ // the hold away from the combining function in elementHoldTag.
+ // However if !paneIsEmpty then it could make sense to use the elementHoldTag here.
+ // Alas, onMerge is forced to add an end of window or garbage collection hold without
+ // knowing whether an element hold is already in place (stopping to check is too expensive).
+ // This it would end up adding an element hold at the end of the window which could
+ // upset the elementHoldTag combining function.
+ context.state().access(EXTRA_HOLD_TAG).add(eowHold);
+ WindowTracing.trace(
+ "WatermarkHold.addEndOfWindowHold: end-of-window hold at {} is on time for "
+ + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
+ eowHold, context.key(), context.window(), inputWM, outputWM);
+ return eowHold;
+ }
+
+ /**
+ * Attempt to add a 'garbage collection hold' if it is required. Return the {@link Instant} at
+ * which the hold was added (ie the end of window time plus allowed lateness),
+ * or {@literal null} if no hold was added.
+ *
+ * <p>We only add the hold if it is distinct from what would be added by
+ * {@link #addEndOfWindowHold}. In other words, {@link WindowingStrategy#getAllowedLateness}
+ * must be non-zero.
+ *
+ * <p>A garbage collection hold is added in two situations:
+ * <ol>
+ * <li>An incoming element came in behind the output watermark, and was too late for placing
+ * the usual element hold or an end of window hold. Place the garbage collection hold so that
+ * we can guarantee when the pane is finally triggered its output will not be dropped due to
+ * excessive lateness by any downstream computation.
+ * <li>The {@link WindowingStrategy#getClosingBehavior()} is
+ * {@link ClosingBehavior#FIRE_ALWAYS}, and thus we guarantee a final pane will be emitted
+ * for all windows which saw at least one element. Again, the garbage collection hold guarantees
+ * that any empty final pane can be given a timestamp which will not be considered beyond
+ * allowed lateness by any downstream computation.
+ * </ol>
+ *
+ * <p>We use {@code paneIsEmpty} to distinguish cases 1 and 2.
+ */
+ @Nullable
+ private Instant addGarbageCollectionHold(
+ ReduceFn<?, ?, ?, W>.Context context, boolean paneIsEmpty) {
+ Instant outputWM = timerInternals.currentOutputWatermarkTime();
+ Instant inputWM = timerInternals.currentInputWatermarkTime();
+ Instant eow = context.window().maxTimestamp();
+ Instant gcHold = eow.plus(windowingStrategy.getAllowedLateness());
+
+ if (!windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO)) {
+ WindowTracing.trace(
+ "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} is unnecessary "
+ + "since no allowed lateness for key:{}; window:{}; inputWatermark:{}; "
+ + "outputWatermark:{}",
+ gcHold, context.key(), context.window(), inputWM, outputWM);
+ return null;
+ }
+
+ if (paneIsEmpty && context.windowingStrategy().getClosingBehavior()
+ == ClosingBehavior.FIRE_IF_NON_EMPTY) {
+ WindowTracing.trace(
+ "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} is unnecessary "
+ + "since empty pane and FIRE_IF_NON_EMPTY for key:{}; window:{}; inputWatermark:{}; "
+ + "outputWatermark:{}",
+ gcHold, context.key(), context.window(), inputWM, outputWM);
+ return null;
+ }
+
+ Preconditions.checkState(!gcHold.isBefore(inputWM),
+ "Garbage collection hold %s cannot be before input watermark %s",
+ gcHold, inputWM);
+ Preconditions.checkState(!gcHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+ "Garbage collection hold %s is beyond end-of-time", gcHold);
+ // Same EXTRA_HOLD_TAG vs elementHoldTag discussion as in addEndOfWindowHold above.
+ context.state().access(EXTRA_HOLD_TAG).add(gcHold);
+
+ WindowTracing.trace(
+ "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} is on time for "
+ + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
+ gcHold, context.key(), context.window(), inputWM, outputWM);
+ return gcHold;
+ }
+
+ /**
+ * Prefetch watermark holds in preparation for merging.
+ */
+ public void prefetchOnMerge(MergingStateAccessor<?, W> state) {
+ StateMerging.prefetchWatermarks(state, elementHoldTag);
+ }
+
+ /**
+ * Updates the watermark hold when windows merge if it is possible the merged value does
+ * not equal all of the existing holds. For example, if the new window implies a later
+ * watermark hold, then earlier holds may be released.
+ */
+ public void onMerge(ReduceFn<?, ?, ?, W>.OnMergeContext context) {
+ WindowTracing.debug("WatermarkHold.onMerge: for key:{}; window:{}; inputWatermark:{}; "
+ + "outputWatermark:{}",
+ context.key(), context.window(), timerInternals.currentInputWatermarkTime(),
+ timerInternals.currentOutputWatermarkTime());
+ StateMerging.mergeWatermarks(context.state(), elementHoldTag, context.window());
+ // If we had a cheap way to determine if we have an element hold then we could
+ // avoid adding an unnecessary end-of-window or garbage collection hold.
+ // Simply reading the above merged watermark would impose an additional read for the
+ // common case that the active window has just one underlying state address window and
+ // the hold depends on the min of the element timestamps.
+ // At least one merged window must be non-empty for the merge to have been triggered.
+ StateMerging.clear(context.state(), EXTRA_HOLD_TAG);
+ addEndOfWindowOrGarbageCollectionHolds(context, false /*paneIsEmpty*/);
+ }
+
+ /**
+ * Result of {@link #extractAndRelease}.
+ */
+ public static class OldAndNewHolds {
+ public final Instant oldHold;
+ @Nullable
+ public final Instant newHold;
+
+ public OldAndNewHolds(Instant oldHold, @Nullable Instant newHold) {
+ this.oldHold = oldHold;
+ this.newHold = newHold;
+ }
+ }
+
+ /**
+ * Return (a future for) the earliest hold for {@code context}. Clear all the holds after
+ * reading, but add/restore an end-of-window or garbage collection hold if required.
+ *
+ * <p>The returned timestamp is the output timestamp according to the {@link OutputTimeFn}
+ * from the windowing strategy of this {@link WatermarkHold}, combined across all the non-late
+ * elements in the current pane. If there is no such value the timestamp is the end
+ * of the window.
+ */
+ public ReadableState<OldAndNewHolds> extractAndRelease(
+ final ReduceFn<?, ?, ?, W>.Context context, final boolean isFinished) {
+ WindowTracing.debug(
+ "WatermarkHold.extractAndRelease: for key:{}; window:{}; inputWatermark:{}; "
+ + "outputWatermark:{}",
+ context.key(), context.window(), timerInternals.currentInputWatermarkTime(),
+ timerInternals.currentOutputWatermarkTime());
+ final WatermarkHoldState<W> elementHoldState = context.state().access(elementHoldTag);
+ final WatermarkHoldState<BoundedWindow> extraHoldState = context.state().access(EXTRA_HOLD_TAG);
+ return new ReadableState<OldAndNewHolds>() {
+ @Override
+ public ReadableState<OldAndNewHolds> readLater() {
+ elementHoldState.readLater();
+ extraHoldState.readLater();
+ return this;
+ }
+
+ @Override
+ public OldAndNewHolds read() {
+ // Read both the element and extra holds.
+ Instant elementHold = elementHoldState.read();
+ Instant extraHold = extraHoldState.read();
+ Instant oldHold;
+ // Find the minimum, accounting for null.
+ if (elementHold == null) {
+ oldHold = extraHold;
+ } else if (extraHold == null) {
+ oldHold = elementHold;
+ } else if (elementHold.isBefore(extraHold)) {
+ oldHold = elementHold;
+ } else {
+ oldHold = extraHold;
+ }
+ if (oldHold == null || oldHold.isAfter(context.window().maxTimestamp())) {
+ // If no hold (eg because all elements came in behind the output watermark), or
+ // the hold was for garbage collection, take the end of window as the result.
+ WindowTracing.debug(
+ "WatermarkHold.extractAndRelease.read: clipping from {} to end of window "
+ + "for key:{}; window:{}",
+ oldHold, context.key(), context.window());
+ oldHold = context.window().maxTimestamp();
+ }
+ WindowTracing.debug("WatermarkHold.extractAndRelease.read: clearing for key:{}; window:{}",
+ context.key(), context.window());
+
+ // Clear the underlying state to allow the output watermark to progress.
+ elementHoldState.clear();
+ extraHoldState.clear();
+
+ @Nullable Instant newHold = null;
+ if (!isFinished) {
+ // Only need to leave behind an end-of-window or garbage collection hold
+ // if future elements will be processed.
+ newHold = addEndOfWindowOrGarbageCollectionHolds(context, true /*paneIsEmpty*/);
+ }
+
+ return new OldAndNewHolds(oldHold, newHold);
+ }
+ };
+ }
+
+ /**
+ * Clear any remaining holds.
+ */
+ public void clearHolds(ReduceFn<?, ?, ?, W>.Context context) {
+ WindowTracing.debug(
+ "WatermarkHold.clearHolds: For key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
+ context.key(), context.window(), timerInternals.currentInputWatermarkTime(),
+ timerInternals.currentOutputWatermarkTime());
+ context.state().access(elementHoldTag).clear();
+ context.state().access(EXTRA_HOLD_TAG).clear();
+ }
+
+ /**
+ * Return the current data hold, or null if none. Does not clear. For debugging only.
+ */
+ @Nullable
+ public Instant getDataCurrent(ReduceFn<?, ?, ?, W>.Context context) {
+ return context.state().access(elementHoldTag).read();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java
new file mode 100644
index 0000000..3e1528f
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaceForTest;
+
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+
+/**
+ * Tests for {@link BatchTimerInternals}.
+ */
+@RunWith(JUnit4.class)
+public class BatchTimerInternalsTest {
+
+ private static final StateNamespace NS1 = new StateNamespaceForTest("NS1");
+
+ @Mock
+ private ReduceFnRunner<?, ?, ?, ?> mockRunner;
+
+ @Before
+ public void setUp() {
+ MockitoAnnotations.initMocks(this);
+ }
+
+ @Test
+ public void testFiringTimers() throws Exception {
+ BatchTimerInternals underTest = new BatchTimerInternals(new Instant(0));
+ TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
+ TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME);
+
+ underTest.setTimer(processingTime1);
+ underTest.setTimer(processingTime2);
+
+ underTest.advanceProcessingTime(mockRunner, new Instant(20));
+ Mockito.verify(mockRunner).onTimer(processingTime1);
+ Mockito.verifyNoMoreInteractions(mockRunner);
+
+ // Advancing just a little shouldn't refire
+ underTest.advanceProcessingTime(mockRunner, new Instant(21));
+ Mockito.verifyNoMoreInteractions(mockRunner);
+
+ // Adding the timer and advancing a little should refire
+ underTest.setTimer(processingTime1);
+ Mockito.verify(mockRunner).onTimer(processingTime1);
+ underTest.advanceProcessingTime(mockRunner, new Instant(21));
+ Mockito.verifyNoMoreInteractions(mockRunner);
+
+ // And advancing the rest of the way should still have the other timer
+ underTest.advanceProcessingTime(mockRunner, new Instant(30));
+ Mockito.verify(mockRunner).onTimer(processingTime2);
+ Mockito.verifyNoMoreInteractions(mockRunner);
+ }
+
+ @Test
+ public void testTimerOrdering() throws Exception {
+ BatchTimerInternals underTest = new BatchTimerInternals(new Instant(0));
+ TimerData watermarkTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME);
+ TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
+ TimerData watermarkTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.EVENT_TIME);
+ TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME);
+
+ underTest.setTimer(processingTime1);
+ underTest.setTimer(watermarkTime1);
+ underTest.setTimer(processingTime2);
+ underTest.setTimer(watermarkTime2);
+
+ underTest.advanceInputWatermark(mockRunner, new Instant(30));
+ Mockito.verify(mockRunner).onTimer(watermarkTime1);
+ Mockito.verify(mockRunner).onTimer(watermarkTime2);
+ Mockito.verifyNoMoreInteractions(mockRunner);
+
+ underTest.advanceProcessingTime(mockRunner, new Instant(30));
+ Mockito.verify(mockRunner).onTimer(processingTime1);
+ Mockito.verify(mockRunner).onTimer(processingTime2);
+ Mockito.verifyNoMoreInteractions(mockRunner);
+ }
+
+ @Test
+ public void testDeduplicate() throws Exception {
+ BatchTimerInternals underTest = new BatchTimerInternals(new Instant(0));
+ TimerData watermarkTime = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME);
+ TimerData processingTime = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
+ underTest.setTimer(watermarkTime);
+ underTest.setTimer(watermarkTime);
+ underTest.setTimer(processingTime);
+ underTest.setTimer(processingTime);
+ underTest.advanceProcessingTime(mockRunner, new Instant(20));
+ underTest.advanceInputWatermark(mockRunner, new Instant(20));
+
+ Mockito.verify(mockRunner).onTimer(processingTime);
+ Mockito.verify(mockRunner).onTimer(watermarkTime);
+ Mockito.verifyNoMoreInteractions(mockRunner);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
new file mode 100644
index 0000000..f653f49
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
@@ -0,0 +1,619 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Properties of {@link GroupAlsoByWindowsDoFn}.
+ *
+ * <p>Some properties may not hold of some implementations, due to restrictions on the context
+ * in which the implementation is applicable. For example, some {@code GroupAlsoByWindows} may not
+ * support merging windows.
+ */
+public class GroupAlsoByWindowsProperties {
+
+ /**
+ * A factory of {@link GroupAlsoByWindowsDoFn} so that the various properties can provide
+ * the appropriate windowing strategy under test.
+ */
+ public interface GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> {
+ <W extends BoundedWindow> GroupAlsoByWindowsDoFn<K, InputT, OutputT, W>
+ forStrategy(WindowingStrategy<?, W> strategy);
+ }
+
+ /**
+ * Tests that for empty input and the given {@link WindowingStrategy}, the provided GABW
+ * implementation produces no output.
+ *
+ * <p>The input type is deliberately left as a wildcard, since it is not relevant.
+ */
+ public static <K, InputT, OutputT> void emptyInputEmptyOutput(
+ GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> gabwFactory)
+ throws Exception {
+
+ WindowingStrategy<?, IntervalWindow> windowingStrategy =
+ WindowingStrategy.of(FixedWindows.of(Duration.millis(10)));
+
+ DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> result = runGABW(
+ gabwFactory,
+ windowingStrategy,
+ (K) null, // key should never be used
+ Collections.<WindowedValue<InputT>>emptyList());
+
+ assertThat(result.peekOutputElements(), hasSize(0));
+ }
+
+ /**
+ * Tests that for a simple sequence of elements on the same key, the given GABW implementation
+ * correctly groups them according to fixed windows.
+ */
+ public static void groupsElementsIntoFixedWindows(
+ GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+ throws Exception {
+
+ WindowingStrategy<?, IntervalWindow> windowingStrategy =
+ WindowingStrategy.of(FixedWindows.of(Duration.millis(10)));
+
+ DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+ runGABW(gabwFactory, windowingStrategy, "key",
+ WindowedValue.of(
+ "v1",
+ new Instant(1),
+ Arrays.asList(window(0, 10)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v2",
+ new Instant(2),
+ Arrays.asList(window(0, 10)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v3",
+ new Instant(13),
+ Arrays.asList(window(10, 20)),
+ PaneInfo.NO_FIRING));
+
+ assertThat(result.peekOutputElements(), hasSize(2));
+
+ TimestampedValue<KV<String, Iterable<String>>> item0 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
+ assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+ assertThat(item0.getTimestamp(), equalTo(window(0, 10).maxTimestamp()));
+
+ TimestampedValue<KV<String, Iterable<String>>> item1 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20)));
+ assertThat(item1.getValue().getValue(), contains("v3"));
+ assertThat(item1.getTimestamp(), equalTo(window(10, 20).maxTimestamp()));
+ }
+
+ /**
+ * Tests that for a simple sequence of elements on the same key, the given GABW implementation
+ * correctly groups them into sliding windows.
+ *
+ * <p>In the input here, each element occurs in multiple windows.
+ */
+ public static void groupsElementsIntoSlidingWindowsWithMinTimestamp(
+ GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+ throws Exception {
+
+ WindowingStrategy<?, IntervalWindow> windowingStrategy = WindowingStrategy.of(
+ SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10)))
+ .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp());
+
+ DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+ runGABW(gabwFactory, windowingStrategy, "key",
+ WindowedValue.of(
+ "v1",
+ new Instant(5),
+ Arrays.asList(window(-10, 10), window(0, 20)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v2",
+ new Instant(15),
+ Arrays.asList(window(0, 20), window(10, 30)),
+ PaneInfo.NO_FIRING));
+
+ assertThat(result.peekOutputElements(), hasSize(3));
+
+ TimestampedValue<KV<String, Iterable<String>>> item0 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(-10, 10)));
+ assertThat(item0.getValue().getValue(), contains("v1"));
+ assertThat(item0.getTimestamp(), equalTo(new Instant(5)));
+
+ TimestampedValue<KV<String, Iterable<String>>> item1 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 20)));
+ assertThat(item1.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+ assertThat(item1.getTimestamp(), equalTo(new Instant(10)));
+
+ TimestampedValue<KV<String, Iterable<String>>> item2 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30)));
+ assertThat(item2.getValue().getValue(), contains("v2"));
+ assertThat(item2.getTimestamp(), equalTo(new Instant(20)));
+ }
+
+ /**
+ * Tests that for a simple sequence of elements on the same key, the given GABW implementation
+ * correctly groups and combines them according to sliding windows.
+ *
+ * <p>In the input here, each element occurs in multiple windows.
+ */
+ public static void combinesElementsInSlidingWindows(
+ GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
+ CombineFn<Long, ?, Long> combineFn)
+ throws Exception {
+
+ WindowingStrategy<?, IntervalWindow> windowingStrategy =
+ WindowingStrategy.of(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10)))
+ .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp());
+
+ DoFnTester<KV<String, Iterable<WindowedValue<Long>>>, KV<String, Long>> result =
+ runGABW(gabwFactory, windowingStrategy, "k",
+ WindowedValue.of(
+ 1L,
+ new Instant(5),
+ Arrays.asList(window(-10, 10), window(0, 20)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ 2L,
+ new Instant(15),
+ Arrays.asList(window(0, 20), window(10, 30)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ 4L,
+ new Instant(18),
+ Arrays.asList(window(0, 20), window(10, 30)),
+ PaneInfo.NO_FIRING));
+
+ assertThat(result.peekOutputElements(), hasSize(3));
+
+ TimestampedValue<KV<String, Long>> item0 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(-10, 10)));
+ assertThat(item0.getValue().getKey(), equalTo("k"));
+ assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L))));
+ assertThat(item0.getTimestamp(), equalTo(new Instant(5L)));
+
+ TimestampedValue<KV<String, Long>> item1 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 20)));
+ assertThat(item1.getValue().getKey(), equalTo("k"));
+ assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L, 4L))));
+ assertThat(item1.getTimestamp(), equalTo(new Instant(5L)));
+
+ TimestampedValue<KV<String, Long>> item2 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30)));
+ assertThat(item2.getValue().getKey(), equalTo("k"));
+ assertThat(item2.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(2L, 4L))));
+ assertThat(item2.getTimestamp(), equalTo(new Instant(15L)));
+ }
+
+ /**
+ * Tests that the given GABW implementation correctly groups elements that fall into overlapping
+ * windows that are not merged.
+ */
+ public static void groupsIntoOverlappingNonmergingWindows(
+ GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+ throws Exception {
+
+ WindowingStrategy<?, IntervalWindow> windowingStrategy =
+ WindowingStrategy.of(FixedWindows.of(Duration.millis(10)));
+
+ DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+ runGABW(gabwFactory, windowingStrategy, "key",
+ WindowedValue.of(
+ "v1",
+ new Instant(1),
+ Arrays.asList(window(0, 5)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v2",
+ new Instant(4),
+ Arrays.asList(window(1, 5)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v3",
+ new Instant(4),
+ Arrays.asList(window(0, 5)),
+ PaneInfo.NO_FIRING));
+
+ assertThat(result.peekOutputElements(), hasSize(2));
+
+ TimestampedValue<KV<String, Iterable<String>>> item0 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 5)));
+ assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v3"));
+ assertThat(item0.getTimestamp(), equalTo(window(1, 5).maxTimestamp()));
+
+ TimestampedValue<KV<String, Iterable<String>>> item1 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(1, 5)));
+ assertThat(item1.getValue().getValue(), contains("v2"));
+ assertThat(item1.getTimestamp(), equalTo(window(0, 5).maxTimestamp()));
+ }
+
+ /**
+ * Tests that the given GABW implementation correctly groups elements into merged sessions.
+ */
+ public static void groupsElementsInMergedSessions(
+ GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+ throws Exception {
+
+ WindowingStrategy<?, IntervalWindow> windowingStrategy =
+ WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)));
+
+ DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+ runGABW(gabwFactory, windowingStrategy, "key",
+ WindowedValue.of(
+ "v1",
+ new Instant(0),
+ Arrays.asList(window(0, 10)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v2",
+ new Instant(5),
+ Arrays.asList(window(5, 15)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v3",
+ new Instant(15),
+ Arrays.asList(window(15, 25)),
+ PaneInfo.NO_FIRING));
+
+ assertThat(result.peekOutputElements(), hasSize(2));
+
+ TimestampedValue<KV<String, Iterable<String>>> item0 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
+ assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+ assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp()));
+
+ TimestampedValue<KV<String, Iterable<String>>> item1 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25)));
+ assertThat(item1.getValue().getValue(), contains("v3"));
+ assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp()));
+ }
+
+ /**
+ * Tests that the given {@link GroupAlsoByWindowsDoFn} implementation combines elements per
+ * session window correctly according to the provided {@link CombineFn}.
+ */
+ public static void combinesElementsPerSession(
+ GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
+ CombineFn<Long, ?, Long> combineFn)
+ throws Exception {
+
+ WindowingStrategy<?, IntervalWindow> windowingStrategy =
+ WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)));
+
+ DoFnTester<KV<String, Iterable<WindowedValue<Long>>>, KV<String, Long>> result =
+ runGABW(gabwFactory, windowingStrategy, "k",
+ WindowedValue.of(
+ 1L,
+ new Instant(0),
+ Arrays.asList(window(0, 10)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ 2L,
+ new Instant(5),
+ Arrays.asList(window(5, 15)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ 4L,
+ new Instant(15),
+ Arrays.asList(window(15, 25)),
+ PaneInfo.NO_FIRING));
+
+ assertThat(result.peekOutputElements(), hasSize(2));
+
+ TimestampedValue<KV<String, Long>> item0 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
+ assertThat(item0.getValue().getKey(), equalTo("k"));
+ assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L))));
+ assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp()));
+
+ TimestampedValue<KV<String, Long>> item1 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25)));
+ assertThat(item1.getValue().getKey(), equalTo("k"));
+ assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(4L))));
+ assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp()));
+ }
+
+ /**
+ * Tests that for a simple sequence of elements on the same key, the given GABW implementation
+ * correctly groups them according to fixed windows and also sets the output timestamp
+ * according to the policy {@link OutputTimeFns#outputAtEndOfWindow()}.
+ */
+ public static void groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp(
+ GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+ throws Exception {
+
+ WindowingStrategy<?, IntervalWindow> windowingStrategy =
+ WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
+ .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
+
+ DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+ runGABW(gabwFactory, windowingStrategy, "key",
+ WindowedValue.of(
+ "v1",
+ new Instant(1),
+ Arrays.asList(window(0, 10)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v2",
+ new Instant(2),
+ Arrays.asList(window(0, 10)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v3",
+ new Instant(13),
+ Arrays.asList(window(10, 20)),
+ PaneInfo.NO_FIRING));
+
+ assertThat(result.peekOutputElements(), hasSize(2));
+
+ TimestampedValue<KV<String, Iterable<String>>> item0 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
+ assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+ assertThat(item0.getTimestamp(), equalTo(window(0, 10).maxTimestamp()));
+
+ TimestampedValue<KV<String, Iterable<String>>> item1 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20)));
+ assertThat(item1.getValue().getValue(), contains("v3"));
+ assertThat(item1.getTimestamp(), equalTo(window(10, 20).maxTimestamp()));
+ }
+
+ /**
+ * Tests that for a simple sequence of elements on the same key, the given GABW implementation
+ * correctly groups them according to fixed windows and also sets the output timestamp
+ * according to the policy {@link OutputTimeFns#outputAtLatestInputTimestamp()}.
+ */
+ public static void groupsElementsIntoFixedWindowsWithLatestTimestamp(
+ GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+ throws Exception {
+
+ WindowingStrategy<?, IntervalWindow> windowingStrategy =
+ WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
+ .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp());
+
+ DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+ runGABW(gabwFactory, windowingStrategy, "k",
+ WindowedValue.of(
+ "v1",
+ new Instant(1),
+ Arrays.asList(window(0, 10)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v2",
+ new Instant(2),
+ Arrays.asList(window(0, 10)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v3",
+ new Instant(13),
+ Arrays.asList(window(10, 20)),
+ PaneInfo.NO_FIRING));
+
+ assertThat(result.peekOutputElements(), hasSize(2));
+
+ TimestampedValue<KV<String, Iterable<String>>> item0 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
+ assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+ assertThat(item0.getTimestamp(), equalTo(new Instant(2)));
+
+ TimestampedValue<KV<String, Iterable<String>>> item1 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20)));
+ assertThat(item1.getValue().getValue(), contains("v3"));
+ assertThat(item1.getTimestamp(), equalTo(new Instant(13)));
+ }
+
+ /**
+ * Tests that the given GABW implementation correctly groups elements into merged sessions
+ * with output timestamps at the end of the merged window.
+ */
+ public static void groupsElementsInMergedSessionsWithEndOfWindowTimestamp(
+ GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+ throws Exception {
+
+ WindowingStrategy<?, IntervalWindow> windowingStrategy =
+ WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)))
+ .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
+
+ DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+ runGABW(gabwFactory, windowingStrategy, "k",
+ WindowedValue.of(
+ "v1",
+ new Instant(0),
+ Arrays.asList(window(0, 10)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v2",
+ new Instant(5),
+ Arrays.asList(window(5, 15)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v3",
+ new Instant(15),
+ Arrays.asList(window(15, 25)),
+ PaneInfo.NO_FIRING));
+
+ assertThat(result.peekOutputElements(), hasSize(2));
+
+ TimestampedValue<KV<String, Iterable<String>>> item0 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
+ assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+ assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp()));
+
+ TimestampedValue<KV<String, Iterable<String>>> item1 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25)));
+ assertThat(item1.getValue().getValue(), contains("v3"));
+ assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp()));
+ }
+
+ /**
+ * Tests that the given GABW implementation correctly groups elements into merged sessions
+ * with output timestamps at the end of the merged window.
+ */
+ public static void groupsElementsInMergedSessionsWithLatestTimestamp(
+ GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
+ throws Exception {
+
+ WindowingStrategy<?, IntervalWindow> windowingStrategy =
+ WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)))
+ .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp());
+
+ BoundedWindow unmergedWindow = window(15, 25);
+ DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
+ runGABW(gabwFactory, windowingStrategy, "k",
+ WindowedValue.of(
+ "v1",
+ new Instant(0),
+ Arrays.asList(window(0, 10)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v2",
+ new Instant(5),
+ Arrays.asList(window(5, 15)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ "v3",
+ new Instant(15),
+ Arrays.asList(unmergedWindow),
+ PaneInfo.NO_FIRING));
+
+ assertThat(result.peekOutputElements(), hasSize(2));
+
+ BoundedWindow mergedWindow = window(0, 15);
+ TimestampedValue<KV<String, Iterable<String>>> item0 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(mergedWindow));
+ assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+ assertThat(item0.getTimestamp(), equalTo(new Instant(5)));
+
+ TimestampedValue<KV<String, Iterable<String>>> item1 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(unmergedWindow));
+ assertThat(item1.getValue().getValue(), contains("v3"));
+ assertThat(item1.getTimestamp(), equalTo(new Instant(15)));
+ }
+
+ /**
+ * Tests that the given {@link GroupAlsoByWindowsDoFn} implementation combines elements per
+ * session window correctly according to the provided {@link CombineFn}.
+ */
+ public static void combinesElementsPerSessionWithEndOfWindowTimestamp(
+ GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
+ CombineFn<Long, ?, Long> combineFn)
+ throws Exception {
+
+ WindowingStrategy<?, IntervalWindow> windowingStrategy =
+ WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)))
+ .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
+
+ BoundedWindow secondWindow = window(15, 25);
+ DoFnTester<?, KV<String, Long>> result =
+ runGABW(gabwFactory, windowingStrategy, "k",
+ WindowedValue.of(
+ 1L,
+ new Instant(0),
+ Arrays.asList(window(0, 10)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ 2L,
+ new Instant(5),
+ Arrays.asList(window(5, 15)),
+ PaneInfo.NO_FIRING),
+ WindowedValue.of(
+ 4L,
+ new Instant(15),
+ Arrays.asList(secondWindow),
+ PaneInfo.NO_FIRING));
+
+ assertThat(result.peekOutputElements(), hasSize(2));
+
+ BoundedWindow firstResultWindow = window(0, 15);
+ TimestampedValue<KV<String, Long>> item0 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(firstResultWindow));
+ assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L))));
+ assertThat(item0.getTimestamp(), equalTo(firstResultWindow.maxTimestamp()));
+
+ TimestampedValue<KV<String, Long>> item1 =
+ Iterables.getOnlyElement(result.peekOutputElementsInWindow(secondWindow));
+ assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(4L))));
+ assertThat(item1.getTimestamp(),
+ equalTo(secondWindow.maxTimestamp()));
+ }
+
+ @SafeVarargs
+ private static <K, InputT, OutputT, W extends BoundedWindow>
+ DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> runGABW(
+ GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> gabwFactory,
+ WindowingStrategy<?, W> windowingStrategy,
+ K key,
+ WindowedValue<InputT>... values) throws Exception {
+ return runGABW(gabwFactory, windowingStrategy, key, Arrays.asList(values));
+ }
+
+ private static <K, InputT, OutputT, W extends BoundedWindow>
+ DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> runGABW(
+ GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> gabwFactory,
+ WindowingStrategy<?, W> windowingStrategy,
+ K key,
+ Collection<WindowedValue<InputT>> values) throws Exception {
+
+ TupleTag<KV<K, OutputT>> outputTag = new TupleTag<>();
+ DoFnRunnerBase.ListOutputManager outputManager = new DoFnRunnerBase.ListOutputManager();
+
+ DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> tester =
+ DoFnTester.of(gabwFactory.forStrategy(windowingStrategy));
+ tester.startBundle();
+ tester.processElement(KV.<K, Iterable<WindowedValue<InputT>>>of(key, values));
+ tester.finishBundle();
+
+ // Sanity check for corruption
+ for (KV<K, OutputT> elem : tester.peekOutputElements()) {
+ assertThat(elem.getKey(), equalTo(key));
+ }
+
+ return tester;
+ }
+
+ private static BoundedWindow window(long start, long end) {
+ return new IntervalWindow(new Instant(start), new Instant(end));
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
new file mode 100644
index 0000000..4ac6164
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.GroupAlsoByWindowsProperties.GroupAlsoByWindowsDoFnFactory;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Unit tests for {@link GroupAlsoByWindowsViaOutputBufferDoFn}.
+ */
+@RunWith(JUnit4.class)
+public class GroupAlsoByWindowsViaOutputBufferDoFnTest {
+
+ private class BufferingGABWViaOutputBufferDoFnFactory<K, InputT>
+ implements GroupAlsoByWindowsDoFnFactory<K, InputT, Iterable<InputT>> {
+
+ private final Coder<InputT> inputCoder;
+
+ public BufferingGABWViaOutputBufferDoFnFactory(Coder<InputT> inputCoder) {
+ this.inputCoder = inputCoder;
+ }
+
+ @Override
+ public <W extends BoundedWindow> GroupAlsoByWindowsDoFn<K, InputT, Iterable<InputT>, W>
+ forStrategy(WindowingStrategy<?, W> windowingStrategy) {
+ return new GroupAlsoByWindowsViaOutputBufferDoFn<K, InputT, Iterable<InputT>, W>(
+ windowingStrategy,
+ SystemReduceFn.<K, InputT, W>buffering(inputCoder));
+ }
+ }
+
+ @Test
+ public void testEmptyInputEmptyOutput() throws Exception {
+ GroupAlsoByWindowsProperties.emptyInputEmptyOutput(
+ new BufferingGABWViaOutputBufferDoFnFactory<>(StringUtf8Coder.of()));
+ }
+
+ @Test
+ public void testGroupsElementsIntoFixedWindows() throws Exception {
+ GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindows(
+ new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+ }
+
+ @Test
+ public void testGroupsElementsIntoSlidingWindows() throws Exception {
+ GroupAlsoByWindowsProperties.groupsElementsIntoSlidingWindowsWithMinTimestamp(
+ new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+ }
+
+ @Test
+ public void testGroupsIntoOverlappingNonmergingWindows() throws Exception {
+ GroupAlsoByWindowsProperties.groupsIntoOverlappingNonmergingWindows(
+ new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+ }
+
+ @Test
+ public void testGroupsIntoSessions() throws Exception {
+ GroupAlsoByWindowsProperties.groupsElementsInMergedSessions(
+ new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+ }
+
+ @Test
+ public void testGroupsElementsIntoFixedWindowsWithEndOfWindowTimestamp() throws Exception {
+ GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp(
+ new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+ }
+
+ @Test
+ public void testGroupsElementsIntoFixedWindowsWithLatestTimestamp() throws Exception {
+ GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindowsWithLatestTimestamp(
+ new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+ }
+
+ @Test
+ public void testGroupsElementsIntoSessionsWithEndOfWindowTimestamp() throws Exception {
+ GroupAlsoByWindowsProperties.groupsElementsInMergedSessionsWithEndOfWindowTimestamp(
+ new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+ }
+
+ @Test
+ public void testGroupsElementsIntoSessionsWithLatestTimestamp() throws Exception {
+ GroupAlsoByWindowsProperties.groupsElementsInMergedSessionsWithLatestTimestamp(
+ new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java
new file mode 100644
index 0000000..d929d39
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.LateDataDroppingDoFnRunner.LateDataFilter;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.util.Arrays;
+
+/**
+ * Unit tests for {@link LateDataDroppingDoFnRunner}.
+ */
+@RunWith(JUnit4.class)
+public class LateDataDroppingDoFnRunnerTest {
+ private static final FixedWindows WINDOW_FN = FixedWindows.of(Duration.millis(10));
+
+ @Mock private TimerInternals mockTimerInternals;
+
+ @Before
+ public void setUp() {
+ MockitoAnnotations.initMocks(this);
+ }
+
+ @Test
+ public void testLateDataFilter() throws Exception {
+ when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(15L));
+
+ InMemoryLongSumAggregator droppedDueToLateness =
+ new InMemoryLongSumAggregator("droppedDueToLateness");
+ LateDataFilter lateDataFilter = new LateDataFilter(
+ WindowingStrategy.of(WINDOW_FN), mockTimerInternals, droppedDueToLateness);
+
+ Iterable<WindowedValue<Integer>> actual = lateDataFilter.filter(
+ "a",
+ ImmutableList.of(
+ createDatum(13, 13L),
+ createDatum(5, 5L), // late element, earlier than 4L.
+ createDatum(16, 16L),
+ createDatum(18, 18L)));
+
+ Iterable<WindowedValue<Integer>> expected = ImmutableList.of(
+ createDatum(13, 13L),
+ createDatum(16, 16L),
+ createDatum(18, 18L));
+ assertThat(expected, containsInAnyOrder(Iterables.toArray(actual, WindowedValue.class)));
+ assertEquals(1, droppedDueToLateness.sum);
+ }
+
+ private <T> WindowedValue<T> createDatum(T element, long timestampMillis) {
+ Instant timestamp = new Instant(timestampMillis);
+ return WindowedValue.of(
+ element,
+ timestamp,
+ Arrays.asList(WINDOW_FN.assignWindow(timestamp)),
+ PaneInfo.NO_FIRING);
+ }
+
+ private static class InMemoryLongSumAggregator implements Aggregator<Long, Long> {
+ private final String name;
+ private long sum = 0;
+
+ public InMemoryLongSumAggregator(String name) {
+ this.name = name;
+ }
+
+ @Override
+ public void addValue(Long value) {
+ sum += value;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public CombineFn<Long, ?, Long> getCombineFn() {
+ return new Sum.SumLongFn();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java
new file mode 100644
index 0000000..8885118
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java
@@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+
+import com.google.common.collect.ImmutableList;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests for {@link PushbackSideInputDoFnRunner}.
+ */
+@RunWith(JUnit4.class)
+public class PushbackSideInputDoFnRunnerTest {
+ @Mock private ReadyCheckingSideInputReader reader;
+ private TestDoFnRunner<Integer, Integer> underlying;
+ private PCollectionView<Integer> singletonView;
+
+ @Before
+ public void setup() {
+ MockitoAnnotations.initMocks(this);
+ TestPipeline p = TestPipeline.create();
+ PCollection<Integer> created = p.apply(Create.of(1, 2, 3));
+ singletonView =
+ created
+ .apply(Window.into(new IdentitySideInputWindowFn()))
+ .apply(Sum.integersGlobally().asSingletonView());
+
+ underlying = new TestDoFnRunner<>();
+ }
+
+ private PushbackSideInputDoFnRunner<Integer, Integer> createRunner(
+ ImmutableList<PCollectionView<?>> views) {
+ PushbackSideInputDoFnRunner<Integer, Integer> runner =
+ PushbackSideInputDoFnRunner.create(underlying, views, reader);
+ runner.startBundle();
+ return runner;
+ }
+
+ @Test
+ public void startFinishBundleDelegates() {
+ PushbackSideInputDoFnRunner runner =
+ createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
+
+ assertThat(underlying.started, is(true));
+ assertThat(underlying.finished, is(false));
+ runner.finishBundle();
+ assertThat(underlying.finished, is(true));
+ }
+
+ @Test
+ public void processElementSideInputNotReady() {
+ when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class)))
+ .thenReturn(false);
+
+ PushbackSideInputDoFnRunner<Integer, Integer> runner =
+ createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
+
+ WindowedValue<Integer> oneWindow =
+ WindowedValue.of(
+ 2,
+ new Instant(-2),
+ new IntervalWindow(new Instant(-500L), new Instant(0L)),
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ Iterable<WindowedValue<Integer>> oneWindowPushback =
+ runner.processElementInReadyWindows(oneWindow);
+ assertThat(oneWindowPushback, containsInAnyOrder(oneWindow));
+ assertThat(underlying.inputElems, Matchers.<WindowedValue<Integer>>emptyIterable());
+ }
+
+ @Test
+ public void processElementSideInputNotReadyMultipleWindows() {
+ when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class)))
+ .thenReturn(false);
+
+ PushbackSideInputDoFnRunner<Integer, Integer> runner =
+ createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
+
+ WindowedValue<Integer> multiWindow =
+ WindowedValue.of(
+ 2,
+ new Instant(-2),
+ ImmutableList.of(
+ new IntervalWindow(new Instant(-500L), new Instant(0L)),
+ new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)),
+ GlobalWindow.INSTANCE),
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ Iterable<WindowedValue<Integer>> multiWindowPushback =
+ runner.processElementInReadyWindows(multiWindow);
+ assertThat(multiWindowPushback, equalTo(multiWindow.explodeWindows()));
+ assertThat(underlying.inputElems, Matchers.<WindowedValue<Integer>>emptyIterable());
+ }
+
+ @Test
+ public void processElementSideInputNotReadySomeWindows() {
+ when(reader.isReady(Mockito.eq(singletonView), Mockito.eq(GlobalWindow.INSTANCE)))
+ .thenReturn(false);
+ when(
+ reader.isReady(
+ Mockito.eq(singletonView),
+ org.mockito.AdditionalMatchers.not(Mockito.eq(GlobalWindow.INSTANCE))))
+ .thenReturn(true);
+
+ PushbackSideInputDoFnRunner<Integer, Integer> runner =
+ createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
+
+ IntervalWindow littleWindow = new IntervalWindow(new Instant(-500L), new Instant(0L));
+ IntervalWindow bigWindow =
+ new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L));
+ WindowedValue<Integer> multiWindow =
+ WindowedValue.of(
+ 2,
+ new Instant(-2),
+ ImmutableList.of(littleWindow, bigWindow, GlobalWindow.INSTANCE),
+ PaneInfo.NO_FIRING);
+ Iterable<WindowedValue<Integer>> multiWindowPushback =
+ runner.processElementInReadyWindows(multiWindow);
+ assertThat(
+ multiWindowPushback,
+ containsInAnyOrder(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(-2L))));
+ assertThat(underlying.inputElems,
+ containsInAnyOrder(WindowedValue.of(2, new Instant(-2), littleWindow, PaneInfo.NO_FIRING),
+ WindowedValue.of(2, new Instant(-2), bigWindow, PaneInfo.NO_FIRING)));
+ }
+
+ @Test
+ public void processElementSideInputReadyAllWindows() {
+ when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class)))
+ .thenReturn(true);
+
+ ImmutableList<PCollectionView<?>> views = ImmutableList.<PCollectionView<?>>of(singletonView);
+ PushbackSideInputDoFnRunner<Integer, Integer> runner = createRunner(views);
+
+ WindowedValue<Integer> multiWindow =
+ WindowedValue.of(
+ 2,
+ new Instant(-2),
+ ImmutableList.of(
+ new IntervalWindow(new Instant(-500L), new Instant(0L)),
+ new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)),
+ GlobalWindow.INSTANCE),
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ Iterable<WindowedValue<Integer>> multiWindowPushback =
+ runner.processElementInReadyWindows(multiWindow);
+ assertThat(multiWindowPushback, emptyIterable());
+ assertThat(underlying.inputElems,
+ containsInAnyOrder(ImmutableList.copyOf(multiWindow.explodeWindows()).toArray()));
+ }
+
+ @Test
+ public void processElementNoSideInputs() {
+ PushbackSideInputDoFnRunner<Integer, Integer> runner =
+ createRunner(ImmutableList.<PCollectionView<?>>of());
+
+ WindowedValue<Integer> multiWindow =
+ WindowedValue.of(
+ 2,
+ new Instant(-2),
+ ImmutableList.of(
+ new IntervalWindow(new Instant(-500L), new Instant(0L)),
+ new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)),
+ GlobalWindow.INSTANCE),
+ PaneInfo.ON_TIME_AND_ONLY_FIRING);
+ Iterable<WindowedValue<Integer>> multiWindowPushback =
+ runner.processElementInReadyWindows(multiWindow);
+ assertThat(multiWindowPushback, emptyIterable());
+ assertThat(underlying.inputElems, containsInAnyOrder(multiWindow));
+ }
+
+ private static class TestDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
+ List<WindowedValue<InputT>> inputElems;
+ private boolean started = false;
+ private boolean finished = false;
+
+ @Override
+ public void startBundle() {
+ started = true;
+ inputElems = new ArrayList<>();
+ }
+
+ @Override
+ public void processElement(WindowedValue<InputT> elem) {
+ inputElems.add(elem);
+ }
+
+ @Override
+ public void finishBundle() {
+ finished = true;
+ }
+ }
+}
[09/12] incubator-beam git commit: Move some easy stuff into
runners/core-java
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java
new file mode 100644
index 0000000..b1442dd
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.PCollectionView;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * A {@link DoFnRunner} that can refuse to process elements that are not ready, instead returning
+ * them via the {@link #processElementInReadyWindows(WindowedValue)}.
+ */
+public class PushbackSideInputDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
+ private final DoFnRunner<InputT, OutputT> underlying;
+ private final Collection<PCollectionView<?>> views;
+ private final ReadyCheckingSideInputReader sideInputReader;
+
+ private Set<BoundedWindow> notReadyWindows;
+
+ public static <InputT, OutputT> PushbackSideInputDoFnRunner<InputT, OutputT> create(
+ DoFnRunner<InputT, OutputT> underlying,
+ Collection<PCollectionView<?>> views,
+ ReadyCheckingSideInputReader sideInputReader) {
+ return new PushbackSideInputDoFnRunner<>(underlying, views, sideInputReader);
+ }
+
+ private PushbackSideInputDoFnRunner(
+ DoFnRunner<InputT, OutputT> underlying,
+ Collection<PCollectionView<?>> views,
+ ReadyCheckingSideInputReader sideInputReader) {
+ this.underlying = underlying;
+ this.views = views;
+ this.sideInputReader = sideInputReader;
+ }
+
+ @Override
+ public void startBundle() {
+ notReadyWindows = new HashSet<>();
+ underlying.startBundle();
+ }
+
+ /**
+ * Call the underlying {@link DoFnRunner#processElement(WindowedValue)} for the provided element
+ * for each window the element is in that is ready.
+ *
+ * @param elem the element to process in all ready windows
+ * @return each element that could not be processed because it requires a side input window
+ * that is not ready.
+ */
+ public Iterable<WindowedValue<InputT>> processElementInReadyWindows(WindowedValue<InputT> elem) {
+ if (views.isEmpty()) {
+ processElement(elem);
+ return Collections.emptyList();
+ }
+ ImmutableList.Builder<WindowedValue<InputT>> pushedBack = ImmutableList.builder();
+ for (WindowedValue<InputT> windowElem : elem.explodeWindows()) {
+ BoundedWindow mainInputWindow = Iterables.getOnlyElement(windowElem.getWindows());
+ boolean isReady = !notReadyWindows.contains(mainInputWindow);
+ for (PCollectionView<?> view : views) {
+ BoundedWindow sideInputWindow =
+ view.getWindowingStrategyInternal()
+ .getWindowFn()
+ .getSideInputWindow(mainInputWindow);
+ if (!sideInputReader.isReady(view, sideInputWindow)) {
+ isReady = false;
+ break;
+ }
+ }
+ if (isReady) {
+ processElement(windowElem);
+ } else {
+ notReadyWindows.add(mainInputWindow);
+ pushedBack.add(windowElem);
+ }
+ }
+ return pushedBack.build();
+ }
+
+ @Override
+ public void processElement(WindowedValue<InputT> elem) {
+ underlying.processElement(elem);
+ }
+
+ /**
+ * Call the underlying {@link DoFnRunner#finishBundle()}.
+ */
+ @Override
+ public void finishBundle() {
+ notReadyWindows = null;
+ underlying.finishBundle();
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java
new file mode 100644
index 0000000..c5ee1e1
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateAccessor;
+
+import org.joda.time.Instant;
+
+import java.io.Serializable;
+
+/**
+ * Specification for processing to happen after elements have been grouped by key.
+ *
+ * @param <K> The type of key being processed.
+ * @param <InputT> The type of input values associated with the key.
+ * @param <OutputT> The output type that will be produced for each key.
+ * @param <W> The type of windows this operates on.
+ */
+public abstract class ReduceFn<K, InputT, OutputT, W extends BoundedWindow>
+ implements Serializable {
+
+ /** Information accessible to all the processing methods in this {@code ReduceFn}. */
+ public abstract class Context {
+ /** Return the key that is being processed. */
+ public abstract K key();
+
+ /** The window that is being processed. */
+ public abstract W window();
+
+ /** Access the current {@link WindowingStrategy}. */
+ public abstract WindowingStrategy<?, W> windowingStrategy();
+
+ /** Return the interface for accessing state. */
+ public abstract StateAccessor<K> state();
+
+ /** Return the interface for accessing timers. */
+ public abstract Timers timers();
+ }
+
+ /** Information accessible within {@link #processValue}. */
+ public abstract class ProcessValueContext extends Context {
+ /** Return the actual value being processed. */
+ public abstract InputT value();
+
+ /** Return the timestamp associated with the value. */
+ public abstract Instant timestamp();
+ }
+
+ /** Information accessible within {@link #onMerge}. */
+ public abstract class OnMergeContext extends Context {
+ /** Return the interface for accessing state. */
+ @Override
+ public abstract MergingStateAccessor<K, W> state();
+ }
+
+ /** Information accessible within {@link #onTrigger}. */
+ public abstract class OnTriggerContext extends Context {
+ /** Returns the {@link PaneInfo} for the trigger firing being processed. */
+ public abstract PaneInfo paneInfo();
+
+ /** Output the given value in the current window. */
+ public abstract void output(OutputT value);
+ }
+
+ //////////////////////////////////////////////////////////////////////////////////////////////////
+
+ /**
+ * Called for each value of type {@code InputT} associated with the current key.
+ */
+ public abstract void processValue(ProcessValueContext c) throws Exception;
+
+ /**
+ * Called when windows are merged.
+ */
+ public abstract void onMerge(OnMergeContext context) throws Exception;
+
+ /**
+ * Called when triggers fire.
+ *
+ * <p>Implementations of {@link ReduceFn} should call {@link OnTriggerContext#output} to emit
+ * any results that should be included in the pane produced by this trigger firing.
+ */
+ public abstract void onTrigger(OnTriggerContext context) throws Exception;
+
+ /**
+ * Called before {@link #onMerge} is invoked to provide an opportunity to prefetch any needed
+ * state.
+ *
+ * @param c Context to use prefetch from.
+ */
+ public void prefetchOnMerge(MergingStateAccessor<K, W> c) throws Exception {}
+
+ /**
+ * Called before {@link #onTrigger} is invoked to provide an opportunity to prefetch any needed
+ * state.
+ *
+ * @param context Context to use prefetch from.
+ */
+ public void prefetchOnTrigger(StateAccessor<K> context) {}
+
+ /**
+ * Called to clear any persisted state that the {@link ReduceFn} may be holding. This will be
+ * called when the windowing is closing and will receive no future interactions.
+ */
+ public abstract void clearState(Context context) throws Exception;
+
+ /**
+ * Returns true if the there is no buffered state.
+ */
+ public abstract ReadableState<Boolean> isEmpty(StateAccessor<K> context);
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
new file mode 100644
index 0000000..c90940e
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateContext;
+import org.apache.beam.sdk.util.state.StateContexts;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace;
+import org.apache.beam.sdk.util.state.StateTag;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+
+import org.joda.time.Instant;
+
+import java.util.Collection;
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+/**
+ * Factory for creating instances of the various {@link ReduceFn} contexts.
+ */
+class ReduceFnContextFactory<K, InputT, OutputT, W extends BoundedWindow> {
+ public interface OnTriggerCallbacks<OutputT> {
+ void output(OutputT toOutput);
+ }
+
+ private final K key;
+ private final ReduceFn<K, InputT, OutputT, W> reduceFn;
+ private final WindowingStrategy<?, W> windowingStrategy;
+ private final StateInternals<K> stateInternals;
+ private final ActiveWindowSet<W> activeWindows;
+ private final TimerInternals timerInternals;
+ private final WindowingInternals<?, ?> windowingInternals;
+ private final PipelineOptions options;
+
+ ReduceFnContextFactory(K key, ReduceFn<K, InputT, OutputT, W> reduceFn,
+ WindowingStrategy<?, W> windowingStrategy, StateInternals<K> stateInternals,
+ ActiveWindowSet<W> activeWindows, TimerInternals timerInternals,
+ WindowingInternals<?, ?> windowingInternals, PipelineOptions options) {
+ this.key = key;
+ this.reduceFn = reduceFn;
+ this.windowingStrategy = windowingStrategy;
+ this.stateInternals = stateInternals;
+ this.activeWindows = activeWindows;
+ this.timerInternals = timerInternals;
+ this.windowingInternals = windowingInternals;
+ this.options = options;
+ }
+
+ /** Where should we look for state associated with a given window? */
+ public static enum StateStyle {
+ /** All state is associated with the window itself. */
+ DIRECT,
+ /** State is associated with the 'state address' windows tracked by the active window set. */
+ RENAMED
+ }
+
+ private StateAccessorImpl<K, W> stateAccessor(W window, StateStyle style) {
+ return new StateAccessorImpl<K, W>(
+ activeWindows, windowingStrategy.getWindowFn().windowCoder(),
+ stateInternals, StateContexts.createFromComponents(options, windowingInternals, window),
+ style);
+ }
+
+ public ReduceFn<K, InputT, OutputT, W>.Context base(W window, StateStyle style) {
+ return new ContextImpl(stateAccessor(window, style));
+ }
+
+ public ReduceFn<K, InputT, OutputT, W>.ProcessValueContext forValue(
+ W window, InputT value, Instant timestamp, StateStyle style) {
+ return new ProcessValueContextImpl(stateAccessor(window, style), value, timestamp);
+ }
+
+ public ReduceFn<K, InputT, OutputT, W>.OnTriggerContext forTrigger(W window,
+ ReadableState<PaneInfo> pane, StateStyle style, OnTriggerCallbacks<OutputT> callbacks) {
+ return new OnTriggerContextImpl(stateAccessor(window, style), pane, callbacks);
+ }
+
+ public ReduceFn<K, InputT, OutputT, W>.OnMergeContext forMerge(
+ Collection<W> activeToBeMerged, W mergeResult, StateStyle style) {
+ return new OnMergeContextImpl(
+ new MergingStateAccessorImpl<K, W>(activeWindows,
+ windowingStrategy.getWindowFn().windowCoder(),
+ stateInternals, style, activeToBeMerged, mergeResult));
+ }
+
+ public ReduceFn<K, InputT, OutputT, W>.OnMergeContext forPremerge(W window) {
+ return new OnPremergeContextImpl(new PremergingStateAccessorImpl<K, W>(
+ activeWindows, windowingStrategy.getWindowFn().windowCoder(), stateInternals, window));
+ }
+
+ private class TimersImpl implements Timers {
+ private final StateNamespace namespace;
+
+ public TimersImpl(StateNamespace namespace) {
+ Preconditions.checkArgument(namespace instanceof WindowNamespace);
+ this.namespace = namespace;
+ }
+
+ @Override
+ public void setTimer(Instant timestamp, TimeDomain timeDomain) {
+ timerInternals.setTimer(TimerData.of(namespace, timestamp, timeDomain));
+ }
+
+ @Override
+ public void deleteTimer(Instant timestamp, TimeDomain timeDomain) {
+ timerInternals.deleteTimer(TimerData.of(namespace, timestamp, timeDomain));
+ }
+
+ @Override
+ public Instant currentProcessingTime() {
+ return timerInternals.currentProcessingTime();
+ }
+
+ @Override
+ @Nullable
+ public Instant currentSynchronizedProcessingTime() {
+ return timerInternals.currentSynchronizedProcessingTime();
+ }
+
+ @Override
+ public Instant currentEventTime() {
+ return timerInternals.currentInputWatermarkTime();
+ }
+ }
+
+ // ======================================================================
+ // StateAccessors
+ // ======================================================================
+ static class StateAccessorImpl<K, W extends BoundedWindow> implements StateAccessor<K> {
+
+
+ protected final ActiveWindowSet<W> activeWindows;
+ protected final StateContext<W> context;
+ protected final StateNamespace windowNamespace;
+ protected final Coder<W> windowCoder;
+ protected final StateInternals<K> stateInternals;
+ protected final StateStyle style;
+
+ public StateAccessorImpl(ActiveWindowSet<W> activeWindows, Coder<W> windowCoder,
+ StateInternals<K> stateInternals, StateContext<W> context, StateStyle style) {
+
+ this.activeWindows = activeWindows;
+ this.windowCoder = windowCoder;
+ this.stateInternals = stateInternals;
+ this.context = checkNotNull(context);
+ this.windowNamespace = namespaceFor(context.window());
+ this.style = style;
+ }
+
+ protected StateNamespace namespaceFor(W window) {
+ return StateNamespaces.window(windowCoder, window);
+ }
+
+ protected StateNamespace windowNamespace() {
+ return windowNamespace;
+ }
+
+ W window() {
+ return context.window();
+ }
+
+ StateNamespace namespace() {
+ return windowNamespace();
+ }
+
+ @Override
+ public <StateT extends State> StateT access(StateTag<? super K, StateT> address) {
+ switch (style) {
+ case DIRECT:
+ return stateInternals.state(windowNamespace(), address, context);
+ case RENAMED:
+ return stateInternals.state(
+ namespaceFor(activeWindows.writeStateAddress(context.window())), address, context);
+ }
+ throw new RuntimeException(); // cases are exhaustive.
+ }
+ }
+
+ static class MergingStateAccessorImpl<K, W extends BoundedWindow>
+ extends StateAccessorImpl<K, W> implements MergingStateAccessor<K, W> {
+ private final Collection<W> activeToBeMerged;
+
+ public MergingStateAccessorImpl(ActiveWindowSet<W> activeWindows, Coder<W> windowCoder,
+ StateInternals<K> stateInternals, StateStyle style, Collection<W> activeToBeMerged,
+ W mergeResult) {
+ super(activeWindows, windowCoder, stateInternals,
+ StateContexts.windowOnly(mergeResult), style);
+ this.activeToBeMerged = activeToBeMerged;
+ }
+
+ @Override
+ public <StateT extends State> StateT access(StateTag<? super K, StateT> address) {
+ switch (style) {
+ case DIRECT:
+ return stateInternals.state(windowNamespace(), address, context);
+ case RENAMED:
+ return stateInternals.state(
+ namespaceFor(activeWindows.mergedWriteStateAddress(
+ activeToBeMerged, context.window())),
+ address,
+ context);
+ }
+ throw new RuntimeException(); // cases are exhaustive.
+ }
+
+ @Override
+ public <StateT extends State> Map<W, StateT> accessInEachMergingWindow(
+ StateTag<? super K, StateT> address) {
+ ImmutableMap.Builder<W, StateT> builder = ImmutableMap.builder();
+ for (W mergingWindow : activeToBeMerged) {
+ StateNamespace namespace = null;
+ switch (style) {
+ case DIRECT:
+ namespace = namespaceFor(mergingWindow);
+ break;
+ case RENAMED:
+ namespace = namespaceFor(activeWindows.writeStateAddress(mergingWindow));
+ break;
+ }
+ Preconditions.checkNotNull(namespace); // cases are exhaustive.
+ builder.put(mergingWindow, stateInternals.state(namespace, address, context));
+ }
+ return builder.build();
+ }
+ }
+
+ static class PremergingStateAccessorImpl<K, W extends BoundedWindow>
+ extends StateAccessorImpl<K, W> implements MergingStateAccessor<K, W> {
+ public PremergingStateAccessorImpl(ActiveWindowSet<W> activeWindows, Coder<W> windowCoder,
+ StateInternals<K> stateInternals, W window) {
+ super(activeWindows, windowCoder, stateInternals,
+ StateContexts.windowOnly(window), StateStyle.RENAMED);
+ }
+
+ Collection<W> mergingWindows() {
+ return activeWindows.readStateAddresses(context.window());
+ }
+
+ @Override
+ public <StateT extends State> Map<W, StateT> accessInEachMergingWindow(
+ StateTag<? super K, StateT> address) {
+ ImmutableMap.Builder<W, StateT> builder = ImmutableMap.builder();
+ for (W stateAddressWindow : activeWindows.readStateAddresses(context.window())) {
+ StateT stateForWindow =
+ stateInternals.state(namespaceFor(stateAddressWindow), address, context);
+ builder.put(stateAddressWindow, stateForWindow);
+ }
+ return builder.build();
+ }
+ }
+
+ // ======================================================================
+ // Contexts
+ // ======================================================================
+
+ private class ContextImpl extends ReduceFn<K, InputT, OutputT, W>.Context {
+ private final StateAccessorImpl<K, W> state;
+ private final TimersImpl timers;
+
+ private ContextImpl(StateAccessorImpl<K, W> state) {
+ reduceFn.super();
+ this.state = state;
+ this.timers = new TimersImpl(state.namespace());
+ }
+
+ @Override
+ public K key() {
+ return key;
+ }
+
+ @Override
+ public W window() {
+ return state.window();
+ }
+
+ @Override
+ public WindowingStrategy<?, W> windowingStrategy() {
+ return windowingStrategy;
+ }
+
+ @Override
+ public StateAccessor<K> state() {
+ return state;
+ }
+
+ @Override
+ public Timers timers() {
+ return timers;
+ }
+ }
+
+ private class ProcessValueContextImpl
+ extends ReduceFn<K, InputT, OutputT, W>.ProcessValueContext {
+ private final InputT value;
+ private final Instant timestamp;
+ private final StateAccessorImpl<K, W> state;
+ private final TimersImpl timers;
+
+ private ProcessValueContextImpl(StateAccessorImpl<K, W> state,
+ InputT value, Instant timestamp) {
+ reduceFn.super();
+ this.state = state;
+ this.value = value;
+ this.timestamp = timestamp;
+ this.timers = new TimersImpl(state.namespace());
+ }
+
+ @Override
+ public K key() {
+ return key;
+ }
+
+ @Override
+ public W window() {
+ return state.window();
+ }
+
+ @Override
+ public WindowingStrategy<?, W> windowingStrategy() {
+ return windowingStrategy;
+ }
+
+ @Override
+ public StateAccessor<K> state() {
+ return state;
+ }
+
+ @Override
+ public InputT value() {
+ return value;
+ }
+
+ @Override
+ public Instant timestamp() {
+ return timestamp;
+ }
+
+ @Override
+ public Timers timers() {
+ return timers;
+ }
+ }
+
+ private class OnTriggerContextImpl extends ReduceFn<K, InputT, OutputT, W>.OnTriggerContext {
+ private final StateAccessorImpl<K, W> state;
+ private final ReadableState<PaneInfo> pane;
+ private final OnTriggerCallbacks<OutputT> callbacks;
+ private final TimersImpl timers;
+
+ private OnTriggerContextImpl(StateAccessorImpl<K, W> state, ReadableState<PaneInfo> pane,
+ OnTriggerCallbacks<OutputT> callbacks) {
+ reduceFn.super();
+ this.state = state;
+ this.pane = pane;
+ this.callbacks = callbacks;
+ this.timers = new TimersImpl(state.namespace());
+ }
+
+ @Override
+ public K key() {
+ return key;
+ }
+
+ @Override
+ public W window() {
+ return state.window();
+ }
+
+ @Override
+ public WindowingStrategy<?, W> windowingStrategy() {
+ return windowingStrategy;
+ }
+
+ @Override
+ public StateAccessor<K> state() {
+ return state;
+ }
+
+ @Override
+ public PaneInfo paneInfo() {
+ return pane.read();
+ }
+
+ @Override
+ public void output(OutputT value) {
+ callbacks.output(value);
+ }
+
+ @Override
+ public Timers timers() {
+ return timers;
+ }
+ }
+
+ private class OnMergeContextImpl extends ReduceFn<K, InputT, OutputT, W>.OnMergeContext {
+ private final MergingStateAccessorImpl<K, W> state;
+ private final TimersImpl timers;
+
+ private OnMergeContextImpl(MergingStateAccessorImpl<K, W> state) {
+ reduceFn.super();
+ this.state = state;
+ this.timers = new TimersImpl(state.namespace());
+ }
+
+ @Override
+ public K key() {
+ return key;
+ }
+
+ @Override
+ public WindowingStrategy<?, W> windowingStrategy() {
+ return windowingStrategy;
+ }
+
+ @Override
+ public MergingStateAccessor<K, W> state() {
+ return state;
+ }
+
+ @Override
+ public W window() {
+ return state.window();
+ }
+
+ @Override
+ public Timers timers() {
+ return timers;
+ }
+ }
+
+ private class OnPremergeContextImpl extends ReduceFn<K, InputT, OutputT, W>.OnMergeContext {
+ private final PremergingStateAccessorImpl<K, W> state;
+ private final TimersImpl timers;
+
+ private OnPremergeContextImpl(PremergingStateAccessorImpl<K, W> state) {
+ reduceFn.super();
+ this.state = state;
+ this.timers = new TimersImpl(state.namespace());
+ }
+
+ @Override
+ public K key() {
+ return key;
+ }
+
+ @Override
+ public WindowingStrategy<?, W> windowingStrategy() {
+ return windowingStrategy;
+ }
+
+ @Override
+ public MergingStateAccessor<K, W> state() {
+ return state;
+ }
+
+ @Override
+ public W window() {
+ return state.window();
+ }
+
+ @Override
+ public Timers timers() {
+ return timers;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
new file mode 100644
index 0000000..864e8e7
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
@@ -0,0 +1,985 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
+import org.apache.beam.sdk.util.ReduceFnContextFactory.OnTriggerCallbacks;
+import org.apache.beam.sdk.util.ReduceFnContextFactory.StateStyle;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.annotation.Nullable;
+
+/**
+ * Manages the execution of a {@link ReduceFn} after a {@link GroupByKeyOnly} has partitioned the
+ * {@link PCollection} by key.
+ *
+ * <p>The {@link #onTrigger} relies on a {@link TriggerRunner} to manage the execution of
+ * the triggering logic. The {@code ReduceFnRunner}s responsibilities are:
+ *
+ * <ul>
+ * <li>Tracking the windows that are active (have buffered data) as elements arrive and
+ * triggers are fired.
+ * <li>Holding the watermark based on the timestamps of elements in a pane and releasing it
+ * when the trigger fires.
+ * <li>Calling the appropriate callbacks on {@link ReduceFn} based on trigger execution, timer
+ * firings, etc, and providing appropriate contexts to the {@link ReduceFn} for actions
+ * such as output.
+ * <li>Scheduling garbage collection of state associated with a specific window, and making that
+ * happen when the appropriate timer fires.
+ * </ul>
+ *
+ * @param <K> The type of key being processed.
+ * @param <InputT> The type of values associated with the key.
+ * @param <OutputT> The output type that will be produced for each key.
+ * @param <W> The type of windows this operates on.
+ */
+public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
+
+ /**
+ * The {@link ReduceFnRunner} depends on most aspects of the {@link WindowingStrategy}.
+ *
+ * <ul>
+ * <li>It runs the trigger from the {@link WindowingStrategy}.</li>
+ * <li>It merges windows according to the {@link WindowingStrategy}.</li>
+ * <li>It chooses how to track active windows and clear out expired windows
+ * according to the {@link WindowingStrategy}, based on the allowed lateness and
+ * whether windows can merge.</li>
+ * <li>It decides whether to emit empty final panes according to whether the
+ * {@link WindowingStrategy} requires it.<li>
+ * <li>It uses discarding or accumulation mode according to the {@link WindowingStrategy}.</li>
+ * </ul>
+ */
+ private final WindowingStrategy<Object, W> windowingStrategy;
+
+ private final OutputWindowedValue<KV<K, OutputT>> outputter;
+
+ private final StateInternals<K> stateInternals;
+
+ private final Aggregator<Long, Long> droppedDueToClosedWindow;
+
+ private final K key;
+
+ /**
+ * Track which windows are still active and the 'state address' windows which hold their state.
+ *
+ * <ul>
+ * <li>State: Global map for all active windows for this computation and key.
+ * <li>Lifetime: Cleared when no active windows need to be tracked. A window lives within
+ * the active window set until its trigger is closed or the window is garbage collected.
+ * </ul>
+ */
+ private final ActiveWindowSet<W> activeWindows;
+
+ /**
+ * Always a {@link SystemReduceFn}.
+ *
+ * <ul>
+ * <li>State: A bag of accumulated values, or the intermediate result of a combiner.
+ * <li>State style: RENAMED
+ * <li>Merging: Concatenate or otherwise combine the state from each merged window.
+ * <li>Lifetime: Cleared when a pane fires if DISCARDING_FIRED_PANES. Otherwise cleared
+ * when trigger is finished or when the window is garbage collected.
+ * </ul>
+ */
+ private final ReduceFn<K, InputT, OutputT, W> reduceFn;
+
+ /**
+ * Manage the setting and firing of timer events.
+ *
+ * <ul>
+ * <li>Merging: End-of-window and garbage collection timers are cancelled when windows are
+ * merged away. Timers created by triggers are never garbage collected and are left to
+ * fire and be ignored.
+ * <li>Lifetime: Timers automatically disappear after they fire.
+ * </ul>
+ */
+ private final TimerInternals timerInternals;
+
+ /**
+ * Manage the execution and state for triggers.
+ *
+ * <ul>
+ * <li>State: Tracks which sub-triggers have finished, and any additional state needed to
+ * determine when the trigger should fire.
+ * <li>State style: DIRECT
+ * <li>Merging: Finished bits are explicitly managed. Other state is eagerly merged as
+ * needed.
+ * <li>Lifetime: Most trigger state is cleared when the final pane is emitted. However
+ * the finished bits are left behind and must be cleared when the window is
+ * garbage collected.
+ * </ul>
+ */
+ private final TriggerRunner<W> triggerRunner;
+
+ /**
+ * Store the output watermark holds for each window.
+ *
+ * <ul>
+ * <li>State: Bag of hold timestamps.
+ * <li>State style: RENAMED
+ * <li>Merging: Depending on {@link OutputTimeFn}, may need to be recalculated on merging.
+ * When a pane fires it may be necessary to add (back) an end-of-window or garbage collection
+ * hold.
+ * <li>Lifetime: Cleared when a pane fires or when the window is garbage collected.
+ * </ul>
+ */
+ private final WatermarkHold<W> watermarkHold;
+
+ private final ReduceFnContextFactory<K, InputT, OutputT, W> contextFactory;
+
+ /**
+ * Store the previously emitted pane (if any) for each window.
+ *
+ * <ul>
+ * <li>State: The previous {@link PaneInfo} passed to the user's {@link DoFn#processElement},
+ * if any.
+ * <li>Style style: DIRECT
+ * <li>Merging: Always keyed by actual window, so does not depend on {@link #activeWindows}.
+ * Cleared when window is merged away.
+ * <li>Lifetime: Cleared when trigger is closed or window is garbage collected.
+ * </ul>
+ */
+ private final PaneInfoTracker paneInfoTracker;
+
+ /**
+ * Store whether we've seen any elements for a window since the last pane was emitted.
+ *
+ * <ul>
+ * <li>State: Unless DISCARDING_FIRED_PANES, a count of number of elements added so far.
+ * <li>State style: RENAMED.
+ * <li>Merging: Counts are summed when windows are merged.
+ * <li>Lifetime: Cleared when pane fires or window is garbage collected.
+ * </ul>
+ */
+ private final NonEmptyPanes<K, W> nonEmptyPanes;
+
+ public ReduceFnRunner(
+ K key,
+ WindowingStrategy<?, W> windowingStrategy,
+ StateInternals<K> stateInternals,
+ TimerInternals timerInternals,
+ WindowingInternals<?, KV<K, OutputT>> windowingInternals,
+ Aggregator<Long, Long> droppedDueToClosedWindow,
+ ReduceFn<K, InputT, OutputT, W> reduceFn,
+ PipelineOptions options) {
+ this.key = key;
+ this.timerInternals = timerInternals;
+ this.paneInfoTracker = new PaneInfoTracker(timerInternals);
+ this.stateInternals = stateInternals;
+ this.outputter = new OutputViaWindowingInternals<>(windowingInternals);
+ this.droppedDueToClosedWindow = droppedDueToClosedWindow;
+ this.reduceFn = reduceFn;
+
+ @SuppressWarnings("unchecked")
+ WindowingStrategy<Object, W> objectWindowingStrategy =
+ (WindowingStrategy<Object, W>) windowingStrategy;
+ this.windowingStrategy = objectWindowingStrategy;
+
+ this.nonEmptyPanes = NonEmptyPanes.create(this.windowingStrategy, this.reduceFn);
+
+ // Note this may incur I/O to load persisted window set data.
+ this.activeWindows = createActiveWindowSet();
+
+ this.contextFactory =
+ new ReduceFnContextFactory<K, InputT, OutputT, W>(key, reduceFn, this.windowingStrategy,
+ stateInternals, this.activeWindows, timerInternals, windowingInternals, options);
+
+ this.watermarkHold = new WatermarkHold<>(timerInternals, windowingStrategy);
+ this.triggerRunner =
+ new TriggerRunner<>(
+ windowingStrategy.getTrigger(),
+ new TriggerContextFactory<>(windowingStrategy, stateInternals, activeWindows));
+ }
+
+ private ActiveWindowSet<W> createActiveWindowSet() {
+ return windowingStrategy.getWindowFn().isNonMerging()
+ ? new NonMergingActiveWindowSet<W>()
+ : new MergingActiveWindowSet<W>(windowingStrategy.getWindowFn(), stateInternals);
+ }
+
+ @VisibleForTesting
+ boolean isFinished(W window) {
+ return triggerRunner.isClosed(contextFactory.base(window, StateStyle.DIRECT).state());
+ }
+
+ @VisibleForTesting
+ boolean hasNoActiveWindows() {
+ return activeWindows.getActiveAndNewWindows().isEmpty();
+ }
+
+ /**
+ * Incorporate {@code values} into the underlying reduce function, and manage holds, timers,
+ * triggers, and window merging.
+ *
+ * <p>The general strategy is:
+ * <ol>
+ * <li>Use {@link WindowedValue#getWindows} (itself determined using
+ * {@link WindowFn#assignWindows}) to determine which windows each element belongs to. Some
+ * of those windows will already have state associated with them. The rest are considered
+ * NEW.
+ * <li>Use {@link WindowFn#mergeWindows} to attempt to merge currently ACTIVE and NEW windows.
+ * Each NEW window will become either ACTIVE or be discardedL.
+ * (See {@link ActiveWindowSet} for definitions of these terms.)
+ * <li>If at all possible, eagerly substitute NEW windows with their ACTIVE state address
+ * windows before any state is associated with the NEW window. In the common case that
+ * windows for new elements are merged into existing ACTIVE windows then no additional
+ * storage or merging overhead will be incurred.
+ * <li>Otherwise, keep track of the state address windows for ACTIVE windows so that their
+ * states can be merged on-demand when a pane fires.
+ * <li>Process the element for each of the windows it's windows have been merged into according
+ * to {@link ActiveWindowSet}. Processing may require running triggers, setting timers,
+ * setting holds, and invoking {@link ReduceFn#onTrigger}.
+ * </ol>
+ */
+ public void processElements(Iterable<WindowedValue<InputT>> values) throws Exception {
+ // If an incoming element introduces a new window, attempt to merge it into an existing
+ // window eagerly.
+ Map<W, W> windowToMergeResult = collectAndMergeWindows(values);
+
+ Set<W> windowsToConsider = new HashSet<>();
+
+ // Process each element, using the updated activeWindows determined by collectAndMergeWindows.
+ for (WindowedValue<InputT> value : values) {
+ windowsToConsider.addAll(processElement(windowToMergeResult, value));
+ }
+
+ // Trigger output from any window for which the trigger is ready
+ for (W mergedWindow : windowsToConsider) {
+ ReduceFn<K, InputT, OutputT, W>.Context directContext =
+ contextFactory.base(mergedWindow, StateStyle.DIRECT);
+ ReduceFn<K, InputT, OutputT, W>.Context renamedContext =
+ contextFactory.base(mergedWindow, StateStyle.RENAMED);
+ triggerRunner.prefetchShouldFire(mergedWindow, directContext.state());
+ emitIfAppropriate(directContext, renamedContext);
+ }
+
+ // We're all done with merging and emitting elements so can compress the activeWindow state.
+ // Any windows which are still NEW must have come in on a new element which was then discarded
+ // due to the window's trigger being closed. We can thus delete them.
+ activeWindows.cleanupTemporaryWindows();
+ }
+
+ public void persist() {
+ activeWindows.persist();
+ }
+
+ /**
+ * Extract the windows associated with the values, and invoke merge. Return a map
+ * from windows to the merge result window. If a window is not in the domain of
+ * the result map then it did not get merged into a different window.
+ */
+ private Map<W, W> collectAndMergeWindows(Iterable<WindowedValue<InputT>> values)
+ throws Exception {
+ // No-op if no merging can take place
+ if (windowingStrategy.getWindowFn().isNonMerging()) {
+ return ImmutableMap.of();
+ }
+
+ // Collect the windows from all elements (except those which are too late) and
+ // make sure they are already in the active window set or are added as NEW windows.
+ for (WindowedValue<?> value : values) {
+ for (BoundedWindow untypedWindow : value.getWindows()) {
+ @SuppressWarnings("unchecked")
+ W window = (W) untypedWindow;
+
+ // For backwards compat with pre 1.4 only.
+ // We may still have ACTIVE windows with multiple state addresses, representing
+ // a window who's state has not yet been eagerly merged.
+ // We'll go ahead and merge that state now so that we don't have to worry about
+ // this legacy case anywhere else.
+ if (activeWindows.isActive(window)) {
+ Set<W> stateAddressWindows = activeWindows.readStateAddresses(window);
+ if (stateAddressWindows.size() > 1) {
+ // This is a legacy window who's state has not been eagerly merged.
+ // Do that now.
+ ReduceFn<K, InputT, OutputT, W>.OnMergeContext premergeContext =
+ contextFactory.forPremerge(window);
+ reduceFn.onMerge(premergeContext);
+ watermarkHold.onMerge(premergeContext);
+ activeWindows.merged(window);
+ }
+ }
+
+ // Add this window as NEW if it is not currently ACTIVE.
+ // If we had already seen this window and closed its trigger, then the
+ // window will not be currently ACTIVE. It will then be added as NEW here,
+ // and fall into the merging logic as usual.
+ activeWindows.ensureWindowExists(window);
+ }
+ }
+
+ // Merge all of the active windows and retain a mapping from source windows to result windows.
+ Map<W, W> windowToMergeResult = new HashMap<>();
+ activeWindows.merge(new OnMergeCallback(windowToMergeResult));
+ return windowToMergeResult;
+ }
+
+ private class OnMergeCallback implements ActiveWindowSet.MergeCallback<W> {
+ private final Map<W, W> windowToMergeResult;
+
+ OnMergeCallback(Map<W, W> windowToMergeResult) {
+ this.windowToMergeResult = windowToMergeResult;
+ }
+
+ /**
+ * Return the subset of {@code windows} which are currently ACTIVE. We only need to worry
+ * about merging state from ACTIVE windows. NEW windows by definition have no existing state.
+ */
+ private List<W> activeWindows(Iterable<W> windows) {
+ List<W> active = new ArrayList<>();
+ for (W window : windows) {
+ if (activeWindows.isActive(window)) {
+ active.add(window);
+ }
+ }
+ return active;
+ }
+
+ /**
+ * Called from the active window set to indicate {@code toBeMerged} (of which only
+ * {@code activeToBeMerged} are ACTIVE and thus have state associated with them) will later
+ * be merged into {@code mergeResult}.
+ */
+ @Override
+ public void prefetchOnMerge(
+ Collection<W> toBeMerged, W mergeResult) throws Exception {
+ List<W> activeToBeMerged = activeWindows(toBeMerged);
+ ReduceFn<K, InputT, OutputT, W>.OnMergeContext directMergeContext =
+ contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.DIRECT);
+ ReduceFn<K, InputT, OutputT, W>.OnMergeContext renamedMergeContext =
+ contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.RENAMED);
+
+ // Prefetch various state.
+ triggerRunner.prefetchForMerge(mergeResult, activeToBeMerged, directMergeContext.state());
+ reduceFn.prefetchOnMerge(renamedMergeContext.state());
+ watermarkHold.prefetchOnMerge(renamedMergeContext.state());
+ nonEmptyPanes.prefetchOnMerge(renamedMergeContext.state());
+ }
+
+ /**
+ * Called from the active window set to indicate {@code toBeMerged} (of which only
+ * {@code activeToBeMerged} are ACTIVE and thus have state associated with them) are about
+ * to be merged into {@code mergeResult}.
+ */
+ @Override
+ public void onMerge(Collection<W> toBeMerged, W mergeResult) throws Exception {
+ // Remember we have merged these windows.
+ for (W window : toBeMerged) {
+ windowToMergeResult.put(window, mergeResult);
+ }
+
+ // At this point activeWindows has NOT incorporated the results of the merge.
+ List<W> activeToBeMerged = activeWindows(toBeMerged);
+ ReduceFn<K, InputT, OutputT, W>.OnMergeContext directMergeContext =
+ contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.DIRECT);
+ ReduceFn<K, InputT, OutputT, W>.OnMergeContext renamedMergeContext =
+ contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.RENAMED);
+
+ // Run the reduceFn to perform any needed merging.
+ reduceFn.onMerge(renamedMergeContext);
+
+ // Merge the watermark holds.
+ watermarkHold.onMerge(renamedMergeContext);
+
+ // Merge non-empty pane state.
+ nonEmptyPanes.onMerge(renamedMergeContext.state());
+
+ // Have the trigger merge state as needed.
+ triggerRunner.onMerge(
+ directMergeContext.window(), directMergeContext.timers(), directMergeContext.state());
+
+ for (W active : activeToBeMerged) {
+ if (active.equals(mergeResult)) {
+ // Not merged away.
+ continue;
+ }
+ // Cleanup flavor A: Currently ACTIVE window is about to be merged away.
+ // Clear any state not already cleared by the onMerge calls above.
+ WindowTracing.debug("ReduceFnRunner.onMerge: Merging {} into {}", active, mergeResult);
+ ReduceFn<K, InputT, OutputT, W>.Context directClearContext =
+ contextFactory.base(active, StateStyle.DIRECT);
+ // No need for the end-of-window or garbage collection timers.
+ // We will establish a new end-of-window or garbage collection timer for the mergeResult
+ // window in processElement below. There must be at least one element for the mergeResult
+ // window since a new element with a new window must have triggered this onMerge.
+ cancelEndOfWindowAndGarbageCollectionTimers(directClearContext);
+ // We no longer care about any previous panes of merged away windows. The
+ // merge result window gets to start fresh if it is new.
+ paneInfoTracker.clear(directClearContext.state());
+ }
+ }
+ }
+
+ /**
+ * Process an element.
+ *
+ * @param value the value being processed
+ * @return the set of windows in which the element was actually processed
+ */
+ private Collection<W> processElement(Map<W, W> windowToMergeResult, WindowedValue<InputT> value)
+ throws Exception {
+ // Redirect element windows to the ACTIVE windows they have been merged into.
+ // The compressed representation (value, {window1, window2, ...}) actually represents
+ // distinct elements (value, window1), (value, window2), ...
+ // so if window1 and window2 merge, the resulting window will contain both copies
+ // of the value.
+ Collection<W> windows = new ArrayList<>();
+ for (BoundedWindow untypedWindow : value.getWindows()) {
+ @SuppressWarnings("unchecked")
+ W window = (W) untypedWindow;
+ W mergeResult = windowToMergeResult.get(window);
+ if (mergeResult == null) {
+ mergeResult = window;
+ }
+ windows.add(mergeResult);
+ }
+
+ // Prefetch in each of the windows if we're going to need to process triggers
+ for (W window : windows) {
+ ReduceFn<K, InputT, OutputT, W>.ProcessValueContext directContext = contextFactory.forValue(
+ window, value.getValue(), value.getTimestamp(), StateStyle.DIRECT);
+ triggerRunner.prefetchForValue(window, directContext.state());
+ }
+
+ // Process the element for each (mergeResultWindow, not closed) window it belongs to.
+ List<W> triggerableWindows = new ArrayList<>(windows.size());
+ for (W window : windows) {
+ ReduceFn<K, InputT, OutputT, W>.ProcessValueContext directContext = contextFactory.forValue(
+ window, value.getValue(), value.getTimestamp(), StateStyle.DIRECT);
+ if (triggerRunner.isClosed(directContext.state())) {
+ // This window has already been closed.
+ droppedDueToClosedWindow.addValue(1L);
+ WindowTracing.debug(
+ "ReduceFnRunner.processElement: Dropping element at {} for key:{}; window:{} "
+ + "since window is no longer active at inputWatermark:{}; outputWatermark:{}",
+ value.getTimestamp(), key, window, timerInternals.currentInputWatermarkTime(),
+ timerInternals.currentOutputWatermarkTime());
+ continue;
+ }
+
+ triggerableWindows.add(window);
+ activeWindows.ensureWindowIsActive(window);
+ ReduceFn<K, InputT, OutputT, W>.ProcessValueContext renamedContext = contextFactory.forValue(
+ window, value.getValue(), value.getTimestamp(), StateStyle.RENAMED);
+
+ nonEmptyPanes.recordContent(renamedContext.state());
+
+ // Make sure we've scheduled the end-of-window or garbage collection timer for this window.
+ Instant timer = scheduleEndOfWindowOrGarbageCollectionTimer(directContext);
+
+ // Hold back progress of the output watermark until we have processed the pane this
+ // element will be included within. If the element is too late for that, place a hold at
+ // the end-of-window or garbage collection time to allow empty panes to contribute elements
+ // which won't be dropped due to lateness by a following computation (assuming the following
+ // computation uses the same allowed lateness value...)
+ @Nullable Instant hold = watermarkHold.addHolds(renamedContext);
+
+ if (hold != null) {
+ // Assert that holds have a proximate timer.
+ boolean holdInWindow = !hold.isAfter(window.maxTimestamp());
+ boolean timerInWindow = !timer.isAfter(window.maxTimestamp());
+ Preconditions.checkState(
+ holdInWindow == timerInWindow,
+ "set a hold at %s, a timer at %s, which disagree as to whether they are in window %s",
+ hold,
+ timer,
+ directContext.window());
+ }
+
+ // Execute the reduceFn, which will buffer the value as appropriate
+ reduceFn.processValue(renamedContext);
+
+ // Run the trigger to update its state
+ triggerRunner.processValue(
+ directContext.window(),
+ directContext.timestamp(),
+ directContext.timers(),
+ directContext.state());
+
+ // At this point, if triggerRunner.shouldFire before the processValue then
+ // triggerRunner.shouldFire after the processValue. In other words adding values
+ // cannot take a trigger state from firing to non-firing.
+ // (We don't actually assert this since it is too slow.)
+ }
+
+ return triggerableWindows;
+ }
+
+ /**
+ * Called when an end-of-window, garbage collection, or trigger-specific timer fires.
+ */
+ public void onTimer(TimerData timer) throws Exception {
+ // Which window is the timer for?
+ Preconditions.checkArgument(timer.getNamespace() instanceof WindowNamespace,
+ "Expected timer to be in WindowNamespace, but was in %s", timer.getNamespace());
+ @SuppressWarnings("unchecked")
+ WindowNamespace<W> windowNamespace = (WindowNamespace<W>) timer.getNamespace();
+ W window = windowNamespace.getWindow();
+ ReduceFn<K, InputT, OutputT, W>.Context directContext =
+ contextFactory.base(window, StateStyle.DIRECT);
+ ReduceFn<K, InputT, OutputT, W>.Context renamedContext =
+ contextFactory.base(window, StateStyle.RENAMED);
+
+ // Has this window had its trigger finish?
+ // - The trigger may implement isClosed as constant false.
+ // - If the window function does not support windowing then all windows will be considered
+ // active.
+ // So we must take conjunction of activeWindows and triggerRunner state.
+ boolean windowIsActiveAndOpen =
+ activeWindows.isActive(window) && !triggerRunner.isClosed(directContext.state());
+
+ if (!windowIsActiveAndOpen) {
+ WindowTracing.debug(
+ "ReduceFnRunner.onTimer: Note that timer {} is for non-ACTIVE window {}", timer, window);
+ }
+
+ // If this is an end-of-window timer then we may need to set a garbage collection timer
+ // if allowed lateness is non-zero.
+ boolean isEndOfWindow = TimeDomain.EVENT_TIME == timer.getDomain()
+ && timer.getTimestamp().equals(window.maxTimestamp());
+
+ // If this is a garbage collection timer then we should trigger and garbage collect the window.
+ // We'll consider any timer at or after the end-of-window time to be a signal to garbage
+ // collect.
+ Instant cleanupTime = garbageCollectionTime(window);
+ boolean isGarbageCollection = TimeDomain.EVENT_TIME == timer.getDomain()
+ && !timer.getTimestamp().isBefore(cleanupTime);
+
+ if (isGarbageCollection) {
+ WindowTracing.debug(
+ "ReduceFnRunner.onTimer: Cleaning up for key:{}; window:{} at {} with "
+ + "inputWatermark:{}; outputWatermark:{}",
+ key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(),
+ timerInternals.currentOutputWatermarkTime());
+
+ if (windowIsActiveAndOpen) {
+ // We need to call onTrigger to emit the final pane if required.
+ // The final pane *may* be ON_TIME if no prior ON_TIME pane has been emitted,
+ // and the watermark has passed the end of the window.
+ @Nullable Instant newHold =
+ onTrigger(directContext, renamedContext, true/* isFinished */, isEndOfWindow);
+ Preconditions.checkState(newHold == null,
+ "Hold placed at %s despite isFinished being true.", newHold);
+ }
+
+ // Cleanup flavor B: Clear all the remaining state for this window since we'll never
+ // see elements for it again.
+ clearAllState(directContext, renamedContext, windowIsActiveAndOpen);
+ } else {
+ WindowTracing.debug(
+ "ReduceFnRunner.onTimer: Triggering for key:{}; window:{} at {} with "
+ + "inputWatermark:{}; outputWatermark:{}",
+ key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(),
+ timerInternals.currentOutputWatermarkTime());
+ if (windowIsActiveAndOpen) {
+ emitIfAppropriate(directContext, renamedContext);
+ }
+
+ if (isEndOfWindow) {
+ // If the window strategy trigger includes a watermark trigger then at this point
+ // there should be no data holds, either because we'd already cleared them on an
+ // earlier onTrigger, or because we just cleared them on the above emitIfAppropriate.
+ // We could assert this but it is very expensive.
+
+ // Since we are processing an on-time firing we should schedule the garbage collection
+ // timer. (If getAllowedLateness is zero then the timer event will be considered a
+ // cleanup event and handled by the above).
+ // Note we must do this even if the trigger is finished so that we are sure to cleanup
+ // any final trigger finished bits.
+ Preconditions.checkState(
+ windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO),
+ "Unexpected zero getAllowedLateness");
+ WindowTracing.debug(
+ "ReduceFnRunner.onTimer: Scheduling cleanup timer for key:{}; window:{} at {} with "
+ + "inputWatermark:{}; outputWatermark:{}",
+ key, directContext.window(), cleanupTime, timerInternals.currentInputWatermarkTime(),
+ timerInternals.currentOutputWatermarkTime());
+ Preconditions.checkState(!cleanupTime.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+ "Cleanup time %s is beyond end-of-time", cleanupTime);
+ directContext.timers().setTimer(cleanupTime, TimeDomain.EVENT_TIME);
+ }
+ }
+ }
+
+ /**
+ * Clear all the state associated with {@code context}'s window.
+ * Should only be invoked if we know all future elements for this window will be considered
+ * beyond allowed lateness.
+ * This is a superset of the clearing done by {@link #emitIfAppropriate} below since:
+ * <ol>
+ * <li>We can clear the trigger finished bits since we'll never need to ask if the trigger is
+ * closed again.
+ * <li>We can clear any remaining garbage collection hold.
+ * </ol>
+ */
+ private void clearAllState(
+ ReduceFn<K, InputT, OutputT, W>.Context directContext,
+ ReduceFn<K, InputT, OutputT, W>.Context renamedContext,
+ boolean windowIsActiveAndOpen)
+ throws Exception {
+ if (windowIsActiveAndOpen) {
+ // Since both the window is in the active window set AND the trigger was not yet closed,
+ // it is possible we still have state.
+ reduceFn.clearState(renamedContext);
+ watermarkHold.clearHolds(renamedContext);
+ nonEmptyPanes.clearPane(renamedContext.state());
+ // These calls work irrespective of whether the window is active or not, but
+ // are unnecessary if the window is not active.
+ triggerRunner.clearState(
+ directContext.window(), directContext.timers(), directContext.state());
+ paneInfoTracker.clear(directContext.state());
+ } else {
+ // If !windowIsActiveAndOpen then !activeWindows.isActive (1) or triggerRunner.isClosed (2).
+ // For (1), if !activeWindows.isActive then the window must be merging and has been
+ // explicitly removed by emitIfAppropriate. But in that case the trigger must have fired
+ // and been closed, so this case reduces to (2).
+ // For (2), if triggerRunner.isClosed then the trigger was fired and entered the
+ // closed state. In that case emitIfAppropriate will have cleared all state in
+ // reduceFn, triggerRunner (except for finished bits), paneInfoTracker and activeWindows.
+ // We also know nonEmptyPanes must have been unconditionally cleared by the trigger.
+ // Since the trigger fired the existing watermark holds must have been cleared, and since
+ // the trigger closed no new end of window or garbage collection hold will have been
+ // placed by WatermarkHold.extractAndRelease.
+ // Thus all the state clearing above is unnecessary.
+ //
+ // But(!) for backwards compatibility we must allow a pipeline to be updated from
+ // an sdk version <= 1.3. In that case it is possible we have an end-of-window or
+ // garbage collection hold keyed by the current window (reached via directContext) rather
+ // than the state address window (reached via renamedContext).
+ // However this can only happen if:
+ // - We have merging windows.
+ // - We are DISCARDING_FIRED_PANES.
+ // - A pane has fired.
+ // - But the trigger is not (yet) closed.
+ if (windowingStrategy.getMode() == AccumulationMode.DISCARDING_FIRED_PANES
+ && !windowingStrategy.getWindowFn().isNonMerging()) {
+ watermarkHold.clearHolds(directContext);
+ }
+ }
+
+ // Don't need to track address state windows anymore.
+ activeWindows.remove(directContext.window());
+ // We'll never need to test for the trigger being closed again.
+ triggerRunner.clearFinished(directContext.state());
+ }
+
+ /** Should the reduce function state be cleared? */
+ private boolean shouldDiscardAfterFiring(boolean isFinished) {
+ if (isFinished) {
+ // This is the last firing for trigger.
+ return true;
+ }
+ if (windowingStrategy.getMode() == AccumulationMode.DISCARDING_FIRED_PANES) {
+ // Nothing should be accumulated between panes.
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * Possibly emit a pane if a trigger is ready to fire or timers require it, and cleanup state.
+ */
+ private void emitIfAppropriate(ReduceFn<K, InputT, OutputT, W>.Context directContext,
+ ReduceFn<K, InputT, OutputT, W>.Context renamedContext)
+ throws Exception {
+ if (!triggerRunner.shouldFire(
+ directContext.window(), directContext.timers(), directContext.state())) {
+ // Ignore unless trigger is ready to fire
+ return;
+ }
+
+ // Inform the trigger of the transition to see if it is finished
+ triggerRunner.onFire(directContext.window(), directContext.timers(), directContext.state());
+ boolean isFinished = triggerRunner.isClosed(directContext.state());
+
+ // Will be able to clear all element state after triggering?
+ boolean shouldDiscard = shouldDiscardAfterFiring(isFinished);
+
+ // Run onTrigger to produce the actual pane contents.
+ // As a side effect it will clear all element holds, but not necessarily any
+ // end-of-window or garbage collection holds.
+ onTrigger(directContext, renamedContext, isFinished, false /*isEndOfWindow*/);
+
+ // Now that we've triggered, the pane is empty.
+ nonEmptyPanes.clearPane(renamedContext.state());
+
+ // Cleanup buffered data if appropriate
+ if (shouldDiscard) {
+ // Cleanup flavor C: The user does not want any buffered data to persist between panes.
+ reduceFn.clearState(renamedContext);
+ }
+
+ if (isFinished) {
+ // Cleanup flavor D: If trigger is closed we will ignore all new incoming elements.
+ // Clear state not otherwise cleared by onTrigger and clearPane above.
+ // Remember the trigger is, indeed, closed until the window is garbage collected.
+ triggerRunner.clearState(
+ directContext.window(), directContext.timers(), directContext.state());
+ paneInfoTracker.clear(directContext.state());
+ activeWindows.remove(directContext.window());
+ }
+ }
+
+ /**
+ * Do we need to emit a pane?
+ */
+ private boolean needToEmit(boolean isEmpty, boolean isFinished, PaneInfo.Timing timing) {
+ if (!isEmpty) {
+ // The pane has elements.
+ return true;
+ }
+ if (timing == Timing.ON_TIME) {
+ // This is the unique ON_TIME pane.
+ return true;
+ }
+ if (isFinished && windowingStrategy.getClosingBehavior() == ClosingBehavior.FIRE_ALWAYS) {
+ // This is known to be the final pane, and the user has requested it even when empty.
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * Run the {@link ReduceFn#onTrigger} method and produce any necessary output.
+ *
+ * @return output watermark hold added, or {@literal null} if none.
+ */
+ @Nullable
+ private Instant onTrigger(
+ final ReduceFn<K, InputT, OutputT, W>.Context directContext,
+ ReduceFn<K, InputT, OutputT, W>.Context renamedContext,
+ boolean isFinished, boolean isEndOfWindow)
+ throws Exception {
+ Instant inputWM = timerInternals.currentInputWatermarkTime();
+
+ // Prefetch necessary states
+ ReadableState<WatermarkHold.OldAndNewHolds> outputTimestampFuture =
+ watermarkHold.extractAndRelease(renamedContext, isFinished).readLater();
+ ReadableState<PaneInfo> paneFuture =
+ paneInfoTracker.getNextPaneInfo(directContext, isFinished).readLater();
+ ReadableState<Boolean> isEmptyFuture =
+ nonEmptyPanes.isEmpty(renamedContext.state()).readLater();
+
+ reduceFn.prefetchOnTrigger(directContext.state());
+ triggerRunner.prefetchOnFire(directContext.window(), directContext.state());
+
+ // Calculate the pane info.
+ final PaneInfo pane = paneFuture.read();
+ // Extract the window hold, and as a side effect clear it.
+
+ WatermarkHold.OldAndNewHolds pair = outputTimestampFuture.read();
+ final Instant outputTimestamp = pair.oldHold;
+ @Nullable Instant newHold = pair.newHold;
+
+ if (newHold != null) {
+ // We can't be finished yet.
+ Preconditions.checkState(
+ !isFinished, "new hold at %s but finished %s", newHold, directContext.window());
+ // The hold cannot be behind the input watermark.
+ Preconditions.checkState(
+ !newHold.isBefore(inputWM), "new hold %s is before input watermark %s", newHold, inputWM);
+ if (newHold.isAfter(directContext.window().maxTimestamp())) {
+ // The hold must be for garbage collection, which can't have happened yet.
+ Preconditions.checkState(
+ newHold.isEqual(garbageCollectionTime(directContext.window())),
+ "new hold %s should be at garbage collection for window %s plus %s",
+ newHold,
+ directContext.window(),
+ windowingStrategy.getAllowedLateness());
+ } else {
+ // The hold must be for the end-of-window, which can't have happened yet.
+ Preconditions.checkState(
+ newHold.isEqual(directContext.window().maxTimestamp()),
+ "new hold %s should be at end of window %s",
+ newHold,
+ directContext.window());
+ Preconditions.checkState(
+ !isEndOfWindow,
+ "new hold at %s for %s but this is the watermark trigger",
+ newHold,
+ directContext.window());
+ }
+ }
+
+ // Only emit a pane if it has data or empty panes are observable.
+ if (needToEmit(isEmptyFuture.read(), isFinished, pane.getTiming())) {
+ // Run reduceFn.onTrigger method.
+ final List<W> windows = Collections.singletonList(directContext.window());
+ ReduceFn<K, InputT, OutputT, W>.OnTriggerContext renamedTriggerContext =
+ contextFactory.forTrigger(directContext.window(), paneFuture, StateStyle.RENAMED,
+ new OnTriggerCallbacks<OutputT>() {
+ @Override
+ public void output(OutputT toOutput) {
+ // We're going to output panes, so commit the (now used) PaneInfo.
+ // TODO: This is unnecessary if the trigger isFinished since the saved
+ // state will be immediately deleted.
+ paneInfoTracker.storeCurrentPaneInfo(directContext, pane);
+
+ // Output the actual value.
+ outputter.outputWindowedValue(
+ KV.of(key, toOutput), outputTimestamp, windows, pane);
+ }
+ });
+
+ reduceFn.onTrigger(renamedTriggerContext);
+ }
+
+ return newHold;
+ }
+
+ /**
+ * Make sure we'll eventually have a timer fire which will tell us to garbage collect
+ * the window state. For efficiency we may need to do this in two steps rather
+ * than one. Return the time at which the timer will fire.
+ *
+ * <ul>
+ * <li>If allowedLateness is zero then we'll garbage collect at the end of the window.
+ * For simplicity we'll set our own timer for this situation even though an
+ * {@link AfterWatermark} trigger may have also set an end-of-window timer.
+ * ({@code setTimer} is idempotent.)
+ * <li>If allowedLateness is non-zero then we could just always set a timer for the garbage
+ * collection time. However if the windows are large (eg hourly) and the allowedLateness is small
+ * (eg seconds) then we'll end up with nearly twice the number of timers in-flight. So we
+ * instead set an end-of-window timer and then roll that forward to a garbage collection timer
+ * when it fires. We use the input watermark to distinguish those cases.
+ * </ul>
+ */
+ private Instant scheduleEndOfWindowOrGarbageCollectionTimer(
+ ReduceFn<?, ?, ?, W>.Context directContext) {
+ Instant inputWM = timerInternals.currentInputWatermarkTime();
+ Instant endOfWindow = directContext.window().maxTimestamp();
+ String which;
+ Instant timer;
+ if (endOfWindow.isBefore(inputWM)) {
+ timer = garbageCollectionTime(directContext.window());
+ which = "garbage collection";
+ } else {
+ timer = endOfWindow;
+ which = "end-of-window";
+ }
+ WindowTracing.trace(
+ "ReduceFnRunner.scheduleEndOfWindowOrGarbageCollectionTimer: Scheduling {} timer at {} for "
+ + "key:{}; window:{} where inputWatermark:{}; outputWatermark:{}",
+ which,
+ timer,
+ key,
+ directContext.window(),
+ inputWM,
+ timerInternals.currentOutputWatermarkTime());
+ Preconditions.checkState(!timer.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+ "Timer %s is beyond end-of-time", timer);
+ directContext.timers().setTimer(timer, TimeDomain.EVENT_TIME);
+ return timer;
+ }
+
+ private void cancelEndOfWindowAndGarbageCollectionTimers(
+ ReduceFn<?, ?, ?, W>.Context directContext) {
+ WindowTracing.debug(
+ "ReduceFnRunner.cancelEndOfWindowAndGarbageCollectionTimers: Deleting timers for "
+ + "key:{}; window:{} where inputWatermark:{}; outputWatermark:{}",
+ key, directContext.window(), timerInternals.currentInputWatermarkTime(),
+ timerInternals.currentOutputWatermarkTime());
+ Instant eow = directContext.window().maxTimestamp();
+ directContext.timers().deleteTimer(eow, TimeDomain.EVENT_TIME);
+ Instant gc = garbageCollectionTime(directContext.window());
+ if (gc.isAfter(eow)) {
+ directContext.timers().deleteTimer(eow, TimeDomain.EVENT_TIME);
+ }
+ }
+
+ /**
+ * 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.
+ */
+ private Instant garbageCollectionTime(W window) {
+
+ // 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(windowingStrategy.getAllowedLateness())
+ .isBefore(window.maxTimestamp())) {
+ return GlobalWindow.INSTANCE.maxTimestamp();
+ } else {
+ return window.maxTimestamp().plus(windowingStrategy.getAllowedLateness());
+ }
+ }
+
+ /**
+ * An object that can output a value with all of its windowing information. This is a deliberately
+ * restricted subinterface of {@link WindowingInternals} to express how it is used here.
+ */
+ private interface OutputWindowedValue<OutputT> {
+ void outputWindowedValue(OutputT output, Instant timestamp,
+ Collection<? extends BoundedWindow> windows, PaneInfo pane);
+ }
+
+ private static class OutputViaWindowingInternals<OutputT>
+ implements OutputWindowedValue<OutputT> {
+
+ private final WindowingInternals<?, OutputT> windowingInternals;
+
+ public OutputViaWindowingInternals(WindowingInternals<?, OutputT> windowingInternals) {
+ this.windowingInternals = windowingInternals;
+ }
+
+ @Override
+ public void outputWindowedValue(
+ OutputT output,
+ Instant timestamp,
+ Collection<? extends BoundedWindow> windows,
+ PaneInfo pane) {
+ windowingInternals.outputWindowedValue(output, timestamp, windows, pane);
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java
new file mode 100644
index 0000000..78377c8
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.util.ExecutionContext.StepContext;
+import org.apache.beam.sdk.util.common.CounterSet.AddCounterMutator;
+import org.apache.beam.sdk.values.TupleTag;
+
+import java.util.List;
+
+/**
+ * Runs a {@link DoFn} by constructing the appropriate contexts and passing them in.
+ *
+ * @param <InputT> the type of the DoFn's (main) input elements
+ * @param <OutputT> the type of the DoFn's (main) output elements
+ */
+public class SimpleDoFnRunner<InputT, OutputT> extends DoFnRunnerBase<InputT, OutputT>{
+
+ protected SimpleDoFnRunner(PipelineOptions options, DoFn<InputT, OutputT> fn,
+ SideInputReader sideInputReader,
+ OutputManager outputManager,
+ TupleTag<OutputT> mainOutputTag, List<TupleTag<?>> sideOutputTags, StepContext stepContext,
+ AddCounterMutator addCounterMutator, WindowingStrategy<?, ?> windowingStrategy) {
+ super(options, fn, sideInputReader, outputManager, mainOutputTag, sideOutputTags, stepContext,
+ addCounterMutator, windowingStrategy);
+ }
+
+ @Override
+ protected void invokeProcessElement(WindowedValue<InputT> elem) {
+ final DoFn<InputT, OutputT>.ProcessContext processContext = createProcessContext(elem);
+ // This can contain user code. Wrap it in case it throws an exception.
+ try {
+ fn.processElement(processContext);
+ } catch (Exception ex) {
+ throw wrapUserCodeException(ex);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java
new file mode 100644
index 0000000..2eeee54
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
+import org.apache.beam.sdk.util.state.BagState;
+import org.apache.beam.sdk.util.state.CombiningState;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateMerging;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+
+/**
+ * {@link ReduceFn} implementing the default reduction behaviors of {@link GroupByKey}.
+ *
+ * @param <K> The type of key being processed.
+ * @param <InputT> The type of values associated with the key.
+ * @param <OutputT> The output type that will be produced for each key.
+ * @param <W> The type of windows this operates on.
+ */
+public abstract class SystemReduceFn<K, InputT, AccumT, OutputT, W extends BoundedWindow>
+ extends ReduceFn<K, InputT, OutputT, W> {
+ private static final String BUFFER_NAME = "buf";
+
+ /**
+ * Create a factory that produces {@link SystemReduceFn} instances that that buffer all of the
+ * input values in persistent state and produces an {@code Iterable<T>}.
+ */
+ public static <K, T, W extends BoundedWindow> SystemReduceFn<K, T, Iterable<T>, Iterable<T>, W>
+ buffering(final Coder<T> inputCoder) {
+ final StateTag<Object, BagState<T>> bufferTag =
+ StateTags.makeSystemTagInternal(StateTags.bag(BUFFER_NAME, inputCoder));
+ return new SystemReduceFn<K, T, Iterable<T>, Iterable<T>, W>(bufferTag) {
+ @Override
+ public void prefetchOnMerge(MergingStateAccessor<K, W> state) throws Exception {
+ StateMerging.prefetchBags(state, bufferTag);
+ }
+
+ @Override
+ public void onMerge(OnMergeContext c) throws Exception {
+ StateMerging.mergeBags(c.state(), bufferTag);
+ }
+ };
+ }
+
+ /**
+ * Create a factory that produces {@link SystemReduceFn} instances that combine all of the input
+ * values using a {@link CombineFn}.
+ */
+ public static <K, InputT, AccumT, OutputT, W extends BoundedWindow> SystemReduceFn<K, InputT,
+ AccumT, OutputT, W>
+ combining(
+ final Coder<K> keyCoder, final AppliedCombineFn<K, InputT, AccumT, OutputT> combineFn) {
+ final StateTag<K, AccumulatorCombiningState<InputT, AccumT, OutputT>> bufferTag;
+ if (combineFn.getFn() instanceof KeyedCombineFnWithContext) {
+ bufferTag = StateTags.makeSystemTagInternal(
+ StateTags.<K, InputT, AccumT, OutputT>keyedCombiningValueWithContext(
+ BUFFER_NAME, combineFn.getAccumulatorCoder(),
+ (KeyedCombineFnWithContext<K, InputT, AccumT, OutputT>) combineFn.getFn()));
+
+ } else {
+ bufferTag = StateTags.makeSystemTagInternal(
+ StateTags.<K, InputT, AccumT, OutputT>keyedCombiningValue(
+ BUFFER_NAME, combineFn.getAccumulatorCoder(),
+ (KeyedCombineFn<K, InputT, AccumT, OutputT>) combineFn.getFn()));
+ }
+ return new SystemReduceFn<K, InputT, AccumT, OutputT, W>(bufferTag) {
+ @Override
+ public void prefetchOnMerge(MergingStateAccessor<K, W> state) throws Exception {
+ StateMerging.prefetchCombiningValues(state, bufferTag);
+ }
+
+ @Override
+ public void onMerge(OnMergeContext c) throws Exception {
+ StateMerging.mergeCombiningValues(c.state(), bufferTag);
+ }
+ };
+ }
+
+ private StateTag<? super K, ? extends CombiningState<InputT, OutputT>> bufferTag;
+
+ public SystemReduceFn(
+ StateTag<? super K, ? extends CombiningState<InputT, OutputT>> bufferTag) {
+ this.bufferTag = bufferTag;
+ }
+
+ @Override
+ public void processValue(ProcessValueContext c) throws Exception {
+ c.state().access(bufferTag).add(c.value());
+ }
+
+ @Override
+ public void prefetchOnTrigger(StateAccessor<K> state) {
+ state.access(bufferTag).readLater();
+ }
+
+ @Override
+ public void onTrigger(OnTriggerContext c) throws Exception {
+ c.output(c.state().access(bufferTag).read());
+ }
+
+ @Override
+ public void clearState(Context c) throws Exception {
+ c.state().access(bufferTag).clear();
+ }
+
+ @Override
+ public ReadableState<Boolean> isEmpty(StateAccessor<K> state) {
+ return state.access(bufferTag).isEmpty();
+ }
+}
[03/12] incubator-beam git commit: Move some easy stuff into
runners/core-java
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
deleted file mode 100644
index 14ec082..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
+++ /dev/null
@@ -1,536 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
-import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.StateMerging;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.util.state.WatermarkHoldState;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-import java.io.Serializable;
-
-import javax.annotation.Nullable;
-
-/**
- * Implements the logic to hold the output watermark for a computation back
- * until it has seen all the elements it needs based on the input watermark for the
- * computation.
- *
- * <p>The backend ensures the output watermark can never progress beyond the
- * input watermark for a computation. GroupAlsoByWindows computations may add a 'hold'
- * to the output watermark in order to prevent it progressing beyond a time within a window.
- * The hold will be 'cleared' when the associated pane is emitted.
- *
- * <p>This class is only intended for use by {@link ReduceFnRunner}. The two evolve together and
- * will likely break any other uses.
- *
- * @param <W> The kind of {@link BoundedWindow} the hold is for.
- */
-class WatermarkHold<W extends BoundedWindow> implements Serializable {
- /**
- * Return tag for state containing the output watermark hold
- * used for elements.
- */
- public static <W extends BoundedWindow>
- StateTag<Object, WatermarkHoldState<W>> watermarkHoldTagForOutputTimeFn(
- OutputTimeFn<? super W> outputTimeFn) {
- return StateTags.<Object, WatermarkHoldState<W>>makeSystemTagInternal(
- StateTags.<W>watermarkStateInternal("hold", outputTimeFn));
- }
-
- /**
- * Tag for state containing end-of-window and garbage collection output watermark holds.
- * (We can't piggy-back on the data hold state since the outputTimeFn may be
- * {@link OutputTimeFns#outputAtLatestInputTimestamp()}, in which case every pane will
- * would take the end-of-window time as its element time.)
- */
- @VisibleForTesting
- public static final StateTag<Object, WatermarkHoldState<BoundedWindow>> EXTRA_HOLD_TAG =
- StateTags.makeSystemTagInternal(StateTags.watermarkStateInternal(
- "extra", OutputTimeFns.outputAtEarliestInputTimestamp()));
-
- private final TimerInternals timerInternals;
- private final WindowingStrategy<?, W> windowingStrategy;
- private final StateTag<Object, WatermarkHoldState<W>> elementHoldTag;
-
- public WatermarkHold(TimerInternals timerInternals, WindowingStrategy<?, W> windowingStrategy) {
- this.timerInternals = timerInternals;
- this.windowingStrategy = windowingStrategy;
- this.elementHoldTag = watermarkHoldTagForOutputTimeFn(windowingStrategy.getOutputTimeFn());
- }
-
- /**
- * Add a hold to prevent the output watermark progressing beyond the (possibly adjusted) timestamp
- * of the element in {@code context}. We allow the actual hold time to be shifted later by
- * {@link OutputTimeFn#assignOutputTime}, but no further than the end of the window. The hold will
- * remain until cleared by {@link #extractAndRelease}. Return the timestamp at which the hold
- * was placed, or {@literal null} if no hold was placed.
- *
- * <p>In the following we'll write {@code E} to represent an element's timestamp after passing
- * through the window strategy's output time function, {@code IWM} for the local input watermark,
- * {@code OWM} for the local output watermark, and {@code GCWM} for the garbage collection
- * watermark (which is at {@code IWM - getAllowedLateness}). Time progresses from left to right,
- * and we write {@code [ ... ]} to denote a bounded window with implied lower bound.
- *
- * <p>Note that the GCWM will be the same as the IWM if {@code getAllowedLateness}
- * is {@code ZERO}.
- *
- * <p>Here are the cases we need to handle. They are conceptually considered in the
- * sequence written since if getAllowedLateness is ZERO the GCWM is the same as the IWM.
- * <ol>
- * <li>(Normal)
- * <pre>
- * |
- * [ | E ]
- * |
- * IWM
- * </pre>
- * This is, hopefully, the common and happy case. The element is locally on-time and can
- * definitely make it to an {@code ON_TIME} pane which we can still set an end-of-window timer
- * for. We place an element hold at E, which may contribute to the {@code ON_TIME} pane's
- * timestamp (depending on the output time function). Thus the OWM will not proceed past E
- * until the next pane fires.
- *
- * <li>(Discard - no target window)
- * <pre>
- * | |
- * [ E ] | |
- * | |
- * GCWM <-getAllowedLateness-> IWM
- * </pre>
- * The element is very locally late. The window has been garbage collected, thus there
- * is no target pane E could be assigned to. We discard E.
- *
- * <li>(Unobservably late)
- * <pre>
- * | |
- * [ | E | ]
- * | |
- * OWM IWM
- * </pre>
- * The element is locally late, however we can still treat this case as for 'Normal' above
- * since the IWM has not yet passed the end of the window and the element is ahead of the
- * OWM. In effect, we get to 'launder' the locally late element and consider it as locally
- * on-time because no downstream computation can observe the difference.
- *
- * <li>(Maybe late 1)
- * <pre>
- * | |
- * [ | E ] |
- * | |
- * OWM IWM
- * </pre>
- * The end-of-window timer may have already fired for this window, and thus an {@code ON_TIME}
- * pane may have already been emitted. However, if timer firings have been delayed then it
- * is possible the {@code ON_TIME} pane has not yet been emitted. We can't place an element
- * hold since we can't be sure if it will be cleared promptly. Thus this element *may* find
- * its way into an {@code ON_TIME} pane, but if so it will *not* contribute to that pane's
- * timestamp. We may however set a garbage collection hold if required.
- *
- * <li>(Maybe late 2)
- * <pre>
- * | |
- * [ E | | ]
- * | |
- * OWM IWM
- * </pre>
- * The end-of-window timer has not yet fired, so this element may still appear in an
- * {@code ON_TIME} pane. However the element is too late to contribute to the output
- * watermark hold, and thus won't contribute to the pane's timestamp. We can still place an
- * end-of-window hold.
- *
- * <li>(Maybe late 3)
- * <pre>
- * | |
- * [ E | ] |
- * | |
- * OWM IWM
- * </pre>
- * As for the (Maybe late 2) case, however we don't even know if the end-of-window timer
- * has already fired, or it is about to fire. We can place only the garbage collection hold,
- * if required.
- *
- * <li>(Definitely late)
- * <pre>
- * | |
- * [ E ] | |
- * | |
- * OWM IWM
- * </pre>
- * The element is definitely too late to make an {@code ON_TIME} pane. We are too late to
- * place an end-of-window hold. We can still place a garbage collection hold if required.
- *
- * </ol>
- */
- @Nullable
- public Instant addHolds(ReduceFn<?, ?, ?, W>.ProcessValueContext context) {
- Instant hold = addElementHold(context);
- if (hold == null) {
- hold = addEndOfWindowOrGarbageCollectionHolds(context, false/*paneIsEmpty*/);
- }
- return hold;
- }
-
- /**
- * 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.getOutputTimeFn().assignOutputTime(timestamp, window);
- Preconditions.checkState(!shifted.isBefore(timestamp),
- "OutputTimeFn moved element from %s to earlier time %s for window %s",
- timestamp, shifted, window);
- Preconditions.checkState(timestamp.isAfter(window.maxTimestamp())
- || !shifted.isAfter(window.maxTimestamp()),
- "OutputTimeFn moved element from %s to %s which is beyond end of "
- + "window %s",
- timestamp, shifted, window);
-
- return shifted;
- }
-
- /**
- * Attempt to add an 'element hold'. Return the {@link Instant} at which the hold was
- * added (ie the element timestamp plus any forward shift requested by the
- * {@link WindowingStrategy#getOutputTimeFn}), or {@literal null} if no hold was added.
- * The hold is only added if both:
- * <ol>
- * <li>The backend will be able to respect it. In other words the output watermark cannot
- * be ahead of the proposed hold time.
- * <li>A timer will be set (by {@link ReduceFnRunner}) to clear the hold by the end of the
- * window. In other words the input watermark cannot be ahead of the end of the window.
- * </ol>
- * The hold ensures the pane which incorporates the element is will not be considered late by
- * any downstream computation when it is eventually emitted.
- */
- @Nullable
- private Instant addElementHold(ReduceFn<?, ?, ?, W>.ProcessValueContext context) {
- // Give the window function a chance to move the hold timestamp forward to encourage progress.
- // (A later hold implies less impediment to the output watermark making progress, which in
- // turn encourages end-of-window triggers to fire earlier in following computations.)
- Instant elementHold = shift(context.timestamp(), context.window());
-
- Instant outputWM = timerInternals.currentOutputWatermarkTime();
- Instant inputWM = timerInternals.currentInputWatermarkTime();
-
- String which;
- boolean tooLate;
- // TODO: These case labels could be tightened.
- // See the case analysis in addHolds above for the motivation.
- if (outputWM != null && elementHold.isBefore(outputWM)) {
- which = "too late to effect output watermark";
- tooLate = true;
- } else if (context.window().maxTimestamp().isBefore(inputWM)) {
- which = "too late for end-of-window timer";
- tooLate = true;
- } else {
- which = "on time";
- tooLate = false;
- Preconditions.checkState(!elementHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
- "Element hold %s is beyond end-of-time", elementHold);
- context.state().access(elementHoldTag).add(elementHold);
- }
- WindowTracing.trace(
- "WatermarkHold.addHolds: element hold at {} is {} for "
- + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
- elementHold, which, context.key(), context.window(), inputWM,
- outputWM);
-
- return tooLate ? null : elementHold;
- }
-
- /**
- * Add an end-of-window hold or, if too late for that, a garbage collection hold (if required).
- * Return the {@link Instant} at which hold was added, or {@literal null} if no hold was added.
- */
- @Nullable
- private Instant addEndOfWindowOrGarbageCollectionHolds(
- ReduceFn<?, ?, ?, W>.Context context, boolean paneIsEmpty) {
- Instant hold = addEndOfWindowHold(context, paneIsEmpty);
- if (hold == null) {
- hold = addGarbageCollectionHold(context, paneIsEmpty);
- }
- return hold;
- }
-
- /**
- * Attempt to add an 'end-of-window hold'. Return the {@link Instant} at which the hold was added
- * (ie the end of window time), or {@literal null} if no end of window hold is possible and we
- * should fallback to a garbage collection hold.
- *
- * <p>We only add the hold if we can be sure a timer will be set (by {@link ReduceFnRunner})
- * to clear it. In other words, the input watermark cannot be ahead of the end of window time.
- *
- * <p>An end-of-window hold is added in two situations:
- * <ol>
- * <li>An incoming element came in behind the output watermark (so we are too late for placing
- * the usual element hold), but it may still be possible to include the element in an
- * {@link Timing#ON_TIME} pane. We place the end of window hold to ensure that pane will
- * not be considered late by any downstream computation.
- * <li>We guarantee an {@link Timing#ON_TIME} pane will be emitted for all windows which saw at
- * least one element, even if that {@link Timing#ON_TIME} pane is empty. Thus when elements in
- * a pane are processed due to a fired trigger we must set both an end of window timer and an end
- * of window hold. Again, the hold ensures the {@link Timing#ON_TIME} pane will not be considered
- * late by any downstream computation.
- * </ol>
- */
- @Nullable
- private Instant addEndOfWindowHold(ReduceFn<?, ?, ?, W>.Context context, boolean paneIsEmpty) {
- Instant outputWM = timerInternals.currentOutputWatermarkTime();
- Instant inputWM = timerInternals.currentInputWatermarkTime();
- Instant eowHold = context.window().maxTimestamp();
-
- if (eowHold.isBefore(inputWM)) {
- WindowTracing.trace(
- "WatermarkHold.addEndOfWindowHold: end-of-window hold at {} is too late for "
- + "end-of-window timer for key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
- eowHold, context.key(), context.window(), inputWM, outputWM);
- return null;
- }
-
- Preconditions.checkState(outputWM == null || !eowHold.isBefore(outputWM),
- "End-of-window hold %s cannot be before output watermark %s",
- eowHold, outputWM);
- Preconditions.checkState(!eowHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
- "End-of-window hold %s is beyond end-of-time", eowHold);
- // If paneIsEmpty then this hold is just for empty ON_TIME panes, so we want to keep
- // the hold away from the combining function in elementHoldTag.
- // However if !paneIsEmpty then it could make sense to use the elementHoldTag here.
- // Alas, onMerge is forced to add an end of window or garbage collection hold without
- // knowing whether an element hold is already in place (stopping to check is too expensive).
- // This it would end up adding an element hold at the end of the window which could
- // upset the elementHoldTag combining function.
- context.state().access(EXTRA_HOLD_TAG).add(eowHold);
- WindowTracing.trace(
- "WatermarkHold.addEndOfWindowHold: end-of-window hold at {} is on time for "
- + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
- eowHold, context.key(), context.window(), inputWM, outputWM);
- return eowHold;
- }
-
- /**
- * Attempt to add a 'garbage collection hold' if it is required. Return the {@link Instant} at
- * which the hold was added (ie the end of window time plus allowed lateness),
- * or {@literal null} if no hold was added.
- *
- * <p>We only add the hold if it is distinct from what would be added by
- * {@link #addEndOfWindowHold}. In other words, {@link WindowingStrategy#getAllowedLateness}
- * must be non-zero.
- *
- * <p>A garbage collection hold is added in two situations:
- * <ol>
- * <li>An incoming element came in behind the output watermark, and was too late for placing
- * the usual element hold or an end of window hold. Place the garbage collection hold so that
- * we can guarantee when the pane is finally triggered its output will not be dropped due to
- * excessive lateness by any downstream computation.
- * <li>The {@link WindowingStrategy#getClosingBehavior()} is
- * {@link ClosingBehavior#FIRE_ALWAYS}, and thus we guarantee a final pane will be emitted
- * for all windows which saw at least one element. Again, the garbage collection hold guarantees
- * that any empty final pane can be given a timestamp which will not be considered beyond
- * allowed lateness by any downstream computation.
- * </ol>
- *
- * <p>We use {@code paneIsEmpty} to distinguish cases 1 and 2.
- */
- @Nullable
- private Instant addGarbageCollectionHold(
- ReduceFn<?, ?, ?, W>.Context context, boolean paneIsEmpty) {
- Instant outputWM = timerInternals.currentOutputWatermarkTime();
- Instant inputWM = timerInternals.currentInputWatermarkTime();
- Instant eow = context.window().maxTimestamp();
- Instant gcHold = eow.plus(windowingStrategy.getAllowedLateness());
-
- if (!windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO)) {
- WindowTracing.trace(
- "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} is unnecessary "
- + "since no allowed lateness for key:{}; window:{}; inputWatermark:{}; "
- + "outputWatermark:{}",
- gcHold, context.key(), context.window(), inputWM, outputWM);
- return null;
- }
-
- if (paneIsEmpty && context.windowingStrategy().getClosingBehavior()
- == ClosingBehavior.FIRE_IF_NON_EMPTY) {
- WindowTracing.trace(
- "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} is unnecessary "
- + "since empty pane and FIRE_IF_NON_EMPTY for key:{}; window:{}; inputWatermark:{}; "
- + "outputWatermark:{}",
- gcHold, context.key(), context.window(), inputWM, outputWM);
- return null;
- }
-
- Preconditions.checkState(!gcHold.isBefore(inputWM),
- "Garbage collection hold %s cannot be before input watermark %s",
- gcHold, inputWM);
- Preconditions.checkState(!gcHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
- "Garbage collection hold %s is beyond end-of-time", gcHold);
- // Same EXTRA_HOLD_TAG vs elementHoldTag discussion as in addEndOfWindowHold above.
- context.state().access(EXTRA_HOLD_TAG).add(gcHold);
-
- WindowTracing.trace(
- "WatermarkHold.addGarbageCollectionHold: garbage collection hold at {} is on time for "
- + "key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
- gcHold, context.key(), context.window(), inputWM, outputWM);
- return gcHold;
- }
-
- /**
- * Prefetch watermark holds in preparation for merging.
- */
- public void prefetchOnMerge(MergingStateAccessor<?, W> state) {
- StateMerging.prefetchWatermarks(state, elementHoldTag);
- }
-
- /**
- * Updates the watermark hold when windows merge if it is possible the merged value does
- * not equal all of the existing holds. For example, if the new window implies a later
- * watermark hold, then earlier holds may be released.
- */
- public void onMerge(ReduceFn<?, ?, ?, W>.OnMergeContext context) {
- WindowTracing.debug("WatermarkHold.onMerge: for key:{}; window:{}; inputWatermark:{}; "
- + "outputWatermark:{}",
- context.key(), context.window(), timerInternals.currentInputWatermarkTime(),
- timerInternals.currentOutputWatermarkTime());
- StateMerging.mergeWatermarks(context.state(), elementHoldTag, context.window());
- // If we had a cheap way to determine if we have an element hold then we could
- // avoid adding an unnecessary end-of-window or garbage collection hold.
- // Simply reading the above merged watermark would impose an additional read for the
- // common case that the active window has just one underlying state address window and
- // the hold depends on the min of the element timestamps.
- // At least one merged window must be non-empty for the merge to have been triggered.
- StateMerging.clear(context.state(), EXTRA_HOLD_TAG);
- addEndOfWindowOrGarbageCollectionHolds(context, false /*paneIsEmpty*/);
- }
-
- /**
- * Result of {@link #extractAndRelease}.
- */
- public static class OldAndNewHolds {
- public final Instant oldHold;
- @Nullable
- public final Instant newHold;
-
- public OldAndNewHolds(Instant oldHold, @Nullable Instant newHold) {
- this.oldHold = oldHold;
- this.newHold = newHold;
- }
- }
-
- /**
- * Return (a future for) the earliest hold for {@code context}. Clear all the holds after
- * reading, but add/restore an end-of-window or garbage collection hold if required.
- *
- * <p>The returned timestamp is the output timestamp according to the {@link OutputTimeFn}
- * from the windowing strategy of this {@link WatermarkHold}, combined across all the non-late
- * elements in the current pane. If there is no such value the timestamp is the end
- * of the window.
- */
- public ReadableState<OldAndNewHolds> extractAndRelease(
- final ReduceFn<?, ?, ?, W>.Context context, final boolean isFinished) {
- WindowTracing.debug(
- "WatermarkHold.extractAndRelease: for key:{}; window:{}; inputWatermark:{}; "
- + "outputWatermark:{}",
- context.key(), context.window(), timerInternals.currentInputWatermarkTime(),
- timerInternals.currentOutputWatermarkTime());
- final WatermarkHoldState<W> elementHoldState = context.state().access(elementHoldTag);
- final WatermarkHoldState<BoundedWindow> extraHoldState = context.state().access(EXTRA_HOLD_TAG);
- return new ReadableState<OldAndNewHolds>() {
- @Override
- public ReadableState<OldAndNewHolds> readLater() {
- elementHoldState.readLater();
- extraHoldState.readLater();
- return this;
- }
-
- @Override
- public OldAndNewHolds read() {
- // Read both the element and extra holds.
- Instant elementHold = elementHoldState.read();
- Instant extraHold = extraHoldState.read();
- Instant oldHold;
- // Find the minimum, accounting for null.
- if (elementHold == null) {
- oldHold = extraHold;
- } else if (extraHold == null) {
- oldHold = elementHold;
- } else if (elementHold.isBefore(extraHold)) {
- oldHold = elementHold;
- } else {
- oldHold = extraHold;
- }
- if (oldHold == null || oldHold.isAfter(context.window().maxTimestamp())) {
- // If no hold (eg because all elements came in behind the output watermark), or
- // the hold was for garbage collection, take the end of window as the result.
- WindowTracing.debug(
- "WatermarkHold.extractAndRelease.read: clipping from {} to end of window "
- + "for key:{}; window:{}",
- oldHold, context.key(), context.window());
- oldHold = context.window().maxTimestamp();
- }
- WindowTracing.debug("WatermarkHold.extractAndRelease.read: clearing for key:{}; window:{}",
- context.key(), context.window());
-
- // Clear the underlying state to allow the output watermark to progress.
- elementHoldState.clear();
- extraHoldState.clear();
-
- @Nullable Instant newHold = null;
- if (!isFinished) {
- // Only need to leave behind an end-of-window or garbage collection hold
- // if future elements will be processed.
- newHold = addEndOfWindowOrGarbageCollectionHolds(context, true /*paneIsEmpty*/);
- }
-
- return new OldAndNewHolds(oldHold, newHold);
- }
- };
- }
-
- /**
- * Clear any remaining holds.
- */
- public void clearHolds(ReduceFn<?, ?, ?, W>.Context context) {
- WindowTracing.debug(
- "WatermarkHold.clearHolds: For key:{}; window:{}; inputWatermark:{}; outputWatermark:{}",
- context.key(), context.window(), timerInternals.currentInputWatermarkTime(),
- timerInternals.currentOutputWatermarkTime());
- context.state().access(elementHoldTag).clear();
- context.state().access(EXTRA_HOLD_TAG).clear();
- }
-
- /**
- * Return the current data hold, or null if none. Does not clear. For debugging only.
- */
- @Nullable
- public Instant getDataCurrent(ReduceFn<?, ?, ?, W>.Context context) {
- return context.state().access(elementHoldTag).read();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java
deleted file mode 100644
index 3e1528f..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.state.StateNamespace;
-import org.apache.beam.sdk.util.state.StateNamespaceForTest;
-
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.MockitoAnnotations;
-
-/**
- * Tests for {@link BatchTimerInternals}.
- */
-@RunWith(JUnit4.class)
-public class BatchTimerInternalsTest {
-
- private static final StateNamespace NS1 = new StateNamespaceForTest("NS1");
-
- @Mock
- private ReduceFnRunner<?, ?, ?, ?> mockRunner;
-
- @Before
- public void setUp() {
- MockitoAnnotations.initMocks(this);
- }
-
- @Test
- public void testFiringTimers() throws Exception {
- BatchTimerInternals underTest = new BatchTimerInternals(new Instant(0));
- TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
- TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME);
-
- underTest.setTimer(processingTime1);
- underTest.setTimer(processingTime2);
-
- underTest.advanceProcessingTime(mockRunner, new Instant(20));
- Mockito.verify(mockRunner).onTimer(processingTime1);
- Mockito.verifyNoMoreInteractions(mockRunner);
-
- // Advancing just a little shouldn't refire
- underTest.advanceProcessingTime(mockRunner, new Instant(21));
- Mockito.verifyNoMoreInteractions(mockRunner);
-
- // Adding the timer and advancing a little should refire
- underTest.setTimer(processingTime1);
- Mockito.verify(mockRunner).onTimer(processingTime1);
- underTest.advanceProcessingTime(mockRunner, new Instant(21));
- Mockito.verifyNoMoreInteractions(mockRunner);
-
- // And advancing the rest of the way should still have the other timer
- underTest.advanceProcessingTime(mockRunner, new Instant(30));
- Mockito.verify(mockRunner).onTimer(processingTime2);
- Mockito.verifyNoMoreInteractions(mockRunner);
- }
-
- @Test
- public void testTimerOrdering() throws Exception {
- BatchTimerInternals underTest = new BatchTimerInternals(new Instant(0));
- TimerData watermarkTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME);
- TimerData processingTime1 = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
- TimerData watermarkTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.EVENT_TIME);
- TimerData processingTime2 = TimerData.of(NS1, new Instant(29), TimeDomain.PROCESSING_TIME);
-
- underTest.setTimer(processingTime1);
- underTest.setTimer(watermarkTime1);
- underTest.setTimer(processingTime2);
- underTest.setTimer(watermarkTime2);
-
- underTest.advanceInputWatermark(mockRunner, new Instant(30));
- Mockito.verify(mockRunner).onTimer(watermarkTime1);
- Mockito.verify(mockRunner).onTimer(watermarkTime2);
- Mockito.verifyNoMoreInteractions(mockRunner);
-
- underTest.advanceProcessingTime(mockRunner, new Instant(30));
- Mockito.verify(mockRunner).onTimer(processingTime1);
- Mockito.verify(mockRunner).onTimer(processingTime2);
- Mockito.verifyNoMoreInteractions(mockRunner);
- }
-
- @Test
- public void testDeduplicate() throws Exception {
- BatchTimerInternals underTest = new BatchTimerInternals(new Instant(0));
- TimerData watermarkTime = TimerData.of(NS1, new Instant(19), TimeDomain.EVENT_TIME);
- TimerData processingTime = TimerData.of(NS1, new Instant(19), TimeDomain.PROCESSING_TIME);
- underTest.setTimer(watermarkTime);
- underTest.setTimer(watermarkTime);
- underTest.setTimer(processingTime);
- underTest.setTimer(processingTime);
- underTest.advanceProcessingTime(mockRunner, new Instant(20));
- underTest.advanceInputWatermark(mockRunner, new Instant(20));
-
- Mockito.verify(mockRunner).onTimer(processingTime);
- Mockito.verify(mockRunner).onTimer(watermarkTime);
- Mockito.verifyNoMoreInteractions(mockRunner);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
deleted file mode 100644
index f653f49..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
+++ /dev/null
@@ -1,619 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.hasSize;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.DoFnTester;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.Sessions;
-import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.apache.beam.sdk.values.TupleTag;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-/**
- * Properties of {@link GroupAlsoByWindowsDoFn}.
- *
- * <p>Some properties may not hold of some implementations, due to restrictions on the context
- * in which the implementation is applicable. For example, some {@code GroupAlsoByWindows} may not
- * support merging windows.
- */
-public class GroupAlsoByWindowsProperties {
-
- /**
- * A factory of {@link GroupAlsoByWindowsDoFn} so that the various properties can provide
- * the appropriate windowing strategy under test.
- */
- public interface GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> {
- <W extends BoundedWindow> GroupAlsoByWindowsDoFn<K, InputT, OutputT, W>
- forStrategy(WindowingStrategy<?, W> strategy);
- }
-
- /**
- * Tests that for empty input and the given {@link WindowingStrategy}, the provided GABW
- * implementation produces no output.
- *
- * <p>The input type is deliberately left as a wildcard, since it is not relevant.
- */
- public static <K, InputT, OutputT> void emptyInputEmptyOutput(
- GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> gabwFactory)
- throws Exception {
-
- WindowingStrategy<?, IntervalWindow> windowingStrategy =
- WindowingStrategy.of(FixedWindows.of(Duration.millis(10)));
-
- DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> result = runGABW(
- gabwFactory,
- windowingStrategy,
- (K) null, // key should never be used
- Collections.<WindowedValue<InputT>>emptyList());
-
- assertThat(result.peekOutputElements(), hasSize(0));
- }
-
- /**
- * Tests that for a simple sequence of elements on the same key, the given GABW implementation
- * correctly groups them according to fixed windows.
- */
- public static void groupsElementsIntoFixedWindows(
- GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
- throws Exception {
-
- WindowingStrategy<?, IntervalWindow> windowingStrategy =
- WindowingStrategy.of(FixedWindows.of(Duration.millis(10)));
-
- DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
- runGABW(gabwFactory, windowingStrategy, "key",
- WindowedValue.of(
- "v1",
- new Instant(1),
- Arrays.asList(window(0, 10)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v2",
- new Instant(2),
- Arrays.asList(window(0, 10)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v3",
- new Instant(13),
- Arrays.asList(window(10, 20)),
- PaneInfo.NO_FIRING));
-
- assertThat(result.peekOutputElements(), hasSize(2));
-
- TimestampedValue<KV<String, Iterable<String>>> item0 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
- assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
- assertThat(item0.getTimestamp(), equalTo(window(0, 10).maxTimestamp()));
-
- TimestampedValue<KV<String, Iterable<String>>> item1 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20)));
- assertThat(item1.getValue().getValue(), contains("v3"));
- assertThat(item1.getTimestamp(), equalTo(window(10, 20).maxTimestamp()));
- }
-
- /**
- * Tests that for a simple sequence of elements on the same key, the given GABW implementation
- * correctly groups them into sliding windows.
- *
- * <p>In the input here, each element occurs in multiple windows.
- */
- public static void groupsElementsIntoSlidingWindowsWithMinTimestamp(
- GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
- throws Exception {
-
- WindowingStrategy<?, IntervalWindow> windowingStrategy = WindowingStrategy.of(
- SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10)))
- .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp());
-
- DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
- runGABW(gabwFactory, windowingStrategy, "key",
- WindowedValue.of(
- "v1",
- new Instant(5),
- Arrays.asList(window(-10, 10), window(0, 20)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v2",
- new Instant(15),
- Arrays.asList(window(0, 20), window(10, 30)),
- PaneInfo.NO_FIRING));
-
- assertThat(result.peekOutputElements(), hasSize(3));
-
- TimestampedValue<KV<String, Iterable<String>>> item0 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(-10, 10)));
- assertThat(item0.getValue().getValue(), contains("v1"));
- assertThat(item0.getTimestamp(), equalTo(new Instant(5)));
-
- TimestampedValue<KV<String, Iterable<String>>> item1 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 20)));
- assertThat(item1.getValue().getValue(), containsInAnyOrder("v1", "v2"));
- assertThat(item1.getTimestamp(), equalTo(new Instant(10)));
-
- TimestampedValue<KV<String, Iterable<String>>> item2 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30)));
- assertThat(item2.getValue().getValue(), contains("v2"));
- assertThat(item2.getTimestamp(), equalTo(new Instant(20)));
- }
-
- /**
- * Tests that for a simple sequence of elements on the same key, the given GABW implementation
- * correctly groups and combines them according to sliding windows.
- *
- * <p>In the input here, each element occurs in multiple windows.
- */
- public static void combinesElementsInSlidingWindows(
- GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
- CombineFn<Long, ?, Long> combineFn)
- throws Exception {
-
- WindowingStrategy<?, IntervalWindow> windowingStrategy =
- WindowingStrategy.of(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10)))
- .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp());
-
- DoFnTester<KV<String, Iterable<WindowedValue<Long>>>, KV<String, Long>> result =
- runGABW(gabwFactory, windowingStrategy, "k",
- WindowedValue.of(
- 1L,
- new Instant(5),
- Arrays.asList(window(-10, 10), window(0, 20)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- 2L,
- new Instant(15),
- Arrays.asList(window(0, 20), window(10, 30)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- 4L,
- new Instant(18),
- Arrays.asList(window(0, 20), window(10, 30)),
- PaneInfo.NO_FIRING));
-
- assertThat(result.peekOutputElements(), hasSize(3));
-
- TimestampedValue<KV<String, Long>> item0 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(-10, 10)));
- assertThat(item0.getValue().getKey(), equalTo("k"));
- assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L))));
- assertThat(item0.getTimestamp(), equalTo(new Instant(5L)));
-
- TimestampedValue<KV<String, Long>> item1 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 20)));
- assertThat(item1.getValue().getKey(), equalTo("k"));
- assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L, 4L))));
- assertThat(item1.getTimestamp(), equalTo(new Instant(5L)));
-
- TimestampedValue<KV<String, Long>> item2 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30)));
- assertThat(item2.getValue().getKey(), equalTo("k"));
- assertThat(item2.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(2L, 4L))));
- assertThat(item2.getTimestamp(), equalTo(new Instant(15L)));
- }
-
- /**
- * Tests that the given GABW implementation correctly groups elements that fall into overlapping
- * windows that are not merged.
- */
- public static void groupsIntoOverlappingNonmergingWindows(
- GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
- throws Exception {
-
- WindowingStrategy<?, IntervalWindow> windowingStrategy =
- WindowingStrategy.of(FixedWindows.of(Duration.millis(10)));
-
- DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
- runGABW(gabwFactory, windowingStrategy, "key",
- WindowedValue.of(
- "v1",
- new Instant(1),
- Arrays.asList(window(0, 5)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v2",
- new Instant(4),
- Arrays.asList(window(1, 5)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v3",
- new Instant(4),
- Arrays.asList(window(0, 5)),
- PaneInfo.NO_FIRING));
-
- assertThat(result.peekOutputElements(), hasSize(2));
-
- TimestampedValue<KV<String, Iterable<String>>> item0 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 5)));
- assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v3"));
- assertThat(item0.getTimestamp(), equalTo(window(1, 5).maxTimestamp()));
-
- TimestampedValue<KV<String, Iterable<String>>> item1 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(1, 5)));
- assertThat(item1.getValue().getValue(), contains("v2"));
- assertThat(item1.getTimestamp(), equalTo(window(0, 5).maxTimestamp()));
- }
-
- /**
- * Tests that the given GABW implementation correctly groups elements into merged sessions.
- */
- public static void groupsElementsInMergedSessions(
- GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
- throws Exception {
-
- WindowingStrategy<?, IntervalWindow> windowingStrategy =
- WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)));
-
- DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
- runGABW(gabwFactory, windowingStrategy, "key",
- WindowedValue.of(
- "v1",
- new Instant(0),
- Arrays.asList(window(0, 10)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v2",
- new Instant(5),
- Arrays.asList(window(5, 15)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v3",
- new Instant(15),
- Arrays.asList(window(15, 25)),
- PaneInfo.NO_FIRING));
-
- assertThat(result.peekOutputElements(), hasSize(2));
-
- TimestampedValue<KV<String, Iterable<String>>> item0 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
- assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
- assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp()));
-
- TimestampedValue<KV<String, Iterable<String>>> item1 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25)));
- assertThat(item1.getValue().getValue(), contains("v3"));
- assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp()));
- }
-
- /**
- * Tests that the given {@link GroupAlsoByWindowsDoFn} implementation combines elements per
- * session window correctly according to the provided {@link CombineFn}.
- */
- public static void combinesElementsPerSession(
- GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
- CombineFn<Long, ?, Long> combineFn)
- throws Exception {
-
- WindowingStrategy<?, IntervalWindow> windowingStrategy =
- WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)));
-
- DoFnTester<KV<String, Iterable<WindowedValue<Long>>>, KV<String, Long>> result =
- runGABW(gabwFactory, windowingStrategy, "k",
- WindowedValue.of(
- 1L,
- new Instant(0),
- Arrays.asList(window(0, 10)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- 2L,
- new Instant(5),
- Arrays.asList(window(5, 15)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- 4L,
- new Instant(15),
- Arrays.asList(window(15, 25)),
- PaneInfo.NO_FIRING));
-
- assertThat(result.peekOutputElements(), hasSize(2));
-
- TimestampedValue<KV<String, Long>> item0 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
- assertThat(item0.getValue().getKey(), equalTo("k"));
- assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L))));
- assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp()));
-
- TimestampedValue<KV<String, Long>> item1 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25)));
- assertThat(item1.getValue().getKey(), equalTo("k"));
- assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(4L))));
- assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp()));
- }
-
- /**
- * Tests that for a simple sequence of elements on the same key, the given GABW implementation
- * correctly groups them according to fixed windows and also sets the output timestamp
- * according to the policy {@link OutputTimeFns#outputAtEndOfWindow()}.
- */
- public static void groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp(
- GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
- throws Exception {
-
- WindowingStrategy<?, IntervalWindow> windowingStrategy =
- WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
- .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
-
- DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
- runGABW(gabwFactory, windowingStrategy, "key",
- WindowedValue.of(
- "v1",
- new Instant(1),
- Arrays.asList(window(0, 10)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v2",
- new Instant(2),
- Arrays.asList(window(0, 10)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v3",
- new Instant(13),
- Arrays.asList(window(10, 20)),
- PaneInfo.NO_FIRING));
-
- assertThat(result.peekOutputElements(), hasSize(2));
-
- TimestampedValue<KV<String, Iterable<String>>> item0 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
- assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
- assertThat(item0.getTimestamp(), equalTo(window(0, 10).maxTimestamp()));
-
- TimestampedValue<KV<String, Iterable<String>>> item1 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20)));
- assertThat(item1.getValue().getValue(), contains("v3"));
- assertThat(item1.getTimestamp(), equalTo(window(10, 20).maxTimestamp()));
- }
-
- /**
- * Tests that for a simple sequence of elements on the same key, the given GABW implementation
- * correctly groups them according to fixed windows and also sets the output timestamp
- * according to the policy {@link OutputTimeFns#outputAtLatestInputTimestamp()}.
- */
- public static void groupsElementsIntoFixedWindowsWithLatestTimestamp(
- GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
- throws Exception {
-
- WindowingStrategy<?, IntervalWindow> windowingStrategy =
- WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
- .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp());
-
- DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
- runGABW(gabwFactory, windowingStrategy, "k",
- WindowedValue.of(
- "v1",
- new Instant(1),
- Arrays.asList(window(0, 10)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v2",
- new Instant(2),
- Arrays.asList(window(0, 10)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v3",
- new Instant(13),
- Arrays.asList(window(10, 20)),
- PaneInfo.NO_FIRING));
-
- assertThat(result.peekOutputElements(), hasSize(2));
-
- TimestampedValue<KV<String, Iterable<String>>> item0 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
- assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
- assertThat(item0.getTimestamp(), equalTo(new Instant(2)));
-
- TimestampedValue<KV<String, Iterable<String>>> item1 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 20)));
- assertThat(item1.getValue().getValue(), contains("v3"));
- assertThat(item1.getTimestamp(), equalTo(new Instant(13)));
- }
-
- /**
- * Tests that the given GABW implementation correctly groups elements into merged sessions
- * with output timestamps at the end of the merged window.
- */
- public static void groupsElementsInMergedSessionsWithEndOfWindowTimestamp(
- GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
- throws Exception {
-
- WindowingStrategy<?, IntervalWindow> windowingStrategy =
- WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)))
- .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
-
- DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
- runGABW(gabwFactory, windowingStrategy, "k",
- WindowedValue.of(
- "v1",
- new Instant(0),
- Arrays.asList(window(0, 10)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v2",
- new Instant(5),
- Arrays.asList(window(5, 15)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v3",
- new Instant(15),
- Arrays.asList(window(15, 25)),
- PaneInfo.NO_FIRING));
-
- assertThat(result.peekOutputElements(), hasSize(2));
-
- TimestampedValue<KV<String, Iterable<String>>> item0 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
- assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
- assertThat(item0.getTimestamp(), equalTo(window(0, 15).maxTimestamp()));
-
- TimestampedValue<KV<String, Iterable<String>>> item1 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(15, 25)));
- assertThat(item1.getValue().getValue(), contains("v3"));
- assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp()));
- }
-
- /**
- * Tests that the given GABW implementation correctly groups elements into merged sessions
- * with output timestamps at the end of the merged window.
- */
- public static void groupsElementsInMergedSessionsWithLatestTimestamp(
- GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
- throws Exception {
-
- WindowingStrategy<?, IntervalWindow> windowingStrategy =
- WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)))
- .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp());
-
- BoundedWindow unmergedWindow = window(15, 25);
- DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
- runGABW(gabwFactory, windowingStrategy, "k",
- WindowedValue.of(
- "v1",
- new Instant(0),
- Arrays.asList(window(0, 10)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v2",
- new Instant(5),
- Arrays.asList(window(5, 15)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- "v3",
- new Instant(15),
- Arrays.asList(unmergedWindow),
- PaneInfo.NO_FIRING));
-
- assertThat(result.peekOutputElements(), hasSize(2));
-
- BoundedWindow mergedWindow = window(0, 15);
- TimestampedValue<KV<String, Iterable<String>>> item0 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(mergedWindow));
- assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2"));
- assertThat(item0.getTimestamp(), equalTo(new Instant(5)));
-
- TimestampedValue<KV<String, Iterable<String>>> item1 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(unmergedWindow));
- assertThat(item1.getValue().getValue(), contains("v3"));
- assertThat(item1.getTimestamp(), equalTo(new Instant(15)));
- }
-
- /**
- * Tests that the given {@link GroupAlsoByWindowsDoFn} implementation combines elements per
- * session window correctly according to the provided {@link CombineFn}.
- */
- public static void combinesElementsPerSessionWithEndOfWindowTimestamp(
- GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
- CombineFn<Long, ?, Long> combineFn)
- throws Exception {
-
- WindowingStrategy<?, IntervalWindow> windowingStrategy =
- WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)))
- .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
-
- BoundedWindow secondWindow = window(15, 25);
- DoFnTester<?, KV<String, Long>> result =
- runGABW(gabwFactory, windowingStrategy, "k",
- WindowedValue.of(
- 1L,
- new Instant(0),
- Arrays.asList(window(0, 10)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- 2L,
- new Instant(5),
- Arrays.asList(window(5, 15)),
- PaneInfo.NO_FIRING),
- WindowedValue.of(
- 4L,
- new Instant(15),
- Arrays.asList(secondWindow),
- PaneInfo.NO_FIRING));
-
- assertThat(result.peekOutputElements(), hasSize(2));
-
- BoundedWindow firstResultWindow = window(0, 15);
- TimestampedValue<KV<String, Long>> item0 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(firstResultWindow));
- assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L))));
- assertThat(item0.getTimestamp(), equalTo(firstResultWindow.maxTimestamp()));
-
- TimestampedValue<KV<String, Long>> item1 =
- Iterables.getOnlyElement(result.peekOutputElementsInWindow(secondWindow));
- assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(4L))));
- assertThat(item1.getTimestamp(),
- equalTo(secondWindow.maxTimestamp()));
- }
-
- @SafeVarargs
- private static <K, InputT, OutputT, W extends BoundedWindow>
- DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> runGABW(
- GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> gabwFactory,
- WindowingStrategy<?, W> windowingStrategy,
- K key,
- WindowedValue<InputT>... values) throws Exception {
- return runGABW(gabwFactory, windowingStrategy, key, Arrays.asList(values));
- }
-
- private static <K, InputT, OutputT, W extends BoundedWindow>
- DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> runGABW(
- GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> gabwFactory,
- WindowingStrategy<?, W> windowingStrategy,
- K key,
- Collection<WindowedValue<InputT>> values) throws Exception {
-
- TupleTag<KV<K, OutputT>> outputTag = new TupleTag<>();
- DoFnRunnerBase.ListOutputManager outputManager = new DoFnRunnerBase.ListOutputManager();
-
- DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> tester =
- DoFnTester.of(gabwFactory.forStrategy(windowingStrategy));
- tester.startBundle();
- tester.processElement(KV.<K, Iterable<WindowedValue<InputT>>>of(key, values));
- tester.finishBundle();
-
- // Sanity check for corruption
- for (KV<K, OutputT> elem : tester.peekOutputElements()) {
- assertThat(elem.getKey(), equalTo(key));
- }
-
- return tester;
- }
-
- private static BoundedWindow window(long start, long end) {
- return new IntervalWindow(new Instant(start), new Instant(end));
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
deleted file mode 100644
index 4ac6164..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.GroupAlsoByWindowsProperties.GroupAlsoByWindowsDoFnFactory;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Unit tests for {@link GroupAlsoByWindowsViaOutputBufferDoFn}.
- */
-@RunWith(JUnit4.class)
-public class GroupAlsoByWindowsViaOutputBufferDoFnTest {
-
- private class BufferingGABWViaOutputBufferDoFnFactory<K, InputT>
- implements GroupAlsoByWindowsDoFnFactory<K, InputT, Iterable<InputT>> {
-
- private final Coder<InputT> inputCoder;
-
- public BufferingGABWViaOutputBufferDoFnFactory(Coder<InputT> inputCoder) {
- this.inputCoder = inputCoder;
- }
-
- @Override
- public <W extends BoundedWindow> GroupAlsoByWindowsDoFn<K, InputT, Iterable<InputT>, W>
- forStrategy(WindowingStrategy<?, W> windowingStrategy) {
- return new GroupAlsoByWindowsViaOutputBufferDoFn<K, InputT, Iterable<InputT>, W>(
- windowingStrategy,
- SystemReduceFn.<K, InputT, W>buffering(inputCoder));
- }
- }
-
- @Test
- public void testEmptyInputEmptyOutput() throws Exception {
- GroupAlsoByWindowsProperties.emptyInputEmptyOutput(
- new BufferingGABWViaOutputBufferDoFnFactory<>(StringUtf8Coder.of()));
- }
-
- @Test
- public void testGroupsElementsIntoFixedWindows() throws Exception {
- GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindows(
- new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
- }
-
- @Test
- public void testGroupsElementsIntoSlidingWindows() throws Exception {
- GroupAlsoByWindowsProperties.groupsElementsIntoSlidingWindowsWithMinTimestamp(
- new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
- }
-
- @Test
- public void testGroupsIntoOverlappingNonmergingWindows() throws Exception {
- GroupAlsoByWindowsProperties.groupsIntoOverlappingNonmergingWindows(
- new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
- }
-
- @Test
- public void testGroupsIntoSessions() throws Exception {
- GroupAlsoByWindowsProperties.groupsElementsInMergedSessions(
- new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
- }
-
- @Test
- public void testGroupsElementsIntoFixedWindowsWithEndOfWindowTimestamp() throws Exception {
- GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp(
- new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
- }
-
- @Test
- public void testGroupsElementsIntoFixedWindowsWithLatestTimestamp() throws Exception {
- GroupAlsoByWindowsProperties.groupsElementsIntoFixedWindowsWithLatestTimestamp(
- new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
- }
-
- @Test
- public void testGroupsElementsIntoSessionsWithEndOfWindowTimestamp() throws Exception {
- GroupAlsoByWindowsProperties.groupsElementsInMergedSessionsWithEndOfWindowTimestamp(
- new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
- }
-
- @Test
- public void testGroupsElementsIntoSessionsWithLatestTimestamp() throws Exception {
- GroupAlsoByWindowsProperties.groupsElementsInMergedSessionsWithLatestTimestamp(
- new BufferingGABWViaOutputBufferDoFnFactory<String, String>(StringUtf8Coder.of()));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java
deleted file mode 100644
index d929d39..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.LateDataDroppingDoFnRunner.LateDataFilter;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import java.util.Arrays;
-
-/**
- * Unit tests for {@link LateDataDroppingDoFnRunner}.
- */
-@RunWith(JUnit4.class)
-public class LateDataDroppingDoFnRunnerTest {
- private static final FixedWindows WINDOW_FN = FixedWindows.of(Duration.millis(10));
-
- @Mock private TimerInternals mockTimerInternals;
-
- @Before
- public void setUp() {
- MockitoAnnotations.initMocks(this);
- }
-
- @Test
- public void testLateDataFilter() throws Exception {
- when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(15L));
-
- InMemoryLongSumAggregator droppedDueToLateness =
- new InMemoryLongSumAggregator("droppedDueToLateness");
- LateDataFilter lateDataFilter = new LateDataFilter(
- WindowingStrategy.of(WINDOW_FN), mockTimerInternals, droppedDueToLateness);
-
- Iterable<WindowedValue<Integer>> actual = lateDataFilter.filter(
- "a",
- ImmutableList.of(
- createDatum(13, 13L),
- createDatum(5, 5L), // late element, earlier than 4L.
- createDatum(16, 16L),
- createDatum(18, 18L)));
-
- Iterable<WindowedValue<Integer>> expected = ImmutableList.of(
- createDatum(13, 13L),
- createDatum(16, 16L),
- createDatum(18, 18L));
- assertThat(expected, containsInAnyOrder(Iterables.toArray(actual, WindowedValue.class)));
- assertEquals(1, droppedDueToLateness.sum);
- }
-
- private <T> WindowedValue<T> createDatum(T element, long timestampMillis) {
- Instant timestamp = new Instant(timestampMillis);
- return WindowedValue.of(
- element,
- timestamp,
- Arrays.asList(WINDOW_FN.assignWindow(timestamp)),
- PaneInfo.NO_FIRING);
- }
-
- private static class InMemoryLongSumAggregator implements Aggregator<Long, Long> {
- private final String name;
- private long sum = 0;
-
- public InMemoryLongSumAggregator(String name) {
- this.name = name;
- }
-
- @Override
- public void addValue(Long value) {
- sum += value;
- }
-
- @Override
- public String getName() {
- return name;
- }
-
- @Override
- public CombineFn<Long, ?, Long> getCombineFn() {
- return new Sum.SumLongFn();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java
deleted file mode 100644
index 8885118..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java
+++ /dev/null
@@ -1,234 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-
-import com.google.common.collect.ImmutableList;
-
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.MockitoAnnotations;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Tests for {@link PushbackSideInputDoFnRunner}.
- */
-@RunWith(JUnit4.class)
-public class PushbackSideInputDoFnRunnerTest {
- @Mock private ReadyCheckingSideInputReader reader;
- private TestDoFnRunner<Integer, Integer> underlying;
- private PCollectionView<Integer> singletonView;
-
- @Before
- public void setup() {
- MockitoAnnotations.initMocks(this);
- TestPipeline p = TestPipeline.create();
- PCollection<Integer> created = p.apply(Create.of(1, 2, 3));
- singletonView =
- created
- .apply(Window.into(new IdentitySideInputWindowFn()))
- .apply(Sum.integersGlobally().asSingletonView());
-
- underlying = new TestDoFnRunner<>();
- }
-
- private PushbackSideInputDoFnRunner<Integer, Integer> createRunner(
- ImmutableList<PCollectionView<?>> views) {
- PushbackSideInputDoFnRunner<Integer, Integer> runner =
- PushbackSideInputDoFnRunner.create(underlying, views, reader);
- runner.startBundle();
- return runner;
- }
-
- @Test
- public void startFinishBundleDelegates() {
- PushbackSideInputDoFnRunner runner =
- createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
-
- assertThat(underlying.started, is(true));
- assertThat(underlying.finished, is(false));
- runner.finishBundle();
- assertThat(underlying.finished, is(true));
- }
-
- @Test
- public void processElementSideInputNotReady() {
- when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class)))
- .thenReturn(false);
-
- PushbackSideInputDoFnRunner<Integer, Integer> runner =
- createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
-
- WindowedValue<Integer> oneWindow =
- WindowedValue.of(
- 2,
- new Instant(-2),
- new IntervalWindow(new Instant(-500L), new Instant(0L)),
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
- Iterable<WindowedValue<Integer>> oneWindowPushback =
- runner.processElementInReadyWindows(oneWindow);
- assertThat(oneWindowPushback, containsInAnyOrder(oneWindow));
- assertThat(underlying.inputElems, Matchers.<WindowedValue<Integer>>emptyIterable());
- }
-
- @Test
- public void processElementSideInputNotReadyMultipleWindows() {
- when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class)))
- .thenReturn(false);
-
- PushbackSideInputDoFnRunner<Integer, Integer> runner =
- createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
-
- WindowedValue<Integer> multiWindow =
- WindowedValue.of(
- 2,
- new Instant(-2),
- ImmutableList.of(
- new IntervalWindow(new Instant(-500L), new Instant(0L)),
- new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)),
- GlobalWindow.INSTANCE),
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
- Iterable<WindowedValue<Integer>> multiWindowPushback =
- runner.processElementInReadyWindows(multiWindow);
- assertThat(multiWindowPushback, equalTo(multiWindow.explodeWindows()));
- assertThat(underlying.inputElems, Matchers.<WindowedValue<Integer>>emptyIterable());
- }
-
- @Test
- public void processElementSideInputNotReadySomeWindows() {
- when(reader.isReady(Mockito.eq(singletonView), Mockito.eq(GlobalWindow.INSTANCE)))
- .thenReturn(false);
- when(
- reader.isReady(
- Mockito.eq(singletonView),
- org.mockito.AdditionalMatchers.not(Mockito.eq(GlobalWindow.INSTANCE))))
- .thenReturn(true);
-
- PushbackSideInputDoFnRunner<Integer, Integer> runner =
- createRunner(ImmutableList.<PCollectionView<?>>of(singletonView));
-
- IntervalWindow littleWindow = new IntervalWindow(new Instant(-500L), new Instant(0L));
- IntervalWindow bigWindow =
- new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L));
- WindowedValue<Integer> multiWindow =
- WindowedValue.of(
- 2,
- new Instant(-2),
- ImmutableList.of(littleWindow, bigWindow, GlobalWindow.INSTANCE),
- PaneInfo.NO_FIRING);
- Iterable<WindowedValue<Integer>> multiWindowPushback =
- runner.processElementInReadyWindows(multiWindow);
- assertThat(
- multiWindowPushback,
- containsInAnyOrder(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(-2L))));
- assertThat(underlying.inputElems,
- containsInAnyOrder(WindowedValue.of(2, new Instant(-2), littleWindow, PaneInfo.NO_FIRING),
- WindowedValue.of(2, new Instant(-2), bigWindow, PaneInfo.NO_FIRING)));
- }
-
- @Test
- public void processElementSideInputReadyAllWindows() {
- when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class)))
- .thenReturn(true);
-
- ImmutableList<PCollectionView<?>> views = ImmutableList.<PCollectionView<?>>of(singletonView);
- PushbackSideInputDoFnRunner<Integer, Integer> runner = createRunner(views);
-
- WindowedValue<Integer> multiWindow =
- WindowedValue.of(
- 2,
- new Instant(-2),
- ImmutableList.of(
- new IntervalWindow(new Instant(-500L), new Instant(0L)),
- new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)),
- GlobalWindow.INSTANCE),
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
- Iterable<WindowedValue<Integer>> multiWindowPushback =
- runner.processElementInReadyWindows(multiWindow);
- assertThat(multiWindowPushback, emptyIterable());
- assertThat(underlying.inputElems,
- containsInAnyOrder(ImmutableList.copyOf(multiWindow.explodeWindows()).toArray()));
- }
-
- @Test
- public void processElementNoSideInputs() {
- PushbackSideInputDoFnRunner<Integer, Integer> runner =
- createRunner(ImmutableList.<PCollectionView<?>>of());
-
- WindowedValue<Integer> multiWindow =
- WindowedValue.of(
- 2,
- new Instant(-2),
- ImmutableList.of(
- new IntervalWindow(new Instant(-500L), new Instant(0L)),
- new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)),
- GlobalWindow.INSTANCE),
- PaneInfo.ON_TIME_AND_ONLY_FIRING);
- Iterable<WindowedValue<Integer>> multiWindowPushback =
- runner.processElementInReadyWindows(multiWindow);
- assertThat(multiWindowPushback, emptyIterable());
- assertThat(underlying.inputElems, containsInAnyOrder(multiWindow));
- }
-
- private static class TestDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
- List<WindowedValue<InputT>> inputElems;
- private boolean started = false;
- private boolean finished = false;
-
- @Override
- public void startBundle() {
- started = true;
- inputElems = new ArrayList<>();
- }
-
- @Override
- public void processElement(WindowedValue<InputT> elem) {
- inputElems.add(elem);
- }
-
- @Override
- public void finishBundle() {
- finished = true;
- }
- }
-}
[05/12] incubator-beam git commit: Move some easy stuff into
runners/core-java
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java
deleted file mode 100644
index 65fc52d..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java
+++ /dev/null
@@ -1,247 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
-import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-
-/**
- * An implementation of {@link GroupByKey} built on top of a lower-level {@link GroupByKeyOnly}
- * primitive.
- *
- * <p>This implementation of {@link GroupByKey} proceeds via the following steps:
- * <ol>
- * <li>{@code ReifyTimestampsAndWindowsDoFn ParDo(ReifyTimestampsAndWindows)}: This embeds
- * the previously-implicit timestamp and window into the elements themselves, so a
- * window-and-timestamp-unaware transform can operate on them.</li>
- * <li>{@code GroupByKeyOnly}: This lower-level primitive groups by keys, ignoring windows
- * and timestamps. Many window-unaware runners have such a primitive already.</li>
- * <li>{@code SortValuesByTimestamp ParDo(SortValuesByTimestamp)}: The values in the iterables
- * output by {@link GroupByKeyOnly} are sorted by timestamp.</li>
- * <li>{@code GroupAlsoByWindow}: This primitive processes the sorted values. Today it is
- * implemented as a {@link ParDo} that calls reserved internal methods.</li>
- * </ol>
- *
- * <p>This implementation of {@link GroupByKey} has severe limitations unless its component
- * transforms are replaced. As-is, it is only applicable for in-memory runners using a batch-style
- * execution strategy. Specifically:
- *
- * <ul>
- * <li>Every iterable output by {@link GroupByKeyOnly} must contain all elements for that key.
- * A streaming-style partition, with multiple elements for the same key, will not yield
- * correct results.</li>
- * <li>Sorting of values by timestamp is performed on an in-memory list. It will not succeed
- * for large iterables.</li>
- * <li>The implementation of {@code GroupAlsoByWindow} does not support timers. This is only
- * appropriate for runners which also do not support timers.</li>
- * </ul>
- */
-public class GroupByKeyViaGroupByKeyOnly<K, V>
- extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
-
- private GroupByKey<K, V> gbkTransform;
-
- public GroupByKeyViaGroupByKeyOnly(GroupByKey<K, V> originalTransform) {
- this.gbkTransform = originalTransform;
- }
-
- @Override
- public PCollection<KV<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
- WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
-
- return input
- // Make each input element's timestamp and assigned windows
- // explicit, in the value part.
- .apply(new ReifyTimestampsAndWindows<K, V>())
-
- // Group by just the key.
- // Combiner lifting will not happen regardless of the disallowCombinerLifting value.
- // There will be no combiners right after the GroupByKeyOnly because of the two ParDos
- // introduced in here.
- .apply(new GroupByKeyOnly<K, WindowedValue<V>>())
-
- // Sort each key's values by timestamp. GroupAlsoByWindow requires
- // its input to be sorted by timestamp.
- .apply(new SortValuesByTimestamp<K, V>())
-
- // Group each key's values by window, merging windows as needed.
- .apply(new GroupAlsoByWindow<K, V>(windowingStrategy))
-
- // And update the windowing strategy as appropriate.
- .setWindowingStrategyInternal(
- gbkTransform.updateWindowingStrategy(windowingStrategy));
- }
-
- /**
- * Runner-specific primitive that groups by key only, ignoring any window assignments. A
- * runner that uses {@link GroupByKeyViaGroupByKeyOnly} should have a primitive way to translate
- * or evaluate this class.
- */
- public static class GroupByKeyOnly<K, V>
- extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
-
- @SuppressWarnings({"rawtypes", "unchecked"})
- @Override
- public PCollection<KV<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
- return PCollection.<KV<K, Iterable<V>>>createPrimitiveOutputInternal(
- input.getPipeline(), input.getWindowingStrategy(), input.isBounded());
- }
-
- @Override
- public Coder<KV<K, Iterable<V>>> getDefaultOutputCoder(PCollection<KV<K, V>> input) {
- return GroupByKey.getOutputKvCoder(input.getCoder());
- }
- }
-
- /**
- * Helper transform that makes timestamps and window assignments explicit in the value part of
- * each key/value pair.
- */
- public static class ReifyTimestampsAndWindows<K, V>
- extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, WindowedValue<V>>>> {
-
- @Override
- public PCollection<KV<K, WindowedValue<V>>> apply(PCollection<KV<K, V>> input) {
-
- // The requirement to use a KvCoder *is* actually a model-level requirement, not specific
- // to this implementation of GBK. All runners need a way to get the key.
- checkArgument(
- input.getCoder() instanceof KvCoder,
- "%s requires its input to use a %s",
- GroupByKey.class.getSimpleName(),
- KvCoder.class.getSimpleName());
-
- @SuppressWarnings("unchecked")
- KvCoder<K, V> inputKvCoder = (KvCoder<K, V>) input.getCoder();
- Coder<K> keyCoder = inputKvCoder.getKeyCoder();
- Coder<V> inputValueCoder = inputKvCoder.getValueCoder();
- Coder<WindowedValue<V>> outputValueCoder =
- FullWindowedValueCoder.of(
- inputValueCoder, input.getWindowingStrategy().getWindowFn().windowCoder());
- Coder<KV<K, WindowedValue<V>>> outputKvCoder = KvCoder.of(keyCoder, outputValueCoder);
- return input
- .apply(ParDo.of(new ReifyTimestampAndWindowsDoFn<K, V>()))
- .setCoder(outputKvCoder);
- }
- }
-
- /**
- * Helper transform that sorts the values associated with each key by timestamp.
- */
- private static class SortValuesByTimestamp<K, V>
- extends PTransform<
- PCollection<KV<K, Iterable<WindowedValue<V>>>>,
- PCollection<KV<K, Iterable<WindowedValue<V>>>>> {
- @Override
- public PCollection<KV<K, Iterable<WindowedValue<V>>>> apply(
- PCollection<KV<K, Iterable<WindowedValue<V>>>> input) {
- return input
- .apply(
- ParDo.of(
- new DoFn<KV<K, Iterable<WindowedValue<V>>>, KV<K, Iterable<WindowedValue<V>>>>() {
- @Override
- public void processElement(ProcessContext c) {
- KV<K, Iterable<WindowedValue<V>>> kvs = c.element();
- K key = kvs.getKey();
- Iterable<WindowedValue<V>> unsortedValues = kvs.getValue();
- List<WindowedValue<V>> sortedValues = new ArrayList<>();
- for (WindowedValue<V> value : unsortedValues) {
- sortedValues.add(value);
- }
- Collections.sort(
- sortedValues,
- new Comparator<WindowedValue<V>>() {
- @Override
- public int compare(WindowedValue<V> e1, WindowedValue<V> e2) {
- return e1.getTimestamp().compareTo(e2.getTimestamp());
- }
- });
- c.output(KV.<K, Iterable<WindowedValue<V>>>of(key, sortedValues));
- }
- }))
- .setCoder(input.getCoder());
- }
- }
-
- /**
- * Helper transform that takes a collection of timestamp-ordered
- * values associated with each key, groups the values by window,
- * combines windows as needed, and for each window in each key,
- * outputs a collection of key/value-list pairs implicitly assigned
- * to the window and with the timestamp derived from that window.
- */
- private static class GroupAlsoByWindow<K, V>
- extends PTransform<
- PCollection<KV<K, Iterable<WindowedValue<V>>>>, PCollection<KV<K, Iterable<V>>>> {
- private final WindowingStrategy<?, ?> windowingStrategy;
-
- public GroupAlsoByWindow(WindowingStrategy<?, ?> windowingStrategy) {
- this.windowingStrategy = windowingStrategy;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public PCollection<KV<K, Iterable<V>>> apply(
- PCollection<KV<K, Iterable<WindowedValue<V>>>> input) {
- @SuppressWarnings("unchecked")
- KvCoder<K, Iterable<WindowedValue<V>>> inputKvCoder =
- (KvCoder<K, Iterable<WindowedValue<V>>>) input.getCoder();
-
- Coder<K> keyCoder = inputKvCoder.getKeyCoder();
- Coder<Iterable<WindowedValue<V>>> inputValueCoder = inputKvCoder.getValueCoder();
-
- IterableCoder<WindowedValue<V>> inputIterableValueCoder =
- (IterableCoder<WindowedValue<V>>) inputValueCoder;
- Coder<WindowedValue<V>> inputIterableElementCoder = inputIterableValueCoder.getElemCoder();
- WindowedValueCoder<V> inputIterableWindowedValueCoder =
- (WindowedValueCoder<V>) inputIterableElementCoder;
-
- Coder<V> inputIterableElementValueCoder = inputIterableWindowedValueCoder.getValueCoder();
- Coder<Iterable<V>> outputValueCoder = IterableCoder.of(inputIterableElementValueCoder);
- Coder<KV<K, Iterable<V>>> outputKvCoder = KvCoder.of(keyCoder, outputValueCoder);
-
- return input
- .apply(ParDo.of(groupAlsoByWindowsFn(windowingStrategy, inputIterableElementValueCoder)))
- .setCoder(outputKvCoder);
- }
-
- private <W extends BoundedWindow>
- GroupAlsoByWindowsViaOutputBufferDoFn<K, V, Iterable<V>, W> groupAlsoByWindowsFn(
- WindowingStrategy<?, W> strategy, Coder<V> inputIterableElementValueCoder) {
- return new GroupAlsoByWindowsViaOutputBufferDoFn<K, V, Iterable<V>, W>(
- strategy, SystemReduceFn.<K, V, W>buffering(inputIterableElementValueCoder));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java
deleted file mode 100644
index 4815162..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.KV;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
-
-import org.joda.time.Instant;
-
-/**
- * A customized {@link DoFnRunner} that handles late data dropping for
- * a {@link KeyedWorkItem} input {@link DoFn}.
- *
- * <p>It expands windows before checking data lateness.
- *
- * <p>{@link KeyedWorkItem KeyedWorkItems} are always in empty windows.
- *
- * @param <K> key type
- * @param <InputT> input value element type
- * @param <OutputT> output value element type
- * @param <W> window type
- */
-public class LateDataDroppingDoFnRunner<K, InputT, OutputT, W extends BoundedWindow>
- implements DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> {
- private final DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> doFnRunner;
- private final LateDataFilter lateDataFilter;
-
- public LateDataDroppingDoFnRunner(
- DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> doFnRunner,
- WindowingStrategy<?, ?> windowingStrategy,
- TimerInternals timerInternals,
- Aggregator<Long, Long> droppedDueToLateness) {
- this.doFnRunner = doFnRunner;
- lateDataFilter = new LateDataFilter(windowingStrategy, timerInternals, droppedDueToLateness);
- }
-
- @Override
- public void startBundle() {
- doFnRunner.startBundle();
- }
-
- @Override
- public void processElement(WindowedValue<KeyedWorkItem<K, InputT>> elem) {
- Iterable<WindowedValue<InputT>> nonLateElements = lateDataFilter.filter(
- elem.getValue().key(), elem.getValue().elementsIterable());
- KeyedWorkItem<K, InputT> keyedWorkItem = KeyedWorkItems.workItem(
- elem.getValue().key(), elem.getValue().timersIterable(), nonLateElements);
- doFnRunner.processElement(elem.withValue(keyedWorkItem));
- }
-
- @Override
- public void finishBundle() {
- doFnRunner.finishBundle();
- }
-
- /**
- * It filters late data in a {@link KeyedWorkItem}.
- */
- @VisibleForTesting
- static class LateDataFilter {
- private final WindowingStrategy<?, ?> windowingStrategy;
- private final TimerInternals timerInternals;
- private final Aggregator<Long, Long> droppedDueToLateness;
-
- public LateDataFilter(
- WindowingStrategy<?, ?> windowingStrategy,
- TimerInternals timerInternals,
- Aggregator<Long, Long> droppedDueToLateness) {
- this.windowingStrategy = windowingStrategy;
- this.timerInternals = timerInternals;
- this.droppedDueToLateness = droppedDueToLateness;
- }
-
- /**
- * Returns an {@code Iterable<WindowedValue<InputT>>} that only contains
- * non-late input elements.
- */
- public <K, InputT> Iterable<WindowedValue<InputT>> filter(
- final K key, Iterable<WindowedValue<InputT>> elements) {
- Iterable<Iterable<WindowedValue<InputT>>> windowsExpandedElements = Iterables.transform(
- elements,
- new Function<WindowedValue<InputT>, Iterable<WindowedValue<InputT>>>() {
- @Override
- public Iterable<WindowedValue<InputT>> apply(final WindowedValue<InputT> input) {
- return Iterables.transform(
- input.getWindows(),
- new Function<BoundedWindow, WindowedValue<InputT>>() {
- @Override
- public WindowedValue<InputT> apply(BoundedWindow window) {
- return WindowedValue.of(
- input.getValue(), input.getTimestamp(), window, input.getPane());
- }
- });
- }});
-
- Iterable<WindowedValue<InputT>> nonLateElements = Iterables.filter(
- Iterables.concat(windowsExpandedElements),
- new Predicate<WindowedValue<InputT>>() {
- @Override
- public boolean apply(WindowedValue<InputT> input) {
- BoundedWindow window = Iterables.getOnlyElement(input.getWindows());
- if (canDropDueToExpiredWindow(window)) {
- // The element is too late for this window.
- droppedDueToLateness.addValue(1L);
- WindowTracing.debug(
- "ReduceFnRunner.processElement: Dropping element at {} for key:{}; window:{} "
- + "since too far behind inputWatermark:{}; outputWatermark:{}",
- input.getTimestamp(), key, window, timerInternals.currentInputWatermarkTime(),
- timerInternals.currentOutputWatermarkTime());
- return false;
- } else {
- return true;
- }
- }
- });
- return nonLateElements;
- }
-
- /** Is {@code window} expired w.r.t. the garbage collection watermark? */
- private boolean canDropDueToExpiredWindow(BoundedWindow window) {
- Instant inputWM = timerInternals.currentInputWatermarkTime();
- return window.maxTimestamp().plus(windowingStrategy.getAllowedLateness()).isBefore(inputWM);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java
deleted file mode 100644
index e809c24..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
-import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateMerging;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-
-/**
- * Tracks which windows have non-empty panes. Specifically, which windows have new elements since
- * their last triggering.
- *
- * @param <W> The kind of windows being tracked.
- */
-public abstract class NonEmptyPanes<K, W extends BoundedWindow> {
-
- static <K, W extends BoundedWindow> NonEmptyPanes<K, W> create(
- WindowingStrategy<?, W> strategy, ReduceFn<K, ?, ?, W> reduceFn) {
- if (strategy.getMode() == AccumulationMode.DISCARDING_FIRED_PANES) {
- return new DiscardingModeNonEmptyPanes<>(reduceFn);
- } else {
- return new GeneralNonEmptyPanes<>();
- }
- }
-
- /**
- * Record that some content has been added to the window in {@code context}, and therefore the
- * current pane is not empty.
- */
- public abstract void recordContent(StateAccessor<K> context);
-
- /**
- * Record that the given pane is empty.
- */
- public abstract void clearPane(StateAccessor<K> state);
-
- /**
- * Return true if the current pane for the window in {@code context} is empty.
- */
- public abstract ReadableState<Boolean> isEmpty(StateAccessor<K> context);
-
- /**
- * Prefetch in preparation for merging.
- */
- public abstract void prefetchOnMerge(MergingStateAccessor<K, W> state);
-
- /**
- * Eagerly merge backing state.
- */
- public abstract void onMerge(MergingStateAccessor<K, W> context);
-
- /**
- * An implementation of {@code NonEmptyPanes} optimized for use with discarding mode. Uses the
- * presence of data in the accumulation buffer to record non-empty panes.
- */
- private static class DiscardingModeNonEmptyPanes<K, W extends BoundedWindow>
- extends NonEmptyPanes<K, W> {
-
- private ReduceFn<K, ?, ?, W> reduceFn;
-
- private DiscardingModeNonEmptyPanes(ReduceFn<K, ?, ?, W> reduceFn) {
- this.reduceFn = reduceFn;
- }
-
- @Override
- public ReadableState<Boolean> isEmpty(StateAccessor<K> state) {
- return reduceFn.isEmpty(state);
- }
-
- @Override
- public void recordContent(StateAccessor<K> state) {
- // Nothing to do -- the reduceFn is tracking contents
- }
-
- @Override
- public void clearPane(StateAccessor<K> state) {
- // Nothing to do -- the reduceFn is tracking contents
- }
-
- @Override
- public void prefetchOnMerge(MergingStateAccessor<K, W> state) {
- // Nothing to do -- the reduceFn is tracking contents
- }
-
- @Override
- public void onMerge(MergingStateAccessor<K, W> context) {
- // Nothing to do -- the reduceFn is tracking contents
- }
- }
-
- /**
- * An implementation of {@code NonEmptyPanes} for general use.
- */
- private static class GeneralNonEmptyPanes<K, W extends BoundedWindow>
- extends NonEmptyPanes<K, W> {
-
- private static final StateTag<Object, AccumulatorCombiningState<Long, long[], Long>>
- PANE_ADDITIONS_TAG =
- StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal(
- "count", VarLongCoder.of(), new Sum.SumLongFn()));
-
- @Override
- public void recordContent(StateAccessor<K> state) {
- state.access(PANE_ADDITIONS_TAG).add(1L);
- }
-
- @Override
- public void clearPane(StateAccessor<K> state) {
- state.access(PANE_ADDITIONS_TAG).clear();
- }
-
- @Override
- public ReadableState<Boolean> isEmpty(StateAccessor<K> state) {
- return state.access(PANE_ADDITIONS_TAG).isEmpty();
- }
-
- @Override
- public void prefetchOnMerge(MergingStateAccessor<K, W> state) {
- StateMerging.prefetchCombiningValues(state, PANE_ADDITIONS_TAG);
- }
-
- @Override
- public void onMerge(MergingStateAccessor<K, W> context) {
- StateMerging.mergeCombiningValues(context, PANE_ADDITIONS_TAG);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
deleted file mode 100644
index 5e08031..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.util.state.ValueState;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-import org.joda.time.Instant;
-
-/**
- * Determine the timing and other properties of a new pane for a given computation, key and window.
- * Incorporates any previous pane, whether the pane has been produced because an
- * on-time {@link AfterWatermark} trigger firing, and the relation between the element's timestamp
- * and the current output watermark.
- */
-public class PaneInfoTracker {
- private TimerInternals timerInternals;
-
- public PaneInfoTracker(TimerInternals timerInternals) {
- this.timerInternals = timerInternals;
- }
-
- @VisibleForTesting
- static final StateTag<Object, ValueState<PaneInfo>> PANE_INFO_TAG =
- StateTags.makeSystemTagInternal(StateTags.value("pane", PaneInfoCoder.INSTANCE));
-
- public void clear(StateAccessor<?> state) {
- state.access(PANE_INFO_TAG).clear();
- }
-
- /**
- * Return a ({@link ReadableState} for) the pane info appropriate for {@code context}. The pane
- * info includes the timing for the pane, who's calculation is quite subtle.
- *
- * @param isFinal should be {@code true} only if the triggering machinery can guarantee
- * no further firings for the
- */
- public ReadableState<PaneInfo> getNextPaneInfo(
- ReduceFn<?, ?, ?, ?>.Context context, final boolean isFinal) {
- final Object key = context.key();
- final ReadableState<PaneInfo> previousPaneFuture =
- context.state().access(PaneInfoTracker.PANE_INFO_TAG);
- final Instant windowMaxTimestamp = context.window().maxTimestamp();
-
- return new ReadableState<PaneInfo>() {
- @Override
- public ReadableState<PaneInfo> readLater() {
- previousPaneFuture.readLater();
- return this;
- }
-
- @Override
- public PaneInfo read() {
- PaneInfo previousPane = previousPaneFuture.read();
- return describePane(key, windowMaxTimestamp, previousPane, isFinal);
- }
- };
- }
-
- public void storeCurrentPaneInfo(ReduceFn<?, ?, ?, ?>.Context context, PaneInfo currentPane) {
- context.state().access(PANE_INFO_TAG).write(currentPane);
- }
-
- private <W> PaneInfo describePane(
- Object key, Instant windowMaxTimestamp, PaneInfo previousPane, boolean isFinal) {
- boolean isFirst = previousPane == null;
- Timing previousTiming = isFirst ? null : previousPane.getTiming();
- long index = isFirst ? 0 : previousPane.getIndex() + 1;
- long nonSpeculativeIndex = isFirst ? 0 : previousPane.getNonSpeculativeIndex() + 1;
- Instant outputWM = timerInternals.currentOutputWatermarkTime();
- Instant inputWM = timerInternals.currentInputWatermarkTime();
-
- // True if it is not possible to assign the element representing this pane a timestamp
- // which will make an ON_TIME pane for any following computation.
- // Ie true if the element's latest possible timestamp is before the current output watermark.
- boolean isLateForOutput = outputWM != null && windowMaxTimestamp.isBefore(outputWM);
-
- // True if all emitted panes (if any) were EARLY panes.
- // Once the ON_TIME pane has fired, all following panes must be considered LATE even
- // if the output watermark is behind the end of the window.
- boolean onlyEarlyPanesSoFar = previousTiming == null || previousTiming == Timing.EARLY;
-
- // True is the input watermark hasn't passed the window's max timestamp.
- boolean isEarlyForInput = !inputWM.isAfter(windowMaxTimestamp);
-
- Timing timing;
- if (isLateForOutput || !onlyEarlyPanesSoFar) {
- // The output watermark has already passed the end of this window, or we have already
- // emitted a non-EARLY pane. Irrespective of how this pane was triggered we must
- // consider this pane LATE.
- timing = Timing.LATE;
- } else if (isEarlyForInput) {
- // This is an EARLY firing.
- timing = Timing.EARLY;
- nonSpeculativeIndex = -1;
- } else {
- // This is the unique ON_TIME firing for the window.
- timing = Timing.ON_TIME;
- }
-
- WindowTracing.debug(
- "describePane: {} pane (prev was {}) for key:{}; windowMaxTimestamp:{}; "
- + "inputWatermark:{}; outputWatermark:{}; isLateForOutput:{}",
- timing, previousTiming, key, windowMaxTimestamp, inputWM, outputWM, isLateForOutput);
-
- if (previousPane != null) {
- // Timing transitions should follow EARLY* ON_TIME? LATE*
- switch (previousTiming) {
- case EARLY:
- Preconditions.checkState(
- timing == Timing.EARLY || timing == Timing.ON_TIME || timing == Timing.LATE,
- "EARLY cannot transition to %s", timing);
- break;
- case ON_TIME:
- Preconditions.checkState(
- timing == Timing.LATE, "ON_TIME cannot transition to %s", timing);
- break;
- case LATE:
- Preconditions.checkState(timing == Timing.LATE, "LATE cannot transtion to %s", timing);
- break;
- case UNKNOWN:
- break;
- }
- Preconditions.checkState(!previousPane.isLast(), "Last pane was not last after all.");
- }
-
- return PaneInfo.createPane(isFirst, isFinal, timing, index, nonSpeculativeIndex);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java
deleted file mode 100644
index b1442dd..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.PCollectionView;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * A {@link DoFnRunner} that can refuse to process elements that are not ready, instead returning
- * them via the {@link #processElementInReadyWindows(WindowedValue)}.
- */
-public class PushbackSideInputDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
- private final DoFnRunner<InputT, OutputT> underlying;
- private final Collection<PCollectionView<?>> views;
- private final ReadyCheckingSideInputReader sideInputReader;
-
- private Set<BoundedWindow> notReadyWindows;
-
- public static <InputT, OutputT> PushbackSideInputDoFnRunner<InputT, OutputT> create(
- DoFnRunner<InputT, OutputT> underlying,
- Collection<PCollectionView<?>> views,
- ReadyCheckingSideInputReader sideInputReader) {
- return new PushbackSideInputDoFnRunner<>(underlying, views, sideInputReader);
- }
-
- private PushbackSideInputDoFnRunner(
- DoFnRunner<InputT, OutputT> underlying,
- Collection<PCollectionView<?>> views,
- ReadyCheckingSideInputReader sideInputReader) {
- this.underlying = underlying;
- this.views = views;
- this.sideInputReader = sideInputReader;
- }
-
- @Override
- public void startBundle() {
- notReadyWindows = new HashSet<>();
- underlying.startBundle();
- }
-
- /**
- * Call the underlying {@link DoFnRunner#processElement(WindowedValue)} for the provided element
- * for each window the element is in that is ready.
- *
- * @param elem the element to process in all ready windows
- * @return each element that could not be processed because it requires a side input window
- * that is not ready.
- */
- public Iterable<WindowedValue<InputT>> processElementInReadyWindows(WindowedValue<InputT> elem) {
- if (views.isEmpty()) {
- processElement(elem);
- return Collections.emptyList();
- }
- ImmutableList.Builder<WindowedValue<InputT>> pushedBack = ImmutableList.builder();
- for (WindowedValue<InputT> windowElem : elem.explodeWindows()) {
- BoundedWindow mainInputWindow = Iterables.getOnlyElement(windowElem.getWindows());
- boolean isReady = !notReadyWindows.contains(mainInputWindow);
- for (PCollectionView<?> view : views) {
- BoundedWindow sideInputWindow =
- view.getWindowingStrategyInternal()
- .getWindowFn()
- .getSideInputWindow(mainInputWindow);
- if (!sideInputReader.isReady(view, sideInputWindow)) {
- isReady = false;
- break;
- }
- }
- if (isReady) {
- processElement(windowElem);
- } else {
- notReadyWindows.add(mainInputWindow);
- pushedBack.add(windowElem);
- }
- }
- return pushedBack.build();
- }
-
- @Override
- public void processElement(WindowedValue<InputT> elem) {
- underlying.processElement(elem);
- }
-
- /**
- * Call the underlying {@link DoFnRunner#finishBundle()}.
- */
- @Override
- public void finishBundle() {
- notReadyWindows = null;
- underlying.finishBundle();
- }
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFn.java
deleted file mode 100644
index c5ee1e1..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFn.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.StateAccessor;
-
-import org.joda.time.Instant;
-
-import java.io.Serializable;
-
-/**
- * Specification for processing to happen after elements have been grouped by key.
- *
- * @param <K> The type of key being processed.
- * @param <InputT> The type of input values associated with the key.
- * @param <OutputT> The output type that will be produced for each key.
- * @param <W> The type of windows this operates on.
- */
-public abstract class ReduceFn<K, InputT, OutputT, W extends BoundedWindow>
- implements Serializable {
-
- /** Information accessible to all the processing methods in this {@code ReduceFn}. */
- public abstract class Context {
- /** Return the key that is being processed. */
- public abstract K key();
-
- /** The window that is being processed. */
- public abstract W window();
-
- /** Access the current {@link WindowingStrategy}. */
- public abstract WindowingStrategy<?, W> windowingStrategy();
-
- /** Return the interface for accessing state. */
- public abstract StateAccessor<K> state();
-
- /** Return the interface for accessing timers. */
- public abstract Timers timers();
- }
-
- /** Information accessible within {@link #processValue}. */
- public abstract class ProcessValueContext extends Context {
- /** Return the actual value being processed. */
- public abstract InputT value();
-
- /** Return the timestamp associated with the value. */
- public abstract Instant timestamp();
- }
-
- /** Information accessible within {@link #onMerge}. */
- public abstract class OnMergeContext extends Context {
- /** Return the interface for accessing state. */
- @Override
- public abstract MergingStateAccessor<K, W> state();
- }
-
- /** Information accessible within {@link #onTrigger}. */
- public abstract class OnTriggerContext extends Context {
- /** Returns the {@link PaneInfo} for the trigger firing being processed. */
- public abstract PaneInfo paneInfo();
-
- /** Output the given value in the current window. */
- public abstract void output(OutputT value);
- }
-
- //////////////////////////////////////////////////////////////////////////////////////////////////
-
- /**
- * Called for each value of type {@code InputT} associated with the current key.
- */
- public abstract void processValue(ProcessValueContext c) throws Exception;
-
- /**
- * Called when windows are merged.
- */
- public abstract void onMerge(OnMergeContext context) throws Exception;
-
- /**
- * Called when triggers fire.
- *
- * <p>Implementations of {@link ReduceFn} should call {@link OnTriggerContext#output} to emit
- * any results that should be included in the pane produced by this trigger firing.
- */
- public abstract void onTrigger(OnTriggerContext context) throws Exception;
-
- /**
- * Called before {@link #onMerge} is invoked to provide an opportunity to prefetch any needed
- * state.
- *
- * @param c Context to use prefetch from.
- */
- public void prefetchOnMerge(MergingStateAccessor<K, W> c) throws Exception {}
-
- /**
- * Called before {@link #onTrigger} is invoked to provide an opportunity to prefetch any needed
- * state.
- *
- * @param context Context to use prefetch from.
- */
- public void prefetchOnTrigger(StateAccessor<K> context) {}
-
- /**
- * Called to clear any persisted state that the {@link ReduceFn} may be holding. This will be
- * called when the windowing is closing and will receive no future interactions.
- */
- public abstract void clearState(Context context) throws Exception;
-
- /**
- * Returns true if the there is no buffered state.
- */
- public abstract ReadableState<Boolean> isEmpty(StateAccessor<K> context);
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
deleted file mode 100644
index c90940e..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
+++ /dev/null
@@ -1,497 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.State;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateContext;
-import org.apache.beam.sdk.util.state.StateContexts;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.util.state.StateNamespace;
-import org.apache.beam.sdk.util.state.StateNamespaces;
-import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace;
-import org.apache.beam.sdk.util.state.StateTag;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableMap;
-
-import org.joda.time.Instant;
-
-import java.util.Collection;
-import java.util.Map;
-
-import javax.annotation.Nullable;
-
-/**
- * Factory for creating instances of the various {@link ReduceFn} contexts.
- */
-class ReduceFnContextFactory<K, InputT, OutputT, W extends BoundedWindow> {
- public interface OnTriggerCallbacks<OutputT> {
- void output(OutputT toOutput);
- }
-
- private final K key;
- private final ReduceFn<K, InputT, OutputT, W> reduceFn;
- private final WindowingStrategy<?, W> windowingStrategy;
- private final StateInternals<K> stateInternals;
- private final ActiveWindowSet<W> activeWindows;
- private final TimerInternals timerInternals;
- private final WindowingInternals<?, ?> windowingInternals;
- private final PipelineOptions options;
-
- ReduceFnContextFactory(K key, ReduceFn<K, InputT, OutputT, W> reduceFn,
- WindowingStrategy<?, W> windowingStrategy, StateInternals<K> stateInternals,
- ActiveWindowSet<W> activeWindows, TimerInternals timerInternals,
- WindowingInternals<?, ?> windowingInternals, PipelineOptions options) {
- this.key = key;
- this.reduceFn = reduceFn;
- this.windowingStrategy = windowingStrategy;
- this.stateInternals = stateInternals;
- this.activeWindows = activeWindows;
- this.timerInternals = timerInternals;
- this.windowingInternals = windowingInternals;
- this.options = options;
- }
-
- /** Where should we look for state associated with a given window? */
- public static enum StateStyle {
- /** All state is associated with the window itself. */
- DIRECT,
- /** State is associated with the 'state address' windows tracked by the active window set. */
- RENAMED
- }
-
- private StateAccessorImpl<K, W> stateAccessor(W window, StateStyle style) {
- return new StateAccessorImpl<K, W>(
- activeWindows, windowingStrategy.getWindowFn().windowCoder(),
- stateInternals, StateContexts.createFromComponents(options, windowingInternals, window),
- style);
- }
-
- public ReduceFn<K, InputT, OutputT, W>.Context base(W window, StateStyle style) {
- return new ContextImpl(stateAccessor(window, style));
- }
-
- public ReduceFn<K, InputT, OutputT, W>.ProcessValueContext forValue(
- W window, InputT value, Instant timestamp, StateStyle style) {
- return new ProcessValueContextImpl(stateAccessor(window, style), value, timestamp);
- }
-
- public ReduceFn<K, InputT, OutputT, W>.OnTriggerContext forTrigger(W window,
- ReadableState<PaneInfo> pane, StateStyle style, OnTriggerCallbacks<OutputT> callbacks) {
- return new OnTriggerContextImpl(stateAccessor(window, style), pane, callbacks);
- }
-
- public ReduceFn<K, InputT, OutputT, W>.OnMergeContext forMerge(
- Collection<W> activeToBeMerged, W mergeResult, StateStyle style) {
- return new OnMergeContextImpl(
- new MergingStateAccessorImpl<K, W>(activeWindows,
- windowingStrategy.getWindowFn().windowCoder(),
- stateInternals, style, activeToBeMerged, mergeResult));
- }
-
- public ReduceFn<K, InputT, OutputT, W>.OnMergeContext forPremerge(W window) {
- return new OnPremergeContextImpl(new PremergingStateAccessorImpl<K, W>(
- activeWindows, windowingStrategy.getWindowFn().windowCoder(), stateInternals, window));
- }
-
- private class TimersImpl implements Timers {
- private final StateNamespace namespace;
-
- public TimersImpl(StateNamespace namespace) {
- Preconditions.checkArgument(namespace instanceof WindowNamespace);
- this.namespace = namespace;
- }
-
- @Override
- public void setTimer(Instant timestamp, TimeDomain timeDomain) {
- timerInternals.setTimer(TimerData.of(namespace, timestamp, timeDomain));
- }
-
- @Override
- public void deleteTimer(Instant timestamp, TimeDomain timeDomain) {
- timerInternals.deleteTimer(TimerData.of(namespace, timestamp, timeDomain));
- }
-
- @Override
- public Instant currentProcessingTime() {
- return timerInternals.currentProcessingTime();
- }
-
- @Override
- @Nullable
- public Instant currentSynchronizedProcessingTime() {
- return timerInternals.currentSynchronizedProcessingTime();
- }
-
- @Override
- public Instant currentEventTime() {
- return timerInternals.currentInputWatermarkTime();
- }
- }
-
- // ======================================================================
- // StateAccessors
- // ======================================================================
- static class StateAccessorImpl<K, W extends BoundedWindow> implements StateAccessor<K> {
-
-
- protected final ActiveWindowSet<W> activeWindows;
- protected final StateContext<W> context;
- protected final StateNamespace windowNamespace;
- protected final Coder<W> windowCoder;
- protected final StateInternals<K> stateInternals;
- protected final StateStyle style;
-
- public StateAccessorImpl(ActiveWindowSet<W> activeWindows, Coder<W> windowCoder,
- StateInternals<K> stateInternals, StateContext<W> context, StateStyle style) {
-
- this.activeWindows = activeWindows;
- this.windowCoder = windowCoder;
- this.stateInternals = stateInternals;
- this.context = checkNotNull(context);
- this.windowNamespace = namespaceFor(context.window());
- this.style = style;
- }
-
- protected StateNamespace namespaceFor(W window) {
- return StateNamespaces.window(windowCoder, window);
- }
-
- protected StateNamespace windowNamespace() {
- return windowNamespace;
- }
-
- W window() {
- return context.window();
- }
-
- StateNamespace namespace() {
- return windowNamespace();
- }
-
- @Override
- public <StateT extends State> StateT access(StateTag<? super K, StateT> address) {
- switch (style) {
- case DIRECT:
- return stateInternals.state(windowNamespace(), address, context);
- case RENAMED:
- return stateInternals.state(
- namespaceFor(activeWindows.writeStateAddress(context.window())), address, context);
- }
- throw new RuntimeException(); // cases are exhaustive.
- }
- }
-
- static class MergingStateAccessorImpl<K, W extends BoundedWindow>
- extends StateAccessorImpl<K, W> implements MergingStateAccessor<K, W> {
- private final Collection<W> activeToBeMerged;
-
- public MergingStateAccessorImpl(ActiveWindowSet<W> activeWindows, Coder<W> windowCoder,
- StateInternals<K> stateInternals, StateStyle style, Collection<W> activeToBeMerged,
- W mergeResult) {
- super(activeWindows, windowCoder, stateInternals,
- StateContexts.windowOnly(mergeResult), style);
- this.activeToBeMerged = activeToBeMerged;
- }
-
- @Override
- public <StateT extends State> StateT access(StateTag<? super K, StateT> address) {
- switch (style) {
- case DIRECT:
- return stateInternals.state(windowNamespace(), address, context);
- case RENAMED:
- return stateInternals.state(
- namespaceFor(activeWindows.mergedWriteStateAddress(
- activeToBeMerged, context.window())),
- address,
- context);
- }
- throw new RuntimeException(); // cases are exhaustive.
- }
-
- @Override
- public <StateT extends State> Map<W, StateT> accessInEachMergingWindow(
- StateTag<? super K, StateT> address) {
- ImmutableMap.Builder<W, StateT> builder = ImmutableMap.builder();
- for (W mergingWindow : activeToBeMerged) {
- StateNamespace namespace = null;
- switch (style) {
- case DIRECT:
- namespace = namespaceFor(mergingWindow);
- break;
- case RENAMED:
- namespace = namespaceFor(activeWindows.writeStateAddress(mergingWindow));
- break;
- }
- Preconditions.checkNotNull(namespace); // cases are exhaustive.
- builder.put(mergingWindow, stateInternals.state(namespace, address, context));
- }
- return builder.build();
- }
- }
-
- static class PremergingStateAccessorImpl<K, W extends BoundedWindow>
- extends StateAccessorImpl<K, W> implements MergingStateAccessor<K, W> {
- public PremergingStateAccessorImpl(ActiveWindowSet<W> activeWindows, Coder<W> windowCoder,
- StateInternals<K> stateInternals, W window) {
- super(activeWindows, windowCoder, stateInternals,
- StateContexts.windowOnly(window), StateStyle.RENAMED);
- }
-
- Collection<W> mergingWindows() {
- return activeWindows.readStateAddresses(context.window());
- }
-
- @Override
- public <StateT extends State> Map<W, StateT> accessInEachMergingWindow(
- StateTag<? super K, StateT> address) {
- ImmutableMap.Builder<W, StateT> builder = ImmutableMap.builder();
- for (W stateAddressWindow : activeWindows.readStateAddresses(context.window())) {
- StateT stateForWindow =
- stateInternals.state(namespaceFor(stateAddressWindow), address, context);
- builder.put(stateAddressWindow, stateForWindow);
- }
- return builder.build();
- }
- }
-
- // ======================================================================
- // Contexts
- // ======================================================================
-
- private class ContextImpl extends ReduceFn<K, InputT, OutputT, W>.Context {
- private final StateAccessorImpl<K, W> state;
- private final TimersImpl timers;
-
- private ContextImpl(StateAccessorImpl<K, W> state) {
- reduceFn.super();
- this.state = state;
- this.timers = new TimersImpl(state.namespace());
- }
-
- @Override
- public K key() {
- return key;
- }
-
- @Override
- public W window() {
- return state.window();
- }
-
- @Override
- public WindowingStrategy<?, W> windowingStrategy() {
- return windowingStrategy;
- }
-
- @Override
- public StateAccessor<K> state() {
- return state;
- }
-
- @Override
- public Timers timers() {
- return timers;
- }
- }
-
- private class ProcessValueContextImpl
- extends ReduceFn<K, InputT, OutputT, W>.ProcessValueContext {
- private final InputT value;
- private final Instant timestamp;
- private final StateAccessorImpl<K, W> state;
- private final TimersImpl timers;
-
- private ProcessValueContextImpl(StateAccessorImpl<K, W> state,
- InputT value, Instant timestamp) {
- reduceFn.super();
- this.state = state;
- this.value = value;
- this.timestamp = timestamp;
- this.timers = new TimersImpl(state.namespace());
- }
-
- @Override
- public K key() {
- return key;
- }
-
- @Override
- public W window() {
- return state.window();
- }
-
- @Override
- public WindowingStrategy<?, W> windowingStrategy() {
- return windowingStrategy;
- }
-
- @Override
- public StateAccessor<K> state() {
- return state;
- }
-
- @Override
- public InputT value() {
- return value;
- }
-
- @Override
- public Instant timestamp() {
- return timestamp;
- }
-
- @Override
- public Timers timers() {
- return timers;
- }
- }
-
- private class OnTriggerContextImpl extends ReduceFn<K, InputT, OutputT, W>.OnTriggerContext {
- private final StateAccessorImpl<K, W> state;
- private final ReadableState<PaneInfo> pane;
- private final OnTriggerCallbacks<OutputT> callbacks;
- private final TimersImpl timers;
-
- private OnTriggerContextImpl(StateAccessorImpl<K, W> state, ReadableState<PaneInfo> pane,
- OnTriggerCallbacks<OutputT> callbacks) {
- reduceFn.super();
- this.state = state;
- this.pane = pane;
- this.callbacks = callbacks;
- this.timers = new TimersImpl(state.namespace());
- }
-
- @Override
- public K key() {
- return key;
- }
-
- @Override
- public W window() {
- return state.window();
- }
-
- @Override
- public WindowingStrategy<?, W> windowingStrategy() {
- return windowingStrategy;
- }
-
- @Override
- public StateAccessor<K> state() {
- return state;
- }
-
- @Override
- public PaneInfo paneInfo() {
- return pane.read();
- }
-
- @Override
- public void output(OutputT value) {
- callbacks.output(value);
- }
-
- @Override
- public Timers timers() {
- return timers;
- }
- }
-
- private class OnMergeContextImpl extends ReduceFn<K, InputT, OutputT, W>.OnMergeContext {
- private final MergingStateAccessorImpl<K, W> state;
- private final TimersImpl timers;
-
- private OnMergeContextImpl(MergingStateAccessorImpl<K, W> state) {
- reduceFn.super();
- this.state = state;
- this.timers = new TimersImpl(state.namespace());
- }
-
- @Override
- public K key() {
- return key;
- }
-
- @Override
- public WindowingStrategy<?, W> windowingStrategy() {
- return windowingStrategy;
- }
-
- @Override
- public MergingStateAccessor<K, W> state() {
- return state;
- }
-
- @Override
- public W window() {
- return state.window();
- }
-
- @Override
- public Timers timers() {
- return timers;
- }
- }
-
- private class OnPremergeContextImpl extends ReduceFn<K, InputT, OutputT, W>.OnMergeContext {
- private final PremergingStateAccessorImpl<K, W> state;
- private final TimersImpl timers;
-
- private OnPremergeContextImpl(PremergingStateAccessorImpl<K, W> state) {
- reduceFn.super();
- this.state = state;
- this.timers = new TimersImpl(state.namespace());
- }
-
- @Override
- public K key() {
- return key;
- }
-
- @Override
- public WindowingStrategy<?, W> windowingStrategy() {
- return windowingStrategy;
- }
-
- @Override
- public MergingStateAccessor<K, W> state() {
- return state;
- }
-
- @Override
- public W window() {
- return state.window();
- }
-
- @Override
- public Timers timers() {
- return timers;
- }
- }
-}
[07/12] incubator-beam git commit: Move some easy stuff into
runners/core-java
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
new file mode 100644
index 0000000..b7ec540
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
@@ -0,0 +1,1448 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import static org.apache.beam.sdk.WindowMatchers.isSingleWindowedValue;
+import static org.apache.beam.sdk.WindowMatchers.isWindowedValue;
+
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.withSettings;
+
+import org.apache.beam.sdk.WindowMatchers;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext;
+import org.apache.beam.sdk.transforms.CombineWithContext.Context;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.AfterEach;
+import org.apache.beam.sdk.transforms.windowing.AfterFirst;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Never;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Matchers;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Tests for {@link ReduceFnRunner}. These tests instantiate a full "stack" of
+ * {@link ReduceFnRunner} with enclosed {@link ReduceFn}, down to the installed {@link Trigger}
+ * (sometimes mocked). They proceed by injecting elements and advancing watermark and
+ * processing time, then verifying produced panes and counters.
+ */
+@RunWith(JUnit4.class)
+public class ReduceFnRunnerTest {
+ @Mock private SideInputReader mockSideInputReader;
+ private Trigger mockTrigger;
+ private PCollectionView<Integer> mockView;
+
+ private IntervalWindow firstWindow;
+
+ private static Trigger.TriggerContext anyTriggerContext() {
+ return Mockito.<Trigger.TriggerContext>any();
+ }
+ private static Trigger.OnElementContext anyElementContext() {
+ return Mockito.<Trigger.OnElementContext>any();
+ }
+
+ @Before
+ public void setUp() {
+ MockitoAnnotations.initMocks(this);
+
+ mockTrigger = mock(Trigger.class, withSettings().serializable());
+ when(mockTrigger.buildTrigger()).thenReturn(mockTrigger);
+
+ @SuppressWarnings("unchecked")
+ PCollectionView<Integer> mockViewUnchecked =
+ mock(PCollectionView.class, withSettings().serializable());
+ mockView = mockViewUnchecked;
+ firstWindow = new IntervalWindow(new Instant(0), new Instant(10));
+ }
+
+ private void injectElement(ReduceFnTester<Integer, ?, IntervalWindow> tester, int element)
+ throws Exception {
+ doNothing().when(mockTrigger).onElement(anyElementContext());
+ tester.injectElements(TimestampedValue.of(element, new Instant(element)));
+ }
+
+ private void triggerShouldFinish(Trigger mockTrigger) throws Exception {
+ doAnswer(new Answer<Void>() {
+ @Override
+ public Void answer(InvocationOnMock invocation) throws Exception {
+ @SuppressWarnings("unchecked")
+ Trigger.TriggerContext context =
+ (Trigger.TriggerContext) invocation.getArguments()[0];
+ context.trigger().setFinished(true);
+ return null;
+ }
+ })
+ .when(mockTrigger).onFire(anyTriggerContext());
+ }
+
+ @Test
+ public void testOnElementBufferingDiscarding() throws Exception {
+ // Test basic execution of a trigger using a non-combining window set and discarding mode.
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+ AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ // Pane of {1, 2}
+ injectElement(tester, 1);
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ injectElement(tester, 2);
+ assertThat(tester.extractOutput(),
+ contains(isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)));
+
+ // Pane of just 3, and finish
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ injectElement(tester, 3);
+ assertThat(tester.extractOutput(),
+ contains(isSingleWindowedValue(containsInAnyOrder(3), 3, 0, 10)));
+ assertTrue(tester.isMarkedFinished(firstWindow));
+ tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
+
+ // This element shouldn't be seen, because the trigger has finished
+ injectElement(tester, 4);
+
+ assertEquals(1, tester.getElementsDroppedDueToClosedWindow());
+ }
+
+ @Test
+ public void testOnElementBufferingAccumulating() throws Exception {
+ // Test basic execution of a trigger using a non-combining window set and accumulating mode.
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+ AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(100),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ injectElement(tester, 1);
+
+ // Fires {1, 2}
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ injectElement(tester, 2);
+
+ // Fires {1, 2, 3} because we are in accumulating mode
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ injectElement(tester, 3);
+
+ // This element shouldn't be seen, because the trigger has finished
+ injectElement(tester, 4);
+
+ assertThat(
+ tester.extractOutput(),
+ contains(
+ isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10),
+ isSingleWindowedValue(containsInAnyOrder(1, 2, 3), 3, 0, 10)));
+ assertTrue(tester.isMarkedFinished(firstWindow));
+ tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
+ }
+
+ @Test
+ public void testOnElementCombiningDiscarding() throws Exception {
+ // Test basic execution of a trigger using a non-combining window set and discarding mode.
+ ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining(
+ FixedWindows.of(Duration.millis(10)), mockTrigger, AccumulationMode.DISCARDING_FIRED_PANES,
+ new Sum.SumIntegerFn().<String>asKeyedFn(), VarIntCoder.of(), Duration.millis(100));
+
+ injectElement(tester, 2);
+
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ injectElement(tester, 3);
+
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ injectElement(tester, 4);
+
+ // This element shouldn't be seen, because the trigger has finished
+ injectElement(tester, 6);
+
+ assertThat(
+ tester.extractOutput(),
+ contains(
+ isSingleWindowedValue(equalTo(5), 2, 0, 10),
+ isSingleWindowedValue(equalTo(4), 4, 0, 10)));
+ assertTrue(tester.isMarkedFinished(firstWindow));
+ tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
+ }
+
+ /**
+ * Tests that the garbage collection time for a fixed window does not overflow the end of time.
+ */
+ @Test
+ public void testFixedWindowEndOfTimeGarbageCollection() throws Exception {
+
+ Duration allowedLateness = Duration.standardDays(365);
+ Duration windowSize = Duration.millis(10);
+ WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(windowSize);
+
+ // This timestamp falls into a window where the end of the window is before the end of the
+ // global window - the "end of time" - yet its expiration time is after.
+ final Instant elementTimestamp =
+ GlobalWindow.INSTANCE.maxTimestamp().minus(allowedLateness).plus(1);
+
+ IntervalWindow window = Iterables.getOnlyElement(
+ windowFn.assignWindows(
+ windowFn.new AssignContext() {
+ @Override
+ public Object element() {
+ throw new UnsupportedOperationException();
+ }
+ @Override
+ public Instant timestamp() {
+ return elementTimestamp;
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ throw new UnsupportedOperationException();
+ }
+ }));
+
+ assertTrue(
+ window.maxTimestamp().isBefore(GlobalWindow.INSTANCE.maxTimestamp()));
+ assertTrue(
+ window.maxTimestamp().plus(allowedLateness).isAfter(GlobalWindow.INSTANCE.maxTimestamp()));
+
+ // Test basic execution of a trigger using a non-combining window set and accumulating mode.
+ ReduceFnTester<Integer, Integer, IntervalWindow> tester =
+ ReduceFnTester.combining(
+ windowFn,
+ AfterWatermark.pastEndOfWindow().withLateFirings(Never.ever()).buildTrigger(),
+ AccumulationMode.DISCARDING_FIRED_PANES,
+ new Sum.SumIntegerFn().<String>asKeyedFn(),
+ VarIntCoder.of(),
+ allowedLateness);
+
+ tester.injectElements(TimestampedValue.of(13, elementTimestamp));
+
+ // Should fire ON_TIME pane and there will be a checkState that the cleanup time
+ // is prior to timestamp max value
+ tester.advanceInputWatermark(window.maxTimestamp());
+
+ // Nothing in the ON_TIME pane (not governed by triggers, but by ReduceFnRunner)
+ assertThat(tester.extractOutput(), emptyIterable());
+
+ tester.injectElements(TimestampedValue.of(42, elementTimestamp));
+
+ // Now the final pane should fire, demonstrating that the GC time was truncated
+ tester.advanceInputWatermark(GlobalWindow.INSTANCE.maxTimestamp());
+ assertThat(tester.extractOutput(), contains(isWindowedValue(equalTo(55))));
+ }
+
+ @Test
+ public void testOnElementCombiningAccumulating() throws Exception {
+ // Test basic execution of a trigger using a non-combining window set and accumulating mode.
+ ReduceFnTester<Integer, Integer, IntervalWindow> tester =
+ ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+ AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().<String>asKeyedFn(),
+ VarIntCoder.of(), Duration.millis(100));
+
+ injectElement(tester, 1);
+
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ injectElement(tester, 2);
+
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ injectElement(tester, 3);
+
+ // This element shouldn't be seen, because the trigger has finished
+ injectElement(tester, 4);
+
+ assertThat(
+ tester.extractOutput(),
+ contains(
+ isSingleWindowedValue(equalTo(3), 1, 0, 10),
+ isSingleWindowedValue(equalTo(6), 3, 0, 10)));
+ assertTrue(tester.isMarkedFinished(firstWindow));
+ tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
+ }
+
+ @Test
+ public void testOnElementCombiningWithContext() throws Exception {
+ Integer expectedValue = 5;
+ WindowingStrategy<?, IntervalWindow> windowingStrategy = WindowingStrategy
+ .of(FixedWindows.of(Duration.millis(10)))
+ .withTrigger(mockTrigger)
+ .withMode(AccumulationMode.DISCARDING_FIRED_PANES)
+ .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
+ .withAllowedLateness(Duration.millis(100));
+
+ TestOptions options = PipelineOptionsFactory.as(TestOptions.class);
+ options.setValue(5);
+
+ when(mockSideInputReader.contains(Matchers.<PCollectionView<Integer>>any())).thenReturn(true);
+ when(mockSideInputReader.get(
+ Matchers.<PCollectionView<Integer>>any(), any(BoundedWindow.class))).thenReturn(5);
+
+ @SuppressWarnings({"rawtypes", "unchecked", "unused"})
+ Object suppressWarningsVar = when(mockView.getWindowingStrategyInternal())
+ .thenReturn((WindowingStrategy) windowingStrategy);
+
+ SumAndVerifyContextFn combineFn = new SumAndVerifyContextFn(mockView, expectedValue);
+ // Test basic execution of a trigger using a non-combining window set and discarding mode.
+ ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining(
+ windowingStrategy, combineFn.<String>asKeyedFn(),
+ VarIntCoder.of(), options, mockSideInputReader);
+
+ injectElement(tester, 2);
+
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ injectElement(tester, 3);
+
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ injectElement(tester, 4);
+
+ // This element shouldn't be seen, because the trigger has finished
+ injectElement(tester, 6);
+
+ assertThat(
+ tester.extractOutput(),
+ contains(
+ isSingleWindowedValue(equalTo(5), 2, 0, 10),
+ isSingleWindowedValue(equalTo(4), 4, 0, 10)));
+ assertTrue(tester.isMarkedFinished(firstWindow));
+ tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
+ }
+
+ @Test
+ public void testWatermarkHoldAndLateData() throws Exception {
+ // Test handling of late data. Specifically, ensure the watermark hold is correct.
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+ AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(10),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ // Input watermark -> null
+ assertEquals(null, tester.getWatermarkHold());
+ assertEquals(null, tester.getOutputWatermark());
+
+ // All on time data, verify watermark hold.
+ injectElement(tester, 1);
+ injectElement(tester, 3);
+ assertEquals(new Instant(1), tester.getWatermarkHold());
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ injectElement(tester, 2);
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+ assertThat(output, contains(
+ isSingleWindowedValue(containsInAnyOrder(1, 2, 3),
+ 1, // timestamp
+ 0, // window start
+ 10))); // window end
+ assertThat(output.get(0).getPane(),
+ equalTo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)));
+
+ // Holding for the end-of-window transition.
+ assertEquals(new Instant(9), tester.getWatermarkHold());
+ // Nothing dropped.
+ assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
+
+ // Input watermark -> 4, output watermark should advance that far as well
+ tester.advanceInputWatermark(new Instant(4));
+ assertEquals(new Instant(4), tester.getOutputWatermark());
+
+ // Some late, some on time. Verify that we only hold to the minimum of on-time.
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
+ tester.advanceInputWatermark(new Instant(4));
+ injectElement(tester, 2);
+ injectElement(tester, 3);
+ assertEquals(new Instant(9), tester.getWatermarkHold());
+ injectElement(tester, 5);
+ assertEquals(new Instant(5), tester.getWatermarkHold());
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ injectElement(tester, 4);
+ output = tester.extractOutput();
+ assertThat(output,
+ contains(
+ isSingleWindowedValue(containsInAnyOrder(
+ 1, 2, 3, // earlier firing
+ 2, 3, 4, 5), // new elements
+ 4, // timestamp
+ 0, // window start
+ 10))); // window end
+ assertThat(output.get(0).getPane(),
+ equalTo(PaneInfo.createPane(false, false, Timing.EARLY, 1, -1)));
+
+ // All late -- output at end of window timestamp.
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
+ tester.advanceInputWatermark(new Instant(8));
+ injectElement(tester, 6);
+ injectElement(tester, 5);
+ assertEquals(new Instant(9), tester.getWatermarkHold());
+ injectElement(tester, 4);
+
+ // Fire the ON_TIME pane
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ tester.advanceInputWatermark(new Instant(10));
+
+ // Output time is end of the window, because all the new data was late, but the pane
+ // is the ON_TIME pane.
+ output = tester.extractOutput();
+ assertThat(output,
+ contains(isSingleWindowedValue(
+ containsInAnyOrder(1, 2, 3, // earlier firing
+ 2, 3, 4, 5, // earlier firing
+ 4, 5, 6), // new elements
+ 9, // timestamp
+ 0, // window start
+ 10))); // window end
+ assertThat(output.get(0).getPane(),
+ equalTo(PaneInfo.createPane(false, false, Timing.ON_TIME, 2, 0)));
+
+ // This is "pending" at the time the watermark makes it way-late.
+ // Because we're about to expire the window, we output it.
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
+ injectElement(tester, 8);
+ assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
+
+ // Exceed the GC limit, triggering the last pane to be fired
+ tester.advanceInputWatermark(new Instant(50));
+ output = tester.extractOutput();
+ // Output time is still end of the window, because the new data (8) was behind
+ // the output watermark.
+ assertThat(output,
+ contains(isSingleWindowedValue(
+ containsInAnyOrder(1, 2, 3, // earlier firing
+ 2, 3, 4, 5, // earlier firing
+ 4, 5, 6, // earlier firing
+ 8), // new element prior to window becoming expired
+ 9, // timestamp
+ 0, // window start
+ 10))); // window end
+ assertThat(
+ output.get(0).getPane(),
+ equalTo(PaneInfo.createPane(false, true, Timing.LATE, 3, 1)));
+ assertEquals(new Instant(50), tester.getOutputWatermark());
+ assertEquals(null, tester.getWatermarkHold());
+
+ // Late timers are ignored
+ tester.fireTimer(new IntervalWindow(new Instant(0), new Instant(10)), new Instant(12),
+ TimeDomain.EVENT_TIME);
+
+ // And because we're past the end of window + allowed lateness, everything should be cleaned up.
+ assertFalse(tester.isMarkedFinished(firstWindow));
+ tester.assertHasOnlyGlobalAndFinishedSetsFor();
+ }
+
+ @Test
+ public void dontSetHoldIfTooLateForEndOfWindowTimer() throws Exception {
+ // Make sure holds are only set if they are accompanied by an end-of-window timer.
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+ AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(10),
+ ClosingBehavior.FIRE_ALWAYS);
+ tester.setAutoAdvanceOutputWatermark(false);
+
+ // Case: Unobservably late
+ tester.advanceInputWatermark(new Instant(15));
+ tester.advanceOutputWatermark(new Instant(11));
+ injectElement(tester, 14);
+ // Hold was applied, waiting for end-of-window timer.
+ assertEquals(new Instant(14), tester.getWatermarkHold());
+ assertEquals(new Instant(19), tester.getNextTimer(TimeDomain.EVENT_TIME));
+
+ // Trigger the end-of-window timer.
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ tester.advanceInputWatermark(new Instant(20));
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
+ // Hold has been replaced with garbage collection hold. Waiting for garbage collection.
+ assertEquals(new Instant(29), tester.getWatermarkHold());
+ assertEquals(new Instant(29), tester.getNextTimer(TimeDomain.EVENT_TIME));
+
+ // Case: Maybe late 1
+ injectElement(tester, 13);
+ // No change to hold or timers.
+ assertEquals(new Instant(29), tester.getWatermarkHold());
+ assertEquals(new Instant(29), tester.getNextTimer(TimeDomain.EVENT_TIME));
+
+ // Trigger the garbage collection timer.
+ tester.advanceInputWatermark(new Instant(30));
+
+ // Everything should be cleaned up.
+ assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(10), new Instant(20))));
+ tester.assertHasOnlyGlobalAndFinishedSetsFor();
+ }
+
+ @Test
+ public void testPaneInfoAllStates() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+ AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ tester.advanceInputWatermark(new Instant(0));
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ injectElement(tester, 1);
+ assertThat(tester.extractOutput(), contains(
+ WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY))));
+
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ injectElement(tester, 2);
+ assertThat(tester.extractOutput(), contains(
+ WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.EARLY, 1, -1))));
+
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
+ tester.advanceInputWatermark(new Instant(15));
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ injectElement(tester, 3);
+ assertThat(tester.extractOutput(), contains(
+ WindowMatchers.valueWithPaneInfo(
+ PaneInfo.createPane(false, false, Timing.ON_TIME, 2, 0))));
+
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ injectElement(tester, 4);
+ assertThat(tester.extractOutput(), contains(
+ WindowMatchers.valueWithPaneInfo(
+ PaneInfo.createPane(false, false, Timing.LATE, 3, 1))));
+
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ injectElement(tester, 5);
+ assertThat(tester.extractOutput(), contains(
+ WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 4, 2))));
+ }
+
+ @Test
+ public void testPaneInfoAllStatesAfterWatermark() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
+ WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
+ .withTrigger(Repeatedly.forever(AfterFirst.of(
+ AfterPane.elementCountAtLeast(2),
+ AfterWatermark.pastEndOfWindow())))
+ .withMode(AccumulationMode.DISCARDING_FIRED_PANES)
+ .withAllowedLateness(Duration.millis(100))
+ .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
+ .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
+
+ tester.advanceInputWatermark(new Instant(0));
+ tester.injectElements(
+ TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2)));
+
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+ assertThat(
+ output,
+ contains(WindowMatchers.valueWithPaneInfo(
+ PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))));
+ assertThat(
+ output,
+ contains(
+ WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)));
+
+ tester.advanceInputWatermark(new Instant(50));
+
+ // We should get the ON_TIME pane even though it is empty,
+ // because we have an AfterWatermark.pastEndOfWindow() trigger.
+ output = tester.extractOutput();
+ assertThat(
+ output,
+ contains(WindowMatchers.valueWithPaneInfo(
+ PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0))));
+ assertThat(
+ output,
+ contains(
+ WindowMatchers.isSingleWindowedValue(emptyIterable(), 9, 0, 10)));
+
+ // We should get the final pane even though it is empty.
+ tester.advanceInputWatermark(new Instant(150));
+ output = tester.extractOutput();
+ assertThat(
+ output,
+ contains(
+ WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 2, 1))));
+ assertThat(
+ output,
+ contains(
+ WindowMatchers.isSingleWindowedValue(emptyIterable(), 9, 0, 10)));
+ }
+
+ @Test
+ public void noEmptyPanesFinalIfNonEmpty() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
+ WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
+ .withTrigger(Repeatedly.<IntervalWindow>forever(AfterFirst.<IntervalWindow>of(
+ AfterPane.elementCountAtLeast(2),
+ AfterWatermark.pastEndOfWindow())))
+ .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES)
+ .withAllowedLateness(Duration.millis(100))
+ .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
+ .withClosingBehavior(ClosingBehavior.FIRE_IF_NON_EMPTY));
+
+ tester.advanceInputWatermark(new Instant(0));
+ tester.injectElements(
+ TimestampedValue.of(1, new Instant(1)),
+ TimestampedValue.of(2, new Instant(2)));
+ tester.advanceInputWatermark(new Instant(20));
+ tester.advanceInputWatermark(new Instant(250));
+
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+ assertThat(output, contains(
+ // Trigger with 2 elements
+ WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10),
+ // Trigger for the empty on time pane
+ WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
+ }
+
+ @Test
+ public void noEmptyPanesFinalAlways() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
+ WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
+ .withTrigger(Repeatedly.<IntervalWindow>forever(AfterFirst.<IntervalWindow>of(
+ AfterPane.elementCountAtLeast(2),
+ AfterWatermark.pastEndOfWindow())))
+ .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES)
+ .withAllowedLateness(Duration.millis(100))
+ .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
+ .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
+
+ tester.advanceInputWatermark(new Instant(0));
+ tester.injectElements(
+ TimestampedValue.of(1, new Instant(1)),
+ TimestampedValue.of(2, new Instant(2)));
+ tester.advanceInputWatermark(new Instant(20));
+ tester.advanceInputWatermark(new Instant(250));
+
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+ assertThat(output, contains(
+ // Trigger with 2 elements
+ WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10),
+ // Trigger for the empty on time pane
+ WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10),
+ // Trigger for the final pane
+ WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
+ }
+
+ @Test
+ public void testPaneInfoAllStatesAfterWatermarkAccumulating() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
+ WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
+ .withTrigger(Repeatedly.forever(AfterFirst.of(
+ AfterPane.elementCountAtLeast(2),
+ AfterWatermark.pastEndOfWindow())))
+ .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES)
+ .withAllowedLateness(Duration.millis(100))
+ .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
+ .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
+
+ tester.advanceInputWatermark(new Instant(0));
+ tester.injectElements(
+ TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2)));
+
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+ assertThat(
+ output,
+ contains(WindowMatchers.valueWithPaneInfo(
+ PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))));
+ assertThat(
+ output,
+ contains(
+ WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)));
+
+ tester.advanceInputWatermark(new Instant(50));
+
+ // We should get the ON_TIME pane even though it is empty,
+ // because we have an AfterWatermark.pastEndOfWindow() trigger.
+ output = tester.extractOutput();
+ assertThat(
+ output,
+ contains(WindowMatchers.valueWithPaneInfo(
+ PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0))));
+ assertThat(
+ output,
+ contains(
+ WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
+
+ // We should get the final pane even though it is empty.
+ tester.advanceInputWatermark(new Instant(150));
+ output = tester.extractOutput();
+ assertThat(
+ output,
+ contains(
+ WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 2, 1))));
+ assertThat(
+ output,
+ contains(
+ WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
+ }
+
+ @Test
+ public void testPaneInfoFinalAndOnTime() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
+ WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
+ .withTrigger(
+ Repeatedly.forever(AfterPane.elementCountAtLeast(2))
+ .orFinally(AfterWatermark.pastEndOfWindow()))
+ .withMode(AccumulationMode.DISCARDING_FIRED_PANES)
+ .withAllowedLateness(Duration.millis(100))
+ .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
+
+ tester.advanceInputWatermark(new Instant(0));
+
+ // Should trigger due to element count
+ tester.injectElements(
+ TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2)));
+
+ assertThat(
+ tester.extractOutput(),
+ contains(WindowMatchers.valueWithPaneInfo(
+ PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))));
+
+ tester.advanceInputWatermark(new Instant(150));
+ assertThat(tester.extractOutput(), contains(
+ WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.ON_TIME, 1, 0))));
+ }
+
+ @Test
+ public void testPaneInfoSkipToFinish() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+ AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ tester.advanceInputWatermark(new Instant(0));
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ injectElement(tester, 1);
+ assertThat(tester.extractOutput(), contains(
+ WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.EARLY))));
+ }
+
+ @Test
+ public void testPaneInfoSkipToNonSpeculativeAndFinish() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+ AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ tester.advanceInputWatermark(new Instant(15));
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ injectElement(tester, 1);
+ assertThat(tester.extractOutput(), contains(
+ WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.LATE))));
+ }
+
+ @Test
+ public void testMergeBeforeFinalizing() throws Exception {
+ // Verify that we merge windows before producing output so users don't see undesired
+ // unmerged windows.
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
+ AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(0),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ // All on time data, verify watermark hold.
+ // These two windows should pre-merge immediately to [1, 20)
+ tester.injectElements(
+ TimestampedValue.of(1, new Instant(1)), // in [1, 11)
+ TimestampedValue.of(10, new Instant(10))); // in [10, 20)
+
+ // And this should fire the end-of-window timer
+ tester.advanceInputWatermark(new Instant(100));
+
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+ assertThat(output.size(), equalTo(1));
+ assertThat(output.get(0),
+ isSingleWindowedValue(containsInAnyOrder(1, 10),
+ 1, // timestamp
+ 1, // window start
+ 20)); // window end
+ assertThat(
+ output.get(0).getPane(),
+ equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
+ }
+
+ /**
+ * It is possible for a session window's trigger to be closed at the point at which
+ * the (merged) session window is garbage collected. Make sure we don't accidentally
+ * assume the window is still active.
+ */
+ @Test
+ public void testMergingWithCloseBeforeGC() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
+ AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ // Two elements in two overlapping session windows.
+ tester.injectElements(
+ TimestampedValue.of(1, new Instant(1)), // in [1, 11)
+ TimestampedValue.of(10, new Instant(10))); // in [10, 20)
+
+ // Close the trigger, but the gargbage collection timer is still pending.
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ tester.advanceInputWatermark(new Instant(30));
+
+ // Now the garbage collection timer will fire, finding the trigger already closed.
+ tester.advanceInputWatermark(new Instant(100));
+
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+ assertThat(output.size(), equalTo(1));
+ assertThat(output.get(0),
+ isSingleWindowedValue(containsInAnyOrder(1, 10),
+ 1, // timestamp
+ 1, // window start
+ 20)); // window end
+ assertThat(
+ output.get(0).getPane(),
+ equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
+ }
+
+ /**
+ * Ensure a closed trigger has its state recorded in the merge result window.
+ */
+ @Test
+ public void testMergingWithCloseTrigger() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
+ AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ // Create a new merged session window.
+ tester.injectElements(TimestampedValue.of(1, new Instant(1)),
+ TimestampedValue.of(2, new Instant(2)));
+
+ // Force the trigger to be closed for the merged window.
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ tester.advanceInputWatermark(new Instant(13));
+
+ // Trigger is now closed.
+ assertTrue(tester.isMarkedFinished(new IntervalWindow(new Instant(1), new Instant(12))));
+
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
+
+ // Revisit the same session window.
+ tester.injectElements(TimestampedValue.of(1, new Instant(1)),
+ TimestampedValue.of(2, new Instant(2)));
+
+ // Trigger is still closed.
+ assertTrue(tester.isMarkedFinished(new IntervalWindow(new Instant(1), new Instant(12))));
+ }
+
+ /**
+ * If a later event tries to reuse an earlier session window which has been closed, we
+ * should reject that element and not fail due to the window no longer being active.
+ */
+ @Test
+ public void testMergingWithReusedWindow() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
+ AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ // One elements in one session window.
+ tester.injectElements(TimestampedValue.of(1, new Instant(1))); // in [1, 11), gc at 21.
+
+ // Close the trigger, but the gargbage collection timer is still pending.
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ tester.advanceInputWatermark(new Instant(15));
+
+ // Another element in the same session window.
+ // Should be discarded with 'window closed'.
+ tester.injectElements(TimestampedValue.of(1, new Instant(1))); // in [1, 11), gc at 21.
+
+ // And nothing should be left in the active window state.
+ assertTrue(tester.hasNoActiveWindows());
+
+ // Now the garbage collection timer will fire, finding the trigger already closed.
+ tester.advanceInputWatermark(new Instant(100));
+
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+ assertThat(output.size(), equalTo(1));
+ assertThat(output.get(0),
+ isSingleWindowedValue(containsInAnyOrder(1),
+ 1, // timestamp
+ 1, // window start
+ 11)); // window end
+ assertThat(
+ output.get(0).getPane(),
+ equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
+ }
+
+ /**
+ * When a merged window's trigger is closed we record that state using the merged window rather
+ * than the original windows.
+ */
+ @Test
+ public void testMergingWithClosedRepresentative() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
+ AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ // 2 elements into merged session window.
+ // Close the trigger, but the garbage collection timer is still pending.
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ tester.injectElements(TimestampedValue.of(1, new Instant(1)), // in [1, 11), gc at 21.
+ TimestampedValue.of(8, new Instant(8))); // in [8, 18), gc at 28.
+
+ // More elements into the same merged session window.
+ // It has not yet been gced.
+ // Should be discarded with 'window closed'.
+ tester.injectElements(TimestampedValue.of(1, new Instant(1)), // in [1, 11), gc at 21.
+ TimestampedValue.of(2, new Instant(2)), // in [2, 12), gc at 22.
+ TimestampedValue.of(8, new Instant(8))); // in [8, 18), gc at 28.
+
+ // Now the garbage collection timer will fire, finding the trigger already closed.
+ tester.advanceInputWatermark(new Instant(100));
+
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+
+ assertThat(output.size(), equalTo(1));
+ assertThat(output.get(0),
+ isSingleWindowedValue(containsInAnyOrder(1, 8),
+ 1, // timestamp
+ 1, // window start
+ 18)); // window end
+ assertThat(
+ output.get(0).getPane(),
+ equalTo(PaneInfo.createPane(true, true, Timing.EARLY, 0, 0)));
+ }
+
+ /**
+ * If an element for a closed session window ends up being merged into other still-open
+ * session windows, the resulting session window is not 'poisoned'.
+ */
+ @Test
+ public void testMergingWithClosedDoesNotPoison() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
+ AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ // 1 element, force its trigger to close.
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ tester.injectElements(TimestampedValue.of(2, new Instant(2)));
+
+ // 3 elements, one already closed.
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
+ tester.injectElements(TimestampedValue.of(1, new Instant(1)),
+ TimestampedValue.of(2, new Instant(2)),
+ TimestampedValue.of(3, new Instant(3)));
+
+ tester.advanceInputWatermark(new Instant(100));
+
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+ assertThat(output.size(), equalTo(2));
+ assertThat(output.get(0),
+ isSingleWindowedValue(containsInAnyOrder(2),
+ 2, // timestamp
+ 2, // window start
+ 12)); // window end
+ assertThat(
+ output.get(0).getPane(),
+ equalTo(PaneInfo.createPane(true, true, Timing.EARLY, 0, 0)));
+ assertThat(output.get(1),
+ isSingleWindowedValue(containsInAnyOrder(1, 2, 3),
+ 1, // timestamp
+ 1, // window start
+ 13)); // window end
+ assertThat(
+ output.get(1).getPane(),
+ equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
+ }
+
+ /**
+ * Tests that when data is assigned to multiple windows but some of those windows have
+ * had their triggers finish, then the data is dropped and counted accurately.
+ */
+ @Test
+ public void testDropDataMultipleWindowsFinishedTrigger() throws Exception {
+ ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining(
+ WindowingStrategy.of(
+ SlidingWindows.of(Duration.millis(100)).every(Duration.millis(30)))
+ .withTrigger(AfterWatermark.pastEndOfWindow())
+ .withAllowedLateness(Duration.millis(1000)),
+ new Sum.SumIntegerFn().<String>asKeyedFn(), VarIntCoder.of());
+
+ tester.injectElements(
+ // assigned to [-60, 40), [-30, 70), [0, 100)
+ TimestampedValue.of(10, new Instant(23)),
+ // assigned to [-30, 70), [0, 100), [30, 130)
+ TimestampedValue.of(12, new Instant(40)));
+
+ assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
+
+ tester.advanceInputWatermark(new Instant(70));
+ tester.injectElements(
+ // assigned to [-30, 70), [0, 100), [30, 130)
+ // but [-30, 70) is closed by the trigger
+ TimestampedValue.of(14, new Instant(60)));
+
+ assertEquals(1, tester.getElementsDroppedDueToClosedWindow());
+
+ tester.advanceInputWatermark(new Instant(130));
+ // assigned to [-30, 70), [0, 100), [30, 130)
+ // but they are all closed
+ tester.injectElements(TimestampedValue.of(16, new Instant(40)));
+
+ assertEquals(4, tester.getElementsDroppedDueToClosedWindow());
+ }
+
+ @Test
+ public void testIdempotentEmptyPanesDiscarding() throws Exception {
+ // Test uninteresting (empty) panes don't increment the index or otherwise
+ // modify PaneInfo.
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+ AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ // Inject a couple of on-time elements and fire at the window end.
+ injectElement(tester, 1);
+ injectElement(tester, 2);
+ tester.advanceInputWatermark(new Instant(12));
+
+ // Fire the on-time pane
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
+
+ // Fire another timer (with no data, so it's an uninteresting pane that should not be output).
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
+
+ // Finish it off with another datum.
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ injectElement(tester, 3);
+
+ // The intermediate trigger firing shouldn't result in any output.
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+ assertThat(output.size(), equalTo(2));
+
+ // The on-time pane is as expected.
+ assertThat(output.get(0), isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10));
+
+ // The late pane has the correct indices.
+ assertThat(output.get(1).getValue(), contains(3));
+ assertThat(
+ output.get(1).getPane(), equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1)));
+
+ assertTrue(tester.isMarkedFinished(firstWindow));
+ tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
+
+ assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
+ }
+
+ @Test
+ public void testIdempotentEmptyPanesAccumulating() throws Exception {
+ // Test uninteresting (empty) panes don't increment the index or otherwise
+ // modify PaneInfo.
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
+ AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(100),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ // Inject a couple of on-time elements and fire at the window end.
+ injectElement(tester, 1);
+ injectElement(tester, 2);
+ tester.advanceInputWatermark(new Instant(12));
+
+ // Trigger the on-time pane
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+ assertThat(output.size(), equalTo(1));
+ assertThat(output.get(0), isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10));
+ assertThat(output.get(0).getPane(),
+ equalTo(PaneInfo.createPane(true, false, Timing.ON_TIME, 0, 0)));
+
+ // Fire another timer with no data; the empty pane should not be output even though the
+ // trigger is ready to fire
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
+ assertThat(tester.extractOutput().size(), equalTo(0));
+
+ // Finish it off with another datum, which is late
+ when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
+ triggerShouldFinish(mockTrigger);
+ injectElement(tester, 3);
+ output = tester.extractOutput();
+ assertThat(output.size(), equalTo(1));
+
+ // The late pane has the correct indices.
+ assertThat(output.get(0).getValue(), containsInAnyOrder(1, 2, 3));
+ assertThat(output.get(0).getPane(),
+ equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1)));
+
+ assertTrue(tester.isMarkedFinished(firstWindow));
+ tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
+
+ assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
+ }
+
+ /**
+ * Test that we receive an empty on-time pane when an or-finally waiting for the watermark fires.
+ * Specifically, verify the proper triggerings and pane-info of a typical speculative/on-time/late
+ * when the on-time pane is empty.
+ */
+ @Test
+ public void testEmptyOnTimeFromOrFinally() throws Exception {
+ ReduceFnTester<Integer, Integer, IntervalWindow> tester =
+ ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)),
+ AfterEach.<IntervalWindow>inOrder(
+ Repeatedly
+ .forever(
+ AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
+ new Duration(5)))
+ .orFinally(AfterWatermark.pastEndOfWindow()),
+ Repeatedly.forever(
+ AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
+ new Duration(25)))),
+ AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().<String>asKeyedFn(),
+ VarIntCoder.of(), Duration.millis(100));
+
+ tester.advanceInputWatermark(new Instant(0));
+ tester.advanceProcessingTime(new Instant(0));
+
+ // Processing time timer for 5
+ tester.injectElements(
+ TimestampedValue.of(1, new Instant(1)),
+ TimestampedValue.of(1, new Instant(3)),
+ TimestampedValue.of(1, new Instant(7)),
+ TimestampedValue.of(1, new Instant(5)));
+
+ // Should fire early pane
+ tester.advanceProcessingTime(new Instant(6));
+
+ // Should fire empty on time pane
+ tester.advanceInputWatermark(new Instant(11));
+ List<WindowedValue<Integer>> output = tester.extractOutput();
+ assertEquals(2, output.size());
+
+ assertThat(output.get(0), WindowMatchers.isSingleWindowedValue(4, 1, 0, 10));
+ assertThat(output.get(1), WindowMatchers.isSingleWindowedValue(4, 9, 0, 10));
+
+ assertThat(
+ output.get(0),
+ WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)));
+ assertThat(
+ output.get(1),
+ WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0)));
+ }
+
+ /**
+ * Tests for processing time firings after the watermark passes the end of the window.
+ * Specifically, verify the proper triggerings and pane-info of a typical speculative/on-time/late
+ * when the on-time pane is non-empty.
+ */
+ @Test
+ public void testProcessingTime() throws Exception {
+ ReduceFnTester<Integer, Integer, IntervalWindow> tester =
+ ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)),
+ AfterEach.<IntervalWindow>inOrder(
+ Repeatedly
+ .forever(
+ AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
+ new Duration(5)))
+ .orFinally(AfterWatermark.pastEndOfWindow()),
+ Repeatedly.forever(
+ AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
+ new Duration(25)))),
+ AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().<String>asKeyedFn(),
+ VarIntCoder.of(), Duration.millis(100));
+
+ tester.advanceInputWatermark(new Instant(0));
+ tester.advanceProcessingTime(new Instant(0));
+
+ tester.injectElements(TimestampedValue.of(1, new Instant(1)),
+ TimestampedValue.of(1, new Instant(3)), TimestampedValue.of(1, new Instant(7)),
+ TimestampedValue.of(1, new Instant(5)));
+ // 4 elements all at processing time 0
+
+ tester.advanceProcessingTime(new Instant(6)); // fire [1,3,7,5] since 6 > 0 + 5
+ tester.injectElements(
+ TimestampedValue.of(1, new Instant(8)),
+ TimestampedValue.of(1, new Instant(4)));
+ // 6 elements
+
+ tester.advanceInputWatermark(new Instant(11)); // fire [1,3,7,5,8,4] since 11 > 9
+ tester.injectElements(
+ TimestampedValue.of(1, new Instant(8)),
+ TimestampedValue.of(1, new Instant(4)),
+ TimestampedValue.of(1, new Instant(5)));
+ // 9 elements
+
+ tester.advanceInputWatermark(new Instant(12));
+ tester.injectElements(
+ TimestampedValue.of(1, new Instant(3)));
+ // 10 elements
+
+ tester.advanceProcessingTime(new Instant(15));
+ tester.injectElements(
+ TimestampedValue.of(1, new Instant(5)));
+ // 11 elements
+ tester.advanceProcessingTime(new Instant(32)); // fire since 32 > 6 + 25
+
+ tester.injectElements(
+ TimestampedValue.of(1, new Instant(3)));
+ // 12 elements
+ // fire [1,3,7,5,8,4,8,4,5,3,5,3] since 125 > 6 + 25
+ tester.advanceInputWatermark(new Instant(125));
+
+ List<WindowedValue<Integer>> output = tester.extractOutput();
+ assertEquals(4, output.size());
+
+ assertThat(output.get(0), WindowMatchers.isSingleWindowedValue(4, 1, 0, 10));
+ assertThat(output.get(1), WindowMatchers.isSingleWindowedValue(6, 4, 0, 10));
+ assertThat(output.get(2), WindowMatchers.isSingleWindowedValue(11, 9, 0, 10));
+ assertThat(output.get(3), WindowMatchers.isSingleWindowedValue(12, 9, 0, 10));
+
+ assertThat(
+ output.get(0),
+ WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)));
+ assertThat(
+ output.get(1),
+ WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0)));
+ assertThat(
+ output.get(2),
+ WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.LATE, 2, 1)));
+ assertThat(
+ output.get(3),
+ WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 3, 2)));
+ }
+
+ /**
+ * We should fire a non-empty ON_TIME pane in the GlobalWindow when the watermark moves to
+ * end-of-time.
+ */
+ @Test
+ public void fireNonEmptyOnDrainInGlobalWindow() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, GlobalWindow> tester =
+ ReduceFnTester.nonCombining(
+ WindowingStrategy.of(new GlobalWindows())
+ .withTrigger(Repeatedly.<GlobalWindow>forever(
+ AfterPane.elementCountAtLeast(3)))
+ .withMode(AccumulationMode.DISCARDING_FIRED_PANES));
+
+ tester.advanceInputWatermark(new Instant(0));
+
+ final int n = 20;
+ for (int i = 0; i < n; i++) {
+ tester.injectElements(TimestampedValue.of(i, new Instant(i)));
+ }
+
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+ assertEquals(n / 3, output.size());
+ for (int i = 0; i < output.size(); i++) {
+ assertEquals(Timing.EARLY, output.get(i).getPane().getTiming());
+ assertEquals(i, output.get(i).getPane().getIndex());
+ assertEquals(3, Iterables.size(output.get(i).getValue()));
+ }
+
+ tester.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+ output = tester.extractOutput();
+ assertEquals(1, output.size());
+ assertEquals(Timing.ON_TIME, output.get(0).getPane().getTiming());
+ assertEquals(n / 3, output.get(0).getPane().getIndex());
+ assertEquals(n - ((n / 3) * 3), Iterables.size(output.get(0).getValue()));
+ }
+
+ /**
+ * We should fire an empty ON_TIME pane in the GlobalWindow when the watermark moves to
+ * end-of-time.
+ */
+ @Test
+ public void fireEmptyOnDrainInGlobalWindowIfRequested() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, GlobalWindow> tester =
+ ReduceFnTester.nonCombining(
+ WindowingStrategy.of(new GlobalWindows())
+ .withTrigger(Repeatedly.<GlobalWindow>forever(
+ AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
+ new Duration(3))))
+ .withMode(AccumulationMode.DISCARDING_FIRED_PANES));
+
+ final int n = 20;
+ for (int i = 0; i < n; i++) {
+ tester.advanceProcessingTime(new Instant(i));
+ tester.injectElements(TimestampedValue.of(i, new Instant(i)));
+ }
+ tester.advanceProcessingTime(new Instant(n + 4));
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+ assertEquals((n + 3) / 4, output.size());
+ for (int i = 0; i < output.size(); i++) {
+ assertEquals(Timing.EARLY, output.get(i).getPane().getTiming());
+ assertEquals(i, output.get(i).getPane().getIndex());
+ assertEquals(4, Iterables.size(output.get(i).getValue()));
+ }
+
+ tester.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+ output = tester.extractOutput();
+ assertEquals(1, output.size());
+ assertEquals(Timing.ON_TIME, output.get(0).getPane().getTiming());
+ assertEquals((n + 3) / 4, output.get(0).getPane().getIndex());
+ assertEquals(0, Iterables.size(output.get(0).getValue()));
+ }
+
+ /**
+ * Late elements should still have a garbage collection hold set so that they
+ * can make a late pane rather than be dropped due to lateness.
+ */
+ @Test
+ public void setGarbageCollectionHoldOnLateElements() throws Exception {
+ ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
+ ReduceFnTester.nonCombining(
+ FixedWindows.of(Duration.millis(10)),
+ AfterWatermark.pastEndOfWindow().withLateFirings(AfterPane.elementCountAtLeast(2)),
+ AccumulationMode.DISCARDING_FIRED_PANES,
+ Duration.millis(100),
+ ClosingBehavior.FIRE_IF_NON_EMPTY);
+
+ tester.advanceInputWatermark(new Instant(0));
+ tester.advanceOutputWatermark(new Instant(0));
+ tester.injectElements(TimestampedValue.of(1, new Instant(1)));
+
+ // Fire ON_TIME pane @ 9 with 1
+
+ tester.advanceInputWatermark(new Instant(109));
+ tester.advanceOutputWatermark(new Instant(109));
+ tester.injectElements(TimestampedValue.of(2, new Instant(2)));
+ // We should have set a garbage collection hold for the final pane.
+ Instant hold = tester.getWatermarkHold();
+ assertEquals(new Instant(109), hold);
+
+ tester.advanceInputWatermark(new Instant(110));
+ tester.advanceOutputWatermark(new Instant(110));
+
+ // Fire final LATE pane @ 9 with 2
+
+ List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
+ assertEquals(2, output.size());
+ }
+
+ private static class SumAndVerifyContextFn extends CombineFnWithContext<Integer, int[], Integer> {
+
+ private final PCollectionView<Integer> view;
+ private final int expectedValue;
+
+ private SumAndVerifyContextFn(PCollectionView<Integer> view, int expectedValue) {
+ this.view = view;
+ this.expectedValue = expectedValue;
+ }
+ @Override
+ public int[] createAccumulator(Context c) {
+ Preconditions.checkArgument(
+ c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
+ Preconditions.checkArgument(c.sideInput(view) == expectedValue);
+ return wrap(0);
+ }
+
+ @Override
+ public int[] addInput(int[] accumulator, Integer input, Context c) {
+ Preconditions.checkArgument(
+ c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
+ Preconditions.checkArgument(c.sideInput(view) == expectedValue);
+ accumulator[0] += input.intValue();
+ return accumulator;
+ }
+
+ @Override
+ public int[] mergeAccumulators(Iterable<int[]> accumulators, Context c) {
+ Preconditions.checkArgument(
+ c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
+ Preconditions.checkArgument(c.sideInput(view) == expectedValue);
+ Iterator<int[]> iter = accumulators.iterator();
+ if (!iter.hasNext()) {
+ return createAccumulator(c);
+ } else {
+ int[] running = iter.next();
+ while (iter.hasNext()) {
+ running[0] += iter.next()[0];
+ }
+ return running;
+ }
+ }
+
+ @Override
+ public Integer extractOutput(int[] accumulator, Context c) {
+ Preconditions.checkArgument(
+ c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
+ Preconditions.checkArgument(c.sideInput(view) == expectedValue);
+ return accumulator[0];
+ }
+
+ private int[] wrap(int value) {
+ return new int[] { value };
+ }
+ }
+
+ /**
+ * A {@link PipelineOptions} to test combining with context.
+ */
+ public interface TestOptions extends PipelineOptions {
+ Integer getValue();
+ void setValue(Integer value);
+ }
+}
[10/12] incubator-beam git commit: Move some easy stuff into
runners/core-java
Posted by ke...@apache.org.
Move some easy stuff into runners/core-java
This change moves a set of classes with no dependents,
leaving them in the same Java packages for now.
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/0fef8e63
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/0fef8e63
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/0fef8e63
Branch: refs/heads/master
Commit: 0fef8e6349216374ef60ef1d3356bdbdcc6f32ee
Parents: efaad32
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Jun 20 11:54:20 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Jun 21 14:09:49 2016 -0700
----------------------------------------------------------------------
runners/core-java/pom.xml | 30 +
.../org/apache/beam/sdk/util/AssignWindows.java | 46 +
.../apache/beam/sdk/util/AssignWindowsDoFn.java | 75 +
.../beam/sdk/util/BatchTimerInternals.java | 140 ++
.../org/apache/beam/sdk/util/DoFnRunner.java | 62 +
.../apache/beam/sdk/util/DoFnRunnerBase.java | 558 +++++++
.../org/apache/beam/sdk/util/DoFnRunners.java | 144 ++
.../beam/sdk/util/GroupAlsoByWindowsDoFn.java | 59 +
.../GroupAlsoByWindowsViaOutputBufferDoFn.java | 100 ++
.../sdk/util/GroupByKeyViaGroupByKeyOnly.java | 212 +++
.../sdk/util/LateDataDroppingDoFnRunner.java | 147 ++
.../org/apache/beam/sdk/util/NonEmptyPanes.java | 150 ++
.../apache/beam/sdk/util/PaneInfoTracker.java | 154 ++
.../sdk/util/PushbackSideInputDoFnRunner.java | 115 ++
.../java/org/apache/beam/sdk/util/ReduceFn.java | 130 ++
.../beam/sdk/util/ReduceFnContextFactory.java | 497 ++++++
.../apache/beam/sdk/util/ReduceFnRunner.java | 985 ++++++++++++
.../apache/beam/sdk/util/SimpleDoFnRunner.java | 56 +
.../apache/beam/sdk/util/SystemReduceFn.java | 135 ++
.../org/apache/beam/sdk/util/TriggerRunner.java | 234 +++
.../org/apache/beam/sdk/util/WatermarkHold.java | 536 +++++++
.../beam/sdk/util/BatchTimerInternalsTest.java | 118 ++
.../sdk/util/GroupAlsoByWindowsProperties.java | 619 ++++++++
...oupAlsoByWindowsViaOutputBufferDoFnTest.java | 106 ++
.../util/LateDataDroppingDoFnRunnerTest.java | 117 ++
.../util/PushbackSideInputDoFnRunnerTest.java | 234 +++
.../beam/sdk/util/ReduceFnRunnerTest.java | 1448 ++++++++++++++++++
.../apache/beam/sdk/util/ReduceFnTester.java | 784 ++++++++++
.../beam/sdk/util/SimpleDoFnRunnerTest.java | 86 ++
.../beam/runners/direct/DirectGroupByKey.java | 2 +-
.../direct/GroupByKeyEvaluatorFactoryTest.java | 2 +-
.../GroupByKeyOnlyEvaluatorFactoryTest.java | 2 +-
.../org/apache/beam/sdk/util/AssignWindows.java | 46 -
.../apache/beam/sdk/util/AssignWindowsDoFn.java | 75 -
.../beam/sdk/util/BatchTimerInternals.java | 140 --
.../org/apache/beam/sdk/util/DoFnRunner.java | 62 -
.../apache/beam/sdk/util/DoFnRunnerBase.java | 558 -------
.../org/apache/beam/sdk/util/DoFnRunners.java | 144 --
.../apache/beam/sdk/util/GatherAllPanes.java | 1 -
.../beam/sdk/util/GroupAlsoByWindowsDoFn.java | 59 -
.../GroupAlsoByWindowsViaOutputBufferDoFn.java | 100 --
.../sdk/util/GroupByKeyViaGroupByKeyOnly.java | 247 ---
.../sdk/util/LateDataDroppingDoFnRunner.java | 147 --
.../org/apache/beam/sdk/util/NonEmptyPanes.java | 150 --
.../apache/beam/sdk/util/PaneInfoTracker.java | 154 --
.../sdk/util/PushbackSideInputDoFnRunner.java | 115 --
.../java/org/apache/beam/sdk/util/ReduceFn.java | 130 --
.../beam/sdk/util/ReduceFnContextFactory.java | 497 ------
.../apache/beam/sdk/util/ReduceFnRunner.java | 985 ------------
.../sdk/util/ReifyTimestampsAndWindows.java | 63 +
.../apache/beam/sdk/util/SimpleDoFnRunner.java | 56 -
.../apache/beam/sdk/util/SystemReduceFn.java | 135 --
.../org/apache/beam/sdk/util/TriggerRunner.java | 234 ---
.../org/apache/beam/sdk/util/WatermarkHold.java | 536 -------
.../beam/sdk/util/BatchTimerInternalsTest.java | 118 --
.../sdk/util/GroupAlsoByWindowsProperties.java | 619 --------
...oupAlsoByWindowsViaOutputBufferDoFnTest.java | 106 --
.../util/LateDataDroppingDoFnRunnerTest.java | 117 --
.../util/PushbackSideInputDoFnRunnerTest.java | 234 ---
.../beam/sdk/util/ReduceFnRunnerTest.java | 1448 ------------------
.../apache/beam/sdk/util/ReduceFnTester.java | 784 ----------
.../beam/sdk/util/SimpleDoFnRunnerTest.java | 86 --
62 files changed, 8143 insertions(+), 8086 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml
index 8ede60b..1587a1a 100644
--- a/runners/core-java/pom.xml
+++ b/runners/core-java/pom.xml
@@ -197,7 +197,31 @@
<!-- build dependencies -->
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>com.google.code.findbugs</groupId>
+ <artifactId>jsr305</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>joda-time</groupId>
+ <artifactId>joda-time</artifactId>
+ </dependency>
+
<!-- test dependencies -->
+
+ <!-- Utilities such as WindowMatchers -->
+ <dependency>
+ <groupId>org.apache.beam</groupId>
+ <artifactId>beam-sdks-java-core</artifactId>
+ <classifier>tests</classifier>
+ <scope>test</scope>
+ </dependency>
+
<dependency>
<groupId>org.hamcrest</groupId>
<artifactId>hamcrest-all</artifactId>
@@ -205,6 +229,12 @@
</dependency>
<dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindows.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
new file mode 100644
index 0000000..af28052
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * {@link PTransform} that uses privileged (non-user-facing) APIs to assign elements of a
+ * {@link PCollection} to windows according to the provided {@link WindowFn}.
+ *
+ * @param <T> Type of elements being windowed
+ * @param <W> Window type
+ */
+public class AssignWindows<T, W extends BoundedWindow>
+ extends PTransform<PCollection<T>, PCollection<T>> {
+
+ private WindowFn<? super T, W> fn;
+
+ public AssignWindows(WindowFn<? super T, W> fn) {
+ this.fn = fn;
+ }
+
+ @Override
+ public PCollection<T> apply(PCollection<T> input) {
+ return input.apply("AssignWindows", ParDo.of(new AssignWindowsDoFn<>(fn)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
new file mode 100644
index 0000000..caec40e
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+
+import org.joda.time.Instant;
+
+import java.util.Collection;
+
+/**
+ * {@link DoFn} that tags elements of a PCollection with windows, according
+ * to the provided {@link WindowFn}.
+ * @param <T> Type of elements being windowed
+ * @param <W> Window type
+ */
+@SystemDoFnInternal
+public class AssignWindowsDoFn<T, W extends BoundedWindow> extends DoFn<T, T> {
+ private WindowFn<? super T, W> fn;
+
+ public AssignWindowsDoFn(WindowFn<? super T, W> fn) {
+ this.fn =
+ checkNotNull(
+ fn,
+ "%s provided to %s cannot be null",
+ WindowFn.class.getSimpleName(),
+ AssignWindowsDoFn.class.getSimpleName());
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void processElement(final ProcessContext c) throws Exception {
+ Collection<W> windows =
+ ((WindowFn<T, W>) fn).assignWindows(
+ ((WindowFn<T, W>) fn).new AssignContext() {
+ @Override
+ public T element() {
+ return c.element();
+ }
+
+ @Override
+ public Instant timestamp() {
+ return c.timestamp();
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ return c.windowingInternals().windows();
+ }
+ });
+
+ c.windowingInternals()
+ .outputWindowedValue(c.element(), c.timestamp(), windows, PaneInfo.NO_FIRING);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
new file mode 100644
index 0000000..d0c0b2f
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Preconditions;
+
+import org.joda.time.Instant;
+
+import java.util.HashSet;
+import java.util.PriorityQueue;
+import java.util.Set;
+
+import javax.annotation.Nullable;
+
+/**
+ * TimerInternals that uses priority queues to manage the timers that are ready to fire.
+ */
+public class BatchTimerInternals implements TimerInternals {
+ /** Set of timers that are scheduled used for deduplicating timers. */
+ private Set<TimerData> existingTimers = new HashSet<>();
+
+ // Keep these queues separate so we can advance over them separately.
+ private PriorityQueue<TimerData> watermarkTimers = new PriorityQueue<>(11);
+ private PriorityQueue<TimerData> processingTimers = new PriorityQueue<>(11);
+
+ private Instant inputWatermarkTime;
+ private Instant processingTime;
+
+ private PriorityQueue<TimerData> queue(TimeDomain domain) {
+ return TimeDomain.EVENT_TIME.equals(domain) ? watermarkTimers : processingTimers;
+ }
+
+ public BatchTimerInternals(Instant processingTime) {
+ this.processingTime = processingTime;
+ this.inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ }
+
+ @Override
+ public void setTimer(TimerData timer) {
+ if (existingTimers.add(timer)) {
+ queue(timer.getDomain()).add(timer);
+ }
+ }
+
+ @Override
+ public void deleteTimer(TimerData timer) {
+ existingTimers.remove(timer);
+ queue(timer.getDomain()).remove(timer);
+ }
+
+ @Override
+ public Instant currentProcessingTime() {
+ return processingTime;
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * @return {@link BoundedWindow#TIMESTAMP_MAX_VALUE}: in batch mode, upstream processing
+ * is already complete.
+ */
+ @Override
+ @Nullable
+ public Instant currentSynchronizedProcessingTime() {
+ return BoundedWindow.TIMESTAMP_MAX_VALUE;
+ }
+
+ @Override
+ public Instant currentInputWatermarkTime() {
+ return inputWatermarkTime;
+ }
+
+ @Override
+ @Nullable
+ public Instant currentOutputWatermarkTime() {
+ // The output watermark is always undefined in batch mode.
+ return null;
+ }
+
+ @Override
+ public String toString() {
+ return MoreObjects.toStringHelper(getClass())
+ .add("watermarkTimers", watermarkTimers)
+ .add("processingTimers", processingTimers)
+ .toString();
+ }
+
+ public void advanceInputWatermark(ReduceFnRunner<?, ?, ?, ?> runner, Instant newInputWatermark)
+ throws Exception {
+ Preconditions.checkState(!newInputWatermark.isBefore(inputWatermarkTime),
+ "Cannot move input watermark time backwards from %s to %s", inputWatermarkTime,
+ newInputWatermark);
+ inputWatermarkTime = newInputWatermark;
+ advance(runner, newInputWatermark, TimeDomain.EVENT_TIME);
+ }
+
+ public void advanceProcessingTime(ReduceFnRunner<?, ?, ?, ?> runner, Instant newProcessingTime)
+ throws Exception {
+ Preconditions.checkState(!newProcessingTime.isBefore(processingTime),
+ "Cannot move processing time backwards from %s to %s", processingTime, newProcessingTime);
+ processingTime = newProcessingTime;
+ advance(runner, newProcessingTime, TimeDomain.PROCESSING_TIME);
+ }
+
+ private void advance(ReduceFnRunner<?, ?, ?, ?> runner, Instant newTime, TimeDomain domain)
+ throws Exception {
+ PriorityQueue<TimerData> timers = queue(domain);
+ boolean shouldFire = false;
+
+ do {
+ TimerData timer = timers.peek();
+ // Timers fire if the new time is ahead of the timer
+ shouldFire = timer != null && newTime.isAfter(timer.getTimestamp());
+ if (shouldFire) {
+ // Remove before firing, so that if the trigger adds another identical
+ // timer we don't remove it.
+ timers.remove();
+ runner.onTimer(timer);
+ }
+ } while (shouldFire);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java
new file mode 100644
index 0000000..4ec8920
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.ProcessContext;
+import org.apache.beam.sdk.values.KV;
+
+/**
+ * An wrapper interface that represents the execution of a {@link DoFn}.
+ */
+public interface DoFnRunner<InputT, OutputT> {
+ /**
+ * Prepares and calls {@link DoFn#startBundle}.
+ */
+ public void startBundle();
+
+ /**
+ * Calls {@link DoFn#processElement} with a {@link ProcessContext} containing the current element.
+ */
+ public void processElement(WindowedValue<InputT> elem);
+
+ /**
+ * Calls {@link DoFn#finishBundle} and performs additional tasks, such as
+ * flushing in-memory states.
+ */
+ public void finishBundle();
+
+ /**
+ * An internal interface for signaling that a {@link DoFn} requires late data dropping.
+ */
+ public interface ReduceFnExecutor<K, InputT, OutputT, W> {
+ /**
+ * Gets this object as a {@link DoFn}.
+ *
+ * Most implementors of this interface are expected to be {@link DoFn} instances, and will
+ * return themselves.
+ */
+ DoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> asDoFn();
+
+ /**
+ * Returns an aggregator that tracks elements that are dropped due to being late.
+ */
+ Aggregator<Long, Long> getDroppedDueToLatenessAggregator();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
new file mode 100644
index 0000000..1ebe72b
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.RequiresWindowAccess;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
+import org.apache.beam.sdk.util.ExecutionContext.StepContext;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.joda.time.Instant;
+import org.joda.time.format.PeriodFormat;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A base implementation of {@link DoFnRunner}.
+ *
+ * <p> Sub-classes should override {@link #invokeProcessElement}.
+ */
+public abstract class DoFnRunnerBase<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
+
+ /** The DoFn being run. */
+ public final DoFn<InputT, OutputT> fn;
+
+ /** The context used for running the DoFn. */
+ public final DoFnContext<InputT, OutputT> context;
+
+ protected DoFnRunnerBase(
+ PipelineOptions options,
+ DoFn<InputT, OutputT> fn,
+ SideInputReader sideInputReader,
+ OutputManager outputManager,
+ TupleTag<OutputT> mainOutputTag,
+ List<TupleTag<?>> sideOutputTags,
+ StepContext stepContext,
+ CounterSet.AddCounterMutator addCounterMutator,
+ WindowingStrategy<?, ?> windowingStrategy) {
+ this.fn = fn;
+ this.context = new DoFnContext<>(
+ options,
+ fn,
+ sideInputReader,
+ outputManager,
+ mainOutputTag,
+ sideOutputTags,
+ stepContext,
+ addCounterMutator,
+ windowingStrategy == null ? null : windowingStrategy.getWindowFn());
+ }
+
+ /**
+ * An implementation of {@code OutputManager} using simple lists, for testing and in-memory
+ * contexts such as the {@link DirectRunner}.
+ */
+ public static class ListOutputManager implements OutputManager {
+
+ private Map<TupleTag<?>, List<WindowedValue<?>>> outputLists = Maps.newHashMap();
+
+ @Override
+ public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ List<WindowedValue<T>> outputList = (List) outputLists.get(tag);
+
+ if (outputList == null) {
+ outputList = Lists.newArrayList();
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ List<WindowedValue<?>> untypedList = (List) outputList;
+ outputLists.put(tag, untypedList);
+ }
+
+ outputList.add(output);
+ }
+
+ public <T> List<WindowedValue<T>> getOutput(TupleTag<T> tag) {
+ // Safe cast by design, inexpressible in Java without rawtypes
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ List<WindowedValue<T>> outputList = (List) outputLists.get(tag);
+ return (outputList != null) ? outputList : Collections.<WindowedValue<T>>emptyList();
+ }
+ }
+
+ @Override
+ public void startBundle() {
+ // This can contain user code. Wrap it in case it throws an exception.
+ try {
+ fn.startBundle(context);
+ } catch (Throwable t) {
+ // Exception in user code.
+ throw wrapUserCodeException(t);
+ }
+ }
+
+ @Override
+ public void processElement(WindowedValue<InputT> elem) {
+ if (elem.getWindows().size() <= 1
+ || (!RequiresWindowAccess.class.isAssignableFrom(fn.getClass())
+ && context.sideInputReader.isEmpty())) {
+ invokeProcessElement(elem);
+ } else {
+ // We could modify the windowed value (and the processContext) to
+ // avoid repeated allocations, but this is more straightforward.
+ for (WindowedValue<InputT> windowedValue : elem.explodeWindows()) {
+ invokeProcessElement(windowedValue);
+ }
+ }
+ }
+
+ /**
+ * Invokes {@link DoFn#processElement} after certain pre-processings has been done in
+ * {@link DoFnRunnerBase#processElement}.
+ */
+ protected abstract void invokeProcessElement(WindowedValue<InputT> elem);
+
+ @Override
+ public void finishBundle() {
+ // This can contain user code. Wrap it in case it throws an exception.
+ try {
+ fn.finishBundle(context);
+ } catch (Throwable t) {
+ // Exception in user code.
+ throw wrapUserCodeException(t);
+ }
+ }
+
+ /**
+ * A concrete implementation of {@code DoFn.Context} used for running a {@link DoFn}.
+ *
+ * @param <InputT> the type of the DoFn's (main) input elements
+ * @param <OutputT> the type of the DoFn's (main) output elements
+ */
+ private static class DoFnContext<InputT, OutputT>
+ extends DoFn<InputT, OutputT>.Context {
+ private static final int MAX_SIDE_OUTPUTS = 1000;
+
+ final PipelineOptions options;
+ final DoFn<InputT, OutputT> fn;
+ final SideInputReader sideInputReader;
+ final OutputManager outputManager;
+ final TupleTag<OutputT> mainOutputTag;
+ final StepContext stepContext;
+ final CounterSet.AddCounterMutator addCounterMutator;
+ final WindowFn<?, ?> windowFn;
+
+ /**
+ * The set of known output tags, some of which may be undeclared, so we can throw an
+ * exception when it exceeds {@link #MAX_SIDE_OUTPUTS}.
+ */
+ private Set<TupleTag<?>> outputTags;
+
+ public DoFnContext(PipelineOptions options,
+ DoFn<InputT, OutputT> fn,
+ SideInputReader sideInputReader,
+ OutputManager outputManager,
+ TupleTag<OutputT> mainOutputTag,
+ List<TupleTag<?>> sideOutputTags,
+ StepContext stepContext,
+ CounterSet.AddCounterMutator addCounterMutator,
+ WindowFn<?, ?> windowFn) {
+ fn.super();
+ this.options = options;
+ this.fn = fn;
+ this.sideInputReader = sideInputReader;
+ this.outputManager = outputManager;
+ this.mainOutputTag = mainOutputTag;
+ this.outputTags = Sets.newHashSet();
+
+ outputTags.add(mainOutputTag);
+ for (TupleTag<?> sideOutputTag : sideOutputTags) {
+ outputTags.add(sideOutputTag);
+ }
+
+ this.stepContext = stepContext;
+ this.addCounterMutator = addCounterMutator;
+ this.windowFn = windowFn;
+ super.setupDelegateAggregators();
+ }
+
+ //////////////////////////////////////////////////////////////////////////////
+
+ @Override
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ <T, W extends BoundedWindow> WindowedValue<T> makeWindowedValue(
+ T output, Instant timestamp, Collection<W> windows, PaneInfo pane) {
+ final Instant inputTimestamp = timestamp;
+
+ if (timestamp == null) {
+ timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ }
+
+ if (windows == null) {
+ try {
+ // The windowFn can never succeed at accessing the element, so its type does not
+ // matter here
+ @SuppressWarnings("unchecked")
+ WindowFn<Object, W> objectWindowFn = (WindowFn<Object, W>) windowFn;
+ windows = objectWindowFn.assignWindows(objectWindowFn.new AssignContext() {
+ @Override
+ public Object element() {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input element when none was available");
+ }
+
+ @Override
+ public Instant timestamp() {
+ if (inputTimestamp == null) {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input timestamp when none was available");
+ }
+ return inputTimestamp;
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input windows when none were available");
+ }
+ });
+ } catch (Exception e) {
+ throw UserCodeException.wrap(e);
+ }
+ }
+
+ return WindowedValue.of(output, timestamp, windows, pane);
+ }
+
+ public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+ if (!sideInputReader.contains(view)) {
+ throw new IllegalArgumentException("calling sideInput() with unknown view");
+ }
+ BoundedWindow sideInputWindow =
+ view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow);
+ return sideInputReader.get(view, sideInputWindow);
+ }
+
+ void outputWindowedValue(
+ OutputT output,
+ Instant timestamp,
+ Collection<? extends BoundedWindow> windows,
+ PaneInfo pane) {
+ outputWindowedValue(makeWindowedValue(output, timestamp, windows, pane));
+ }
+
+ void outputWindowedValue(WindowedValue<OutputT> windowedElem) {
+ outputManager.output(mainOutputTag, windowedElem);
+ if (stepContext != null) {
+ stepContext.noteOutput(windowedElem);
+ }
+ }
+
+ protected <T> void sideOutputWindowedValue(TupleTag<T> tag,
+ T output,
+ Instant timestamp,
+ Collection<? extends BoundedWindow> windows,
+ PaneInfo pane) {
+ sideOutputWindowedValue(tag, makeWindowedValue(output, timestamp, windows, pane));
+ }
+
+ protected <T> void sideOutputWindowedValue(TupleTag<T> tag, WindowedValue<T> windowedElem) {
+ if (!outputTags.contains(tag)) {
+ // This tag wasn't declared nor was it seen before during this execution.
+ // Thus, this must be a new, undeclared and unconsumed output.
+ // To prevent likely user errors, enforce the limit on the number of side
+ // outputs.
+ if (outputTags.size() >= MAX_SIDE_OUTPUTS) {
+ throw new IllegalArgumentException(
+ "the number of side outputs has exceeded a limit of " + MAX_SIDE_OUTPUTS);
+ }
+ outputTags.add(tag);
+ }
+
+ outputManager.output(tag, windowedElem);
+ if (stepContext != null) {
+ stepContext.noteSideOutput(tag, windowedElem);
+ }
+ }
+
+ // Following implementations of output, outputWithTimestamp, and sideOutput
+ // are only accessible in DoFn.startBundle and DoFn.finishBundle, and will be shadowed by
+ // ProcessContext's versions in DoFn.processElement.
+ @Override
+ public void output(OutputT output) {
+ outputWindowedValue(output, null, null, PaneInfo.NO_FIRING);
+ }
+
+ @Override
+ public void outputWithTimestamp(OutputT output, Instant timestamp) {
+ outputWindowedValue(output, timestamp, null, PaneInfo.NO_FIRING);
+ }
+
+ @Override
+ public <T> void sideOutput(TupleTag<T> tag, T output) {
+ Preconditions.checkNotNull(tag, "TupleTag passed to sideOutput cannot be null");
+ sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING);
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+ Preconditions.checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null");
+ sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING);
+ }
+
+ private String generateInternalAggregatorName(String userName) {
+ boolean system = fn.getClass().isAnnotationPresent(SystemDoFnInternal.class);
+ return (system ? "" : "user-") + stepContext.getStepName() + "-" + userName;
+ }
+
+ @Override
+ protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+ String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+ Preconditions.checkNotNull(combiner,
+ "Combiner passed to createAggregator cannot be null");
+ return new CounterAggregator<>(generateInternalAggregatorName(name),
+ combiner, addCounterMutator);
+ }
+ }
+
+ /**
+ * Returns a new {@code DoFn.ProcessContext} for the given element.
+ */
+ protected DoFn<InputT, OutputT>.ProcessContext createProcessContext(WindowedValue<InputT> elem) {
+ return new DoFnProcessContext<InputT, OutputT>(fn, context, elem);
+ }
+
+ protected RuntimeException wrapUserCodeException(Throwable t) {
+ throw UserCodeException.wrapIf(!isSystemDoFn(), t);
+ }
+
+ private boolean isSystemDoFn() {
+ return fn.getClass().isAnnotationPresent(SystemDoFnInternal.class);
+ }
+
+ /**
+ * A concrete implementation of {@code DoFn.ProcessContext} used for
+ * running a {@link DoFn} over a single element.
+ *
+ * @param <InputT> the type of the DoFn's (main) input elements
+ * @param <OutputT> the type of the DoFn's (main) output elements
+ */
+ static class DoFnProcessContext<InputT, OutputT>
+ extends DoFn<InputT, OutputT>.ProcessContext {
+
+
+ final DoFn<InputT, OutputT> fn;
+ final DoFnContext<InputT, OutputT> context;
+ final WindowedValue<InputT> windowedValue;
+
+ public DoFnProcessContext(DoFn<InputT, OutputT> fn,
+ DoFnContext<InputT, OutputT> context,
+ WindowedValue<InputT> windowedValue) {
+ fn.super();
+ this.fn = fn;
+ this.context = context;
+ this.windowedValue = windowedValue;
+ }
+
+ @Override
+ public PipelineOptions getPipelineOptions() {
+ return context.getPipelineOptions();
+ }
+
+ @Override
+ public InputT element() {
+ return windowedValue.getValue();
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view) {
+ Preconditions.checkNotNull(view, "View passed to sideInput cannot be null");
+ Iterator<? extends BoundedWindow> windowIter = windows().iterator();
+ BoundedWindow window;
+ if (!windowIter.hasNext()) {
+ if (context.windowFn instanceof GlobalWindows) {
+ // TODO: Remove this once GroupByKeyOnly no longer outputs elements
+ // without windows
+ window = GlobalWindow.INSTANCE;
+ } else {
+ throw new IllegalStateException(
+ "sideInput called when main input element is not in any windows");
+ }
+ } else {
+ window = windowIter.next();
+ if (windowIter.hasNext()) {
+ throw new IllegalStateException(
+ "sideInput called when main input element is in multiple windows");
+ }
+ }
+ return context.sideInput(view, window);
+ }
+
+ @Override
+ public BoundedWindow window() {
+ if (!(fn instanceof RequiresWindowAccess)) {
+ throw new UnsupportedOperationException(
+ "window() is only available in the context of a DoFn marked as RequiresWindow.");
+ }
+ return Iterables.getOnlyElement(windows());
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return windowedValue.getPane();
+ }
+
+ @Override
+ public void output(OutputT output) {
+ context.outputWindowedValue(windowedValue.withValue(output));
+ }
+
+ @Override
+ public void outputWithTimestamp(OutputT output, Instant timestamp) {
+ checkTimestamp(timestamp);
+ context.outputWindowedValue(output, timestamp,
+ windowedValue.getWindows(), windowedValue.getPane());
+ }
+
+ void outputWindowedValue(
+ OutputT output,
+ Instant timestamp,
+ Collection<? extends BoundedWindow> windows,
+ PaneInfo pane) {
+ context.outputWindowedValue(output, timestamp, windows, pane);
+ }
+
+ @Override
+ public <T> void sideOutput(TupleTag<T> tag, T output) {
+ Preconditions.checkNotNull(tag, "Tag passed to sideOutput cannot be null");
+ context.sideOutputWindowedValue(tag, windowedValue.withValue(output));
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+ Preconditions.checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null");
+ checkTimestamp(timestamp);
+ context.sideOutputWindowedValue(
+ tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane());
+ }
+
+ @Override
+ public Instant timestamp() {
+ return windowedValue.getTimestamp();
+ }
+
+ public Collection<? extends BoundedWindow> windows() {
+ return windowedValue.getWindows();
+ }
+
+ private void checkTimestamp(Instant timestamp) {
+ if (timestamp.isBefore(windowedValue.getTimestamp().minus(fn.getAllowedTimestampSkew()))) {
+ throw new IllegalArgumentException(String.format(
+ "Cannot output with timestamp %s. Output timestamps must be no earlier than the "
+ + "timestamp of the current input (%s) minus the allowed skew (%s). See the "
+ + "DoFn#getAllowedTimestampSkew() Javadoc for details on changing the allowed skew.",
+ timestamp, windowedValue.getTimestamp(),
+ PeriodFormat.getDefault().print(fn.getAllowedTimestampSkew().toPeriod())));
+ }
+ }
+
+ @Override
+ public WindowingInternals<InputT, OutputT> windowingInternals() {
+ return new WindowingInternals<InputT, OutputT>() {
+ @Override
+ public void outputWindowedValue(OutputT output, Instant timestamp,
+ Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ context.outputWindowedValue(output, timestamp, windows, pane);
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ return windowedValue.getWindows();
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return windowedValue.getPane();
+ }
+
+ @Override
+ public TimerInternals timerInternals() {
+ return context.stepContext.timerInternals();
+ }
+
+ @Override
+ public <T> void writePCollectionViewData(
+ TupleTag<?> tag,
+ Iterable<WindowedValue<T>> data,
+ Coder<T> elemCoder) throws IOException {
+ @SuppressWarnings("unchecked")
+ Coder<BoundedWindow> windowCoder = (Coder<BoundedWindow>) context.windowFn.windowCoder();
+
+ context.stepContext.writePCollectionViewData(
+ tag, data, IterableCoder.of(WindowedValue.getFullCoder(elemCoder, windowCoder)),
+ window(), windowCoder);
+ }
+
+ @Override
+ public StateInternals<?> stateInternals() {
+ return context.stepContext.stateInternals();
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+ return context.sideInput(view, mainInputWindow);
+ }
+ };
+ }
+
+ @Override
+ protected <AggregatorInputT, AggregatorOutputT> Aggregator<AggregatorInputT, AggregatorOutputT>
+ createAggregatorInternal(
+ String name, CombineFn<AggregatorInputT, ?, AggregatorOutputT> combiner) {
+ return context.createAggregatorInternal(name, combiner);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java
new file mode 100644
index 0000000..648a281
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.DoFnRunner.ReduceFnExecutor;
+import org.apache.beam.sdk.util.ExecutionContext.StepContext;
+import org.apache.beam.sdk.util.common.CounterSet;
+import org.apache.beam.sdk.util.common.CounterSet.AddCounterMutator;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TupleTag;
+
+import java.util.List;
+
+/**
+ * Static utility methods that provide {@link DoFnRunner} implementations.
+ */
+public class DoFnRunners {
+ /**
+ * Information about how to create output receivers and output to them.
+ */
+ public interface OutputManager {
+ /**
+ * Outputs a single element to the receiver indicated by the given {@link TupleTag}.
+ */
+ public <T> void output(TupleTag<T> tag, WindowedValue<T> output);
+ }
+
+ /**
+ * Returns a basic implementation of {@link DoFnRunner} that works for most {@link DoFn DoFns}.
+ *
+ * <p>It invokes {@link DoFn#processElement} for each input.
+ */
+ public static <InputT, OutputT> DoFnRunner<InputT, OutputT> simpleRunner(
+ PipelineOptions options,
+ DoFn<InputT, OutputT> fn,
+ SideInputReader sideInputReader,
+ OutputManager outputManager,
+ TupleTag<OutputT> mainOutputTag,
+ List<TupleTag<?>> sideOutputTags,
+ StepContext stepContext,
+ CounterSet.AddCounterMutator addCounterMutator,
+ WindowingStrategy<?, ?> windowingStrategy) {
+ return new SimpleDoFnRunner<>(
+ options,
+ fn,
+ sideInputReader,
+ outputManager,
+ mainOutputTag,
+ sideOutputTags,
+ stepContext,
+ addCounterMutator,
+ windowingStrategy);
+ }
+
+ /**
+ * Returns an implementation of {@link DoFnRunner} that handles late data dropping.
+ *
+ * <p>It drops elements from expired windows before they reach the underlying {@link DoFn}.
+ */
+ public static <K, InputT, OutputT, W extends BoundedWindow>
+ DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> lateDataDroppingRunner(
+ PipelineOptions options,
+ ReduceFnExecutor<K, InputT, OutputT, W> reduceFnExecutor,
+ SideInputReader sideInputReader,
+ OutputManager outputManager,
+ TupleTag<KV<K, OutputT>> mainOutputTag,
+ List<TupleTag<?>> sideOutputTags,
+ StepContext stepContext,
+ CounterSet.AddCounterMutator addCounterMutator,
+ WindowingStrategy<?, W> windowingStrategy) {
+ DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> simpleDoFnRunner =
+ simpleRunner(
+ options,
+ reduceFnExecutor.asDoFn(),
+ sideInputReader,
+ outputManager,
+ mainOutputTag,
+ sideOutputTags,
+ stepContext,
+ addCounterMutator,
+ windowingStrategy);
+ return new LateDataDroppingDoFnRunner<>(
+ simpleDoFnRunner,
+ windowingStrategy,
+ stepContext.timerInternals(),
+ reduceFnExecutor.getDroppedDueToLatenessAggregator());
+ }
+
+ public static <InputT, OutputT> DoFnRunner<InputT, OutputT> createDefault(
+ PipelineOptions options,
+ DoFn<InputT, OutputT> doFn,
+ SideInputReader sideInputReader,
+ OutputManager outputManager,
+ TupleTag<OutputT> mainOutputTag,
+ List<TupleTag<?>> sideOutputTags,
+ StepContext stepContext,
+ AddCounterMutator addCounterMutator,
+ WindowingStrategy<?, ?> windowingStrategy) {
+ if (doFn instanceof ReduceFnExecutor) {
+ @SuppressWarnings("rawtypes")
+ ReduceFnExecutor fn = (ReduceFnExecutor) doFn;
+ @SuppressWarnings({"unchecked", "cast", "rawtypes"})
+ DoFnRunner<InputT, OutputT> runner = (DoFnRunner<InputT, OutputT>) lateDataDroppingRunner(
+ options,
+ fn,
+ sideInputReader,
+ outputManager,
+ (TupleTag) mainOutputTag,
+ sideOutputTags,
+ stepContext,
+ addCounterMutator,
+ (WindowingStrategy) windowingStrategy);
+ return runner;
+ }
+ return simpleRunner(
+ options,
+ doFn,
+ sideInputReader,
+ outputManager,
+ mainOutputTag,
+ sideOutputTags,
+ stepContext,
+ addCounterMutator,
+ windowingStrategy);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java
new file mode 100644
index 0000000..f5de0bc
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
+
+/**
+ * DoFn that merges windows and groups elements in those windows, optionally
+ * combining values.
+ *
+ * @param <K> key type
+ * @param <InputT> input value element type
+ * @param <OutputT> output value element type
+ * @param <W> window type
+ */
+@SystemDoFnInternal
+public abstract class GroupAlsoByWindowsDoFn<K, InputT, OutputT, W extends BoundedWindow>
+ extends DoFn<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> {
+ public static final String DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER = "DroppedDueToClosedWindow";
+ public static final String DROPPED_DUE_TO_LATENESS_COUNTER = "DroppedDueToLateness";
+
+ protected final Aggregator<Long, Long> droppedDueToClosedWindow =
+ createAggregator(DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER, new Sum.SumLongFn());
+ protected final Aggregator<Long, Long> droppedDueToLateness =
+ createAggregator(DROPPED_DUE_TO_LATENESS_COUNTER, new Sum.SumLongFn());
+
+ /**
+ * Create the default {@link GroupAlsoByWindowsDoFn}, which uses window sets to implement the
+ * grouping.
+ *
+ * @param windowingStrategy The window function and trigger to use for grouping
+ * @param inputCoder the input coder to use
+ */
+ public static <K, V, W extends BoundedWindow> GroupAlsoByWindowsDoFn<K, V, Iterable<V>, W>
+ createDefault(WindowingStrategy<?, W> windowingStrategy, Coder<V> inputCoder) {
+ return new GroupAlsoByWindowsViaOutputBufferDoFn<>(
+ windowingStrategy, SystemReduceFn.<K, V, W>buffering(inputCoder));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java
new file mode 100644
index 0000000..d364168
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.KV;
+
+import com.google.common.collect.Iterables;
+
+import org.joda.time.Instant;
+
+import java.util.List;
+
+/**
+ * The default batch {@link GroupAlsoByWindowsDoFn} implementation, if no specialized "fast path"
+ * implementation is applicable.
+ */
+@SystemDoFnInternal
+public class GroupAlsoByWindowsViaOutputBufferDoFn<K, InputT, OutputT, W extends BoundedWindow>
+ extends GroupAlsoByWindowsDoFn<K, InputT, OutputT, W> {
+
+ private final WindowingStrategy<?, W> strategy;
+ private SystemReduceFn<K, InputT, ?, OutputT, W> reduceFn;
+
+ public GroupAlsoByWindowsViaOutputBufferDoFn(
+ WindowingStrategy<?, W> windowingStrategy,
+ SystemReduceFn<K, InputT, ?, OutputT, W> reduceFn) {
+ this.strategy = windowingStrategy;
+ this.reduceFn = reduceFn;
+ }
+
+ @Override
+ public void processElement(
+ DoFn<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>>.ProcessContext c)
+ throws Exception {
+ K key = c.element().getKey();
+ // Used with Batch, we know that all the data is available for this key. We can't use the
+ // timer manager from the context because it doesn't exist. So we create one and emulate the
+ // watermark, knowing that we have all data and it is in timestamp order.
+ BatchTimerInternals timerInternals = new BatchTimerInternals(Instant.now());
+
+ // It is the responsibility of the user of GroupAlsoByWindowsViaOutputBufferDoFn to only
+ // provide a WindowingInternals instance with the appropriate key type for StateInternals.
+ @SuppressWarnings("unchecked")
+ StateInternals<K> stateInternals = (StateInternals<K>) c.windowingInternals().stateInternals();
+
+ ReduceFnRunner<K, InputT, OutputT, W> reduceFnRunner =
+ new ReduceFnRunner<K, InputT, OutputT, W>(
+ key,
+ strategy,
+ stateInternals,
+ timerInternals,
+ c.windowingInternals(),
+ droppedDueToClosedWindow,
+ reduceFn,
+ c.getPipelineOptions());
+
+ Iterable<List<WindowedValue<InputT>>> chunks =
+ Iterables.partition(c.element().getValue(), 1000);
+ for (Iterable<WindowedValue<InputT>> chunk : chunks) {
+ // Process the chunk of elements.
+ reduceFnRunner.processElements(chunk);
+
+ // Then, since elements are sorted by their timestamp, advance the input watermark
+ // to the first element, and fire any timers that may have been scheduled.
+ timerInternals.advanceInputWatermark(reduceFnRunner, chunk.iterator().next().getTimestamp());
+
+ // Fire any processing timers that need to fire
+ timerInternals.advanceProcessingTime(reduceFnRunner, Instant.now());
+
+ // Leave the output watermark undefined. Since there's no late data in batch mode
+ // there's really no need to track it as we do for streaming.
+ }
+
+ // Finish any pending windows by advancing the input watermark to infinity.
+ timerInternals.advanceInputWatermark(reduceFnRunner, BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+ // Finally, advance the processing time to infinity to fire any timers.
+ timerInternals.advanceProcessingTime(reduceFnRunner, BoundedWindow.TIMESTAMP_MAX_VALUE);
+
+ reduceFnRunner.persist();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java
new file mode 100644
index 0000000..9450495
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+/**
+ * An implementation of {@link GroupByKey} built on top of a lower-level {@link GroupByKeyOnly}
+ * primitive.
+ *
+ * <p>This implementation of {@link GroupByKey} proceeds via the following steps:
+ * <ol>
+ * <li>{@code ReifyTimestampsAndWindowsDoFn ParDo(ReifyTimestampsAndWindows)}: This embeds
+ * the previously-implicit timestamp and window into the elements themselves, so a
+ * window-and-timestamp-unaware transform can operate on them.</li>
+ * <li>{@code GroupByKeyOnly}: This lower-level primitive groups by keys, ignoring windows
+ * and timestamps. Many window-unaware runners have such a primitive already.</li>
+ * <li>{@code SortValuesByTimestamp ParDo(SortValuesByTimestamp)}: The values in the iterables
+ * output by {@link GroupByKeyOnly} are sorted by timestamp.</li>
+ * <li>{@code GroupAlsoByWindow}: This primitive processes the sorted values. Today it is
+ * implemented as a {@link ParDo} that calls reserved internal methods.</li>
+ * </ol>
+ *
+ * <p>This implementation of {@link GroupByKey} has severe limitations unless its component
+ * transforms are replaced. As-is, it is only applicable for in-memory runners using a batch-style
+ * execution strategy. Specifically:
+ *
+ * <ul>
+ * <li>Every iterable output by {@link GroupByKeyOnly} must contain all elements for that key.
+ * A streaming-style partition, with multiple elements for the same key, will not yield
+ * correct results.</li>
+ * <li>Sorting of values by timestamp is performed on an in-memory list. It will not succeed
+ * for large iterables.</li>
+ * <li>The implementation of {@code GroupAlsoByWindow} does not support timers. This is only
+ * appropriate for runners which also do not support timers.</li>
+ * </ul>
+ */
+public class GroupByKeyViaGroupByKeyOnly<K, V>
+ extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
+
+ private GroupByKey<K, V> gbkTransform;
+
+ public GroupByKeyViaGroupByKeyOnly(GroupByKey<K, V> originalTransform) {
+ this.gbkTransform = originalTransform;
+ }
+
+ @Override
+ public PCollection<KV<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
+ WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
+
+ return input
+ // Make each input element's timestamp and assigned windows
+ // explicit, in the value part.
+ .apply(new ReifyTimestampsAndWindows<K, V>())
+
+ // Group by just the key.
+ // Combiner lifting will not happen regardless of the disallowCombinerLifting value.
+ // There will be no combiners right after the GroupByKeyOnly because of the two ParDos
+ // introduced in here.
+ .apply(new GroupByKeyOnly<K, WindowedValue<V>>())
+
+ // Sort each key's values by timestamp. GroupAlsoByWindow requires
+ // its input to be sorted by timestamp.
+ .apply(new SortValuesByTimestamp<K, V>())
+
+ // Group each key's values by window, merging windows as needed.
+ .apply(new GroupAlsoByWindow<K, V>(windowingStrategy))
+
+ // And update the windowing strategy as appropriate.
+ .setWindowingStrategyInternal(
+ gbkTransform.updateWindowingStrategy(windowingStrategy));
+ }
+
+ /**
+ * Runner-specific primitive that groups by key only, ignoring any window assignments. A
+ * runner that uses {@link GroupByKeyViaGroupByKeyOnly} should have a primitive way to translate
+ * or evaluate this class.
+ */
+ public static class GroupByKeyOnly<K, V>
+ extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
+
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ @Override
+ public PCollection<KV<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
+ return PCollection.<KV<K, Iterable<V>>>createPrimitiveOutputInternal(
+ input.getPipeline(), input.getWindowingStrategy(), input.isBounded());
+ }
+
+ @Override
+ public Coder<KV<K, Iterable<V>>> getDefaultOutputCoder(PCollection<KV<K, V>> input) {
+ return GroupByKey.getOutputKvCoder(input.getCoder());
+ }
+ }
+
+ /**
+ * Helper transform that sorts the values associated with each key by timestamp.
+ */
+ private static class SortValuesByTimestamp<K, V>
+ extends PTransform<
+ PCollection<KV<K, Iterable<WindowedValue<V>>>>,
+ PCollection<KV<K, Iterable<WindowedValue<V>>>>> {
+ @Override
+ public PCollection<KV<K, Iterable<WindowedValue<V>>>> apply(
+ PCollection<KV<K, Iterable<WindowedValue<V>>>> input) {
+ return input
+ .apply(
+ ParDo.of(
+ new DoFn<KV<K, Iterable<WindowedValue<V>>>, KV<K, Iterable<WindowedValue<V>>>>() {
+ @Override
+ public void processElement(ProcessContext c) {
+ KV<K, Iterable<WindowedValue<V>>> kvs = c.element();
+ K key = kvs.getKey();
+ Iterable<WindowedValue<V>> unsortedValues = kvs.getValue();
+ List<WindowedValue<V>> sortedValues = new ArrayList<>();
+ for (WindowedValue<V> value : unsortedValues) {
+ sortedValues.add(value);
+ }
+ Collections.sort(
+ sortedValues,
+ new Comparator<WindowedValue<V>>() {
+ @Override
+ public int compare(WindowedValue<V> e1, WindowedValue<V> e2) {
+ return e1.getTimestamp().compareTo(e2.getTimestamp());
+ }
+ });
+ c.output(KV.<K, Iterable<WindowedValue<V>>>of(key, sortedValues));
+ }
+ }))
+ .setCoder(input.getCoder());
+ }
+ }
+
+ /**
+ * Helper transform that takes a collection of timestamp-ordered
+ * values associated with each key, groups the values by window,
+ * combines windows as needed, and for each window in each key,
+ * outputs a collection of key/value-list pairs implicitly assigned
+ * to the window and with the timestamp derived from that window.
+ */
+ private static class GroupAlsoByWindow<K, V>
+ extends PTransform<
+ PCollection<KV<K, Iterable<WindowedValue<V>>>>, PCollection<KV<K, Iterable<V>>>> {
+ private final WindowingStrategy<?, ?> windowingStrategy;
+
+ public GroupAlsoByWindow(WindowingStrategy<?, ?> windowingStrategy) {
+ this.windowingStrategy = windowingStrategy;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public PCollection<KV<K, Iterable<V>>> apply(
+ PCollection<KV<K, Iterable<WindowedValue<V>>>> input) {
+ @SuppressWarnings("unchecked")
+ KvCoder<K, Iterable<WindowedValue<V>>> inputKvCoder =
+ (KvCoder<K, Iterable<WindowedValue<V>>>) input.getCoder();
+
+ Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+ Coder<Iterable<WindowedValue<V>>> inputValueCoder = inputKvCoder.getValueCoder();
+
+ IterableCoder<WindowedValue<V>> inputIterableValueCoder =
+ (IterableCoder<WindowedValue<V>>) inputValueCoder;
+ Coder<WindowedValue<V>> inputIterableElementCoder = inputIterableValueCoder.getElemCoder();
+ WindowedValueCoder<V> inputIterableWindowedValueCoder =
+ (WindowedValueCoder<V>) inputIterableElementCoder;
+
+ Coder<V> inputIterableElementValueCoder = inputIterableWindowedValueCoder.getValueCoder();
+ Coder<Iterable<V>> outputValueCoder = IterableCoder.of(inputIterableElementValueCoder);
+ Coder<KV<K, Iterable<V>>> outputKvCoder = KvCoder.of(keyCoder, outputValueCoder);
+
+ return input
+ .apply(ParDo.of(groupAlsoByWindowsFn(windowingStrategy, inputIterableElementValueCoder)))
+ .setCoder(outputKvCoder);
+ }
+
+ private <W extends BoundedWindow>
+ GroupAlsoByWindowsViaOutputBufferDoFn<K, V, Iterable<V>, W> groupAlsoByWindowsFn(
+ WindowingStrategy<?, W> strategy, Coder<V> inputIterableElementValueCoder) {
+ return new GroupAlsoByWindowsViaOutputBufferDoFn<K, V, Iterable<V>, W>(
+ strategy, SystemReduceFn.<K, V, W>buffering(inputIterableElementValueCoder));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java
new file mode 100644
index 0000000..4815162
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.KV;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+import org.joda.time.Instant;
+
+/**
+ * A customized {@link DoFnRunner} that handles late data dropping for
+ * a {@link KeyedWorkItem} input {@link DoFn}.
+ *
+ * <p>It expands windows before checking data lateness.
+ *
+ * <p>{@link KeyedWorkItem KeyedWorkItems} are always in empty windows.
+ *
+ * @param <K> key type
+ * @param <InputT> input value element type
+ * @param <OutputT> output value element type
+ * @param <W> window type
+ */
+public class LateDataDroppingDoFnRunner<K, InputT, OutputT, W extends BoundedWindow>
+ implements DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> {
+ private final DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> doFnRunner;
+ private final LateDataFilter lateDataFilter;
+
+ public LateDataDroppingDoFnRunner(
+ DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> doFnRunner,
+ WindowingStrategy<?, ?> windowingStrategy,
+ TimerInternals timerInternals,
+ Aggregator<Long, Long> droppedDueToLateness) {
+ this.doFnRunner = doFnRunner;
+ lateDataFilter = new LateDataFilter(windowingStrategy, timerInternals, droppedDueToLateness);
+ }
+
+ @Override
+ public void startBundle() {
+ doFnRunner.startBundle();
+ }
+
+ @Override
+ public void processElement(WindowedValue<KeyedWorkItem<K, InputT>> elem) {
+ Iterable<WindowedValue<InputT>> nonLateElements = lateDataFilter.filter(
+ elem.getValue().key(), elem.getValue().elementsIterable());
+ KeyedWorkItem<K, InputT> keyedWorkItem = KeyedWorkItems.workItem(
+ elem.getValue().key(), elem.getValue().timersIterable(), nonLateElements);
+ doFnRunner.processElement(elem.withValue(keyedWorkItem));
+ }
+
+ @Override
+ public void finishBundle() {
+ doFnRunner.finishBundle();
+ }
+
+ /**
+ * It filters late data in a {@link KeyedWorkItem}.
+ */
+ @VisibleForTesting
+ static class LateDataFilter {
+ private final WindowingStrategy<?, ?> windowingStrategy;
+ private final TimerInternals timerInternals;
+ private final Aggregator<Long, Long> droppedDueToLateness;
+
+ public LateDataFilter(
+ WindowingStrategy<?, ?> windowingStrategy,
+ TimerInternals timerInternals,
+ Aggregator<Long, Long> droppedDueToLateness) {
+ this.windowingStrategy = windowingStrategy;
+ this.timerInternals = timerInternals;
+ this.droppedDueToLateness = droppedDueToLateness;
+ }
+
+ /**
+ * Returns an {@code Iterable<WindowedValue<InputT>>} that only contains
+ * non-late input elements.
+ */
+ public <K, InputT> Iterable<WindowedValue<InputT>> filter(
+ final K key, Iterable<WindowedValue<InputT>> elements) {
+ Iterable<Iterable<WindowedValue<InputT>>> windowsExpandedElements = Iterables.transform(
+ elements,
+ new Function<WindowedValue<InputT>, Iterable<WindowedValue<InputT>>>() {
+ @Override
+ public Iterable<WindowedValue<InputT>> apply(final WindowedValue<InputT> input) {
+ return Iterables.transform(
+ input.getWindows(),
+ new Function<BoundedWindow, WindowedValue<InputT>>() {
+ @Override
+ public WindowedValue<InputT> apply(BoundedWindow window) {
+ return WindowedValue.of(
+ input.getValue(), input.getTimestamp(), window, input.getPane());
+ }
+ });
+ }});
+
+ Iterable<WindowedValue<InputT>> nonLateElements = Iterables.filter(
+ Iterables.concat(windowsExpandedElements),
+ new Predicate<WindowedValue<InputT>>() {
+ @Override
+ public boolean apply(WindowedValue<InputT> input) {
+ BoundedWindow window = Iterables.getOnlyElement(input.getWindows());
+ if (canDropDueToExpiredWindow(window)) {
+ // The element is too late for this window.
+ droppedDueToLateness.addValue(1L);
+ WindowTracing.debug(
+ "ReduceFnRunner.processElement: Dropping element at {} for key:{}; window:{} "
+ + "since too far behind inputWatermark:{}; outputWatermark:{}",
+ input.getTimestamp(), key, window, timerInternals.currentInputWatermarkTime(),
+ timerInternals.currentOutputWatermarkTime());
+ return false;
+ } else {
+ return true;
+ }
+ }
+ });
+ return nonLateElements;
+ }
+
+ /** Is {@code window} expired w.r.t. the garbage collection watermark? */
+ private boolean canDropDueToExpiredWindow(BoundedWindow window) {
+ Instant inputWM = timerInternals.currentInputWatermarkTime();
+ return window.maxTimestamp().plus(windowingStrategy.getAllowedLateness()).isBefore(inputWM);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java
new file mode 100644
index 0000000..e809c24
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
+import org.apache.beam.sdk.util.state.MergingStateAccessor;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateMerging;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+
+/**
+ * Tracks which windows have non-empty panes. Specifically, which windows have new elements since
+ * their last triggering.
+ *
+ * @param <W> The kind of windows being tracked.
+ */
+public abstract class NonEmptyPanes<K, W extends BoundedWindow> {
+
+ static <K, W extends BoundedWindow> NonEmptyPanes<K, W> create(
+ WindowingStrategy<?, W> strategy, ReduceFn<K, ?, ?, W> reduceFn) {
+ if (strategy.getMode() == AccumulationMode.DISCARDING_FIRED_PANES) {
+ return new DiscardingModeNonEmptyPanes<>(reduceFn);
+ } else {
+ return new GeneralNonEmptyPanes<>();
+ }
+ }
+
+ /**
+ * Record that some content has been added to the window in {@code context}, and therefore the
+ * current pane is not empty.
+ */
+ public abstract void recordContent(StateAccessor<K> context);
+
+ /**
+ * Record that the given pane is empty.
+ */
+ public abstract void clearPane(StateAccessor<K> state);
+
+ /**
+ * Return true if the current pane for the window in {@code context} is empty.
+ */
+ public abstract ReadableState<Boolean> isEmpty(StateAccessor<K> context);
+
+ /**
+ * Prefetch in preparation for merging.
+ */
+ public abstract void prefetchOnMerge(MergingStateAccessor<K, W> state);
+
+ /**
+ * Eagerly merge backing state.
+ */
+ public abstract void onMerge(MergingStateAccessor<K, W> context);
+
+ /**
+ * An implementation of {@code NonEmptyPanes} optimized for use with discarding mode. Uses the
+ * presence of data in the accumulation buffer to record non-empty panes.
+ */
+ private static class DiscardingModeNonEmptyPanes<K, W extends BoundedWindow>
+ extends NonEmptyPanes<K, W> {
+
+ private ReduceFn<K, ?, ?, W> reduceFn;
+
+ private DiscardingModeNonEmptyPanes(ReduceFn<K, ?, ?, W> reduceFn) {
+ this.reduceFn = reduceFn;
+ }
+
+ @Override
+ public ReadableState<Boolean> isEmpty(StateAccessor<K> state) {
+ return reduceFn.isEmpty(state);
+ }
+
+ @Override
+ public void recordContent(StateAccessor<K> state) {
+ // Nothing to do -- the reduceFn is tracking contents
+ }
+
+ @Override
+ public void clearPane(StateAccessor<K> state) {
+ // Nothing to do -- the reduceFn is tracking contents
+ }
+
+ @Override
+ public void prefetchOnMerge(MergingStateAccessor<K, W> state) {
+ // Nothing to do -- the reduceFn is tracking contents
+ }
+
+ @Override
+ public void onMerge(MergingStateAccessor<K, W> context) {
+ // Nothing to do -- the reduceFn is tracking contents
+ }
+ }
+
+ /**
+ * An implementation of {@code NonEmptyPanes} for general use.
+ */
+ private static class GeneralNonEmptyPanes<K, W extends BoundedWindow>
+ extends NonEmptyPanes<K, W> {
+
+ private static final StateTag<Object, AccumulatorCombiningState<Long, long[], Long>>
+ PANE_ADDITIONS_TAG =
+ StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal(
+ "count", VarLongCoder.of(), new Sum.SumLongFn()));
+
+ @Override
+ public void recordContent(StateAccessor<K> state) {
+ state.access(PANE_ADDITIONS_TAG).add(1L);
+ }
+
+ @Override
+ public void clearPane(StateAccessor<K> state) {
+ state.access(PANE_ADDITIONS_TAG).clear();
+ }
+
+ @Override
+ public ReadableState<Boolean> isEmpty(StateAccessor<K> state) {
+ return state.access(PANE_ADDITIONS_TAG).isEmpty();
+ }
+
+ @Override
+ public void prefetchOnMerge(MergingStateAccessor<K, W> state) {
+ StateMerging.prefetchCombiningValues(state, PANE_ADDITIONS_TAG);
+ }
+
+ @Override
+ public void onMerge(MergingStateAccessor<K, W> context) {
+ StateMerging.mergeCombiningValues(context, PANE_ADDITIONS_TAG);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
new file mode 100644
index 0000000..5e08031
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.util.state.ReadableState;
+import org.apache.beam.sdk.util.state.StateAccessor;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.ValueState;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.joda.time.Instant;
+
+/**
+ * Determine the timing and other properties of a new pane for a given computation, key and window.
+ * Incorporates any previous pane, whether the pane has been produced because an
+ * on-time {@link AfterWatermark} trigger firing, and the relation between the element's timestamp
+ * and the current output watermark.
+ */
+public class PaneInfoTracker {
+ private TimerInternals timerInternals;
+
+ public PaneInfoTracker(TimerInternals timerInternals) {
+ this.timerInternals = timerInternals;
+ }
+
+ @VisibleForTesting
+ static final StateTag<Object, ValueState<PaneInfo>> PANE_INFO_TAG =
+ StateTags.makeSystemTagInternal(StateTags.value("pane", PaneInfoCoder.INSTANCE));
+
+ public void clear(StateAccessor<?> state) {
+ state.access(PANE_INFO_TAG).clear();
+ }
+
+ /**
+ * Return a ({@link ReadableState} for) the pane info appropriate for {@code context}. The pane
+ * info includes the timing for the pane, who's calculation is quite subtle.
+ *
+ * @param isFinal should be {@code true} only if the triggering machinery can guarantee
+ * no further firings for the
+ */
+ public ReadableState<PaneInfo> getNextPaneInfo(
+ ReduceFn<?, ?, ?, ?>.Context context, final boolean isFinal) {
+ final Object key = context.key();
+ final ReadableState<PaneInfo> previousPaneFuture =
+ context.state().access(PaneInfoTracker.PANE_INFO_TAG);
+ final Instant windowMaxTimestamp = context.window().maxTimestamp();
+
+ return new ReadableState<PaneInfo>() {
+ @Override
+ public ReadableState<PaneInfo> readLater() {
+ previousPaneFuture.readLater();
+ return this;
+ }
+
+ @Override
+ public PaneInfo read() {
+ PaneInfo previousPane = previousPaneFuture.read();
+ return describePane(key, windowMaxTimestamp, previousPane, isFinal);
+ }
+ };
+ }
+
+ public void storeCurrentPaneInfo(ReduceFn<?, ?, ?, ?>.Context context, PaneInfo currentPane) {
+ context.state().access(PANE_INFO_TAG).write(currentPane);
+ }
+
+ private <W> PaneInfo describePane(
+ Object key, Instant windowMaxTimestamp, PaneInfo previousPane, boolean isFinal) {
+ boolean isFirst = previousPane == null;
+ Timing previousTiming = isFirst ? null : previousPane.getTiming();
+ long index = isFirst ? 0 : previousPane.getIndex() + 1;
+ long nonSpeculativeIndex = isFirst ? 0 : previousPane.getNonSpeculativeIndex() + 1;
+ Instant outputWM = timerInternals.currentOutputWatermarkTime();
+ Instant inputWM = timerInternals.currentInputWatermarkTime();
+
+ // True if it is not possible to assign the element representing this pane a timestamp
+ // which will make an ON_TIME pane for any following computation.
+ // Ie true if the element's latest possible timestamp is before the current output watermark.
+ boolean isLateForOutput = outputWM != null && windowMaxTimestamp.isBefore(outputWM);
+
+ // True if all emitted panes (if any) were EARLY panes.
+ // Once the ON_TIME pane has fired, all following panes must be considered LATE even
+ // if the output watermark is behind the end of the window.
+ boolean onlyEarlyPanesSoFar = previousTiming == null || previousTiming == Timing.EARLY;
+
+ // True is the input watermark hasn't passed the window's max timestamp.
+ boolean isEarlyForInput = !inputWM.isAfter(windowMaxTimestamp);
+
+ Timing timing;
+ if (isLateForOutput || !onlyEarlyPanesSoFar) {
+ // The output watermark has already passed the end of this window, or we have already
+ // emitted a non-EARLY pane. Irrespective of how this pane was triggered we must
+ // consider this pane LATE.
+ timing = Timing.LATE;
+ } else if (isEarlyForInput) {
+ // This is an EARLY firing.
+ timing = Timing.EARLY;
+ nonSpeculativeIndex = -1;
+ } else {
+ // This is the unique ON_TIME firing for the window.
+ timing = Timing.ON_TIME;
+ }
+
+ WindowTracing.debug(
+ "describePane: {} pane (prev was {}) for key:{}; windowMaxTimestamp:{}; "
+ + "inputWatermark:{}; outputWatermark:{}; isLateForOutput:{}",
+ timing, previousTiming, key, windowMaxTimestamp, inputWM, outputWM, isLateForOutput);
+
+ if (previousPane != null) {
+ // Timing transitions should follow EARLY* ON_TIME? LATE*
+ switch (previousTiming) {
+ case EARLY:
+ Preconditions.checkState(
+ timing == Timing.EARLY || timing == Timing.ON_TIME || timing == Timing.LATE,
+ "EARLY cannot transition to %s", timing);
+ break;
+ case ON_TIME:
+ Preconditions.checkState(
+ timing == Timing.LATE, "ON_TIME cannot transition to %s", timing);
+ break;
+ case LATE:
+ Preconditions.checkState(timing == Timing.LATE, "LATE cannot transtion to %s", timing);
+ break;
+ case UNKNOWN:
+ break;
+ }
+ Preconditions.checkState(!previousPane.isLast(), "Last pane was not last after all.");
+ }
+
+ return PaneInfo.createPane(isFirst, isFinal, timing, index, nonSpeculativeIndex);
+ }
+}
[02/12] incubator-beam git commit: Move some easy stuff into
runners/core-java
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
deleted file mode 100644
index b7ec540..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
+++ /dev/null
@@ -1,1448 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static org.apache.beam.sdk.WindowMatchers.isSingleWindowedValue;
-import static org.apache.beam.sdk.WindowMatchers.isWindowedValue;
-
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.withSettings;
-
-import org.apache.beam.sdk.WindowMatchers;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext;
-import org.apache.beam.sdk.transforms.CombineWithContext.Context;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.AfterEach;
-import org.apache.beam.sdk.transforms.windowing.AfterFirst;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.Never;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-import org.apache.beam.sdk.transforms.windowing.Sessions;
-import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TimestampedValue;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterables;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Matchers;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.MockitoAnnotations;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-
-/**
- * Tests for {@link ReduceFnRunner}. These tests instantiate a full "stack" of
- * {@link ReduceFnRunner} with enclosed {@link ReduceFn}, down to the installed {@link Trigger}
- * (sometimes mocked). They proceed by injecting elements and advancing watermark and
- * processing time, then verifying produced panes and counters.
- */
-@RunWith(JUnit4.class)
-public class ReduceFnRunnerTest {
- @Mock private SideInputReader mockSideInputReader;
- private Trigger mockTrigger;
- private PCollectionView<Integer> mockView;
-
- private IntervalWindow firstWindow;
-
- private static Trigger.TriggerContext anyTriggerContext() {
- return Mockito.<Trigger.TriggerContext>any();
- }
- private static Trigger.OnElementContext anyElementContext() {
- return Mockito.<Trigger.OnElementContext>any();
- }
-
- @Before
- public void setUp() {
- MockitoAnnotations.initMocks(this);
-
- mockTrigger = mock(Trigger.class, withSettings().serializable());
- when(mockTrigger.buildTrigger()).thenReturn(mockTrigger);
-
- @SuppressWarnings("unchecked")
- PCollectionView<Integer> mockViewUnchecked =
- mock(PCollectionView.class, withSettings().serializable());
- mockView = mockViewUnchecked;
- firstWindow = new IntervalWindow(new Instant(0), new Instant(10));
- }
-
- private void injectElement(ReduceFnTester<Integer, ?, IntervalWindow> tester, int element)
- throws Exception {
- doNothing().when(mockTrigger).onElement(anyElementContext());
- tester.injectElements(TimestampedValue.of(element, new Instant(element)));
- }
-
- private void triggerShouldFinish(Trigger mockTrigger) throws Exception {
- doAnswer(new Answer<Void>() {
- @Override
- public Void answer(InvocationOnMock invocation) throws Exception {
- @SuppressWarnings("unchecked")
- Trigger.TriggerContext context =
- (Trigger.TriggerContext) invocation.getArguments()[0];
- context.trigger().setFinished(true);
- return null;
- }
- })
- .when(mockTrigger).onFire(anyTriggerContext());
- }
-
- @Test
- public void testOnElementBufferingDiscarding() throws Exception {
- // Test basic execution of a trigger using a non-combining window set and discarding mode.
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
- AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- // Pane of {1, 2}
- injectElement(tester, 1);
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- injectElement(tester, 2);
- assertThat(tester.extractOutput(),
- contains(isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)));
-
- // Pane of just 3, and finish
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- injectElement(tester, 3);
- assertThat(tester.extractOutput(),
- contains(isSingleWindowedValue(containsInAnyOrder(3), 3, 0, 10)));
- assertTrue(tester.isMarkedFinished(firstWindow));
- tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
-
- // This element shouldn't be seen, because the trigger has finished
- injectElement(tester, 4);
-
- assertEquals(1, tester.getElementsDroppedDueToClosedWindow());
- }
-
- @Test
- public void testOnElementBufferingAccumulating() throws Exception {
- // Test basic execution of a trigger using a non-combining window set and accumulating mode.
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
- AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(100),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- injectElement(tester, 1);
-
- // Fires {1, 2}
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- injectElement(tester, 2);
-
- // Fires {1, 2, 3} because we are in accumulating mode
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- injectElement(tester, 3);
-
- // This element shouldn't be seen, because the trigger has finished
- injectElement(tester, 4);
-
- assertThat(
- tester.extractOutput(),
- contains(
- isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10),
- isSingleWindowedValue(containsInAnyOrder(1, 2, 3), 3, 0, 10)));
- assertTrue(tester.isMarkedFinished(firstWindow));
- tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
- }
-
- @Test
- public void testOnElementCombiningDiscarding() throws Exception {
- // Test basic execution of a trigger using a non-combining window set and discarding mode.
- ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining(
- FixedWindows.of(Duration.millis(10)), mockTrigger, AccumulationMode.DISCARDING_FIRED_PANES,
- new Sum.SumIntegerFn().<String>asKeyedFn(), VarIntCoder.of(), Duration.millis(100));
-
- injectElement(tester, 2);
-
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- injectElement(tester, 3);
-
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- injectElement(tester, 4);
-
- // This element shouldn't be seen, because the trigger has finished
- injectElement(tester, 6);
-
- assertThat(
- tester.extractOutput(),
- contains(
- isSingleWindowedValue(equalTo(5), 2, 0, 10),
- isSingleWindowedValue(equalTo(4), 4, 0, 10)));
- assertTrue(tester.isMarkedFinished(firstWindow));
- tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
- }
-
- /**
- * Tests that the garbage collection time for a fixed window does not overflow the end of time.
- */
- @Test
- public void testFixedWindowEndOfTimeGarbageCollection() throws Exception {
-
- Duration allowedLateness = Duration.standardDays(365);
- Duration windowSize = Duration.millis(10);
- WindowFn<Object, IntervalWindow> windowFn = FixedWindows.of(windowSize);
-
- // This timestamp falls into a window where the end of the window is before the end of the
- // global window - the "end of time" - yet its expiration time is after.
- final Instant elementTimestamp =
- GlobalWindow.INSTANCE.maxTimestamp().minus(allowedLateness).plus(1);
-
- IntervalWindow window = Iterables.getOnlyElement(
- windowFn.assignWindows(
- windowFn.new AssignContext() {
- @Override
- public Object element() {
- throw new UnsupportedOperationException();
- }
- @Override
- public Instant timestamp() {
- return elementTimestamp;
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- throw new UnsupportedOperationException();
- }
- }));
-
- assertTrue(
- window.maxTimestamp().isBefore(GlobalWindow.INSTANCE.maxTimestamp()));
- assertTrue(
- window.maxTimestamp().plus(allowedLateness).isAfter(GlobalWindow.INSTANCE.maxTimestamp()));
-
- // Test basic execution of a trigger using a non-combining window set and accumulating mode.
- ReduceFnTester<Integer, Integer, IntervalWindow> tester =
- ReduceFnTester.combining(
- windowFn,
- AfterWatermark.pastEndOfWindow().withLateFirings(Never.ever()).buildTrigger(),
- AccumulationMode.DISCARDING_FIRED_PANES,
- new Sum.SumIntegerFn().<String>asKeyedFn(),
- VarIntCoder.of(),
- allowedLateness);
-
- tester.injectElements(TimestampedValue.of(13, elementTimestamp));
-
- // Should fire ON_TIME pane and there will be a checkState that the cleanup time
- // is prior to timestamp max value
- tester.advanceInputWatermark(window.maxTimestamp());
-
- // Nothing in the ON_TIME pane (not governed by triggers, but by ReduceFnRunner)
- assertThat(tester.extractOutput(), emptyIterable());
-
- tester.injectElements(TimestampedValue.of(42, elementTimestamp));
-
- // Now the final pane should fire, demonstrating that the GC time was truncated
- tester.advanceInputWatermark(GlobalWindow.INSTANCE.maxTimestamp());
- assertThat(tester.extractOutput(), contains(isWindowedValue(equalTo(55))));
- }
-
- @Test
- public void testOnElementCombiningAccumulating() throws Exception {
- // Test basic execution of a trigger using a non-combining window set and accumulating mode.
- ReduceFnTester<Integer, Integer, IntervalWindow> tester =
- ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)), mockTrigger,
- AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().<String>asKeyedFn(),
- VarIntCoder.of(), Duration.millis(100));
-
- injectElement(tester, 1);
-
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- injectElement(tester, 2);
-
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- injectElement(tester, 3);
-
- // This element shouldn't be seen, because the trigger has finished
- injectElement(tester, 4);
-
- assertThat(
- tester.extractOutput(),
- contains(
- isSingleWindowedValue(equalTo(3), 1, 0, 10),
- isSingleWindowedValue(equalTo(6), 3, 0, 10)));
- assertTrue(tester.isMarkedFinished(firstWindow));
- tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
- }
-
- @Test
- public void testOnElementCombiningWithContext() throws Exception {
- Integer expectedValue = 5;
- WindowingStrategy<?, IntervalWindow> windowingStrategy = WindowingStrategy
- .of(FixedWindows.of(Duration.millis(10)))
- .withTrigger(mockTrigger)
- .withMode(AccumulationMode.DISCARDING_FIRED_PANES)
- .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
- .withAllowedLateness(Duration.millis(100));
-
- TestOptions options = PipelineOptionsFactory.as(TestOptions.class);
- options.setValue(5);
-
- when(mockSideInputReader.contains(Matchers.<PCollectionView<Integer>>any())).thenReturn(true);
- when(mockSideInputReader.get(
- Matchers.<PCollectionView<Integer>>any(), any(BoundedWindow.class))).thenReturn(5);
-
- @SuppressWarnings({"rawtypes", "unchecked", "unused"})
- Object suppressWarningsVar = when(mockView.getWindowingStrategyInternal())
- .thenReturn((WindowingStrategy) windowingStrategy);
-
- SumAndVerifyContextFn combineFn = new SumAndVerifyContextFn(mockView, expectedValue);
- // Test basic execution of a trigger using a non-combining window set and discarding mode.
- ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining(
- windowingStrategy, combineFn.<String>asKeyedFn(),
- VarIntCoder.of(), options, mockSideInputReader);
-
- injectElement(tester, 2);
-
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- injectElement(tester, 3);
-
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- injectElement(tester, 4);
-
- // This element shouldn't be seen, because the trigger has finished
- injectElement(tester, 6);
-
- assertThat(
- tester.extractOutput(),
- contains(
- isSingleWindowedValue(equalTo(5), 2, 0, 10),
- isSingleWindowedValue(equalTo(4), 4, 0, 10)));
- assertTrue(tester.isMarkedFinished(firstWindow));
- tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
- }
-
- @Test
- public void testWatermarkHoldAndLateData() throws Exception {
- // Test handling of late data. Specifically, ensure the watermark hold is correct.
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
- AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(10),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- // Input watermark -> null
- assertEquals(null, tester.getWatermarkHold());
- assertEquals(null, tester.getOutputWatermark());
-
- // All on time data, verify watermark hold.
- injectElement(tester, 1);
- injectElement(tester, 3);
- assertEquals(new Instant(1), tester.getWatermarkHold());
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- injectElement(tester, 2);
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
- assertThat(output, contains(
- isSingleWindowedValue(containsInAnyOrder(1, 2, 3),
- 1, // timestamp
- 0, // window start
- 10))); // window end
- assertThat(output.get(0).getPane(),
- equalTo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)));
-
- // Holding for the end-of-window transition.
- assertEquals(new Instant(9), tester.getWatermarkHold());
- // Nothing dropped.
- assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
-
- // Input watermark -> 4, output watermark should advance that far as well
- tester.advanceInputWatermark(new Instant(4));
- assertEquals(new Instant(4), tester.getOutputWatermark());
-
- // Some late, some on time. Verify that we only hold to the minimum of on-time.
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
- tester.advanceInputWatermark(new Instant(4));
- injectElement(tester, 2);
- injectElement(tester, 3);
- assertEquals(new Instant(9), tester.getWatermarkHold());
- injectElement(tester, 5);
- assertEquals(new Instant(5), tester.getWatermarkHold());
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- injectElement(tester, 4);
- output = tester.extractOutput();
- assertThat(output,
- contains(
- isSingleWindowedValue(containsInAnyOrder(
- 1, 2, 3, // earlier firing
- 2, 3, 4, 5), // new elements
- 4, // timestamp
- 0, // window start
- 10))); // window end
- assertThat(output.get(0).getPane(),
- equalTo(PaneInfo.createPane(false, false, Timing.EARLY, 1, -1)));
-
- // All late -- output at end of window timestamp.
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
- tester.advanceInputWatermark(new Instant(8));
- injectElement(tester, 6);
- injectElement(tester, 5);
- assertEquals(new Instant(9), tester.getWatermarkHold());
- injectElement(tester, 4);
-
- // Fire the ON_TIME pane
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- tester.advanceInputWatermark(new Instant(10));
-
- // Output time is end of the window, because all the new data was late, but the pane
- // is the ON_TIME pane.
- output = tester.extractOutput();
- assertThat(output,
- contains(isSingleWindowedValue(
- containsInAnyOrder(1, 2, 3, // earlier firing
- 2, 3, 4, 5, // earlier firing
- 4, 5, 6), // new elements
- 9, // timestamp
- 0, // window start
- 10))); // window end
- assertThat(output.get(0).getPane(),
- equalTo(PaneInfo.createPane(false, false, Timing.ON_TIME, 2, 0)));
-
- // This is "pending" at the time the watermark makes it way-late.
- // Because we're about to expire the window, we output it.
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
- injectElement(tester, 8);
- assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
-
- // Exceed the GC limit, triggering the last pane to be fired
- tester.advanceInputWatermark(new Instant(50));
- output = tester.extractOutput();
- // Output time is still end of the window, because the new data (8) was behind
- // the output watermark.
- assertThat(output,
- contains(isSingleWindowedValue(
- containsInAnyOrder(1, 2, 3, // earlier firing
- 2, 3, 4, 5, // earlier firing
- 4, 5, 6, // earlier firing
- 8), // new element prior to window becoming expired
- 9, // timestamp
- 0, // window start
- 10))); // window end
- assertThat(
- output.get(0).getPane(),
- equalTo(PaneInfo.createPane(false, true, Timing.LATE, 3, 1)));
- assertEquals(new Instant(50), tester.getOutputWatermark());
- assertEquals(null, tester.getWatermarkHold());
-
- // Late timers are ignored
- tester.fireTimer(new IntervalWindow(new Instant(0), new Instant(10)), new Instant(12),
- TimeDomain.EVENT_TIME);
-
- // And because we're past the end of window + allowed lateness, everything should be cleaned up.
- assertFalse(tester.isMarkedFinished(firstWindow));
- tester.assertHasOnlyGlobalAndFinishedSetsFor();
- }
-
- @Test
- public void dontSetHoldIfTooLateForEndOfWindowTimer() throws Exception {
- // Make sure holds are only set if they are accompanied by an end-of-window timer.
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
- AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(10),
- ClosingBehavior.FIRE_ALWAYS);
- tester.setAutoAdvanceOutputWatermark(false);
-
- // Case: Unobservably late
- tester.advanceInputWatermark(new Instant(15));
- tester.advanceOutputWatermark(new Instant(11));
- injectElement(tester, 14);
- // Hold was applied, waiting for end-of-window timer.
- assertEquals(new Instant(14), tester.getWatermarkHold());
- assertEquals(new Instant(19), tester.getNextTimer(TimeDomain.EVENT_TIME));
-
- // Trigger the end-of-window timer.
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- tester.advanceInputWatermark(new Instant(20));
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
- // Hold has been replaced with garbage collection hold. Waiting for garbage collection.
- assertEquals(new Instant(29), tester.getWatermarkHold());
- assertEquals(new Instant(29), tester.getNextTimer(TimeDomain.EVENT_TIME));
-
- // Case: Maybe late 1
- injectElement(tester, 13);
- // No change to hold or timers.
- assertEquals(new Instant(29), tester.getWatermarkHold());
- assertEquals(new Instant(29), tester.getNextTimer(TimeDomain.EVENT_TIME));
-
- // Trigger the garbage collection timer.
- tester.advanceInputWatermark(new Instant(30));
-
- // Everything should be cleaned up.
- assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(10), new Instant(20))));
- tester.assertHasOnlyGlobalAndFinishedSetsFor();
- }
-
- @Test
- public void testPaneInfoAllStates() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
- AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- tester.advanceInputWatermark(new Instant(0));
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- injectElement(tester, 1);
- assertThat(tester.extractOutput(), contains(
- WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY))));
-
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- injectElement(tester, 2);
- assertThat(tester.extractOutput(), contains(
- WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.EARLY, 1, -1))));
-
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
- tester.advanceInputWatermark(new Instant(15));
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- injectElement(tester, 3);
- assertThat(tester.extractOutput(), contains(
- WindowMatchers.valueWithPaneInfo(
- PaneInfo.createPane(false, false, Timing.ON_TIME, 2, 0))));
-
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- injectElement(tester, 4);
- assertThat(tester.extractOutput(), contains(
- WindowMatchers.valueWithPaneInfo(
- PaneInfo.createPane(false, false, Timing.LATE, 3, 1))));
-
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- injectElement(tester, 5);
- assertThat(tester.extractOutput(), contains(
- WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 4, 2))));
- }
-
- @Test
- public void testPaneInfoAllStatesAfterWatermark() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
- WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
- .withTrigger(Repeatedly.forever(AfterFirst.of(
- AfterPane.elementCountAtLeast(2),
- AfterWatermark.pastEndOfWindow())))
- .withMode(AccumulationMode.DISCARDING_FIRED_PANES)
- .withAllowedLateness(Duration.millis(100))
- .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
- .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
-
- tester.advanceInputWatermark(new Instant(0));
- tester.injectElements(
- TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2)));
-
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
- assertThat(
- output,
- contains(WindowMatchers.valueWithPaneInfo(
- PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))));
- assertThat(
- output,
- contains(
- WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)));
-
- tester.advanceInputWatermark(new Instant(50));
-
- // We should get the ON_TIME pane even though it is empty,
- // because we have an AfterWatermark.pastEndOfWindow() trigger.
- output = tester.extractOutput();
- assertThat(
- output,
- contains(WindowMatchers.valueWithPaneInfo(
- PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0))));
- assertThat(
- output,
- contains(
- WindowMatchers.isSingleWindowedValue(emptyIterable(), 9, 0, 10)));
-
- // We should get the final pane even though it is empty.
- tester.advanceInputWatermark(new Instant(150));
- output = tester.extractOutput();
- assertThat(
- output,
- contains(
- WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 2, 1))));
- assertThat(
- output,
- contains(
- WindowMatchers.isSingleWindowedValue(emptyIterable(), 9, 0, 10)));
- }
-
- @Test
- public void noEmptyPanesFinalIfNonEmpty() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
- WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
- .withTrigger(Repeatedly.<IntervalWindow>forever(AfterFirst.<IntervalWindow>of(
- AfterPane.elementCountAtLeast(2),
- AfterWatermark.pastEndOfWindow())))
- .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES)
- .withAllowedLateness(Duration.millis(100))
- .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
- .withClosingBehavior(ClosingBehavior.FIRE_IF_NON_EMPTY));
-
- tester.advanceInputWatermark(new Instant(0));
- tester.injectElements(
- TimestampedValue.of(1, new Instant(1)),
- TimestampedValue.of(2, new Instant(2)));
- tester.advanceInputWatermark(new Instant(20));
- tester.advanceInputWatermark(new Instant(250));
-
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
- assertThat(output, contains(
- // Trigger with 2 elements
- WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10),
- // Trigger for the empty on time pane
- WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
- }
-
- @Test
- public void noEmptyPanesFinalAlways() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
- WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
- .withTrigger(Repeatedly.<IntervalWindow>forever(AfterFirst.<IntervalWindow>of(
- AfterPane.elementCountAtLeast(2),
- AfterWatermark.pastEndOfWindow())))
- .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES)
- .withAllowedLateness(Duration.millis(100))
- .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
- .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
-
- tester.advanceInputWatermark(new Instant(0));
- tester.injectElements(
- TimestampedValue.of(1, new Instant(1)),
- TimestampedValue.of(2, new Instant(2)));
- tester.advanceInputWatermark(new Instant(20));
- tester.advanceInputWatermark(new Instant(250));
-
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
- assertThat(output, contains(
- // Trigger with 2 elements
- WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10),
- // Trigger for the empty on time pane
- WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10),
- // Trigger for the final pane
- WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
- }
-
- @Test
- public void testPaneInfoAllStatesAfterWatermarkAccumulating() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
- WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
- .withTrigger(Repeatedly.forever(AfterFirst.of(
- AfterPane.elementCountAtLeast(2),
- AfterWatermark.pastEndOfWindow())))
- .withMode(AccumulationMode.ACCUMULATING_FIRED_PANES)
- .withAllowedLateness(Duration.millis(100))
- .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
- .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
-
- tester.advanceInputWatermark(new Instant(0));
- tester.injectElements(
- TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2)));
-
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
- assertThat(
- output,
- contains(WindowMatchers.valueWithPaneInfo(
- PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))));
- assertThat(
- output,
- contains(
- WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)));
-
- tester.advanceInputWatermark(new Instant(50));
-
- // We should get the ON_TIME pane even though it is empty,
- // because we have an AfterWatermark.pastEndOfWindow() trigger.
- output = tester.extractOutput();
- assertThat(
- output,
- contains(WindowMatchers.valueWithPaneInfo(
- PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0))));
- assertThat(
- output,
- contains(
- WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
-
- // We should get the final pane even though it is empty.
- tester.advanceInputWatermark(new Instant(150));
- output = tester.extractOutput();
- assertThat(
- output,
- contains(
- WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 2, 1))));
- assertThat(
- output,
- contains(
- WindowMatchers.isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10)));
- }
-
- @Test
- public void testPaneInfoFinalAndOnTime() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester = ReduceFnTester.nonCombining(
- WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
- .withTrigger(
- Repeatedly.forever(AfterPane.elementCountAtLeast(2))
- .orFinally(AfterWatermark.pastEndOfWindow()))
- .withMode(AccumulationMode.DISCARDING_FIRED_PANES)
- .withAllowedLateness(Duration.millis(100))
- .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS));
-
- tester.advanceInputWatermark(new Instant(0));
-
- // Should trigger due to element count
- tester.injectElements(
- TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2)));
-
- assertThat(
- tester.extractOutput(),
- contains(WindowMatchers.valueWithPaneInfo(
- PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))));
-
- tester.advanceInputWatermark(new Instant(150));
- assertThat(tester.extractOutput(), contains(
- WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.ON_TIME, 1, 0))));
- }
-
- @Test
- public void testPaneInfoSkipToFinish() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
- AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- tester.advanceInputWatermark(new Instant(0));
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- injectElement(tester, 1);
- assertThat(tester.extractOutput(), contains(
- WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.EARLY))));
- }
-
- @Test
- public void testPaneInfoSkipToNonSpeculativeAndFinish() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
- AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- tester.advanceInputWatermark(new Instant(15));
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- injectElement(tester, 1);
- assertThat(tester.extractOutput(), contains(
- WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, true, Timing.LATE))));
- }
-
- @Test
- public void testMergeBeforeFinalizing() throws Exception {
- // Verify that we merge windows before producing output so users don't see undesired
- // unmerged windows.
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
- AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(0),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- // All on time data, verify watermark hold.
- // These two windows should pre-merge immediately to [1, 20)
- tester.injectElements(
- TimestampedValue.of(1, new Instant(1)), // in [1, 11)
- TimestampedValue.of(10, new Instant(10))); // in [10, 20)
-
- // And this should fire the end-of-window timer
- tester.advanceInputWatermark(new Instant(100));
-
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
- assertThat(output.size(), equalTo(1));
- assertThat(output.get(0),
- isSingleWindowedValue(containsInAnyOrder(1, 10),
- 1, // timestamp
- 1, // window start
- 20)); // window end
- assertThat(
- output.get(0).getPane(),
- equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
- }
-
- /**
- * It is possible for a session window's trigger to be closed at the point at which
- * the (merged) session window is garbage collected. Make sure we don't accidentally
- * assume the window is still active.
- */
- @Test
- public void testMergingWithCloseBeforeGC() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
- AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- // Two elements in two overlapping session windows.
- tester.injectElements(
- TimestampedValue.of(1, new Instant(1)), // in [1, 11)
- TimestampedValue.of(10, new Instant(10))); // in [10, 20)
-
- // Close the trigger, but the gargbage collection timer is still pending.
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- tester.advanceInputWatermark(new Instant(30));
-
- // Now the garbage collection timer will fire, finding the trigger already closed.
- tester.advanceInputWatermark(new Instant(100));
-
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
- assertThat(output.size(), equalTo(1));
- assertThat(output.get(0),
- isSingleWindowedValue(containsInAnyOrder(1, 10),
- 1, // timestamp
- 1, // window start
- 20)); // window end
- assertThat(
- output.get(0).getPane(),
- equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
- }
-
- /**
- * Ensure a closed trigger has its state recorded in the merge result window.
- */
- @Test
- public void testMergingWithCloseTrigger() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
- AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- // Create a new merged session window.
- tester.injectElements(TimestampedValue.of(1, new Instant(1)),
- TimestampedValue.of(2, new Instant(2)));
-
- // Force the trigger to be closed for the merged window.
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- tester.advanceInputWatermark(new Instant(13));
-
- // Trigger is now closed.
- assertTrue(tester.isMarkedFinished(new IntervalWindow(new Instant(1), new Instant(12))));
-
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
-
- // Revisit the same session window.
- tester.injectElements(TimestampedValue.of(1, new Instant(1)),
- TimestampedValue.of(2, new Instant(2)));
-
- // Trigger is still closed.
- assertTrue(tester.isMarkedFinished(new IntervalWindow(new Instant(1), new Instant(12))));
- }
-
- /**
- * If a later event tries to reuse an earlier session window which has been closed, we
- * should reject that element and not fail due to the window no longer being active.
- */
- @Test
- public void testMergingWithReusedWindow() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
- AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- // One elements in one session window.
- tester.injectElements(TimestampedValue.of(1, new Instant(1))); // in [1, 11), gc at 21.
-
- // Close the trigger, but the gargbage collection timer is still pending.
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- tester.advanceInputWatermark(new Instant(15));
-
- // Another element in the same session window.
- // Should be discarded with 'window closed'.
- tester.injectElements(TimestampedValue.of(1, new Instant(1))); // in [1, 11), gc at 21.
-
- // And nothing should be left in the active window state.
- assertTrue(tester.hasNoActiveWindows());
-
- // Now the garbage collection timer will fire, finding the trigger already closed.
- tester.advanceInputWatermark(new Instant(100));
-
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
- assertThat(output.size(), equalTo(1));
- assertThat(output.get(0),
- isSingleWindowedValue(containsInAnyOrder(1),
- 1, // timestamp
- 1, // window start
- 11)); // window end
- assertThat(
- output.get(0).getPane(),
- equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
- }
-
- /**
- * When a merged window's trigger is closed we record that state using the merged window rather
- * than the original windows.
- */
- @Test
- public void testMergingWithClosedRepresentative() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
- AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- // 2 elements into merged session window.
- // Close the trigger, but the garbage collection timer is still pending.
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- tester.injectElements(TimestampedValue.of(1, new Instant(1)), // in [1, 11), gc at 21.
- TimestampedValue.of(8, new Instant(8))); // in [8, 18), gc at 28.
-
- // More elements into the same merged session window.
- // It has not yet been gced.
- // Should be discarded with 'window closed'.
- tester.injectElements(TimestampedValue.of(1, new Instant(1)), // in [1, 11), gc at 21.
- TimestampedValue.of(2, new Instant(2)), // in [2, 12), gc at 22.
- TimestampedValue.of(8, new Instant(8))); // in [8, 18), gc at 28.
-
- // Now the garbage collection timer will fire, finding the trigger already closed.
- tester.advanceInputWatermark(new Instant(100));
-
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
-
- assertThat(output.size(), equalTo(1));
- assertThat(output.get(0),
- isSingleWindowedValue(containsInAnyOrder(1, 8),
- 1, // timestamp
- 1, // window start
- 18)); // window end
- assertThat(
- output.get(0).getPane(),
- equalTo(PaneInfo.createPane(true, true, Timing.EARLY, 0, 0)));
- }
-
- /**
- * If an element for a closed session window ends up being merged into other still-open
- * session windows, the resulting session window is not 'poisoned'.
- */
- @Test
- public void testMergingWithClosedDoesNotPoison() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(Sessions.withGapDuration(Duration.millis(10)), mockTrigger,
- AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(50),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- // 1 element, force its trigger to close.
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- tester.injectElements(TimestampedValue.of(2, new Instant(2)));
-
- // 3 elements, one already closed.
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(false);
- tester.injectElements(TimestampedValue.of(1, new Instant(1)),
- TimestampedValue.of(2, new Instant(2)),
- TimestampedValue.of(3, new Instant(3)));
-
- tester.advanceInputWatermark(new Instant(100));
-
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
- assertThat(output.size(), equalTo(2));
- assertThat(output.get(0),
- isSingleWindowedValue(containsInAnyOrder(2),
- 2, // timestamp
- 2, // window start
- 12)); // window end
- assertThat(
- output.get(0).getPane(),
- equalTo(PaneInfo.createPane(true, true, Timing.EARLY, 0, 0)));
- assertThat(output.get(1),
- isSingleWindowedValue(containsInAnyOrder(1, 2, 3),
- 1, // timestamp
- 1, // window start
- 13)); // window end
- assertThat(
- output.get(1).getPane(),
- equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0)));
- }
-
- /**
- * Tests that when data is assigned to multiple windows but some of those windows have
- * had their triggers finish, then the data is dropped and counted accurately.
- */
- @Test
- public void testDropDataMultipleWindowsFinishedTrigger() throws Exception {
- ReduceFnTester<Integer, Integer, IntervalWindow> tester = ReduceFnTester.combining(
- WindowingStrategy.of(
- SlidingWindows.of(Duration.millis(100)).every(Duration.millis(30)))
- .withTrigger(AfterWatermark.pastEndOfWindow())
- .withAllowedLateness(Duration.millis(1000)),
- new Sum.SumIntegerFn().<String>asKeyedFn(), VarIntCoder.of());
-
- tester.injectElements(
- // assigned to [-60, 40), [-30, 70), [0, 100)
- TimestampedValue.of(10, new Instant(23)),
- // assigned to [-30, 70), [0, 100), [30, 130)
- TimestampedValue.of(12, new Instant(40)));
-
- assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
-
- tester.advanceInputWatermark(new Instant(70));
- tester.injectElements(
- // assigned to [-30, 70), [0, 100), [30, 130)
- // but [-30, 70) is closed by the trigger
- TimestampedValue.of(14, new Instant(60)));
-
- assertEquals(1, tester.getElementsDroppedDueToClosedWindow());
-
- tester.advanceInputWatermark(new Instant(130));
- // assigned to [-30, 70), [0, 100), [30, 130)
- // but they are all closed
- tester.injectElements(TimestampedValue.of(16, new Instant(40)));
-
- assertEquals(4, tester.getElementsDroppedDueToClosedWindow());
- }
-
- @Test
- public void testIdempotentEmptyPanesDiscarding() throws Exception {
- // Test uninteresting (empty) panes don't increment the index or otherwise
- // modify PaneInfo.
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
- AccumulationMode.DISCARDING_FIRED_PANES, Duration.millis(100),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- // Inject a couple of on-time elements and fire at the window end.
- injectElement(tester, 1);
- injectElement(tester, 2);
- tester.advanceInputWatermark(new Instant(12));
-
- // Fire the on-time pane
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
-
- // Fire another timer (with no data, so it's an uninteresting pane that should not be output).
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
-
- // Finish it off with another datum.
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- injectElement(tester, 3);
-
- // The intermediate trigger firing shouldn't result in any output.
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
- assertThat(output.size(), equalTo(2));
-
- // The on-time pane is as expected.
- assertThat(output.get(0), isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10));
-
- // The late pane has the correct indices.
- assertThat(output.get(1).getValue(), contains(3));
- assertThat(
- output.get(1).getPane(), equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1)));
-
- assertTrue(tester.isMarkedFinished(firstWindow));
- tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
-
- assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
- }
-
- @Test
- public void testIdempotentEmptyPanesAccumulating() throws Exception {
- // Test uninteresting (empty) panes don't increment the index or otherwise
- // modify PaneInfo.
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(FixedWindows.of(Duration.millis(10)), mockTrigger,
- AccumulationMode.ACCUMULATING_FIRED_PANES, Duration.millis(100),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- // Inject a couple of on-time elements and fire at the window end.
- injectElement(tester, 1);
- injectElement(tester, 2);
- tester.advanceInputWatermark(new Instant(12));
-
- // Trigger the on-time pane
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
- assertThat(output.size(), equalTo(1));
- assertThat(output.get(0), isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10));
- assertThat(output.get(0).getPane(),
- equalTo(PaneInfo.createPane(true, false, Timing.ON_TIME, 0, 0)));
-
- // Fire another timer with no data; the empty pane should not be output even though the
- // trigger is ready to fire
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME);
- assertThat(tester.extractOutput().size(), equalTo(0));
-
- // Finish it off with another datum, which is late
- when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true);
- triggerShouldFinish(mockTrigger);
- injectElement(tester, 3);
- output = tester.extractOutput();
- assertThat(output.size(), equalTo(1));
-
- // The late pane has the correct indices.
- assertThat(output.get(0).getValue(), containsInAnyOrder(1, 2, 3));
- assertThat(output.get(0).getPane(),
- equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1)));
-
- assertTrue(tester.isMarkedFinished(firstWindow));
- tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow);
-
- assertEquals(0, tester.getElementsDroppedDueToClosedWindow());
- }
-
- /**
- * Test that we receive an empty on-time pane when an or-finally waiting for the watermark fires.
- * Specifically, verify the proper triggerings and pane-info of a typical speculative/on-time/late
- * when the on-time pane is empty.
- */
- @Test
- public void testEmptyOnTimeFromOrFinally() throws Exception {
- ReduceFnTester<Integer, Integer, IntervalWindow> tester =
- ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)),
- AfterEach.<IntervalWindow>inOrder(
- Repeatedly
- .forever(
- AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
- new Duration(5)))
- .orFinally(AfterWatermark.pastEndOfWindow()),
- Repeatedly.forever(
- AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
- new Duration(25)))),
- AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().<String>asKeyedFn(),
- VarIntCoder.of(), Duration.millis(100));
-
- tester.advanceInputWatermark(new Instant(0));
- tester.advanceProcessingTime(new Instant(0));
-
- // Processing time timer for 5
- tester.injectElements(
- TimestampedValue.of(1, new Instant(1)),
- TimestampedValue.of(1, new Instant(3)),
- TimestampedValue.of(1, new Instant(7)),
- TimestampedValue.of(1, new Instant(5)));
-
- // Should fire early pane
- tester.advanceProcessingTime(new Instant(6));
-
- // Should fire empty on time pane
- tester.advanceInputWatermark(new Instant(11));
- List<WindowedValue<Integer>> output = tester.extractOutput();
- assertEquals(2, output.size());
-
- assertThat(output.get(0), WindowMatchers.isSingleWindowedValue(4, 1, 0, 10));
- assertThat(output.get(1), WindowMatchers.isSingleWindowedValue(4, 9, 0, 10));
-
- assertThat(
- output.get(0),
- WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)));
- assertThat(
- output.get(1),
- WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0)));
- }
-
- /**
- * Tests for processing time firings after the watermark passes the end of the window.
- * Specifically, verify the proper triggerings and pane-info of a typical speculative/on-time/late
- * when the on-time pane is non-empty.
- */
- @Test
- public void testProcessingTime() throws Exception {
- ReduceFnTester<Integer, Integer, IntervalWindow> tester =
- ReduceFnTester.combining(FixedWindows.of(Duration.millis(10)),
- AfterEach.<IntervalWindow>inOrder(
- Repeatedly
- .forever(
- AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
- new Duration(5)))
- .orFinally(AfterWatermark.pastEndOfWindow()),
- Repeatedly.forever(
- AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
- new Duration(25)))),
- AccumulationMode.ACCUMULATING_FIRED_PANES, new Sum.SumIntegerFn().<String>asKeyedFn(),
- VarIntCoder.of(), Duration.millis(100));
-
- tester.advanceInputWatermark(new Instant(0));
- tester.advanceProcessingTime(new Instant(0));
-
- tester.injectElements(TimestampedValue.of(1, new Instant(1)),
- TimestampedValue.of(1, new Instant(3)), TimestampedValue.of(1, new Instant(7)),
- TimestampedValue.of(1, new Instant(5)));
- // 4 elements all at processing time 0
-
- tester.advanceProcessingTime(new Instant(6)); // fire [1,3,7,5] since 6 > 0 + 5
- tester.injectElements(
- TimestampedValue.of(1, new Instant(8)),
- TimestampedValue.of(1, new Instant(4)));
- // 6 elements
-
- tester.advanceInputWatermark(new Instant(11)); // fire [1,3,7,5,8,4] since 11 > 9
- tester.injectElements(
- TimestampedValue.of(1, new Instant(8)),
- TimestampedValue.of(1, new Instant(4)),
- TimestampedValue.of(1, new Instant(5)));
- // 9 elements
-
- tester.advanceInputWatermark(new Instant(12));
- tester.injectElements(
- TimestampedValue.of(1, new Instant(3)));
- // 10 elements
-
- tester.advanceProcessingTime(new Instant(15));
- tester.injectElements(
- TimestampedValue.of(1, new Instant(5)));
- // 11 elements
- tester.advanceProcessingTime(new Instant(32)); // fire since 32 > 6 + 25
-
- tester.injectElements(
- TimestampedValue.of(1, new Instant(3)));
- // 12 elements
- // fire [1,3,7,5,8,4,8,4,5,3,5,3] since 125 > 6 + 25
- tester.advanceInputWatermark(new Instant(125));
-
- List<WindowedValue<Integer>> output = tester.extractOutput();
- assertEquals(4, output.size());
-
- assertThat(output.get(0), WindowMatchers.isSingleWindowedValue(4, 1, 0, 10));
- assertThat(output.get(1), WindowMatchers.isSingleWindowedValue(6, 4, 0, 10));
- assertThat(output.get(2), WindowMatchers.isSingleWindowedValue(11, 9, 0, 10));
- assertThat(output.get(3), WindowMatchers.isSingleWindowedValue(12, 9, 0, 10));
-
- assertThat(
- output.get(0),
- WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)));
- assertThat(
- output.get(1),
- WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.ON_TIME, 1, 0)));
- assertThat(
- output.get(2),
- WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, false, Timing.LATE, 2, 1)));
- assertThat(
- output.get(3),
- WindowMatchers.valueWithPaneInfo(PaneInfo.createPane(false, true, Timing.LATE, 3, 2)));
- }
-
- /**
- * We should fire a non-empty ON_TIME pane in the GlobalWindow when the watermark moves to
- * end-of-time.
- */
- @Test
- public void fireNonEmptyOnDrainInGlobalWindow() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, GlobalWindow> tester =
- ReduceFnTester.nonCombining(
- WindowingStrategy.of(new GlobalWindows())
- .withTrigger(Repeatedly.<GlobalWindow>forever(
- AfterPane.elementCountAtLeast(3)))
- .withMode(AccumulationMode.DISCARDING_FIRED_PANES));
-
- tester.advanceInputWatermark(new Instant(0));
-
- final int n = 20;
- for (int i = 0; i < n; i++) {
- tester.injectElements(TimestampedValue.of(i, new Instant(i)));
- }
-
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
- assertEquals(n / 3, output.size());
- for (int i = 0; i < output.size(); i++) {
- assertEquals(Timing.EARLY, output.get(i).getPane().getTiming());
- assertEquals(i, output.get(i).getPane().getIndex());
- assertEquals(3, Iterables.size(output.get(i).getValue()));
- }
-
- tester.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE);
-
- output = tester.extractOutput();
- assertEquals(1, output.size());
- assertEquals(Timing.ON_TIME, output.get(0).getPane().getTiming());
- assertEquals(n / 3, output.get(0).getPane().getIndex());
- assertEquals(n - ((n / 3) * 3), Iterables.size(output.get(0).getValue()));
- }
-
- /**
- * We should fire an empty ON_TIME pane in the GlobalWindow when the watermark moves to
- * end-of-time.
- */
- @Test
- public void fireEmptyOnDrainInGlobalWindowIfRequested() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, GlobalWindow> tester =
- ReduceFnTester.nonCombining(
- WindowingStrategy.of(new GlobalWindows())
- .withTrigger(Repeatedly.<GlobalWindow>forever(
- AfterProcessingTime.pastFirstElementInPane().plusDelayOf(
- new Duration(3))))
- .withMode(AccumulationMode.DISCARDING_FIRED_PANES));
-
- final int n = 20;
- for (int i = 0; i < n; i++) {
- tester.advanceProcessingTime(new Instant(i));
- tester.injectElements(TimestampedValue.of(i, new Instant(i)));
- }
- tester.advanceProcessingTime(new Instant(n + 4));
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
- assertEquals((n + 3) / 4, output.size());
- for (int i = 0; i < output.size(); i++) {
- assertEquals(Timing.EARLY, output.get(i).getPane().getTiming());
- assertEquals(i, output.get(i).getPane().getIndex());
- assertEquals(4, Iterables.size(output.get(i).getValue()));
- }
-
- tester.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE);
-
- output = tester.extractOutput();
- assertEquals(1, output.size());
- assertEquals(Timing.ON_TIME, output.get(0).getPane().getTiming());
- assertEquals((n + 3) / 4, output.get(0).getPane().getIndex());
- assertEquals(0, Iterables.size(output.get(0).getValue()));
- }
-
- /**
- * Late elements should still have a garbage collection hold set so that they
- * can make a late pane rather than be dropped due to lateness.
- */
- @Test
- public void setGarbageCollectionHoldOnLateElements() throws Exception {
- ReduceFnTester<Integer, Iterable<Integer>, IntervalWindow> tester =
- ReduceFnTester.nonCombining(
- FixedWindows.of(Duration.millis(10)),
- AfterWatermark.pastEndOfWindow().withLateFirings(AfterPane.elementCountAtLeast(2)),
- AccumulationMode.DISCARDING_FIRED_PANES,
- Duration.millis(100),
- ClosingBehavior.FIRE_IF_NON_EMPTY);
-
- tester.advanceInputWatermark(new Instant(0));
- tester.advanceOutputWatermark(new Instant(0));
- tester.injectElements(TimestampedValue.of(1, new Instant(1)));
-
- // Fire ON_TIME pane @ 9 with 1
-
- tester.advanceInputWatermark(new Instant(109));
- tester.advanceOutputWatermark(new Instant(109));
- tester.injectElements(TimestampedValue.of(2, new Instant(2)));
- // We should have set a garbage collection hold for the final pane.
- Instant hold = tester.getWatermarkHold();
- assertEquals(new Instant(109), hold);
-
- tester.advanceInputWatermark(new Instant(110));
- tester.advanceOutputWatermark(new Instant(110));
-
- // Fire final LATE pane @ 9 with 2
-
- List<WindowedValue<Iterable<Integer>>> output = tester.extractOutput();
- assertEquals(2, output.size());
- }
-
- private static class SumAndVerifyContextFn extends CombineFnWithContext<Integer, int[], Integer> {
-
- private final PCollectionView<Integer> view;
- private final int expectedValue;
-
- private SumAndVerifyContextFn(PCollectionView<Integer> view, int expectedValue) {
- this.view = view;
- this.expectedValue = expectedValue;
- }
- @Override
- public int[] createAccumulator(Context c) {
- Preconditions.checkArgument(
- c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
- Preconditions.checkArgument(c.sideInput(view) == expectedValue);
- return wrap(0);
- }
-
- @Override
- public int[] addInput(int[] accumulator, Integer input, Context c) {
- Preconditions.checkArgument(
- c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
- Preconditions.checkArgument(c.sideInput(view) == expectedValue);
- accumulator[0] += input.intValue();
- return accumulator;
- }
-
- @Override
- public int[] mergeAccumulators(Iterable<int[]> accumulators, Context c) {
- Preconditions.checkArgument(
- c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
- Preconditions.checkArgument(c.sideInput(view) == expectedValue);
- Iterator<int[]> iter = accumulators.iterator();
- if (!iter.hasNext()) {
- return createAccumulator(c);
- } else {
- int[] running = iter.next();
- while (iter.hasNext()) {
- running[0] += iter.next()[0];
- }
- return running;
- }
- }
-
- @Override
- public Integer extractOutput(int[] accumulator, Context c) {
- Preconditions.checkArgument(
- c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
- Preconditions.checkArgument(c.sideInput(view) == expectedValue);
- return accumulator[0];
- }
-
- private int[] wrap(int value) {
- return new int[] { value };
- }
- }
-
- /**
- * A {@link PipelineOptions} to test combining with context.
- */
- public interface TestOptions extends PipelineOptions {
- Integer getValue();
- void setValue(Integer value);
- }
-}
[12/12] incubator-beam git commit: This closes #510
Posted by ke...@apache.org.
This closes #510
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/e255cd6b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/e255cd6b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/e255cd6b
Branch: refs/heads/master
Commit: e255cd6be4eef7072c05bdf8720a766a2a597ea5
Parents: 73862b4 0fef8e6
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Jun 21 19:21:53 2016 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Jun 21 19:21:53 2016 -0700
----------------------------------------------------------------------
runners/core-java/pom.xml | 30 +
.../org/apache/beam/sdk/util/AssignWindows.java | 46 +
.../apache/beam/sdk/util/AssignWindowsDoFn.java | 75 +
.../beam/sdk/util/BatchTimerInternals.java | 140 ++
.../org/apache/beam/sdk/util/DoFnRunner.java | 62 +
.../apache/beam/sdk/util/DoFnRunnerBase.java | 558 +++++++
.../org/apache/beam/sdk/util/DoFnRunners.java | 144 ++
.../beam/sdk/util/GroupAlsoByWindowsDoFn.java | 59 +
.../GroupAlsoByWindowsViaOutputBufferDoFn.java | 100 ++
.../sdk/util/GroupByKeyViaGroupByKeyOnly.java | 212 +++
.../sdk/util/LateDataDroppingDoFnRunner.java | 147 ++
.../org/apache/beam/sdk/util/NonEmptyPanes.java | 150 ++
.../apache/beam/sdk/util/PaneInfoTracker.java | 154 ++
.../sdk/util/PushbackSideInputDoFnRunner.java | 115 ++
.../java/org/apache/beam/sdk/util/ReduceFn.java | 130 ++
.../beam/sdk/util/ReduceFnContextFactory.java | 497 ++++++
.../apache/beam/sdk/util/ReduceFnRunner.java | 985 ++++++++++++
.../apache/beam/sdk/util/SimpleDoFnRunner.java | 56 +
.../apache/beam/sdk/util/SystemReduceFn.java | 135 ++
.../org/apache/beam/sdk/util/TriggerRunner.java | 234 +++
.../org/apache/beam/sdk/util/WatermarkHold.java | 536 +++++++
.../beam/sdk/util/BatchTimerInternalsTest.java | 118 ++
.../sdk/util/GroupAlsoByWindowsProperties.java | 619 ++++++++
...oupAlsoByWindowsViaOutputBufferDoFnTest.java | 106 ++
.../util/LateDataDroppingDoFnRunnerTest.java | 117 ++
.../util/PushbackSideInputDoFnRunnerTest.java | 234 +++
.../beam/sdk/util/ReduceFnRunnerTest.java | 1448 ++++++++++++++++++
.../apache/beam/sdk/util/ReduceFnTester.java | 784 ++++++++++
.../beam/sdk/util/SimpleDoFnRunnerTest.java | 86 ++
.../beam/runners/direct/DirectGroupByKey.java | 2 +-
.../direct/GroupByKeyEvaluatorFactoryTest.java | 2 +-
.../GroupByKeyOnlyEvaluatorFactoryTest.java | 2 +-
.../org/apache/beam/sdk/util/AssignWindows.java | 46 -
.../apache/beam/sdk/util/AssignWindowsDoFn.java | 75 -
.../beam/sdk/util/BatchTimerInternals.java | 140 --
.../org/apache/beam/sdk/util/DoFnRunner.java | 62 -
.../apache/beam/sdk/util/DoFnRunnerBase.java | 558 -------
.../org/apache/beam/sdk/util/DoFnRunners.java | 144 --
.../apache/beam/sdk/util/GatherAllPanes.java | 1 -
.../beam/sdk/util/GroupAlsoByWindowsDoFn.java | 59 -
.../GroupAlsoByWindowsViaOutputBufferDoFn.java | 100 --
.../sdk/util/GroupByKeyViaGroupByKeyOnly.java | 247 ---
.../sdk/util/LateDataDroppingDoFnRunner.java | 147 --
.../org/apache/beam/sdk/util/NonEmptyPanes.java | 150 --
.../apache/beam/sdk/util/PaneInfoTracker.java | 154 --
.../sdk/util/PushbackSideInputDoFnRunner.java | 115 --
.../java/org/apache/beam/sdk/util/ReduceFn.java | 130 --
.../beam/sdk/util/ReduceFnContextFactory.java | 497 ------
.../apache/beam/sdk/util/ReduceFnRunner.java | 985 ------------
.../sdk/util/ReifyTimestampsAndWindows.java | 63 +
.../apache/beam/sdk/util/SimpleDoFnRunner.java | 56 -
.../apache/beam/sdk/util/SystemReduceFn.java | 135 --
.../org/apache/beam/sdk/util/TriggerRunner.java | 234 ---
.../org/apache/beam/sdk/util/WatermarkHold.java | 536 -------
.../beam/sdk/util/BatchTimerInternalsTest.java | 118 --
.../sdk/util/GroupAlsoByWindowsProperties.java | 619 --------
...oupAlsoByWindowsViaOutputBufferDoFnTest.java | 106 --
.../util/LateDataDroppingDoFnRunnerTest.java | 117 --
.../util/PushbackSideInputDoFnRunnerTest.java | 234 ---
.../beam/sdk/util/ReduceFnRunnerTest.java | 1448 ------------------
.../apache/beam/sdk/util/ReduceFnTester.java | 784 ----------
.../beam/sdk/util/SimpleDoFnRunnerTest.java | 86 --
62 files changed, 8143 insertions(+), 8086 deletions(-)
----------------------------------------------------------------------
[06/12] incubator-beam git commit: Move some easy stuff into
runners/core-java
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
new file mode 100644
index 0000000..9916c5c
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
@@ -0,0 +1,784 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
+import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.Trigger;
+import org.apache.beam.sdk.transforms.windowing.TriggerBuilder;
+import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
+import org.apache.beam.sdk.util.state.InMemoryStateInternals;
+import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+
+import com.google.common.base.Function;
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.PriorityQueue;
+import java.util.Set;
+
+import javax.annotation.Nullable;
+
+/**
+ * Test utility that runs a {@link ReduceFn}, {@link WindowFn}, {@link Trigger} using in-memory stub
+ * implementations to provide the {@link TimerInternals} and {@link WindowingInternals} needed to
+ * run {@code Trigger}s and {@code ReduceFn}s.
+ *
+ * @param <InputT> The element types.
+ * @param <OutputT> The final type for elements in the window (for instance,
+ * {@code Iterable<InputT>})
+ * @param <W> The type of windows being used.
+ */
+public class ReduceFnTester<InputT, OutputT, W extends BoundedWindow> {
+ private static final String KEY = "TEST_KEY";
+
+ private final TestInMemoryStateInternals<String> stateInternals =
+ new TestInMemoryStateInternals<>(KEY);
+ private final TestTimerInternals timerInternals = new TestTimerInternals();
+
+ private final WindowFn<Object, W> windowFn;
+ private final TestWindowingInternals windowingInternals;
+ private final Coder<OutputT> outputCoder;
+ private final WindowingStrategy<Object, W> objectStrategy;
+ private final ReduceFn<String, InputT, OutputT, W> reduceFn;
+ private final PipelineOptions options;
+
+ /**
+ * If true, the output watermark is automatically advanced to the latest possible
+ * point when the input watermark is advanced. This is the default for most tests.
+ * If false, the output watermark must be explicitly advanced by the test, which can
+ * be used to exercise some of the more subtle behavior of WatermarkHold.
+ */
+ private boolean autoAdvanceOutputWatermark;
+
+ private ExecutableTrigger executableTrigger;
+
+ private final InMemoryLongSumAggregator droppedDueToClosedWindow =
+ new InMemoryLongSumAggregator(GroupAlsoByWindowsDoFn.DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER);
+
+ public static <W extends BoundedWindow> ReduceFnTester<Integer, Iterable<Integer>, W>
+ nonCombining(WindowingStrategy<?, W> windowingStrategy) throws Exception {
+ return new ReduceFnTester<Integer, Iterable<Integer>, W>(
+ windowingStrategy,
+ SystemReduceFn.<String, Integer, W>buffering(VarIntCoder.of()),
+ IterableCoder.of(VarIntCoder.of()),
+ PipelineOptionsFactory.create(),
+ NullSideInputReader.empty());
+ }
+
+ public static <W extends BoundedWindow> ReduceFnTester<Integer, Iterable<Integer>, W>
+ nonCombining(WindowFn<?, W> windowFn, TriggerBuilder trigger, AccumulationMode mode,
+ Duration allowedDataLateness, ClosingBehavior closingBehavior) throws Exception {
+ WindowingStrategy<?, W> strategy =
+ WindowingStrategy.of(windowFn)
+ .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
+ .withTrigger(trigger.buildTrigger())
+ .withMode(mode)
+ .withAllowedLateness(allowedDataLateness)
+ .withClosingBehavior(closingBehavior);
+ return nonCombining(strategy);
+ }
+
+ public static <W extends BoundedWindow, AccumT, OutputT> ReduceFnTester<Integer, OutputT, W>
+ combining(WindowingStrategy<?, W> strategy,
+ KeyedCombineFn<String, Integer, AccumT, OutputT> combineFn,
+ Coder<OutputT> outputCoder) throws Exception {
+
+ CoderRegistry registry = new CoderRegistry();
+ registry.registerStandardCoders();
+ AppliedCombineFn<String, Integer, AccumT, OutputT> fn =
+ AppliedCombineFn.<String, Integer, AccumT, OutputT>withInputCoder(
+ combineFn, registry, KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()));
+
+ return new ReduceFnTester<Integer, OutputT, W>(
+ strategy,
+ SystemReduceFn.<String, Integer, AccumT, OutputT, W>combining(StringUtf8Coder.of(), fn),
+ outputCoder,
+ PipelineOptionsFactory.create(),
+ NullSideInputReader.empty());
+ }
+
+ public static <W extends BoundedWindow, AccumT, OutputT> ReduceFnTester<Integer, OutputT, W>
+ combining(WindowingStrategy<?, W> strategy,
+ KeyedCombineFnWithContext<String, Integer, AccumT, OutputT> combineFn,
+ Coder<OutputT> outputCoder,
+ PipelineOptions options,
+ SideInputReader sideInputReader) throws Exception {
+ CoderRegistry registry = new CoderRegistry();
+ registry.registerStandardCoders();
+ AppliedCombineFn<String, Integer, AccumT, OutputT> fn =
+ AppliedCombineFn.<String, Integer, AccumT, OutputT>withInputCoder(
+ combineFn, registry, KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()));
+
+ return new ReduceFnTester<Integer, OutputT, W>(
+ strategy,
+ SystemReduceFn.<String, Integer, AccumT, OutputT, W>combining(StringUtf8Coder.of(), fn),
+ outputCoder,
+ options,
+ sideInputReader);
+ }
+ public static <W extends BoundedWindow, AccumT, OutputT> ReduceFnTester<Integer, OutputT, W>
+ combining(WindowFn<?, W> windowFn, Trigger trigger, AccumulationMode mode,
+ KeyedCombineFn<String, Integer, AccumT, OutputT> combineFn, Coder<OutputT> outputCoder,
+ Duration allowedDataLateness) throws Exception {
+
+ WindowingStrategy<?, W> strategy =
+ WindowingStrategy.of(windowFn)
+ .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
+ .withTrigger(trigger)
+ .withMode(mode)
+ .withAllowedLateness(allowedDataLateness);
+
+ return combining(strategy, combineFn, outputCoder);
+ }
+
+ private ReduceFnTester(WindowingStrategy<?, W> wildcardStrategy,
+ ReduceFn<String, InputT, OutputT, W> reduceFn, Coder<OutputT> outputCoder,
+ PipelineOptions options, SideInputReader sideInputReader) throws Exception {
+ @SuppressWarnings("unchecked")
+ WindowingStrategy<Object, W> objectStrategy = (WindowingStrategy<Object, W>) wildcardStrategy;
+
+ this.objectStrategy = objectStrategy;
+ this.reduceFn = reduceFn;
+ this.windowFn = objectStrategy.getWindowFn();
+ this.windowingInternals = new TestWindowingInternals(sideInputReader);
+ this.outputCoder = outputCoder;
+ this.autoAdvanceOutputWatermark = true;
+ this.executableTrigger = wildcardStrategy.getTrigger();
+ this.options = options;
+ }
+
+ public void setAutoAdvanceOutputWatermark(boolean autoAdvanceOutputWatermark) {
+ this.autoAdvanceOutputWatermark = autoAdvanceOutputWatermark;
+ }
+
+ @Nullable
+ public Instant getNextTimer(TimeDomain domain) {
+ return timerInternals.getNextTimer(domain);
+ }
+
+ ReduceFnRunner<String, InputT, OutputT, W> createRunner() {
+ return new ReduceFnRunner<>(
+ KEY,
+ objectStrategy,
+ stateInternals,
+ timerInternals,
+ windowingInternals,
+ droppedDueToClosedWindow,
+ reduceFn,
+ options);
+ }
+
+ public ExecutableTrigger getTrigger() {
+ return executableTrigger;
+ }
+
+ public boolean isMarkedFinished(W window) {
+ return createRunner().isFinished(window);
+ }
+
+ public boolean hasNoActiveWindows() {
+ return createRunner().hasNoActiveWindows();
+ }
+
+ @SafeVarargs
+ public final void assertHasOnlyGlobalAndFinishedSetsFor(W... expectedWindows) {
+ assertHasOnlyGlobalAndAllowedTags(
+ ImmutableSet.copyOf(expectedWindows),
+ ImmutableSet.<StateTag<? super String, ?>>of(TriggerRunner.FINISHED_BITS_TAG));
+ }
+
+ @SafeVarargs
+ public final void assertHasOnlyGlobalAndFinishedSetsAndPaneInfoFor(W... expectedWindows) {
+ assertHasOnlyGlobalAndAllowedTags(
+ ImmutableSet.copyOf(expectedWindows),
+ ImmutableSet.<StateTag<? super String, ?>>of(
+ TriggerRunner.FINISHED_BITS_TAG, PaneInfoTracker.PANE_INFO_TAG,
+ WatermarkHold.watermarkHoldTagForOutputTimeFn(objectStrategy.getOutputTimeFn()),
+ WatermarkHold.EXTRA_HOLD_TAG));
+ }
+
+ public final void assertHasOnlyGlobalState() {
+ assertHasOnlyGlobalAndAllowedTags(
+ Collections.<W>emptySet(), Collections.<StateTag<? super String, ?>>emptySet());
+ }
+
+ @SafeVarargs
+ public final void assertHasOnlyGlobalAndPaneInfoFor(W... expectedWindows) {
+ assertHasOnlyGlobalAndAllowedTags(
+ ImmutableSet.copyOf(expectedWindows),
+ ImmutableSet.<StateTag<? super String, ?>>of(
+ PaneInfoTracker.PANE_INFO_TAG,
+ WatermarkHold.watermarkHoldTagForOutputTimeFn(objectStrategy.getOutputTimeFn()),
+ WatermarkHold.EXTRA_HOLD_TAG));
+ }
+
+ /**
+ * Verifies that the the set of windows that have any state stored is exactly
+ * {@code expectedWindows} and that each of these windows has only tags from {@code allowedTags}.
+ */
+ private void assertHasOnlyGlobalAndAllowedTags(
+ Set<W> expectedWindows, Set<StateTag<? super String, ?>> allowedTags) {
+ Set<StateNamespace> expectedWindowsSet = new HashSet<>();
+ for (W expectedWindow : expectedWindows) {
+ expectedWindowsSet.add(windowNamespace(expectedWindow));
+ }
+ Map<StateNamespace, Set<StateTag<? super String, ?>>> actualWindows = new HashMap<>();
+
+ for (StateNamespace namespace : stateInternals.getNamespacesInUse()) {
+ if (namespace instanceof StateNamespaces.GlobalNamespace) {
+ continue;
+ } else if (namespace instanceof StateNamespaces.WindowNamespace) {
+ Set<StateTag<? super String, ?>> tagsInUse = stateInternals.getTagsInUse(namespace);
+ if (tagsInUse.isEmpty()) {
+ continue;
+ }
+ actualWindows.put(namespace, tagsInUse);
+ Set<StateTag<? super String, ?>> unexpected = Sets.difference(tagsInUse, allowedTags);
+ if (unexpected.isEmpty()) {
+ continue;
+ } else {
+ fail(namespace + " has unexpected states: " + tagsInUse);
+ }
+ } else if (namespace instanceof StateNamespaces.WindowAndTriggerNamespace) {
+ Set<StateTag<? super String, ?>> tagsInUse = stateInternals.getTagsInUse(namespace);
+ assertTrue(namespace + " contains " + tagsInUse, tagsInUse.isEmpty());
+ } else {
+ fail("Unrecognized namespace " + namespace);
+ }
+ }
+
+ assertEquals("Still in use: " + actualWindows.toString(), expectedWindowsSet,
+ actualWindows.keySet());
+ }
+
+ private StateNamespace windowNamespace(W window) {
+ return StateNamespaces.window(windowFn.windowCoder(), window);
+ }
+
+ public Instant getWatermarkHold() {
+ return stateInternals.earliestWatermarkHold();
+ }
+
+ public Instant getOutputWatermark() {
+ return timerInternals.currentOutputWatermarkTime();
+ }
+
+ public long getElementsDroppedDueToClosedWindow() {
+ return droppedDueToClosedWindow.getSum();
+ }
+
+ /**
+ * How many panes do we have in the output?
+ */
+ public int getOutputSize() {
+ return windowingInternals.outputs.size();
+ }
+
+ /**
+ * Retrieve the values that have been output to this time, and clear out the output accumulator.
+ */
+ public List<WindowedValue<OutputT>> extractOutput() {
+ ImmutableList<WindowedValue<OutputT>> result =
+ FluentIterable.from(windowingInternals.outputs)
+ .transform(new Function<WindowedValue<KV<String, OutputT>>, WindowedValue<OutputT>>() {
+ @Override
+ public WindowedValue<OutputT> apply(WindowedValue<KV<String, OutputT>> input) {
+ return input.withValue(input.getValue().getValue());
+ }
+ })
+ .toList();
+ windowingInternals.outputs.clear();
+ return result;
+ }
+
+ /**
+ * Advance the input watermark to the specified time, firing any timers that should
+ * fire. Then advance the output watermark as far as possible.
+ */
+ public void advanceInputWatermark(Instant newInputWatermark) throws Exception {
+ ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
+ timerInternals.advanceInputWatermark(runner, newInputWatermark);
+ runner.persist();
+ }
+
+ /**
+ * If {@link #autoAdvanceOutputWatermark} is {@literal false}, advance the output watermark
+ * to the given value. Otherwise throw.
+ */
+ public void advanceOutputWatermark(Instant newOutputWatermark) throws Exception {
+ timerInternals.advanceOutputWatermark(newOutputWatermark);
+ }
+
+ /** Advance the processing time to the specified time, firing any timers that should fire. */
+ public void advanceProcessingTime(Instant newProcessingTime) throws Exception {
+ ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
+ timerInternals.advanceProcessingTime(runner, newProcessingTime);
+ runner.persist();
+ }
+
+ /**
+ * Advance the synchronized processing time to the specified time,
+ * firing any timers that should fire.
+ */
+ public void advanceSynchronizedProcessingTime(Instant newProcessingTime) throws Exception {
+ ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
+ timerInternals.advanceSynchronizedProcessingTime(runner, newProcessingTime);
+ runner.persist();
+ }
+
+ /**
+ * Inject all the timestamped values (after passing through the window function) as if they
+ * arrived in a single chunk of a bundle (or work-unit).
+ */
+ @SafeVarargs
+ public final void injectElements(TimestampedValue<InputT>... values) throws Exception {
+ for (TimestampedValue<InputT> value : values) {
+ WindowTracing.trace("TriggerTester.injectElements: {}", value);
+ }
+ ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
+ runner.processElements(Iterables.transform(
+ Arrays.asList(values), new Function<TimestampedValue<InputT>, WindowedValue<InputT>>() {
+ @Override
+ public WindowedValue<InputT> apply(TimestampedValue<InputT> input) {
+ try {
+ InputT value = input.getValue();
+ Instant timestamp = input.getTimestamp();
+ Collection<W> windows = windowFn.assignWindows(new TestAssignContext<W>(
+ windowFn, value, timestamp, Arrays.asList(GlobalWindow.INSTANCE)));
+ return WindowedValue.of(value, timestamp, windows, PaneInfo.NO_FIRING);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }));
+
+ // Persist after each bundle.
+ runner.persist();
+ }
+
+ public void fireTimer(W window, Instant timestamp, TimeDomain domain) throws Exception {
+ ReduceFnRunner<String, InputT, OutputT, W> runner = createRunner();
+ runner.onTimer(
+ TimerData.of(StateNamespaces.window(windowFn.windowCoder(), window), timestamp, domain));
+ runner.persist();
+ }
+
+ /**
+ * Simulate state.
+ */
+ private static class TestInMemoryStateInternals<K> extends InMemoryStateInternals<K> {
+
+ public TestInMemoryStateInternals(K key) {
+ super(key);
+ }
+
+ public Set<StateTag<? super K, ?>> getTagsInUse(StateNamespace namespace) {
+ Set<StateTag<? super K, ?>> inUse = new HashSet<>();
+ for (Entry<StateTag<? super K, ?>, State> entry :
+ inMemoryState.getTagsInUse(namespace).entrySet()) {
+ if (!isEmptyForTesting(entry.getValue())) {
+ inUse.add(entry.getKey());
+ }
+ }
+ return inUse;
+ }
+
+ public Set<StateNamespace> getNamespacesInUse() {
+ return inMemoryState.getNamespacesInUse();
+ }
+
+ /** Return the earliest output watermark hold in state, or null if none. */
+ public Instant earliestWatermarkHold() {
+ Instant minimum = null;
+ for (State storage : inMemoryState.values()) {
+ if (storage instanceof WatermarkHoldState) {
+ Instant hold = ((WatermarkHoldState<?>) storage).read();
+ if (minimum == null || (hold != null && hold.isBefore(minimum))) {
+ minimum = hold;
+ }
+ }
+ }
+ return minimum;
+ }
+ }
+
+ /**
+ * Convey the simulated state and implement {@link #outputWindowedValue} to capture all output
+ * elements.
+ */
+ private class TestWindowingInternals implements WindowingInternals<InputT, KV<String, OutputT>> {
+ private List<WindowedValue<KV<String, OutputT>>> outputs = new ArrayList<>();
+ private SideInputReader sideInputReader;
+
+ private TestWindowingInternals(SideInputReader sideInputReader) {
+ this.sideInputReader = sideInputReader;
+ }
+
+ @Override
+ public void outputWindowedValue(KV<String, OutputT> output, Instant timestamp,
+ Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ // Copy the output value (using coders) before capturing it.
+ KV<String, OutputT> copy = SerializableUtils.<KV<String, OutputT>>ensureSerializableByCoder(
+ KvCoder.of(StringUtf8Coder.of(), outputCoder), output, "outputForWindow");
+ WindowedValue<KV<String, OutputT>> value = WindowedValue.of(copy, timestamp, windows, pane);
+ outputs.add(value);
+ }
+
+ @Override
+ public TimerInternals timerInternals() {
+ throw new UnsupportedOperationException(
+ "Testing triggers should not use timers from WindowingInternals.");
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ throw new UnsupportedOperationException(
+ "Testing triggers should not use windows from WindowingInternals.");
+ }
+
+ @Override
+ public PaneInfo pane() {
+ throw new UnsupportedOperationException(
+ "Testing triggers should not use pane from WindowingInternals.");
+ }
+
+ @Override
+ public <T> void writePCollectionViewData(
+ TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
+ throw new UnsupportedOperationException(
+ "Testing triggers should not use writePCollectionViewData from WindowingInternals.");
+ }
+
+ @Override
+ public StateInternals<Object> stateInternals() {
+ // Safe for testing only
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ TestInMemoryStateInternals<Object> untypedStateInternals =
+ (TestInMemoryStateInternals) stateInternals;
+ return untypedStateInternals;
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+ if (!sideInputReader.contains(view)) {
+ throw new IllegalArgumentException("calling sideInput() with unknown view");
+ }
+ BoundedWindow sideInputWindow =
+ view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow);
+ return sideInputReader.get(view, sideInputWindow);
+ }
+ }
+
+ private static class TestAssignContext<W extends BoundedWindow>
+ extends WindowFn<Object, W>.AssignContext {
+ private Object element;
+ private Instant timestamp;
+ private Collection<? extends BoundedWindow> windows;
+
+ public TestAssignContext(WindowFn<Object, W> windowFn, Object element, Instant timestamp,
+ Collection<? extends BoundedWindow> windows) {
+ windowFn.super();
+ this.element = element;
+ this.timestamp = timestamp;
+ this.windows = windows;
+ }
+
+ @Override
+ public Object element() {
+ return element;
+ }
+
+ @Override
+ public Instant timestamp() {
+ return timestamp;
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ return windows;
+ }
+ }
+
+ private static class InMemoryLongSumAggregator implements Aggregator<Long, Long> {
+ private final String name;
+ private long sum = 0;
+
+ public InMemoryLongSumAggregator(String name) {
+ this.name = name;
+ }
+
+ @Override
+ public void addValue(Long value) {
+ sum += value;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public CombineFn<Long, ?, Long> getCombineFn() {
+ return new Sum.SumLongFn();
+ }
+
+ public long getSum() {
+ return sum;
+ }
+ }
+
+ /**
+ * Simulate the firing of timers and progression of input and output watermarks for a
+ * single computation and key in a Windmill-like streaming environment. Similar to
+ * {@link BatchTimerInternals}, but also tracks the output watermark.
+ */
+ private class TestTimerInternals implements TimerInternals {
+ /** At most one timer per timestamp is kept. */
+ private Set<TimerData> existingTimers = new HashSet<>();
+
+ /** Pending input watermark timers, in timestamp order. */
+ private PriorityQueue<TimerData> watermarkTimers = new PriorityQueue<>(11);
+
+ /** Pending processing time timers, in timestamp order. */
+ private PriorityQueue<TimerData> processingTimers = new PriorityQueue<>(11);
+
+ /** Current input watermark. */
+ @Nullable
+ private Instant inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
+
+ /** Current output watermark. */
+ @Nullable
+ private Instant outputWatermarkTime = null;
+
+ /** Current processing time. */
+ private Instant processingTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
+
+ /** Current synchronized processing time. */
+ @Nullable
+ private Instant synchronizedProcessingTime = null;
+
+ @Nullable
+ public Instant getNextTimer(TimeDomain domain) {
+ TimerData data = null;
+ switch (domain) {
+ case EVENT_TIME:
+ data = watermarkTimers.peek();
+ break;
+ case PROCESSING_TIME:
+ case SYNCHRONIZED_PROCESSING_TIME:
+ data = processingTimers.peek();
+ break;
+ }
+ Preconditions.checkNotNull(data); // cases exhaustive
+ return data == null ? null : data.getTimestamp();
+ }
+
+ private PriorityQueue<TimerData> queue(TimeDomain domain) {
+ switch (domain) {
+ case EVENT_TIME:
+ return watermarkTimers;
+ case PROCESSING_TIME:
+ case SYNCHRONIZED_PROCESSING_TIME:
+ return processingTimers;
+ }
+ throw new RuntimeException(); // cases exhaustive
+ }
+
+ @Override
+ public void setTimer(TimerData timer) {
+ WindowTracing.trace("TestTimerInternals.setTimer: {}", timer);
+ if (existingTimers.add(timer)) {
+ queue(timer.getDomain()).add(timer);
+ }
+ }
+
+ @Override
+ public void deleteTimer(TimerData timer) {
+ WindowTracing.trace("TestTimerInternals.deleteTimer: {}", timer);
+ existingTimers.remove(timer);
+ queue(timer.getDomain()).remove(timer);
+ }
+
+ @Override
+ public Instant currentProcessingTime() {
+ return processingTime;
+ }
+
+ @Override
+ @Nullable
+ public Instant currentSynchronizedProcessingTime() {
+ return synchronizedProcessingTime;
+ }
+
+ @Override
+ public Instant currentInputWatermarkTime() {
+ return Preconditions.checkNotNull(inputWatermarkTime);
+ }
+
+ @Override
+ @Nullable
+ public Instant currentOutputWatermarkTime() {
+ return outputWatermarkTime;
+ }
+
+ @Override
+ public String toString() {
+ return MoreObjects.toStringHelper(getClass())
+ .add("watermarkTimers", watermarkTimers)
+ .add("processingTimers", processingTimers)
+ .add("inputWatermarkTime", inputWatermarkTime)
+ .add("outputWatermarkTime", outputWatermarkTime)
+ .add("processingTime", processingTime)
+ .toString();
+ }
+
+ public void advanceInputWatermark(
+ ReduceFnRunner<?, ?, ?, ?> runner, Instant newInputWatermark) throws Exception {
+ Preconditions.checkNotNull(newInputWatermark);
+ Preconditions.checkState(
+ !newInputWatermark.isBefore(inputWatermarkTime),
+ "Cannot move input watermark time backwards from %s to %s", inputWatermarkTime,
+ newInputWatermark);
+ WindowTracing.trace("TestTimerInternals.advanceInputWatermark: from {} to {}",
+ inputWatermarkTime, newInputWatermark);
+ inputWatermarkTime = newInputWatermark;
+ advanceAndFire(runner, newInputWatermark, TimeDomain.EVENT_TIME);
+
+ Instant hold = stateInternals.earliestWatermarkHold();
+ if (hold == null) {
+ WindowTracing.trace("TestTimerInternals.advanceInputWatermark: no holds, "
+ + "so output watermark = input watermark");
+ hold = inputWatermarkTime;
+ }
+ if (autoAdvanceOutputWatermark) {
+ advanceOutputWatermark(hold);
+ }
+ }
+
+ public void advanceOutputWatermark(Instant newOutputWatermark) {
+ Preconditions.checkNotNull(newOutputWatermark);
+ if (newOutputWatermark.isAfter(inputWatermarkTime)) {
+ WindowTracing.trace(
+ "TestTimerInternals.advanceOutputWatermark: clipping output watermark from {} to {}",
+ newOutputWatermark, inputWatermarkTime);
+ newOutputWatermark = inputWatermarkTime;
+ }
+ Preconditions.checkState(
+ outputWatermarkTime == null || !newOutputWatermark.isBefore(outputWatermarkTime),
+ "Cannot move output watermark time backwards from %s to %s", outputWatermarkTime,
+ newOutputWatermark);
+ WindowTracing.trace("TestTimerInternals.advanceOutputWatermark: from {} to {}",
+ outputWatermarkTime, newOutputWatermark);
+ outputWatermarkTime = newOutputWatermark;
+ }
+
+ public void advanceProcessingTime(
+ ReduceFnRunner<?, ?, ?, ?> runner, Instant newProcessingTime) throws Exception {
+ Preconditions.checkState(!newProcessingTime.isBefore(processingTime),
+ "Cannot move processing time backwards from %s to %s", processingTime, newProcessingTime);
+ WindowTracing.trace("TestTimerInternals.advanceProcessingTime: from {} to {}", processingTime,
+ newProcessingTime);
+ processingTime = newProcessingTime;
+ advanceAndFire(runner, newProcessingTime, TimeDomain.PROCESSING_TIME);
+ }
+
+ public void advanceSynchronizedProcessingTime(
+ ReduceFnRunner<?, ?, ?, ?> runner, Instant newSynchronizedProcessingTime) throws Exception {
+ Preconditions.checkState(!newSynchronizedProcessingTime.isBefore(synchronizedProcessingTime),
+ "Cannot move processing time backwards from %s to %s", processingTime,
+ newSynchronizedProcessingTime);
+ WindowTracing.trace("TestTimerInternals.advanceProcessingTime: from {} to {}",
+ synchronizedProcessingTime, newSynchronizedProcessingTime);
+ synchronizedProcessingTime = newSynchronizedProcessingTime;
+ advanceAndFire(
+ runner, newSynchronizedProcessingTime, TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
+ }
+
+ private void advanceAndFire(
+ ReduceFnRunner<?, ?, ?, ?> runner, Instant currentTime, TimeDomain domain)
+ throws Exception {
+ PriorityQueue<TimerData> queue = queue(domain);
+ boolean shouldFire = false;
+
+ do {
+ TimerData timer = queue.peek();
+ // Timers fire when the current time progresses past the timer time.
+ shouldFire = timer != null && currentTime.isAfter(timer.getTimestamp());
+ if (shouldFire) {
+ WindowTracing.trace(
+ "TestTimerInternals.advanceAndFire: firing {} at {}", timer, currentTime);
+ // Remove before firing, so that if the trigger adds another identical
+ // timer we don't remove it.
+ queue.remove();
+
+ runner.onTimer(timer);
+ }
+ } while (shouldFire);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
new file mode 100644
index 0000000..fb74fc6
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.util;
+
+import static org.hamcrest.Matchers.is;
+import static org.mockito.Mockito.mock;
+
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.BaseExecutionContext.StepContext;
+import org.apache.beam.sdk.values.TupleTag;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Tests for base {@link DoFnRunnerBase} functionality.
+ */
+@RunWith(JUnit4.class)
+public class SimpleDoFnRunnerTest {
+ @Rule
+ public ExpectedException thrown = ExpectedException.none();
+
+ @Test
+ public void testExceptionsWrappedAsUserCodeException() {
+ ThrowingDoFn fn = new ThrowingDoFn();
+ DoFnRunner<String, String> runner = createRunner(fn);
+
+ thrown.expect(UserCodeException.class);
+ thrown.expectCause(is(fn.exceptionToThrow));
+
+ runner.processElement(WindowedValue.valueInGlobalWindow("anyValue"));
+ }
+
+ @Test
+ public void testSystemDoFnInternalExceptionsNotWrapped() {
+ ThrowingSystemDoFn fn = new ThrowingSystemDoFn();
+ DoFnRunner<String, String> runner = createRunner(fn);
+
+ thrown.expect(is(fn.exceptionToThrow));
+
+ runner.processElement(WindowedValue.valueInGlobalWindow("anyValue"));
+ }
+
+ private DoFnRunner<String, String> createRunner(DoFn<String, String> fn) {
+ // Pass in only necessary parameters for the test
+ List<TupleTag<?>> sideOutputTags = Arrays.asList();
+ StepContext context = mock(StepContext.class);
+ return new SimpleDoFnRunner<>(
+ null, fn, null, null, null, sideOutputTags, context, null, null);
+ }
+
+ static class ThrowingDoFn extends DoFn<String, String> {
+ final Exception exceptionToThrow =
+ new UnsupportedOperationException("Expected exception");
+
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ throw exceptionToThrow;
+ }
+ }
+
+ @SystemDoFnInternal
+ static class ThrowingSystemDoFn extends ThrowingDoFn {
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java
index 0200676..14103a6 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java
@@ -25,9 +25,9 @@ import org.apache.beam.sdk.coders.IterableCoder;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.transforms.GroupByKey;
import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
import org.apache.beam.sdk.util.KeyedWorkItem;
import org.apache.beam.sdk.util.KeyedWorkItemCoder;
+import org.apache.beam.sdk.util.ReifyTimestampsAndWindows;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.util.WindowingStrategy;
import org.apache.beam.sdk.values.KV;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
index a529043..78736c4 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
@@ -30,9 +30,9 @@ import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
import org.apache.beam.sdk.util.KeyedWorkItem;
import org.apache.beam.sdk.util.KeyedWorkItems;
+import org.apache.beam.sdk.util.ReifyTimestampsAndWindows;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java
index 5fb5a76..4afd64b 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java
@@ -30,9 +30,9 @@ import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
import org.apache.beam.sdk.util.KeyedWorkItem;
import org.apache.beam.sdk.util.KeyedWorkItems;
+import org.apache.beam.sdk.util.ReifyTimestampsAndWindows;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
deleted file mode 100644
index af28052..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * {@link PTransform} that uses privileged (non-user-facing) APIs to assign elements of a
- * {@link PCollection} to windows according to the provided {@link WindowFn}.
- *
- * @param <T> Type of elements being windowed
- * @param <W> Window type
- */
-public class AssignWindows<T, W extends BoundedWindow>
- extends PTransform<PCollection<T>, PCollection<T>> {
-
- private WindowFn<? super T, W> fn;
-
- public AssignWindows(WindowFn<? super T, W> fn) {
- this.fn = fn;
- }
-
- @Override
- public PCollection<T> apply(PCollection<T> input) {
- return input.apply("AssignWindows", ParDo.of(new AssignWindowsDoFn<>(fn)));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
deleted file mode 100644
index caec40e..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-
-import org.joda.time.Instant;
-
-import java.util.Collection;
-
-/**
- * {@link DoFn} that tags elements of a PCollection with windows, according
- * to the provided {@link WindowFn}.
- * @param <T> Type of elements being windowed
- * @param <W> Window type
- */
-@SystemDoFnInternal
-public class AssignWindowsDoFn<T, W extends BoundedWindow> extends DoFn<T, T> {
- private WindowFn<? super T, W> fn;
-
- public AssignWindowsDoFn(WindowFn<? super T, W> fn) {
- this.fn =
- checkNotNull(
- fn,
- "%s provided to %s cannot be null",
- WindowFn.class.getSimpleName(),
- AssignWindowsDoFn.class.getSimpleName());
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public void processElement(final ProcessContext c) throws Exception {
- Collection<W> windows =
- ((WindowFn<T, W>) fn).assignWindows(
- ((WindowFn<T, W>) fn).new AssignContext() {
- @Override
- public T element() {
- return c.element();
- }
-
- @Override
- public Instant timestamp() {
- return c.timestamp();
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- return c.windowingInternals().windows();
- }
- });
-
- c.windowingInternals()
- .outputWindowedValue(c.element(), c.timestamp(), windows, PaneInfo.NO_FIRING);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
deleted file mode 100644
index d0c0b2f..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
-
-import org.joda.time.Instant;
-
-import java.util.HashSet;
-import java.util.PriorityQueue;
-import java.util.Set;
-
-import javax.annotation.Nullable;
-
-/**
- * TimerInternals that uses priority queues to manage the timers that are ready to fire.
- */
-public class BatchTimerInternals implements TimerInternals {
- /** Set of timers that are scheduled used for deduplicating timers. */
- private Set<TimerData> existingTimers = new HashSet<>();
-
- // Keep these queues separate so we can advance over them separately.
- private PriorityQueue<TimerData> watermarkTimers = new PriorityQueue<>(11);
- private PriorityQueue<TimerData> processingTimers = new PriorityQueue<>(11);
-
- private Instant inputWatermarkTime;
- private Instant processingTime;
-
- private PriorityQueue<TimerData> queue(TimeDomain domain) {
- return TimeDomain.EVENT_TIME.equals(domain) ? watermarkTimers : processingTimers;
- }
-
- public BatchTimerInternals(Instant processingTime) {
- this.processingTime = processingTime;
- this.inputWatermarkTime = BoundedWindow.TIMESTAMP_MIN_VALUE;
- }
-
- @Override
- public void setTimer(TimerData timer) {
- if (existingTimers.add(timer)) {
- queue(timer.getDomain()).add(timer);
- }
- }
-
- @Override
- public void deleteTimer(TimerData timer) {
- existingTimers.remove(timer);
- queue(timer.getDomain()).remove(timer);
- }
-
- @Override
- public Instant currentProcessingTime() {
- return processingTime;
- }
-
- /**
- * {@inheritDoc}
- *
- * @return {@link BoundedWindow#TIMESTAMP_MAX_VALUE}: in batch mode, upstream processing
- * is already complete.
- */
- @Override
- @Nullable
- public Instant currentSynchronizedProcessingTime() {
- return BoundedWindow.TIMESTAMP_MAX_VALUE;
- }
-
- @Override
- public Instant currentInputWatermarkTime() {
- return inputWatermarkTime;
- }
-
- @Override
- @Nullable
- public Instant currentOutputWatermarkTime() {
- // The output watermark is always undefined in batch mode.
- return null;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("watermarkTimers", watermarkTimers)
- .add("processingTimers", processingTimers)
- .toString();
- }
-
- public void advanceInputWatermark(ReduceFnRunner<?, ?, ?, ?> runner, Instant newInputWatermark)
- throws Exception {
- Preconditions.checkState(!newInputWatermark.isBefore(inputWatermarkTime),
- "Cannot move input watermark time backwards from %s to %s", inputWatermarkTime,
- newInputWatermark);
- inputWatermarkTime = newInputWatermark;
- advance(runner, newInputWatermark, TimeDomain.EVENT_TIME);
- }
-
- public void advanceProcessingTime(ReduceFnRunner<?, ?, ?, ?> runner, Instant newProcessingTime)
- throws Exception {
- Preconditions.checkState(!newProcessingTime.isBefore(processingTime),
- "Cannot move processing time backwards from %s to %s", processingTime, newProcessingTime);
- processingTime = newProcessingTime;
- advance(runner, newProcessingTime, TimeDomain.PROCESSING_TIME);
- }
-
- private void advance(ReduceFnRunner<?, ?, ?, ?> runner, Instant newTime, TimeDomain domain)
- throws Exception {
- PriorityQueue<TimerData> timers = queue(domain);
- boolean shouldFire = false;
-
- do {
- TimerData timer = timers.peek();
- // Timers fire if the new time is ahead of the timer
- shouldFire = timer != null && newTime.isAfter(timer.getTimestamp());
- if (shouldFire) {
- // Remove before firing, so that if the trigger adds another identical
- // timer we don't remove it.
- timers.remove();
- runner.onTimer(timer);
- }
- } while (shouldFire);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java
deleted file mode 100644
index 4ec8920..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.DoFn.ProcessContext;
-import org.apache.beam.sdk.values.KV;
-
-/**
- * An wrapper interface that represents the execution of a {@link DoFn}.
- */
-public interface DoFnRunner<InputT, OutputT> {
- /**
- * Prepares and calls {@link DoFn#startBundle}.
- */
- public void startBundle();
-
- /**
- * Calls {@link DoFn#processElement} with a {@link ProcessContext} containing the current element.
- */
- public void processElement(WindowedValue<InputT> elem);
-
- /**
- * Calls {@link DoFn#finishBundle} and performs additional tasks, such as
- * flushing in-memory states.
- */
- public void finishBundle();
-
- /**
- * An internal interface for signaling that a {@link DoFn} requires late data dropping.
- */
- public interface ReduceFnExecutor<K, InputT, OutputT, W> {
- /**
- * Gets this object as a {@link DoFn}.
- *
- * Most implementors of this interface are expected to be {@link DoFn} instances, and will
- * return themselves.
- */
- DoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> asDoFn();
-
- /**
- * Returns an aggregator that tracks elements that are dropped due to being late.
- */
- Aggregator<Long, Long> getDroppedDueToLatenessAggregator();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
deleted file mode 100644
index 1ebe72b..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
+++ /dev/null
@@ -1,558 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.DoFn.RequiresWindowAccess;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
-import org.apache.beam.sdk.util.ExecutionContext.StepContext;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TupleTag;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-
-import org.joda.time.Instant;
-import org.joda.time.format.PeriodFormat;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * A base implementation of {@link DoFnRunner}.
- *
- * <p> Sub-classes should override {@link #invokeProcessElement}.
- */
-public abstract class DoFnRunnerBase<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
-
- /** The DoFn being run. */
- public final DoFn<InputT, OutputT> fn;
-
- /** The context used for running the DoFn. */
- public final DoFnContext<InputT, OutputT> context;
-
- protected DoFnRunnerBase(
- PipelineOptions options,
- DoFn<InputT, OutputT> fn,
- SideInputReader sideInputReader,
- OutputManager outputManager,
- TupleTag<OutputT> mainOutputTag,
- List<TupleTag<?>> sideOutputTags,
- StepContext stepContext,
- CounterSet.AddCounterMutator addCounterMutator,
- WindowingStrategy<?, ?> windowingStrategy) {
- this.fn = fn;
- this.context = new DoFnContext<>(
- options,
- fn,
- sideInputReader,
- outputManager,
- mainOutputTag,
- sideOutputTags,
- stepContext,
- addCounterMutator,
- windowingStrategy == null ? null : windowingStrategy.getWindowFn());
- }
-
- /**
- * An implementation of {@code OutputManager} using simple lists, for testing and in-memory
- * contexts such as the {@link DirectRunner}.
- */
- public static class ListOutputManager implements OutputManager {
-
- private Map<TupleTag<?>, List<WindowedValue<?>>> outputLists = Maps.newHashMap();
-
- @Override
- public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
- @SuppressWarnings({"rawtypes", "unchecked"})
- List<WindowedValue<T>> outputList = (List) outputLists.get(tag);
-
- if (outputList == null) {
- outputList = Lists.newArrayList();
- @SuppressWarnings({"rawtypes", "unchecked"})
- List<WindowedValue<?>> untypedList = (List) outputList;
- outputLists.put(tag, untypedList);
- }
-
- outputList.add(output);
- }
-
- public <T> List<WindowedValue<T>> getOutput(TupleTag<T> tag) {
- // Safe cast by design, inexpressible in Java without rawtypes
- @SuppressWarnings({"rawtypes", "unchecked"})
- List<WindowedValue<T>> outputList = (List) outputLists.get(tag);
- return (outputList != null) ? outputList : Collections.<WindowedValue<T>>emptyList();
- }
- }
-
- @Override
- public void startBundle() {
- // This can contain user code. Wrap it in case it throws an exception.
- try {
- fn.startBundle(context);
- } catch (Throwable t) {
- // Exception in user code.
- throw wrapUserCodeException(t);
- }
- }
-
- @Override
- public void processElement(WindowedValue<InputT> elem) {
- if (elem.getWindows().size() <= 1
- || (!RequiresWindowAccess.class.isAssignableFrom(fn.getClass())
- && context.sideInputReader.isEmpty())) {
- invokeProcessElement(elem);
- } else {
- // We could modify the windowed value (and the processContext) to
- // avoid repeated allocations, but this is more straightforward.
- for (WindowedValue<InputT> windowedValue : elem.explodeWindows()) {
- invokeProcessElement(windowedValue);
- }
- }
- }
-
- /**
- * Invokes {@link DoFn#processElement} after certain pre-processings has been done in
- * {@link DoFnRunnerBase#processElement}.
- */
- protected abstract void invokeProcessElement(WindowedValue<InputT> elem);
-
- @Override
- public void finishBundle() {
- // This can contain user code. Wrap it in case it throws an exception.
- try {
- fn.finishBundle(context);
- } catch (Throwable t) {
- // Exception in user code.
- throw wrapUserCodeException(t);
- }
- }
-
- /**
- * A concrete implementation of {@code DoFn.Context} used for running a {@link DoFn}.
- *
- * @param <InputT> the type of the DoFn's (main) input elements
- * @param <OutputT> the type of the DoFn's (main) output elements
- */
- private static class DoFnContext<InputT, OutputT>
- extends DoFn<InputT, OutputT>.Context {
- private static final int MAX_SIDE_OUTPUTS = 1000;
-
- final PipelineOptions options;
- final DoFn<InputT, OutputT> fn;
- final SideInputReader sideInputReader;
- final OutputManager outputManager;
- final TupleTag<OutputT> mainOutputTag;
- final StepContext stepContext;
- final CounterSet.AddCounterMutator addCounterMutator;
- final WindowFn<?, ?> windowFn;
-
- /**
- * The set of known output tags, some of which may be undeclared, so we can throw an
- * exception when it exceeds {@link #MAX_SIDE_OUTPUTS}.
- */
- private Set<TupleTag<?>> outputTags;
-
- public DoFnContext(PipelineOptions options,
- DoFn<InputT, OutputT> fn,
- SideInputReader sideInputReader,
- OutputManager outputManager,
- TupleTag<OutputT> mainOutputTag,
- List<TupleTag<?>> sideOutputTags,
- StepContext stepContext,
- CounterSet.AddCounterMutator addCounterMutator,
- WindowFn<?, ?> windowFn) {
- fn.super();
- this.options = options;
- this.fn = fn;
- this.sideInputReader = sideInputReader;
- this.outputManager = outputManager;
- this.mainOutputTag = mainOutputTag;
- this.outputTags = Sets.newHashSet();
-
- outputTags.add(mainOutputTag);
- for (TupleTag<?> sideOutputTag : sideOutputTags) {
- outputTags.add(sideOutputTag);
- }
-
- this.stepContext = stepContext;
- this.addCounterMutator = addCounterMutator;
- this.windowFn = windowFn;
- super.setupDelegateAggregators();
- }
-
- //////////////////////////////////////////////////////////////////////////////
-
- @Override
- public PipelineOptions getPipelineOptions() {
- return options;
- }
-
- <T, W extends BoundedWindow> WindowedValue<T> makeWindowedValue(
- T output, Instant timestamp, Collection<W> windows, PaneInfo pane) {
- final Instant inputTimestamp = timestamp;
-
- if (timestamp == null) {
- timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
- }
-
- if (windows == null) {
- try {
- // The windowFn can never succeed at accessing the element, so its type does not
- // matter here
- @SuppressWarnings("unchecked")
- WindowFn<Object, W> objectWindowFn = (WindowFn<Object, W>) windowFn;
- windows = objectWindowFn.assignWindows(objectWindowFn.new AssignContext() {
- @Override
- public Object element() {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input element when none was available");
- }
-
- @Override
- public Instant timestamp() {
- if (inputTimestamp == null) {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input timestamp when none was available");
- }
- return inputTimestamp;
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input windows when none were available");
- }
- });
- } catch (Exception e) {
- throw UserCodeException.wrap(e);
- }
- }
-
- return WindowedValue.of(output, timestamp, windows, pane);
- }
-
- public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
- if (!sideInputReader.contains(view)) {
- throw new IllegalArgumentException("calling sideInput() with unknown view");
- }
- BoundedWindow sideInputWindow =
- view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow);
- return sideInputReader.get(view, sideInputWindow);
- }
-
- void outputWindowedValue(
- OutputT output,
- Instant timestamp,
- Collection<? extends BoundedWindow> windows,
- PaneInfo pane) {
- outputWindowedValue(makeWindowedValue(output, timestamp, windows, pane));
- }
-
- void outputWindowedValue(WindowedValue<OutputT> windowedElem) {
- outputManager.output(mainOutputTag, windowedElem);
- if (stepContext != null) {
- stepContext.noteOutput(windowedElem);
- }
- }
-
- protected <T> void sideOutputWindowedValue(TupleTag<T> tag,
- T output,
- Instant timestamp,
- Collection<? extends BoundedWindow> windows,
- PaneInfo pane) {
- sideOutputWindowedValue(tag, makeWindowedValue(output, timestamp, windows, pane));
- }
-
- protected <T> void sideOutputWindowedValue(TupleTag<T> tag, WindowedValue<T> windowedElem) {
- if (!outputTags.contains(tag)) {
- // This tag wasn't declared nor was it seen before during this execution.
- // Thus, this must be a new, undeclared and unconsumed output.
- // To prevent likely user errors, enforce the limit on the number of side
- // outputs.
- if (outputTags.size() >= MAX_SIDE_OUTPUTS) {
- throw new IllegalArgumentException(
- "the number of side outputs has exceeded a limit of " + MAX_SIDE_OUTPUTS);
- }
- outputTags.add(tag);
- }
-
- outputManager.output(tag, windowedElem);
- if (stepContext != null) {
- stepContext.noteSideOutput(tag, windowedElem);
- }
- }
-
- // Following implementations of output, outputWithTimestamp, and sideOutput
- // are only accessible in DoFn.startBundle and DoFn.finishBundle, and will be shadowed by
- // ProcessContext's versions in DoFn.processElement.
- @Override
- public void output(OutputT output) {
- outputWindowedValue(output, null, null, PaneInfo.NO_FIRING);
- }
-
- @Override
- public void outputWithTimestamp(OutputT output, Instant timestamp) {
- outputWindowedValue(output, timestamp, null, PaneInfo.NO_FIRING);
- }
-
- @Override
- public <T> void sideOutput(TupleTag<T> tag, T output) {
- Preconditions.checkNotNull(tag, "TupleTag passed to sideOutput cannot be null");
- sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING);
- }
-
- @Override
- public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
- Preconditions.checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null");
- sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING);
- }
-
- private String generateInternalAggregatorName(String userName) {
- boolean system = fn.getClass().isAnnotationPresent(SystemDoFnInternal.class);
- return (system ? "" : "user-") + stepContext.getStepName() + "-" + userName;
- }
-
- @Override
- protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
- String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
- Preconditions.checkNotNull(combiner,
- "Combiner passed to createAggregator cannot be null");
- return new CounterAggregator<>(generateInternalAggregatorName(name),
- combiner, addCounterMutator);
- }
- }
-
- /**
- * Returns a new {@code DoFn.ProcessContext} for the given element.
- */
- protected DoFn<InputT, OutputT>.ProcessContext createProcessContext(WindowedValue<InputT> elem) {
- return new DoFnProcessContext<InputT, OutputT>(fn, context, elem);
- }
-
- protected RuntimeException wrapUserCodeException(Throwable t) {
- throw UserCodeException.wrapIf(!isSystemDoFn(), t);
- }
-
- private boolean isSystemDoFn() {
- return fn.getClass().isAnnotationPresent(SystemDoFnInternal.class);
- }
-
- /**
- * A concrete implementation of {@code DoFn.ProcessContext} used for
- * running a {@link DoFn} over a single element.
- *
- * @param <InputT> the type of the DoFn's (main) input elements
- * @param <OutputT> the type of the DoFn's (main) output elements
- */
- static class DoFnProcessContext<InputT, OutputT>
- extends DoFn<InputT, OutputT>.ProcessContext {
-
-
- final DoFn<InputT, OutputT> fn;
- final DoFnContext<InputT, OutputT> context;
- final WindowedValue<InputT> windowedValue;
-
- public DoFnProcessContext(DoFn<InputT, OutputT> fn,
- DoFnContext<InputT, OutputT> context,
- WindowedValue<InputT> windowedValue) {
- fn.super();
- this.fn = fn;
- this.context = context;
- this.windowedValue = windowedValue;
- }
-
- @Override
- public PipelineOptions getPipelineOptions() {
- return context.getPipelineOptions();
- }
-
- @Override
- public InputT element() {
- return windowedValue.getValue();
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view) {
- Preconditions.checkNotNull(view, "View passed to sideInput cannot be null");
- Iterator<? extends BoundedWindow> windowIter = windows().iterator();
- BoundedWindow window;
- if (!windowIter.hasNext()) {
- if (context.windowFn instanceof GlobalWindows) {
- // TODO: Remove this once GroupByKeyOnly no longer outputs elements
- // without windows
- window = GlobalWindow.INSTANCE;
- } else {
- throw new IllegalStateException(
- "sideInput called when main input element is not in any windows");
- }
- } else {
- window = windowIter.next();
- if (windowIter.hasNext()) {
- throw new IllegalStateException(
- "sideInput called when main input element is in multiple windows");
- }
- }
- return context.sideInput(view, window);
- }
-
- @Override
- public BoundedWindow window() {
- if (!(fn instanceof RequiresWindowAccess)) {
- throw new UnsupportedOperationException(
- "window() is only available in the context of a DoFn marked as RequiresWindow.");
- }
- return Iterables.getOnlyElement(windows());
- }
-
- @Override
- public PaneInfo pane() {
- return windowedValue.getPane();
- }
-
- @Override
- public void output(OutputT output) {
- context.outputWindowedValue(windowedValue.withValue(output));
- }
-
- @Override
- public void outputWithTimestamp(OutputT output, Instant timestamp) {
- checkTimestamp(timestamp);
- context.outputWindowedValue(output, timestamp,
- windowedValue.getWindows(), windowedValue.getPane());
- }
-
- void outputWindowedValue(
- OutputT output,
- Instant timestamp,
- Collection<? extends BoundedWindow> windows,
- PaneInfo pane) {
- context.outputWindowedValue(output, timestamp, windows, pane);
- }
-
- @Override
- public <T> void sideOutput(TupleTag<T> tag, T output) {
- Preconditions.checkNotNull(tag, "Tag passed to sideOutput cannot be null");
- context.sideOutputWindowedValue(tag, windowedValue.withValue(output));
- }
-
- @Override
- public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
- Preconditions.checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null");
- checkTimestamp(timestamp);
- context.sideOutputWindowedValue(
- tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane());
- }
-
- @Override
- public Instant timestamp() {
- return windowedValue.getTimestamp();
- }
-
- public Collection<? extends BoundedWindow> windows() {
- return windowedValue.getWindows();
- }
-
- private void checkTimestamp(Instant timestamp) {
- if (timestamp.isBefore(windowedValue.getTimestamp().minus(fn.getAllowedTimestampSkew()))) {
- throw new IllegalArgumentException(String.format(
- "Cannot output with timestamp %s. Output timestamps must be no earlier than the "
- + "timestamp of the current input (%s) minus the allowed skew (%s). See the "
- + "DoFn#getAllowedTimestampSkew() Javadoc for details on changing the allowed skew.",
- timestamp, windowedValue.getTimestamp(),
- PeriodFormat.getDefault().print(fn.getAllowedTimestampSkew().toPeriod())));
- }
- }
-
- @Override
- public WindowingInternals<InputT, OutputT> windowingInternals() {
- return new WindowingInternals<InputT, OutputT>() {
- @Override
- public void outputWindowedValue(OutputT output, Instant timestamp,
- Collection<? extends BoundedWindow> windows, PaneInfo pane) {
- context.outputWindowedValue(output, timestamp, windows, pane);
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- return windowedValue.getWindows();
- }
-
- @Override
- public PaneInfo pane() {
- return windowedValue.getPane();
- }
-
- @Override
- public TimerInternals timerInternals() {
- return context.stepContext.timerInternals();
- }
-
- @Override
- public <T> void writePCollectionViewData(
- TupleTag<?> tag,
- Iterable<WindowedValue<T>> data,
- Coder<T> elemCoder) throws IOException {
- @SuppressWarnings("unchecked")
- Coder<BoundedWindow> windowCoder = (Coder<BoundedWindow>) context.windowFn.windowCoder();
-
- context.stepContext.writePCollectionViewData(
- tag, data, IterableCoder.of(WindowedValue.getFullCoder(elemCoder, windowCoder)),
- window(), windowCoder);
- }
-
- @Override
- public StateInternals<?> stateInternals() {
- return context.stepContext.stateInternals();
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
- return context.sideInput(view, mainInputWindow);
- }
- };
- }
-
- @Override
- protected <AggregatorInputT, AggregatorOutputT> Aggregator<AggregatorInputT, AggregatorOutputT>
- createAggregatorInternal(
- String name, CombineFn<AggregatorInputT, ?, AggregatorOutputT> combiner) {
- return context.createAggregatorInternal(name, combiner);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java
deleted file mode 100644
index 648a281..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.DoFnRunner.ReduceFnExecutor;
-import org.apache.beam.sdk.util.ExecutionContext.StepContext;
-import org.apache.beam.sdk.util.common.CounterSet;
-import org.apache.beam.sdk.util.common.CounterSet.AddCounterMutator;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.TupleTag;
-
-import java.util.List;
-
-/**
- * Static utility methods that provide {@link DoFnRunner} implementations.
- */
-public class DoFnRunners {
- /**
- * Information about how to create output receivers and output to them.
- */
- public interface OutputManager {
- /**
- * Outputs a single element to the receiver indicated by the given {@link TupleTag}.
- */
- public <T> void output(TupleTag<T> tag, WindowedValue<T> output);
- }
-
- /**
- * Returns a basic implementation of {@link DoFnRunner} that works for most {@link DoFn DoFns}.
- *
- * <p>It invokes {@link DoFn#processElement} for each input.
- */
- public static <InputT, OutputT> DoFnRunner<InputT, OutputT> simpleRunner(
- PipelineOptions options,
- DoFn<InputT, OutputT> fn,
- SideInputReader sideInputReader,
- OutputManager outputManager,
- TupleTag<OutputT> mainOutputTag,
- List<TupleTag<?>> sideOutputTags,
- StepContext stepContext,
- CounterSet.AddCounterMutator addCounterMutator,
- WindowingStrategy<?, ?> windowingStrategy) {
- return new SimpleDoFnRunner<>(
- options,
- fn,
- sideInputReader,
- outputManager,
- mainOutputTag,
- sideOutputTags,
- stepContext,
- addCounterMutator,
- windowingStrategy);
- }
-
- /**
- * Returns an implementation of {@link DoFnRunner} that handles late data dropping.
- *
- * <p>It drops elements from expired windows before they reach the underlying {@link DoFn}.
- */
- public static <K, InputT, OutputT, W extends BoundedWindow>
- DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> lateDataDroppingRunner(
- PipelineOptions options,
- ReduceFnExecutor<K, InputT, OutputT, W> reduceFnExecutor,
- SideInputReader sideInputReader,
- OutputManager outputManager,
- TupleTag<KV<K, OutputT>> mainOutputTag,
- List<TupleTag<?>> sideOutputTags,
- StepContext stepContext,
- CounterSet.AddCounterMutator addCounterMutator,
- WindowingStrategy<?, W> windowingStrategy) {
- DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> simpleDoFnRunner =
- simpleRunner(
- options,
- reduceFnExecutor.asDoFn(),
- sideInputReader,
- outputManager,
- mainOutputTag,
- sideOutputTags,
- stepContext,
- addCounterMutator,
- windowingStrategy);
- return new LateDataDroppingDoFnRunner<>(
- simpleDoFnRunner,
- windowingStrategy,
- stepContext.timerInternals(),
- reduceFnExecutor.getDroppedDueToLatenessAggregator());
- }
-
- public static <InputT, OutputT> DoFnRunner<InputT, OutputT> createDefault(
- PipelineOptions options,
- DoFn<InputT, OutputT> doFn,
- SideInputReader sideInputReader,
- OutputManager outputManager,
- TupleTag<OutputT> mainOutputTag,
- List<TupleTag<?>> sideOutputTags,
- StepContext stepContext,
- AddCounterMutator addCounterMutator,
- WindowingStrategy<?, ?> windowingStrategy) {
- if (doFn instanceof ReduceFnExecutor) {
- @SuppressWarnings("rawtypes")
- ReduceFnExecutor fn = (ReduceFnExecutor) doFn;
- @SuppressWarnings({"unchecked", "cast", "rawtypes"})
- DoFnRunner<InputT, OutputT> runner = (DoFnRunner<InputT, OutputT>) lateDataDroppingRunner(
- options,
- fn,
- sideInputReader,
- outputManager,
- (TupleTag) mainOutputTag,
- sideOutputTags,
- stepContext,
- addCounterMutator,
- (WindowingStrategy) windowingStrategy);
- return runner;
- }
- return simpleRunner(
- options,
- doFn,
- sideInputReader,
- outputManager,
- mainOutputTag,
- sideOutputTags,
- stepContext,
- addCounterMutator,
- windowingStrategy);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GatherAllPanes.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GatherAllPanes.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GatherAllPanes.java
index 958d710..5a01c28 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GatherAllPanes.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GatherAllPanes.java
@@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.WithKeys;
import org.apache.beam.sdk.transforms.windowing.Never;
import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.TypeDescriptor;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java
deleted file mode 100644
index f5de0bc..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.KV;
-
-/**
- * DoFn that merges windows and groups elements in those windows, optionally
- * combining values.
- *
- * @param <K> key type
- * @param <InputT> input value element type
- * @param <OutputT> output value element type
- * @param <W> window type
- */
-@SystemDoFnInternal
-public abstract class GroupAlsoByWindowsDoFn<K, InputT, OutputT, W extends BoundedWindow>
- extends DoFn<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> {
- public static final String DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER = "DroppedDueToClosedWindow";
- public static final String DROPPED_DUE_TO_LATENESS_COUNTER = "DroppedDueToLateness";
-
- protected final Aggregator<Long, Long> droppedDueToClosedWindow =
- createAggregator(DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER, new Sum.SumLongFn());
- protected final Aggregator<Long, Long> droppedDueToLateness =
- createAggregator(DROPPED_DUE_TO_LATENESS_COUNTER, new Sum.SumLongFn());
-
- /**
- * Create the default {@link GroupAlsoByWindowsDoFn}, which uses window sets to implement the
- * grouping.
- *
- * @param windowingStrategy The window function and trigger to use for grouping
- * @param inputCoder the input coder to use
- */
- public static <K, V, W extends BoundedWindow> GroupAlsoByWindowsDoFn<K, V, Iterable<V>, W>
- createDefault(WindowingStrategy<?, W> windowingStrategy, Coder<V> inputCoder) {
- return new GroupAlsoByWindowsViaOutputBufferDoFn<>(
- windowingStrategy, SystemReduceFn.<K, V, W>buffering(inputCoder));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java
deleted file mode 100644
index d364168..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.values.KV;
-
-import com.google.common.collect.Iterables;
-
-import org.joda.time.Instant;
-
-import java.util.List;
-
-/**
- * The default batch {@link GroupAlsoByWindowsDoFn} implementation, if no specialized "fast path"
- * implementation is applicable.
- */
-@SystemDoFnInternal
-public class GroupAlsoByWindowsViaOutputBufferDoFn<K, InputT, OutputT, W extends BoundedWindow>
- extends GroupAlsoByWindowsDoFn<K, InputT, OutputT, W> {
-
- private final WindowingStrategy<?, W> strategy;
- private SystemReduceFn<K, InputT, ?, OutputT, W> reduceFn;
-
- public GroupAlsoByWindowsViaOutputBufferDoFn(
- WindowingStrategy<?, W> windowingStrategy,
- SystemReduceFn<K, InputT, ?, OutputT, W> reduceFn) {
- this.strategy = windowingStrategy;
- this.reduceFn = reduceFn;
- }
-
- @Override
- public void processElement(
- DoFn<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>>.ProcessContext c)
- throws Exception {
- K key = c.element().getKey();
- // Used with Batch, we know that all the data is available for this key. We can't use the
- // timer manager from the context because it doesn't exist. So we create one and emulate the
- // watermark, knowing that we have all data and it is in timestamp order.
- BatchTimerInternals timerInternals = new BatchTimerInternals(Instant.now());
-
- // It is the responsibility of the user of GroupAlsoByWindowsViaOutputBufferDoFn to only
- // provide a WindowingInternals instance with the appropriate key type for StateInternals.
- @SuppressWarnings("unchecked")
- StateInternals<K> stateInternals = (StateInternals<K>) c.windowingInternals().stateInternals();
-
- ReduceFnRunner<K, InputT, OutputT, W> reduceFnRunner =
- new ReduceFnRunner<K, InputT, OutputT, W>(
- key,
- strategy,
- stateInternals,
- timerInternals,
- c.windowingInternals(),
- droppedDueToClosedWindow,
- reduceFn,
- c.getPipelineOptions());
-
- Iterable<List<WindowedValue<InputT>>> chunks =
- Iterables.partition(c.element().getValue(), 1000);
- for (Iterable<WindowedValue<InputT>> chunk : chunks) {
- // Process the chunk of elements.
- reduceFnRunner.processElements(chunk);
-
- // Then, since elements are sorted by their timestamp, advance the input watermark
- // to the first element, and fire any timers that may have been scheduled.
- timerInternals.advanceInputWatermark(reduceFnRunner, chunk.iterator().next().getTimestamp());
-
- // Fire any processing timers that need to fire
- timerInternals.advanceProcessingTime(reduceFnRunner, Instant.now());
-
- // Leave the output watermark undefined. Since there's no late data in batch mode
- // there's really no need to track it as we do for streaming.
- }
-
- // Finish any pending windows by advancing the input watermark to infinity.
- timerInternals.advanceInputWatermark(reduceFnRunner, BoundedWindow.TIMESTAMP_MAX_VALUE);
-
- // Finally, advance the processing time to infinity to fire any timers.
- timerInternals.advanceProcessingTime(reduceFnRunner, BoundedWindow.TIMESTAMP_MAX_VALUE);
-
- reduceFnRunner.persist();
- }
-}
[04/12] incubator-beam git commit: Move some easy stuff into
runners/core-java
Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
deleted file mode 100644
index 864e8e7..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
+++ /dev/null
@@ -1,985 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
-import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
-import org.apache.beam.sdk.util.ReduceFnContextFactory.OnTriggerCallbacks;
-import org.apache.beam.sdk.util.ReduceFnContextFactory.StateStyle;
-import org.apache.beam.sdk.util.TimerInternals.TimerData;
-import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.StateInternals;
-import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableMap;
-
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import javax.annotation.Nullable;
-
-/**
- * Manages the execution of a {@link ReduceFn} after a {@link GroupByKeyOnly} has partitioned the
- * {@link PCollection} by key.
- *
- * <p>The {@link #onTrigger} relies on a {@link TriggerRunner} to manage the execution of
- * the triggering logic. The {@code ReduceFnRunner}s responsibilities are:
- *
- * <ul>
- * <li>Tracking the windows that are active (have buffered data) as elements arrive and
- * triggers are fired.
- * <li>Holding the watermark based on the timestamps of elements in a pane and releasing it
- * when the trigger fires.
- * <li>Calling the appropriate callbacks on {@link ReduceFn} based on trigger execution, timer
- * firings, etc, and providing appropriate contexts to the {@link ReduceFn} for actions
- * such as output.
- * <li>Scheduling garbage collection of state associated with a specific window, and making that
- * happen when the appropriate timer fires.
- * </ul>
- *
- * @param <K> The type of key being processed.
- * @param <InputT> The type of values associated with the key.
- * @param <OutputT> The output type that will be produced for each key.
- * @param <W> The type of windows this operates on.
- */
-public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
-
- /**
- * The {@link ReduceFnRunner} depends on most aspects of the {@link WindowingStrategy}.
- *
- * <ul>
- * <li>It runs the trigger from the {@link WindowingStrategy}.</li>
- * <li>It merges windows according to the {@link WindowingStrategy}.</li>
- * <li>It chooses how to track active windows and clear out expired windows
- * according to the {@link WindowingStrategy}, based on the allowed lateness and
- * whether windows can merge.</li>
- * <li>It decides whether to emit empty final panes according to whether the
- * {@link WindowingStrategy} requires it.<li>
- * <li>It uses discarding or accumulation mode according to the {@link WindowingStrategy}.</li>
- * </ul>
- */
- private final WindowingStrategy<Object, W> windowingStrategy;
-
- private final OutputWindowedValue<KV<K, OutputT>> outputter;
-
- private final StateInternals<K> stateInternals;
-
- private final Aggregator<Long, Long> droppedDueToClosedWindow;
-
- private final K key;
-
- /**
- * Track which windows are still active and the 'state address' windows which hold their state.
- *
- * <ul>
- * <li>State: Global map for all active windows for this computation and key.
- * <li>Lifetime: Cleared when no active windows need to be tracked. A window lives within
- * the active window set until its trigger is closed or the window is garbage collected.
- * </ul>
- */
- private final ActiveWindowSet<W> activeWindows;
-
- /**
- * Always a {@link SystemReduceFn}.
- *
- * <ul>
- * <li>State: A bag of accumulated values, or the intermediate result of a combiner.
- * <li>State style: RENAMED
- * <li>Merging: Concatenate or otherwise combine the state from each merged window.
- * <li>Lifetime: Cleared when a pane fires if DISCARDING_FIRED_PANES. Otherwise cleared
- * when trigger is finished or when the window is garbage collected.
- * </ul>
- */
- private final ReduceFn<K, InputT, OutputT, W> reduceFn;
-
- /**
- * Manage the setting and firing of timer events.
- *
- * <ul>
- * <li>Merging: End-of-window and garbage collection timers are cancelled when windows are
- * merged away. Timers created by triggers are never garbage collected and are left to
- * fire and be ignored.
- * <li>Lifetime: Timers automatically disappear after they fire.
- * </ul>
- */
- private final TimerInternals timerInternals;
-
- /**
- * Manage the execution and state for triggers.
- *
- * <ul>
- * <li>State: Tracks which sub-triggers have finished, and any additional state needed to
- * determine when the trigger should fire.
- * <li>State style: DIRECT
- * <li>Merging: Finished bits are explicitly managed. Other state is eagerly merged as
- * needed.
- * <li>Lifetime: Most trigger state is cleared when the final pane is emitted. However
- * the finished bits are left behind and must be cleared when the window is
- * garbage collected.
- * </ul>
- */
- private final TriggerRunner<W> triggerRunner;
-
- /**
- * Store the output watermark holds for each window.
- *
- * <ul>
- * <li>State: Bag of hold timestamps.
- * <li>State style: RENAMED
- * <li>Merging: Depending on {@link OutputTimeFn}, may need to be recalculated on merging.
- * When a pane fires it may be necessary to add (back) an end-of-window or garbage collection
- * hold.
- * <li>Lifetime: Cleared when a pane fires or when the window is garbage collected.
- * </ul>
- */
- private final WatermarkHold<W> watermarkHold;
-
- private final ReduceFnContextFactory<K, InputT, OutputT, W> contextFactory;
-
- /**
- * Store the previously emitted pane (if any) for each window.
- *
- * <ul>
- * <li>State: The previous {@link PaneInfo} passed to the user's {@link DoFn#processElement},
- * if any.
- * <li>Style style: DIRECT
- * <li>Merging: Always keyed by actual window, so does not depend on {@link #activeWindows}.
- * Cleared when window is merged away.
- * <li>Lifetime: Cleared when trigger is closed or window is garbage collected.
- * </ul>
- */
- private final PaneInfoTracker paneInfoTracker;
-
- /**
- * Store whether we've seen any elements for a window since the last pane was emitted.
- *
- * <ul>
- * <li>State: Unless DISCARDING_FIRED_PANES, a count of number of elements added so far.
- * <li>State style: RENAMED.
- * <li>Merging: Counts are summed when windows are merged.
- * <li>Lifetime: Cleared when pane fires or window is garbage collected.
- * </ul>
- */
- private final NonEmptyPanes<K, W> nonEmptyPanes;
-
- public ReduceFnRunner(
- K key,
- WindowingStrategy<?, W> windowingStrategy,
- StateInternals<K> stateInternals,
- TimerInternals timerInternals,
- WindowingInternals<?, KV<K, OutputT>> windowingInternals,
- Aggregator<Long, Long> droppedDueToClosedWindow,
- ReduceFn<K, InputT, OutputT, W> reduceFn,
- PipelineOptions options) {
- this.key = key;
- this.timerInternals = timerInternals;
- this.paneInfoTracker = new PaneInfoTracker(timerInternals);
- this.stateInternals = stateInternals;
- this.outputter = new OutputViaWindowingInternals<>(windowingInternals);
- this.droppedDueToClosedWindow = droppedDueToClosedWindow;
- this.reduceFn = reduceFn;
-
- @SuppressWarnings("unchecked")
- WindowingStrategy<Object, W> objectWindowingStrategy =
- (WindowingStrategy<Object, W>) windowingStrategy;
- this.windowingStrategy = objectWindowingStrategy;
-
- this.nonEmptyPanes = NonEmptyPanes.create(this.windowingStrategy, this.reduceFn);
-
- // Note this may incur I/O to load persisted window set data.
- this.activeWindows = createActiveWindowSet();
-
- this.contextFactory =
- new ReduceFnContextFactory<K, InputT, OutputT, W>(key, reduceFn, this.windowingStrategy,
- stateInternals, this.activeWindows, timerInternals, windowingInternals, options);
-
- this.watermarkHold = new WatermarkHold<>(timerInternals, windowingStrategy);
- this.triggerRunner =
- new TriggerRunner<>(
- windowingStrategy.getTrigger(),
- new TriggerContextFactory<>(windowingStrategy, stateInternals, activeWindows));
- }
-
- private ActiveWindowSet<W> createActiveWindowSet() {
- return windowingStrategy.getWindowFn().isNonMerging()
- ? new NonMergingActiveWindowSet<W>()
- : new MergingActiveWindowSet<W>(windowingStrategy.getWindowFn(), stateInternals);
- }
-
- @VisibleForTesting
- boolean isFinished(W window) {
- return triggerRunner.isClosed(contextFactory.base(window, StateStyle.DIRECT).state());
- }
-
- @VisibleForTesting
- boolean hasNoActiveWindows() {
- return activeWindows.getActiveAndNewWindows().isEmpty();
- }
-
- /**
- * Incorporate {@code values} into the underlying reduce function, and manage holds, timers,
- * triggers, and window merging.
- *
- * <p>The general strategy is:
- * <ol>
- * <li>Use {@link WindowedValue#getWindows} (itself determined using
- * {@link WindowFn#assignWindows}) to determine which windows each element belongs to. Some
- * of those windows will already have state associated with them. The rest are considered
- * NEW.
- * <li>Use {@link WindowFn#mergeWindows} to attempt to merge currently ACTIVE and NEW windows.
- * Each NEW window will become either ACTIVE or be discardedL.
- * (See {@link ActiveWindowSet} for definitions of these terms.)
- * <li>If at all possible, eagerly substitute NEW windows with their ACTIVE state address
- * windows before any state is associated with the NEW window. In the common case that
- * windows for new elements are merged into existing ACTIVE windows then no additional
- * storage or merging overhead will be incurred.
- * <li>Otherwise, keep track of the state address windows for ACTIVE windows so that their
- * states can be merged on-demand when a pane fires.
- * <li>Process the element for each of the windows it's windows have been merged into according
- * to {@link ActiveWindowSet}. Processing may require running triggers, setting timers,
- * setting holds, and invoking {@link ReduceFn#onTrigger}.
- * </ol>
- */
- public void processElements(Iterable<WindowedValue<InputT>> values) throws Exception {
- // If an incoming element introduces a new window, attempt to merge it into an existing
- // window eagerly.
- Map<W, W> windowToMergeResult = collectAndMergeWindows(values);
-
- Set<W> windowsToConsider = new HashSet<>();
-
- // Process each element, using the updated activeWindows determined by collectAndMergeWindows.
- for (WindowedValue<InputT> value : values) {
- windowsToConsider.addAll(processElement(windowToMergeResult, value));
- }
-
- // Trigger output from any window for which the trigger is ready
- for (W mergedWindow : windowsToConsider) {
- ReduceFn<K, InputT, OutputT, W>.Context directContext =
- contextFactory.base(mergedWindow, StateStyle.DIRECT);
- ReduceFn<K, InputT, OutputT, W>.Context renamedContext =
- contextFactory.base(mergedWindow, StateStyle.RENAMED);
- triggerRunner.prefetchShouldFire(mergedWindow, directContext.state());
- emitIfAppropriate(directContext, renamedContext);
- }
-
- // We're all done with merging and emitting elements so can compress the activeWindow state.
- // Any windows which are still NEW must have come in on a new element which was then discarded
- // due to the window's trigger being closed. We can thus delete them.
- activeWindows.cleanupTemporaryWindows();
- }
-
- public void persist() {
- activeWindows.persist();
- }
-
- /**
- * Extract the windows associated with the values, and invoke merge. Return a map
- * from windows to the merge result window. If a window is not in the domain of
- * the result map then it did not get merged into a different window.
- */
- private Map<W, W> collectAndMergeWindows(Iterable<WindowedValue<InputT>> values)
- throws Exception {
- // No-op if no merging can take place
- if (windowingStrategy.getWindowFn().isNonMerging()) {
- return ImmutableMap.of();
- }
-
- // Collect the windows from all elements (except those which are too late) and
- // make sure they are already in the active window set or are added as NEW windows.
- for (WindowedValue<?> value : values) {
- for (BoundedWindow untypedWindow : value.getWindows()) {
- @SuppressWarnings("unchecked")
- W window = (W) untypedWindow;
-
- // For backwards compat with pre 1.4 only.
- // We may still have ACTIVE windows with multiple state addresses, representing
- // a window who's state has not yet been eagerly merged.
- // We'll go ahead and merge that state now so that we don't have to worry about
- // this legacy case anywhere else.
- if (activeWindows.isActive(window)) {
- Set<W> stateAddressWindows = activeWindows.readStateAddresses(window);
- if (stateAddressWindows.size() > 1) {
- // This is a legacy window who's state has not been eagerly merged.
- // Do that now.
- ReduceFn<K, InputT, OutputT, W>.OnMergeContext premergeContext =
- contextFactory.forPremerge(window);
- reduceFn.onMerge(premergeContext);
- watermarkHold.onMerge(premergeContext);
- activeWindows.merged(window);
- }
- }
-
- // Add this window as NEW if it is not currently ACTIVE.
- // If we had already seen this window and closed its trigger, then the
- // window will not be currently ACTIVE. It will then be added as NEW here,
- // and fall into the merging logic as usual.
- activeWindows.ensureWindowExists(window);
- }
- }
-
- // Merge all of the active windows and retain a mapping from source windows to result windows.
- Map<W, W> windowToMergeResult = new HashMap<>();
- activeWindows.merge(new OnMergeCallback(windowToMergeResult));
- return windowToMergeResult;
- }
-
- private class OnMergeCallback implements ActiveWindowSet.MergeCallback<W> {
- private final Map<W, W> windowToMergeResult;
-
- OnMergeCallback(Map<W, W> windowToMergeResult) {
- this.windowToMergeResult = windowToMergeResult;
- }
-
- /**
- * Return the subset of {@code windows} which are currently ACTIVE. We only need to worry
- * about merging state from ACTIVE windows. NEW windows by definition have no existing state.
- */
- private List<W> activeWindows(Iterable<W> windows) {
- List<W> active = new ArrayList<>();
- for (W window : windows) {
- if (activeWindows.isActive(window)) {
- active.add(window);
- }
- }
- return active;
- }
-
- /**
- * Called from the active window set to indicate {@code toBeMerged} (of which only
- * {@code activeToBeMerged} are ACTIVE and thus have state associated with them) will later
- * be merged into {@code mergeResult}.
- */
- @Override
- public void prefetchOnMerge(
- Collection<W> toBeMerged, W mergeResult) throws Exception {
- List<W> activeToBeMerged = activeWindows(toBeMerged);
- ReduceFn<K, InputT, OutputT, W>.OnMergeContext directMergeContext =
- contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.DIRECT);
- ReduceFn<K, InputT, OutputT, W>.OnMergeContext renamedMergeContext =
- contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.RENAMED);
-
- // Prefetch various state.
- triggerRunner.prefetchForMerge(mergeResult, activeToBeMerged, directMergeContext.state());
- reduceFn.prefetchOnMerge(renamedMergeContext.state());
- watermarkHold.prefetchOnMerge(renamedMergeContext.state());
- nonEmptyPanes.prefetchOnMerge(renamedMergeContext.state());
- }
-
- /**
- * Called from the active window set to indicate {@code toBeMerged} (of which only
- * {@code activeToBeMerged} are ACTIVE and thus have state associated with them) are about
- * to be merged into {@code mergeResult}.
- */
- @Override
- public void onMerge(Collection<W> toBeMerged, W mergeResult) throws Exception {
- // Remember we have merged these windows.
- for (W window : toBeMerged) {
- windowToMergeResult.put(window, mergeResult);
- }
-
- // At this point activeWindows has NOT incorporated the results of the merge.
- List<W> activeToBeMerged = activeWindows(toBeMerged);
- ReduceFn<K, InputT, OutputT, W>.OnMergeContext directMergeContext =
- contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.DIRECT);
- ReduceFn<K, InputT, OutputT, W>.OnMergeContext renamedMergeContext =
- contextFactory.forMerge(activeToBeMerged, mergeResult, StateStyle.RENAMED);
-
- // Run the reduceFn to perform any needed merging.
- reduceFn.onMerge(renamedMergeContext);
-
- // Merge the watermark holds.
- watermarkHold.onMerge(renamedMergeContext);
-
- // Merge non-empty pane state.
- nonEmptyPanes.onMerge(renamedMergeContext.state());
-
- // Have the trigger merge state as needed.
- triggerRunner.onMerge(
- directMergeContext.window(), directMergeContext.timers(), directMergeContext.state());
-
- for (W active : activeToBeMerged) {
- if (active.equals(mergeResult)) {
- // Not merged away.
- continue;
- }
- // Cleanup flavor A: Currently ACTIVE window is about to be merged away.
- // Clear any state not already cleared by the onMerge calls above.
- WindowTracing.debug("ReduceFnRunner.onMerge: Merging {} into {}", active, mergeResult);
- ReduceFn<K, InputT, OutputT, W>.Context directClearContext =
- contextFactory.base(active, StateStyle.DIRECT);
- // No need for the end-of-window or garbage collection timers.
- // We will establish a new end-of-window or garbage collection timer for the mergeResult
- // window in processElement below. There must be at least one element for the mergeResult
- // window since a new element with a new window must have triggered this onMerge.
- cancelEndOfWindowAndGarbageCollectionTimers(directClearContext);
- // We no longer care about any previous panes of merged away windows. The
- // merge result window gets to start fresh if it is new.
- paneInfoTracker.clear(directClearContext.state());
- }
- }
- }
-
- /**
- * Process an element.
- *
- * @param value the value being processed
- * @return the set of windows in which the element was actually processed
- */
- private Collection<W> processElement(Map<W, W> windowToMergeResult, WindowedValue<InputT> value)
- throws Exception {
- // Redirect element windows to the ACTIVE windows they have been merged into.
- // The compressed representation (value, {window1, window2, ...}) actually represents
- // distinct elements (value, window1), (value, window2), ...
- // so if window1 and window2 merge, the resulting window will contain both copies
- // of the value.
- Collection<W> windows = new ArrayList<>();
- for (BoundedWindow untypedWindow : value.getWindows()) {
- @SuppressWarnings("unchecked")
- W window = (W) untypedWindow;
- W mergeResult = windowToMergeResult.get(window);
- if (mergeResult == null) {
- mergeResult = window;
- }
- windows.add(mergeResult);
- }
-
- // Prefetch in each of the windows if we're going to need to process triggers
- for (W window : windows) {
- ReduceFn<K, InputT, OutputT, W>.ProcessValueContext directContext = contextFactory.forValue(
- window, value.getValue(), value.getTimestamp(), StateStyle.DIRECT);
- triggerRunner.prefetchForValue(window, directContext.state());
- }
-
- // Process the element for each (mergeResultWindow, not closed) window it belongs to.
- List<W> triggerableWindows = new ArrayList<>(windows.size());
- for (W window : windows) {
- ReduceFn<K, InputT, OutputT, W>.ProcessValueContext directContext = contextFactory.forValue(
- window, value.getValue(), value.getTimestamp(), StateStyle.DIRECT);
- if (triggerRunner.isClosed(directContext.state())) {
- // This window has already been closed.
- droppedDueToClosedWindow.addValue(1L);
- WindowTracing.debug(
- "ReduceFnRunner.processElement: Dropping element at {} for key:{}; window:{} "
- + "since window is no longer active at inputWatermark:{}; outputWatermark:{}",
- value.getTimestamp(), key, window, timerInternals.currentInputWatermarkTime(),
- timerInternals.currentOutputWatermarkTime());
- continue;
- }
-
- triggerableWindows.add(window);
- activeWindows.ensureWindowIsActive(window);
- ReduceFn<K, InputT, OutputT, W>.ProcessValueContext renamedContext = contextFactory.forValue(
- window, value.getValue(), value.getTimestamp(), StateStyle.RENAMED);
-
- nonEmptyPanes.recordContent(renamedContext.state());
-
- // Make sure we've scheduled the end-of-window or garbage collection timer for this window.
- Instant timer = scheduleEndOfWindowOrGarbageCollectionTimer(directContext);
-
- // Hold back progress of the output watermark until we have processed the pane this
- // element will be included within. If the element is too late for that, place a hold at
- // the end-of-window or garbage collection time to allow empty panes to contribute elements
- // which won't be dropped due to lateness by a following computation (assuming the following
- // computation uses the same allowed lateness value...)
- @Nullable Instant hold = watermarkHold.addHolds(renamedContext);
-
- if (hold != null) {
- // Assert that holds have a proximate timer.
- boolean holdInWindow = !hold.isAfter(window.maxTimestamp());
- boolean timerInWindow = !timer.isAfter(window.maxTimestamp());
- Preconditions.checkState(
- holdInWindow == timerInWindow,
- "set a hold at %s, a timer at %s, which disagree as to whether they are in window %s",
- hold,
- timer,
- directContext.window());
- }
-
- // Execute the reduceFn, which will buffer the value as appropriate
- reduceFn.processValue(renamedContext);
-
- // Run the trigger to update its state
- triggerRunner.processValue(
- directContext.window(),
- directContext.timestamp(),
- directContext.timers(),
- directContext.state());
-
- // At this point, if triggerRunner.shouldFire before the processValue then
- // triggerRunner.shouldFire after the processValue. In other words adding values
- // cannot take a trigger state from firing to non-firing.
- // (We don't actually assert this since it is too slow.)
- }
-
- return triggerableWindows;
- }
-
- /**
- * Called when an end-of-window, garbage collection, or trigger-specific timer fires.
- */
- public void onTimer(TimerData timer) throws Exception {
- // Which window is the timer for?
- Preconditions.checkArgument(timer.getNamespace() instanceof WindowNamespace,
- "Expected timer to be in WindowNamespace, but was in %s", timer.getNamespace());
- @SuppressWarnings("unchecked")
- WindowNamespace<W> windowNamespace = (WindowNamespace<W>) timer.getNamespace();
- W window = windowNamespace.getWindow();
- ReduceFn<K, InputT, OutputT, W>.Context directContext =
- contextFactory.base(window, StateStyle.DIRECT);
- ReduceFn<K, InputT, OutputT, W>.Context renamedContext =
- contextFactory.base(window, StateStyle.RENAMED);
-
- // Has this window had its trigger finish?
- // - The trigger may implement isClosed as constant false.
- // - If the window function does not support windowing then all windows will be considered
- // active.
- // So we must take conjunction of activeWindows and triggerRunner state.
- boolean windowIsActiveAndOpen =
- activeWindows.isActive(window) && !triggerRunner.isClosed(directContext.state());
-
- if (!windowIsActiveAndOpen) {
- WindowTracing.debug(
- "ReduceFnRunner.onTimer: Note that timer {} is for non-ACTIVE window {}", timer, window);
- }
-
- // If this is an end-of-window timer then we may need to set a garbage collection timer
- // if allowed lateness is non-zero.
- boolean isEndOfWindow = TimeDomain.EVENT_TIME == timer.getDomain()
- && timer.getTimestamp().equals(window.maxTimestamp());
-
- // If this is a garbage collection timer then we should trigger and garbage collect the window.
- // We'll consider any timer at or after the end-of-window time to be a signal to garbage
- // collect.
- Instant cleanupTime = garbageCollectionTime(window);
- boolean isGarbageCollection = TimeDomain.EVENT_TIME == timer.getDomain()
- && !timer.getTimestamp().isBefore(cleanupTime);
-
- if (isGarbageCollection) {
- WindowTracing.debug(
- "ReduceFnRunner.onTimer: Cleaning up for key:{}; window:{} at {} with "
- + "inputWatermark:{}; outputWatermark:{}",
- key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(),
- timerInternals.currentOutputWatermarkTime());
-
- if (windowIsActiveAndOpen) {
- // We need to call onTrigger to emit the final pane if required.
- // The final pane *may* be ON_TIME if no prior ON_TIME pane has been emitted,
- // and the watermark has passed the end of the window.
- @Nullable Instant newHold =
- onTrigger(directContext, renamedContext, true/* isFinished */, isEndOfWindow);
- Preconditions.checkState(newHold == null,
- "Hold placed at %s despite isFinished being true.", newHold);
- }
-
- // Cleanup flavor B: Clear all the remaining state for this window since we'll never
- // see elements for it again.
- clearAllState(directContext, renamedContext, windowIsActiveAndOpen);
- } else {
- WindowTracing.debug(
- "ReduceFnRunner.onTimer: Triggering for key:{}; window:{} at {} with "
- + "inputWatermark:{}; outputWatermark:{}",
- key, window, timer.getTimestamp(), timerInternals.currentInputWatermarkTime(),
- timerInternals.currentOutputWatermarkTime());
- if (windowIsActiveAndOpen) {
- emitIfAppropriate(directContext, renamedContext);
- }
-
- if (isEndOfWindow) {
- // If the window strategy trigger includes a watermark trigger then at this point
- // there should be no data holds, either because we'd already cleared them on an
- // earlier onTrigger, or because we just cleared them on the above emitIfAppropriate.
- // We could assert this but it is very expensive.
-
- // Since we are processing an on-time firing we should schedule the garbage collection
- // timer. (If getAllowedLateness is zero then the timer event will be considered a
- // cleanup event and handled by the above).
- // Note we must do this even if the trigger is finished so that we are sure to cleanup
- // any final trigger finished bits.
- Preconditions.checkState(
- windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO),
- "Unexpected zero getAllowedLateness");
- WindowTracing.debug(
- "ReduceFnRunner.onTimer: Scheduling cleanup timer for key:{}; window:{} at {} with "
- + "inputWatermark:{}; outputWatermark:{}",
- key, directContext.window(), cleanupTime, timerInternals.currentInputWatermarkTime(),
- timerInternals.currentOutputWatermarkTime());
- Preconditions.checkState(!cleanupTime.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
- "Cleanup time %s is beyond end-of-time", cleanupTime);
- directContext.timers().setTimer(cleanupTime, TimeDomain.EVENT_TIME);
- }
- }
- }
-
- /**
- * Clear all the state associated with {@code context}'s window.
- * Should only be invoked if we know all future elements for this window will be considered
- * beyond allowed lateness.
- * This is a superset of the clearing done by {@link #emitIfAppropriate} below since:
- * <ol>
- * <li>We can clear the trigger finished bits since we'll never need to ask if the trigger is
- * closed again.
- * <li>We can clear any remaining garbage collection hold.
- * </ol>
- */
- private void clearAllState(
- ReduceFn<K, InputT, OutputT, W>.Context directContext,
- ReduceFn<K, InputT, OutputT, W>.Context renamedContext,
- boolean windowIsActiveAndOpen)
- throws Exception {
- if (windowIsActiveAndOpen) {
- // Since both the window is in the active window set AND the trigger was not yet closed,
- // it is possible we still have state.
- reduceFn.clearState(renamedContext);
- watermarkHold.clearHolds(renamedContext);
- nonEmptyPanes.clearPane(renamedContext.state());
- // These calls work irrespective of whether the window is active or not, but
- // are unnecessary if the window is not active.
- triggerRunner.clearState(
- directContext.window(), directContext.timers(), directContext.state());
- paneInfoTracker.clear(directContext.state());
- } else {
- // If !windowIsActiveAndOpen then !activeWindows.isActive (1) or triggerRunner.isClosed (2).
- // For (1), if !activeWindows.isActive then the window must be merging and has been
- // explicitly removed by emitIfAppropriate. But in that case the trigger must have fired
- // and been closed, so this case reduces to (2).
- // For (2), if triggerRunner.isClosed then the trigger was fired and entered the
- // closed state. In that case emitIfAppropriate will have cleared all state in
- // reduceFn, triggerRunner (except for finished bits), paneInfoTracker and activeWindows.
- // We also know nonEmptyPanes must have been unconditionally cleared by the trigger.
- // Since the trigger fired the existing watermark holds must have been cleared, and since
- // the trigger closed no new end of window or garbage collection hold will have been
- // placed by WatermarkHold.extractAndRelease.
- // Thus all the state clearing above is unnecessary.
- //
- // But(!) for backwards compatibility we must allow a pipeline to be updated from
- // an sdk version <= 1.3. In that case it is possible we have an end-of-window or
- // garbage collection hold keyed by the current window (reached via directContext) rather
- // than the state address window (reached via renamedContext).
- // However this can only happen if:
- // - We have merging windows.
- // - We are DISCARDING_FIRED_PANES.
- // - A pane has fired.
- // - But the trigger is not (yet) closed.
- if (windowingStrategy.getMode() == AccumulationMode.DISCARDING_FIRED_PANES
- && !windowingStrategy.getWindowFn().isNonMerging()) {
- watermarkHold.clearHolds(directContext);
- }
- }
-
- // Don't need to track address state windows anymore.
- activeWindows.remove(directContext.window());
- // We'll never need to test for the trigger being closed again.
- triggerRunner.clearFinished(directContext.state());
- }
-
- /** Should the reduce function state be cleared? */
- private boolean shouldDiscardAfterFiring(boolean isFinished) {
- if (isFinished) {
- // This is the last firing for trigger.
- return true;
- }
- if (windowingStrategy.getMode() == AccumulationMode.DISCARDING_FIRED_PANES) {
- // Nothing should be accumulated between panes.
- return true;
- }
- return false;
- }
-
- /**
- * Possibly emit a pane if a trigger is ready to fire or timers require it, and cleanup state.
- */
- private void emitIfAppropriate(ReduceFn<K, InputT, OutputT, W>.Context directContext,
- ReduceFn<K, InputT, OutputT, W>.Context renamedContext)
- throws Exception {
- if (!triggerRunner.shouldFire(
- directContext.window(), directContext.timers(), directContext.state())) {
- // Ignore unless trigger is ready to fire
- return;
- }
-
- // Inform the trigger of the transition to see if it is finished
- triggerRunner.onFire(directContext.window(), directContext.timers(), directContext.state());
- boolean isFinished = triggerRunner.isClosed(directContext.state());
-
- // Will be able to clear all element state after triggering?
- boolean shouldDiscard = shouldDiscardAfterFiring(isFinished);
-
- // Run onTrigger to produce the actual pane contents.
- // As a side effect it will clear all element holds, but not necessarily any
- // end-of-window or garbage collection holds.
- onTrigger(directContext, renamedContext, isFinished, false /*isEndOfWindow*/);
-
- // Now that we've triggered, the pane is empty.
- nonEmptyPanes.clearPane(renamedContext.state());
-
- // Cleanup buffered data if appropriate
- if (shouldDiscard) {
- // Cleanup flavor C: The user does not want any buffered data to persist between panes.
- reduceFn.clearState(renamedContext);
- }
-
- if (isFinished) {
- // Cleanup flavor D: If trigger is closed we will ignore all new incoming elements.
- // Clear state not otherwise cleared by onTrigger and clearPane above.
- // Remember the trigger is, indeed, closed until the window is garbage collected.
- triggerRunner.clearState(
- directContext.window(), directContext.timers(), directContext.state());
- paneInfoTracker.clear(directContext.state());
- activeWindows.remove(directContext.window());
- }
- }
-
- /**
- * Do we need to emit a pane?
- */
- private boolean needToEmit(boolean isEmpty, boolean isFinished, PaneInfo.Timing timing) {
- if (!isEmpty) {
- // The pane has elements.
- return true;
- }
- if (timing == Timing.ON_TIME) {
- // This is the unique ON_TIME pane.
- return true;
- }
- if (isFinished && windowingStrategy.getClosingBehavior() == ClosingBehavior.FIRE_ALWAYS) {
- // This is known to be the final pane, and the user has requested it even when empty.
- return true;
- }
- return false;
- }
-
- /**
- * Run the {@link ReduceFn#onTrigger} method and produce any necessary output.
- *
- * @return output watermark hold added, or {@literal null} if none.
- */
- @Nullable
- private Instant onTrigger(
- final ReduceFn<K, InputT, OutputT, W>.Context directContext,
- ReduceFn<K, InputT, OutputT, W>.Context renamedContext,
- boolean isFinished, boolean isEndOfWindow)
- throws Exception {
- Instant inputWM = timerInternals.currentInputWatermarkTime();
-
- // Prefetch necessary states
- ReadableState<WatermarkHold.OldAndNewHolds> outputTimestampFuture =
- watermarkHold.extractAndRelease(renamedContext, isFinished).readLater();
- ReadableState<PaneInfo> paneFuture =
- paneInfoTracker.getNextPaneInfo(directContext, isFinished).readLater();
- ReadableState<Boolean> isEmptyFuture =
- nonEmptyPanes.isEmpty(renamedContext.state()).readLater();
-
- reduceFn.prefetchOnTrigger(directContext.state());
- triggerRunner.prefetchOnFire(directContext.window(), directContext.state());
-
- // Calculate the pane info.
- final PaneInfo pane = paneFuture.read();
- // Extract the window hold, and as a side effect clear it.
-
- WatermarkHold.OldAndNewHolds pair = outputTimestampFuture.read();
- final Instant outputTimestamp = pair.oldHold;
- @Nullable Instant newHold = pair.newHold;
-
- if (newHold != null) {
- // We can't be finished yet.
- Preconditions.checkState(
- !isFinished, "new hold at %s but finished %s", newHold, directContext.window());
- // The hold cannot be behind the input watermark.
- Preconditions.checkState(
- !newHold.isBefore(inputWM), "new hold %s is before input watermark %s", newHold, inputWM);
- if (newHold.isAfter(directContext.window().maxTimestamp())) {
- // The hold must be for garbage collection, which can't have happened yet.
- Preconditions.checkState(
- newHold.isEqual(garbageCollectionTime(directContext.window())),
- "new hold %s should be at garbage collection for window %s plus %s",
- newHold,
- directContext.window(),
- windowingStrategy.getAllowedLateness());
- } else {
- // The hold must be for the end-of-window, which can't have happened yet.
- Preconditions.checkState(
- newHold.isEqual(directContext.window().maxTimestamp()),
- "new hold %s should be at end of window %s",
- newHold,
- directContext.window());
- Preconditions.checkState(
- !isEndOfWindow,
- "new hold at %s for %s but this is the watermark trigger",
- newHold,
- directContext.window());
- }
- }
-
- // Only emit a pane if it has data or empty panes are observable.
- if (needToEmit(isEmptyFuture.read(), isFinished, pane.getTiming())) {
- // Run reduceFn.onTrigger method.
- final List<W> windows = Collections.singletonList(directContext.window());
- ReduceFn<K, InputT, OutputT, W>.OnTriggerContext renamedTriggerContext =
- contextFactory.forTrigger(directContext.window(), paneFuture, StateStyle.RENAMED,
- new OnTriggerCallbacks<OutputT>() {
- @Override
- public void output(OutputT toOutput) {
- // We're going to output panes, so commit the (now used) PaneInfo.
- // TODO: This is unnecessary if the trigger isFinished since the saved
- // state will be immediately deleted.
- paneInfoTracker.storeCurrentPaneInfo(directContext, pane);
-
- // Output the actual value.
- outputter.outputWindowedValue(
- KV.of(key, toOutput), outputTimestamp, windows, pane);
- }
- });
-
- reduceFn.onTrigger(renamedTriggerContext);
- }
-
- return newHold;
- }
-
- /**
- * Make sure we'll eventually have a timer fire which will tell us to garbage collect
- * the window state. For efficiency we may need to do this in two steps rather
- * than one. Return the time at which the timer will fire.
- *
- * <ul>
- * <li>If allowedLateness is zero then we'll garbage collect at the end of the window.
- * For simplicity we'll set our own timer for this situation even though an
- * {@link AfterWatermark} trigger may have also set an end-of-window timer.
- * ({@code setTimer} is idempotent.)
- * <li>If allowedLateness is non-zero then we could just always set a timer for the garbage
- * collection time. However if the windows are large (eg hourly) and the allowedLateness is small
- * (eg seconds) then we'll end up with nearly twice the number of timers in-flight. So we
- * instead set an end-of-window timer and then roll that forward to a garbage collection timer
- * when it fires. We use the input watermark to distinguish those cases.
- * </ul>
- */
- private Instant scheduleEndOfWindowOrGarbageCollectionTimer(
- ReduceFn<?, ?, ?, W>.Context directContext) {
- Instant inputWM = timerInternals.currentInputWatermarkTime();
- Instant endOfWindow = directContext.window().maxTimestamp();
- String which;
- Instant timer;
- if (endOfWindow.isBefore(inputWM)) {
- timer = garbageCollectionTime(directContext.window());
- which = "garbage collection";
- } else {
- timer = endOfWindow;
- which = "end-of-window";
- }
- WindowTracing.trace(
- "ReduceFnRunner.scheduleEndOfWindowOrGarbageCollectionTimer: Scheduling {} timer at {} for "
- + "key:{}; window:{} where inputWatermark:{}; outputWatermark:{}",
- which,
- timer,
- key,
- directContext.window(),
- inputWM,
- timerInternals.currentOutputWatermarkTime());
- Preconditions.checkState(!timer.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
- "Timer %s is beyond end-of-time", timer);
- directContext.timers().setTimer(timer, TimeDomain.EVENT_TIME);
- return timer;
- }
-
- private void cancelEndOfWindowAndGarbageCollectionTimers(
- ReduceFn<?, ?, ?, W>.Context directContext) {
- WindowTracing.debug(
- "ReduceFnRunner.cancelEndOfWindowAndGarbageCollectionTimers: Deleting timers for "
- + "key:{}; window:{} where inputWatermark:{}; outputWatermark:{}",
- key, directContext.window(), timerInternals.currentInputWatermarkTime(),
- timerInternals.currentOutputWatermarkTime());
- Instant eow = directContext.window().maxTimestamp();
- directContext.timers().deleteTimer(eow, TimeDomain.EVENT_TIME);
- Instant gc = garbageCollectionTime(directContext.window());
- if (gc.isAfter(eow)) {
- directContext.timers().deleteTimer(eow, TimeDomain.EVENT_TIME);
- }
- }
-
- /**
- * 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.
- */
- private Instant garbageCollectionTime(W window) {
-
- // 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(windowingStrategy.getAllowedLateness())
- .isBefore(window.maxTimestamp())) {
- return GlobalWindow.INSTANCE.maxTimestamp();
- } else {
- return window.maxTimestamp().plus(windowingStrategy.getAllowedLateness());
- }
- }
-
- /**
- * An object that can output a value with all of its windowing information. This is a deliberately
- * restricted subinterface of {@link WindowingInternals} to express how it is used here.
- */
- private interface OutputWindowedValue<OutputT> {
- void outputWindowedValue(OutputT output, Instant timestamp,
- Collection<? extends BoundedWindow> windows, PaneInfo pane);
- }
-
- private static class OutputViaWindowingInternals<OutputT>
- implements OutputWindowedValue<OutputT> {
-
- private final WindowingInternals<?, OutputT> windowingInternals;
-
- public OutputViaWindowingInternals(WindowingInternals<?, OutputT> windowingInternals) {
- this.windowingInternals = windowingInternals;
- }
-
- @Override
- public void outputWindowedValue(
- OutputT output,
- Instant timestamp,
- Collection<? extends BoundedWindow> windows,
- PaneInfo pane) {
- windowingInternals.outputWindowedValue(output, timestamp, windows, pane);
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampsAndWindows.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampsAndWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampsAndWindows.java
new file mode 100644
index 0000000..d129c8e
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampsAndWindows.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ * implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.util;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Helper transform that makes timestamps and window assignments explicit in the value part of
+ * each key/value pair.
+ */
+public class ReifyTimestampsAndWindows<K, V>
+ extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, WindowedValue<V>>>> {
+
+ @Override
+ public PCollection<KV<K, WindowedValue<V>>> apply(PCollection<KV<K, V>> input) {
+
+ // The requirement to use a KvCoder *is* actually a model-level requirement, not specific
+ // to this implementation of GBK. All runners need a way to get the key.
+ checkArgument(
+ input.getCoder() instanceof KvCoder,
+ "%s requires its input to use a %s",
+ GroupByKey.class.getSimpleName(),
+ KvCoder.class.getSimpleName());
+
+ @SuppressWarnings("unchecked")
+ KvCoder<K, V> inputKvCoder = (KvCoder<K, V>) input.getCoder();
+ Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+ Coder<V> inputValueCoder = inputKvCoder.getValueCoder();
+ Coder<WindowedValue<V>> outputValueCoder =
+ FullWindowedValueCoder.of(
+ inputValueCoder, input.getWindowingStrategy().getWindowFn().windowCoder());
+ Coder<KV<K, WindowedValue<V>>> outputKvCoder = KvCoder.of(keyCoder, outputValueCoder);
+ return input
+ .apply(ParDo.of(new ReifyTimestampAndWindowsDoFn<K, V>()))
+ .setCoder(outputKvCoder);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java
deleted file mode 100644
index 78377c8..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.util.DoFnRunners.OutputManager;
-import org.apache.beam.sdk.util.ExecutionContext.StepContext;
-import org.apache.beam.sdk.util.common.CounterSet.AddCounterMutator;
-import org.apache.beam.sdk.values.TupleTag;
-
-import java.util.List;
-
-/**
- * Runs a {@link DoFn} by constructing the appropriate contexts and passing them in.
- *
- * @param <InputT> the type of the DoFn's (main) input elements
- * @param <OutputT> the type of the DoFn's (main) output elements
- */
-public class SimpleDoFnRunner<InputT, OutputT> extends DoFnRunnerBase<InputT, OutputT>{
-
- protected SimpleDoFnRunner(PipelineOptions options, DoFn<InputT, OutputT> fn,
- SideInputReader sideInputReader,
- OutputManager outputManager,
- TupleTag<OutputT> mainOutputTag, List<TupleTag<?>> sideOutputTags, StepContext stepContext,
- AddCounterMutator addCounterMutator, WindowingStrategy<?, ?> windowingStrategy) {
- super(options, fn, sideInputReader, outputManager, mainOutputTag, sideOutputTags, stepContext,
- addCounterMutator, windowingStrategy);
- }
-
- @Override
- protected void invokeProcessElement(WindowedValue<InputT> elem) {
- final DoFn<InputT, OutputT>.ProcessContext processContext = createProcessContext(elem);
- // This can contain user code. Wrap it in case it throws an exception.
- try {
- fn.processElement(processContext);
- } catch (Exception ex) {
- throw wrapUserCodeException(ex);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java
deleted file mode 100644
index 2eeee54..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn;
-import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.state.AccumulatorCombiningState;
-import org.apache.beam.sdk.util.state.BagState;
-import org.apache.beam.sdk.util.state.CombiningState;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.ReadableState;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateMerging;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-
-/**
- * {@link ReduceFn} implementing the default reduction behaviors of {@link GroupByKey}.
- *
- * @param <K> The type of key being processed.
- * @param <InputT> The type of values associated with the key.
- * @param <OutputT> The output type that will be produced for each key.
- * @param <W> The type of windows this operates on.
- */
-public abstract class SystemReduceFn<K, InputT, AccumT, OutputT, W extends BoundedWindow>
- extends ReduceFn<K, InputT, OutputT, W> {
- private static final String BUFFER_NAME = "buf";
-
- /**
- * Create a factory that produces {@link SystemReduceFn} instances that that buffer all of the
- * input values in persistent state and produces an {@code Iterable<T>}.
- */
- public static <K, T, W extends BoundedWindow> SystemReduceFn<K, T, Iterable<T>, Iterable<T>, W>
- buffering(final Coder<T> inputCoder) {
- final StateTag<Object, BagState<T>> bufferTag =
- StateTags.makeSystemTagInternal(StateTags.bag(BUFFER_NAME, inputCoder));
- return new SystemReduceFn<K, T, Iterable<T>, Iterable<T>, W>(bufferTag) {
- @Override
- public void prefetchOnMerge(MergingStateAccessor<K, W> state) throws Exception {
- StateMerging.prefetchBags(state, bufferTag);
- }
-
- @Override
- public void onMerge(OnMergeContext c) throws Exception {
- StateMerging.mergeBags(c.state(), bufferTag);
- }
- };
- }
-
- /**
- * Create a factory that produces {@link SystemReduceFn} instances that combine all of the input
- * values using a {@link CombineFn}.
- */
- public static <K, InputT, AccumT, OutputT, W extends BoundedWindow> SystemReduceFn<K, InputT,
- AccumT, OutputT, W>
- combining(
- final Coder<K> keyCoder, final AppliedCombineFn<K, InputT, AccumT, OutputT> combineFn) {
- final StateTag<K, AccumulatorCombiningState<InputT, AccumT, OutputT>> bufferTag;
- if (combineFn.getFn() instanceof KeyedCombineFnWithContext) {
- bufferTag = StateTags.makeSystemTagInternal(
- StateTags.<K, InputT, AccumT, OutputT>keyedCombiningValueWithContext(
- BUFFER_NAME, combineFn.getAccumulatorCoder(),
- (KeyedCombineFnWithContext<K, InputT, AccumT, OutputT>) combineFn.getFn()));
-
- } else {
- bufferTag = StateTags.makeSystemTagInternal(
- StateTags.<K, InputT, AccumT, OutputT>keyedCombiningValue(
- BUFFER_NAME, combineFn.getAccumulatorCoder(),
- (KeyedCombineFn<K, InputT, AccumT, OutputT>) combineFn.getFn()));
- }
- return new SystemReduceFn<K, InputT, AccumT, OutputT, W>(bufferTag) {
- @Override
- public void prefetchOnMerge(MergingStateAccessor<K, W> state) throws Exception {
- StateMerging.prefetchCombiningValues(state, bufferTag);
- }
-
- @Override
- public void onMerge(OnMergeContext c) throws Exception {
- StateMerging.mergeCombiningValues(c.state(), bufferTag);
- }
- };
- }
-
- private StateTag<? super K, ? extends CombiningState<InputT, OutputT>> bufferTag;
-
- public SystemReduceFn(
- StateTag<? super K, ? extends CombiningState<InputT, OutputT>> bufferTag) {
- this.bufferTag = bufferTag;
- }
-
- @Override
- public void processValue(ProcessValueContext c) throws Exception {
- c.state().access(bufferTag).add(c.value());
- }
-
- @Override
- public void prefetchOnTrigger(StateAccessor<K> state) {
- state.access(bufferTag).readLater();
- }
-
- @Override
- public void onTrigger(OnTriggerContext c) throws Exception {
- c.output(c.state().access(bufferTag).read());
- }
-
- @Override
- public void clearState(Context c) throws Exception {
- c.state().access(bufferTag).clear();
- }
-
- @Override
- public ReadableState<Boolean> isEmpty(StateAccessor<K> state) {
- return state.access(bufferTag).isEmpty();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fef8e63/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
deleted file mode 100644
index f104f6a..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
+++ /dev/null
@@ -1,234 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.util;
-
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.util.state.MergingStateAccessor;
-import org.apache.beam.sdk.util.state.StateAccessor;
-import org.apache.beam.sdk.util.state.StateTag;
-import org.apache.beam.sdk.util.state.StateTags;
-import org.apache.beam.sdk.util.state.ValueState;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableMap;
-
-import org.joda.time.Instant;
-
-import java.util.BitSet;
-import java.util.Collection;
-import java.util.Map;
-
-/**
- * Executes a trigger while managing persistence of information about which subtriggers are
- * finished. Subtriggers include all recursive trigger expressions as well as the entire trigger.
- *
- * <p>Specifically, the responsibilities are:
- *
- * <ul>
- * <li>Invoking the trigger's methods via its {@link ExecutableTrigger} wrapper by
- * constructing the appropriate trigger contexts.</li>
- * <li>Committing a record of which subtriggers are finished to persistent state.</li>
- * <li>Restoring the record of which subtriggers are finished from persistent state.</li>
- * <li>Clearing out the persisted finished set when a caller indicates
- * (via {#link #clearFinished}) that it is no longer needed.</li>
- * </ul>
- *
- * <p>These responsibilities are intertwined: trigger contexts include mutable information about
- * which subtriggers are finished. This class provides the information when building the contexts
- * and commits the information when the method of the {@link ExecutableTrigger} returns.
- *
- * @param <W> The kind of windows being processed.
- */
-public class TriggerRunner<W extends BoundedWindow> {
- @VisibleForTesting
- static final StateTag<Object, ValueState<BitSet>> FINISHED_BITS_TAG =
- StateTags.makeSystemTagInternal(StateTags.value("closed", BitSetCoder.of()));
-
- private final ExecutableTrigger rootTrigger;
- private final TriggerContextFactory<W> contextFactory;
-
- public TriggerRunner(ExecutableTrigger rootTrigger, TriggerContextFactory<W> contextFactory) {
- Preconditions.checkState(rootTrigger.getTriggerIndex() == 0);
- this.rootTrigger = rootTrigger;
- this.contextFactory = contextFactory;
- }
-
- private FinishedTriggersBitSet readFinishedBits(ValueState<BitSet> state) {
- if (!isFinishedSetNeeded()) {
- // If no trigger in the tree will ever have finished bits, then we don't need to read them.
- // So that the code can be agnostic to that fact, we create a BitSet that is all 0 (not
- // finished) for each trigger in the tree.
- return FinishedTriggersBitSet.emptyWithCapacity(rootTrigger.getFirstIndexAfterSubtree());
- }
-
- BitSet bitSet = state.read();
- return bitSet == null
- ? FinishedTriggersBitSet.emptyWithCapacity(rootTrigger.getFirstIndexAfterSubtree())
- : FinishedTriggersBitSet.fromBitSet(bitSet);
- }
-
-
- private void clearFinishedBits(ValueState<BitSet> state) {
- if (!isFinishedSetNeeded()) {
- // Nothing to clear.
- return;
- }
- state.clear();
- }
-
- /** Return true if the trigger is closed in the window corresponding to the specified state. */
- public boolean isClosed(StateAccessor<?> state) {
- return readFinishedBits(state.access(FINISHED_BITS_TAG)).isFinished(rootTrigger);
- }
-
- public void prefetchForValue(W window, StateAccessor<?> state) {
- if (isFinishedSetNeeded()) {
- state.access(FINISHED_BITS_TAG).readLater();
- }
- rootTrigger.getSpec().prefetchOnElement(
- contextFactory.createStateAccessor(window, rootTrigger));
- }
-
- public void prefetchOnFire(W window, StateAccessor<?> state) {
- if (isFinishedSetNeeded()) {
- state.access(FINISHED_BITS_TAG).readLater();
- }
- rootTrigger.getSpec().prefetchOnFire(contextFactory.createStateAccessor(window, rootTrigger));
- }
-
- public void prefetchShouldFire(W window, StateAccessor<?> state) {
- if (isFinishedSetNeeded()) {
- state.access(FINISHED_BITS_TAG).readLater();
- }
- rootTrigger.getSpec().prefetchShouldFire(
- contextFactory.createStateAccessor(window, rootTrigger));
- }
-
- /**
- * Run the trigger logic to deal with a new value.
- */
- public void processValue(W window, Instant timestamp, Timers timers, StateAccessor<?> state)
- throws Exception {
- // Clone so that we can detect changes and so that changes here don't pollute merging.
- FinishedTriggersBitSet finishedSet =
- readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
- Trigger.OnElementContext triggerContext = contextFactory.createOnElementContext(
- window, timers, timestamp, rootTrigger, finishedSet);
- rootTrigger.invokeOnElement(triggerContext);
- persistFinishedSet(state, finishedSet);
- }
-
- public void prefetchForMerge(
- W window, Collection<W> mergingWindows, MergingStateAccessor<?, W> state) {
- if (isFinishedSetNeeded()) {
- for (ValueState<?> value : state.accessInEachMergingWindow(FINISHED_BITS_TAG).values()) {
- value.readLater();
- }
- }
- rootTrigger.getSpec().prefetchOnMerge(contextFactory.createMergingStateAccessor(
- window, mergingWindows, rootTrigger));
- }
-
- /**
- * Run the trigger merging logic as part of executing the specified merge.
- */
- public void onMerge(W window, Timers timers, MergingStateAccessor<?, W> state) throws Exception {
- // Clone so that we can detect changes and so that changes here don't pollute merging.
- FinishedTriggersBitSet finishedSet =
- readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
-
- // And read the finished bits in each merging window.
- ImmutableMap.Builder<W, FinishedTriggers> builder = ImmutableMap.builder();
- for (Map.Entry<W, ValueState<BitSet>> entry :
- state.accessInEachMergingWindow(FINISHED_BITS_TAG).entrySet()) {
- // Don't need to clone these, since the trigger context doesn't allow modification
- builder.put(entry.getKey(), readFinishedBits(entry.getValue()));
- // Clear the underlying finished bits.
- clearFinishedBits(entry.getValue());
- }
- ImmutableMap<W, FinishedTriggers> mergingFinishedSets = builder.build();
-
- Trigger.OnMergeContext mergeContext = contextFactory.createOnMergeContext(
- window, timers, rootTrigger, finishedSet, mergingFinishedSets);
-
- // Run the merge from the trigger
- rootTrigger.invokeOnMerge(mergeContext);
-
- persistFinishedSet(state, finishedSet);
- }
-
- public boolean shouldFire(W window, Timers timers, StateAccessor<?> state) throws Exception {
- FinishedTriggers finishedSet = readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
- Trigger.TriggerContext context = contextFactory.base(window, timers,
- rootTrigger, finishedSet);
- return rootTrigger.invokeShouldFire(context);
- }
-
- public void onFire(W window, Timers timers, StateAccessor<?> state) throws Exception {
- // shouldFire should be false.
- // However it is too expensive to assert.
- FinishedTriggersBitSet finishedSet =
- readFinishedBits(state.access(FINISHED_BITS_TAG)).copy();
- Trigger.TriggerContext context = contextFactory.base(window, timers,
- rootTrigger, finishedSet);
- rootTrigger.invokeOnFire(context);
- persistFinishedSet(state, finishedSet);
- }
-
- private void persistFinishedSet(
- StateAccessor<?> state, FinishedTriggersBitSet modifiedFinishedSet) {
- if (!isFinishedSetNeeded()) {
- return;
- }
-
- ValueState<BitSet> finishedSetState = state.access(FINISHED_BITS_TAG);
- if (!readFinishedBits(finishedSetState).equals(modifiedFinishedSet)) {
- if (modifiedFinishedSet.getBitSet().isEmpty()) {
- finishedSetState.clear();
- } else {
- finishedSetState.write(modifiedFinishedSet.getBitSet());
- }
- }
- }
-
- /**
- * Clear the finished bits.
- */
- public void clearFinished(StateAccessor<?> state) {
- clearFinishedBits(state.access(FINISHED_BITS_TAG));
- }
-
- /**
- * Clear the state used for executing triggers, but leave the finished set to indicate
- * the window is closed.
- */
- public void clearState(W window, Timers timers, StateAccessor<?> state) throws Exception {
- // Don't need to clone, because we'll be clearing the finished bits anyways.
- FinishedTriggers finishedSet = readFinishedBits(state.access(FINISHED_BITS_TAG));
- rootTrigger.invokeClear(contextFactory.base(window, timers, rootTrigger, finishedSet));
- }
-
- private boolean isFinishedSetNeeded() {
- // TODO: If we know that no trigger in the tree will ever finish, we don't need to do the
- // lookup. Right now, we special case this for the DefaultTrigger.
- return !(rootTrigger.getSpec() instanceof DefaultTrigger);
- }
-}