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);
-  }
-}