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/12/06 16:40:51 UTC

[01/50] [abbrv] incubator-beam git commit: Move TransformHierarchy Maintenance into it

Repository: incubator-beam
Updated Branches:
  refs/heads/gearpump-runner 28124050d -> 88de0cb23


Move TransformHierarchy Maintenance into it

This reduces the complexity of Pipeline.applyInternal by keeping the
responsiblities to passing a node into the Transform Hierarchy,
enforcing name uniqueness, and causing the runner to expand the
PTransform. This logic is moved to the appropriate application sites.


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/ab1f1ad0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/ab1f1ad0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/ab1f1ad0

Branch: refs/heads/gearpump-runner
Commit: ab1f1ad012bc559cdb099319a516e4437eed2825
Parents: 0c875ba
Author: Thomas Groh <tg...@google.com>
Authored: Tue Nov 29 14:29:47 2016 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Thu Dec 1 12:55:25 2016 -0800

----------------------------------------------------------------------
 .../direct/KeyedPValueTrackingVisitor.java      |   2 +-
 .../DataflowPipelineTranslatorTest.java         |   2 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java | 117 +++---------
 .../beam/sdk/runners/TransformHierarchy.java    | 126 ++++++++-----
 .../beam/sdk/runners/TransformTreeNode.java     | 165 +++++++++--------
 .../sdk/runners/TransformHierarchyTest.java     | 180 ++++++++++++++-----
 .../beam/sdk/runners/TransformTreeTest.java     |   4 +-
 7 files changed, 340 insertions(+), 256 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ab1f1ad0/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
index 7c4376a..47b0857 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
@@ -74,7 +74,7 @@ class KeyedPValueTrackingVisitor implements PipelineVisitor {
     if (node.isRootNode()) {
       finalized = true;
     } else if (producesKeyedOutputs.contains(node.getTransform().getClass())) {
-      keyedValues.addAll(node.getExpandedOutputs());
+      keyedValues.addAll(node.getOutput().expand());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ab1f1ad0/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
index c925454..95c7132 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
@@ -669,7 +669,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     PCollection<Integer> input = p.begin()
         .apply(Create.of(1, 2, 3));
 
-    thrown.expect(IllegalStateException.class);
+    thrown.expect(IllegalArgumentException.class);
     input.apply(new PartiallyBoundOutputCreator());
 
     Assert.fail("Failure expected from use of partially bound output");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ab1f1ad0/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
index 9edf496..c8a4439 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
@@ -17,10 +17,11 @@
  */
 package org.apache.beam.sdk;
 
+import static com.google.common.base.Preconditions.checkState;
+
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import org.apache.beam.sdk.coders.CoderRegistry;
@@ -31,7 +32,6 @@ import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.runners.TransformTreeNode;
 import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.util.UserCodeException;
@@ -282,14 +282,12 @@ public class Pipeline {
    * <p>Typically invoked by {@link PipelineRunner} subclasses.
    */
   public void traverseTopologically(PipelineVisitor visitor) {
-    Set<PValue> visitedValues = new HashSet<>();
-    // Visit all the transforms, which should implicitly visit all the values.
-    transforms.visit(visitor, visitedValues);
-    if (!visitedValues.containsAll(values)) {
-      throw new RuntimeException(
-          "internal error: should have visited all the values "
-          + "after visiting all the transforms");
-    }
+    Set<PValue> visitedValues =
+        // Visit all the transforms, which should implicitly visit all the values.
+        transforms.visit(visitor);
+    checkState(
+        visitedValues.containsAll(values),
+        "internal error: should have visited all the values after visiting all the transforms");
   }
 
   /**
@@ -351,53 +349,43 @@ public class Pipeline {
    *
    * @see Pipeline#apply
    */
-  private <InputT extends PInput, OutputT extends POutput>
-  OutputT applyInternal(String name, InputT input,
-      PTransform<? super InputT, OutputT> transform) {
-    input.finishSpecifying();
+  private <InputT extends PInput, OutputT extends POutput> OutputT applyInternal(
+      String name, InputT input, PTransform<? super InputT, OutputT> transform) {
+    String namePrefix = transforms.getCurrent().getFullName();
+    String uniqueName = uniquifyInternal(namePrefix, name);
 
-    TransformTreeNode parent = transforms.getCurrent();
-    String namePrefix = parent.getFullName();
-    String fullName = uniquifyInternal(namePrefix, name);
-
-    boolean nameIsUnique = fullName.equals(buildName(namePrefix, name));
+    boolean nameIsUnique = uniqueName.equals(buildName(namePrefix, name));
 
     if (!nameIsUnique) {
       switch (getOptions().getStableUniqueNames()) {
         case OFF:
           break;
         case WARNING:
-          LOG.warn("Transform {} does not have a stable unique name. "
-              + "This will prevent updating of pipelines.", fullName);
+          LOG.warn(
+              "Transform {} does not have a stable unique name. "
+                  + "This will prevent updating of pipelines.",
+              uniqueName);
           break;
         case ERROR:
           throw new IllegalStateException(
-              "Transform " + fullName + " does not have a stable unique name. "
-              + "This will prevent updating of pipelines.");
+              "Transform "
+                  + uniqueName
+                  + " does not have a stable unique name. "
+                  + "This will prevent updating of pipelines.");
         default:
           throw new IllegalArgumentException(
               "Unrecognized value for stable unique names: " + getOptions().getStableUniqueNames());
       }
     }
 
-    TransformTreeNode child =
-        new TransformTreeNode(parent, transform, fullName, input);
-    parent.addComposite(child);
-
-    transforms.addInput(child, input);
-
     LOG.debug("Adding {} to {}", transform, this);
+    transforms.pushNode(uniqueName, input, transform);
     try {
-      transforms.pushNode(child);
+      transforms.finishSpecifyingInput();
       transform.validate(input);
       OutputT output = runner.apply(transform, input);
-      transforms.setOutput(child, output);
+      transforms.setOutput(output);
 
-      AppliedPTransform<?, ?, ?> applied = AppliedPTransform.of(
-          child.getFullName(), input, output, transform);
-      // recordAsOutput is a NOOP if already called;
-      output.recordAsOutput(applied);
-      verifyOutputState(output, child);
       return output;
     } finally {
       transforms.popNode();
@@ -405,63 +393,6 @@ public class Pipeline {
   }
 
   /**
-   * Returns all producing transforms for the {@link PValue PValues} contained
-   * in {@code output}.
-   */
-  private List<AppliedPTransform<?, ?, ?>> getProducingTransforms(POutput output) {
-    List<AppliedPTransform<?, ?, ?>> producingTransforms = new ArrayList<>();
-    for (PValue value : output.expand()) {
-      AppliedPTransform<?, ?, ?> transform = value.getProducingTransformInternal();
-      if (transform != null) {
-        producingTransforms.add(transform);
-      }
-    }
-    return producingTransforms;
-  }
-
-  /**
-   * Verifies that the output of a {@link PTransform} is correctly configured in its
-   * {@link TransformTreeNode} in the {@link Pipeline} graph.
-   *
-   * <p>A non-composite {@link PTransform} must have all
-   * of its outputs registered as produced by that {@link PTransform}.
-   *
-   * <p>A composite {@link PTransform} must have all of its outputs
-   * registered as produced by the contained primitive {@link PTransform PTransforms}.
-   * They have each had the above check performed already, when
-   * they were applied, so the only possible failure state is
-   * that the composite {@link PTransform} has returned a primitive output.
-   */
-  private void verifyOutputState(POutput output, TransformTreeNode node) {
-    if (!node.isCompositeNode()) {
-      PTransform<?, ?> thisTransform = node.getTransform();
-      List<AppliedPTransform<?, ?, ?>> producingTransforms = getProducingTransforms(output);
-      for (AppliedPTransform<?, ?, ?> producingTransform : producingTransforms) {
-        // Using != because object identity indicates that the transforms
-        // are the same node in the pipeline
-        if (thisTransform != producingTransform.getTransform()) {
-          throw new IllegalArgumentException("Output of non-composite transform "
-              + thisTransform + " is registered as being produced by"
-              + " a different transform: " + producingTransform);
-        }
-      }
-    } else {
-      PTransform<?, ?> thisTransform = node.getTransform();
-      List<AppliedPTransform<?, ?, ?>> producingTransforms = getProducingTransforms(output);
-      for (AppliedPTransform<?, ?, ?> producingTransform : producingTransforms) {
-        // Using == because object identity indicates that the transforms
-        // are the same node in the pipeline
-        if (thisTransform == producingTransform.getTransform()) {
-          throw new IllegalStateException("Output of composite transform "
-              + thisTransform + " is registered as being produced by it,"
-              + " but the output of every composite transform should be"
-              + " produced by a primitive transform contained therein.");
-        }
-      }
-    }
-  }
-
-  /**
    * Returns the configured {@link PipelineRunner}.
    */
   public PipelineRunner<?> getRunner() {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ab1f1ad0/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
index 0a4bb08..d3fd497 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
@@ -17,14 +17,17 @@
  */
 package org.apache.beam.sdk.runners;
 
+import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
 
-import java.util.Deque;
+import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.LinkedList;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
@@ -34,70 +37,109 @@ import org.apache.beam.sdk.values.PValue;
  * associated {@link PValue}s.
  */
 public class TransformHierarchy {
-  private final Deque<TransformTreeNode> transformStack = new LinkedList<>();
-  private final Map<PInput, TransformTreeNode> producingTransformNode = new HashMap<>();
+  private final TransformTreeNode root;
+  private final Map<POutput, TransformTreeNode> producers;
+  // Maintain a stack based on the enclosing nodes
+  private TransformTreeNode current;
 
-  /**
-   * Create a {@code TransformHierarchy} containing a root node.
-   */
   public TransformHierarchy() {
-    // First element in the stack is the root node, holding all child nodes.
-    transformStack.add(new TransformTreeNode(null, null, "", null));
+    root = TransformTreeNode.root(this);
+    current = root;
+    producers = new HashMap<>();
   }
 
   /**
-   * Returns the last TransformTreeNode on the stack.
+   * Adds the named {@link PTransform} consuming the provided {@link PInput} as a node in this
+   * {@link TransformHierarchy} as a child of the current node, and sets it to be the current node.
+   *
+   * <p>This call should be finished by expanding and recursively calling {@link #pushNode(String,
+   * PInput, PTransform)}, calling {@link #finishSpecifyingInput()}, setting the output with {@link
+   * #setOutput(POutput)}, and ending with a call to {@link #popNode()}.
+   *
+   * @return the added node
    */
-  public TransformTreeNode getCurrent() {
-    return transformStack.peek();
+  public TransformTreeNode pushNode(String name, PInput input, PTransform<?, ?> transform) {
+    checkNotNull(
+        transform, "A %s must be provided for all Nodes", PTransform.class.getSimpleName());
+    checkNotNull(
+        name, "A name must be provided for all %s Nodes", PTransform.class.getSimpleName());
+    checkNotNull(
+        input, "An input must be provided for all %s Nodes", PTransform.class.getSimpleName());
+    current = TransformTreeNode.subtransform(current, transform, name, input);
+    return current;
   }
 
   /**
-   * Add a TransformTreeNode to the stack.
+   * Finish specifying all of the input {@link PValue PValues} of the current {@link
+   * TransformTreeNode}. Ensures that all of the inputs to the current node have been fully
+   * specified, and have been produced by a node in this graph.
    */
-  public void pushNode(TransformTreeNode current) {
-    transformStack.push(current);
+  public void finishSpecifyingInput() {
+    // Inputs must be completely specified before they are consumed by a transform.
+    current.getInput().finishSpecifying();
+    for (PValue inputValue : current.getInput().expand()) {
+      checkState(producers.get(inputValue) != null, "Producer unknown for input %s", inputValue);
+      inputValue.finishSpecifying();
+    }
   }
 
   /**
-   * Removes the last TransformTreeNode from the stack.
+   * Set the output of the current {@link TransformTreeNode}. If the output is new (setOutput has
+   * not previously been called with it as the parameter), the current node is set as the producer
+   * of that {@link POutput}.
+   *
+   * <p>Also validates the output - specifically, a Primitive {@link PTransform} produces all of
+   * its outputs, and a Composite {@link PTransform} produces none of its outputs. Verifies that the
+   * expanded output does not contain {@link PValue PValues} produced by both this node and other
+   * nodes.
    */
-  public void popNode() {
-    transformStack.pop();
-    checkState(!transformStack.isEmpty());
+  public void setOutput(POutput output) {
+    for (PValue value : output.expand()) {
+      if (!producers.containsKey(value)) {
+        producers.put(value, current);
+      }
+    }
+    current.setOutput(output);
+    // TODO: Replace with a "generateDefaultNames" method.
+    output.recordAsOutput(current.toAppliedPTransform());
   }
 
   /**
-   * Adds an input to the given node.
-   *
-   * <p>This forces the producing node to be finished.
+   * Pops the current node off the top of the stack, finishing it. Outputs of the node are finished
+   * once they are consumed as input.
    */
-  public void addInput(TransformTreeNode node, PInput input) {
-    for (PValue i : input.expand()) {
-      TransformTreeNode producer = producingTransformNode.get(i);
-      checkState(producer != null, "Producer unknown for input: %s", i);
+  public void popNode() {
+    current.finishSpecifying();
+    current = current.getEnclosingNode();
+    checkState(current != null, "Can't pop the root node of a TransformHierarchy");
+  }
 
-      producer.finishSpecifying();
-      node.addInputProducer(i, producer);
-    }
+  TransformTreeNode getProducer(PValue produced) {
+    return producers.get(produced);
   }
 
   /**
-   * Sets the output of a transform node.
+   * Returns all producing transforms for the {@link PValue PValues} contained
+   * in {@code output}.
    */
-  public void setOutput(TransformTreeNode producer, POutput output) {
-    producer.setOutput(output);
-
-    for (PValue o : output.expand()) {
-      producingTransformNode.put(o, producer);
+  List<TransformTreeNode> getProducingTransforms(POutput output) {
+    List<TransformTreeNode> producingTransforms = new ArrayList<>();
+    for (PValue value : output.expand()) {
+      TransformTreeNode producer = getProducer(value);
+      if (producer != null) {
+        producingTransforms.add(producer);
+      }
     }
+    return producingTransforms;
   }
 
-  /**
-   * Visits all nodes in the transform hierarchy, in transitive order.
-   */
-  public void visit(Pipeline.PipelineVisitor visitor,
-                    Set<PValue> visitedNodes) {
-    transformStack.peekFirst().visit(visitor, visitedNodes);
+  public Set<PValue> visit(PipelineVisitor visitor) {
+    Set<PValue> visitedValues = new HashSet<>();
+    root.visit(visitor, visitedValues);
+    return visitedValues;
+  }
+
+  public TransformTreeNode getCurrent() {
+    return current;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ab1f1ad0/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java
index d16b828..ea94bd9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java
@@ -17,18 +17,19 @@
  */
 package org.apache.beam.sdk.runners;
 
-import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
 
+import com.google.common.annotations.VisibleForTesting;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
+import java.util.HashSet;
 import java.util.Set;
 import javax.annotation.Nullable;
+import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
@@ -39,6 +40,7 @@ import org.apache.beam.sdk.values.PValue;
  * for initialization and ordered visitation.
  */
 public class TransformTreeNode {
+  private final TransformHierarchy hierarchy;
   private final TransformTreeNode enclosingNode;
 
   // The PTransform for this node, which may be a composite PTransform.
@@ -51,10 +53,6 @@ public class TransformTreeNode {
   // Nodes for sub-transforms of a composite transform.
   private final Collection<TransformTreeNode> parts = new ArrayList<>();
 
-  // Inputs to the transform, in expanded form and mapped to the producer
-  // of the input.
-  private final Map<PValue, TransformTreeNode> inputs = new HashMap<>();
-
   // Input to the transform, in unexpanded form.
   private final PInput input;
 
@@ -62,28 +60,57 @@ public class TransformTreeNode {
   // Output of the transform, in unexpanded form.
   private POutput output;
 
-  private boolean finishedSpecifying = false;
+  @VisibleForTesting
+  boolean finishedSpecifying = false;
+
+  /**
+   * Create a root {@link TransformTreeNode}. This transform is the root of the provided {@link
+   * TransformHierarchy} - it has no enclosing node, no {@link PTransform}, no {@link PInput input},
+   * no {@link POutput output}, and an empty name. It contains all {@link PTransform transforms}
+   * within a {@link Pipeline} as component transforms.
+   */
+  public static TransformTreeNode root(TransformHierarchy hierarchy) {
+    return new TransformTreeNode(hierarchy, null, null, "", null);
+  }
+
+  /**
+   * Create a subtransform of the provided {@link TransformTreeNode node}. The enclosing node is a
+   * composite that contains this transform.
+   *
+   * <p>The returned node is a component node of the enclosing node.
+   */
+  public static TransformTreeNode subtransform(
+      TransformTreeNode enclosing, PTransform<?, ?> transform, String fullName, PInput input) {
+    checkNotNull(enclosing);
+    checkNotNull(transform);
+    checkNotNull(fullName);
+    checkNotNull(input);
+    TransformTreeNode node =
+        new TransformTreeNode(enclosing.hierarchy, enclosing, transform, fullName, input);
+    enclosing.addComposite(node);
+    return node;
+  }
 
   /**
    * Creates a new TransformTreeNode with the given parent and transform.
    *
-   * <p>EnclosingNode and transform may both be null for
-   * a root-level node, which holds all other nodes.
+   * <p>EnclosingNode and transform may both be null for a root-level node, which holds all other
+   * nodes.
    *
    * @param enclosingNode the composite node containing this node
    * @param transform the PTransform tracked by this node
    * @param fullName the fully qualified name of the transform
    * @param input the unexpanded input to the transform
    */
-  public TransformTreeNode(@Nullable TransformTreeNode enclosingNode,
-                           @Nullable PTransform<?, ?> transform,
-                           String fullName,
-                           @Nullable PInput input) {
+  private TransformTreeNode(
+      TransformHierarchy hierarchy,
+      @Nullable TransformTreeNode enclosingNode,
+      @Nullable PTransform<?, ?> transform,
+      String fullName,
+      @Nullable PInput input) {
+    this.hierarchy = hierarchy;
     this.enclosingNode = enclosingNode;
     this.transform = transform;
-    checkArgument((enclosingNode == null && transform == null)
-        || (enclosingNode != null && transform != null),
-        "EnclosingNode and transform must both be specified, or both be null");
     this.fullName = fullName;
     this.input = input;
   }
@@ -113,21 +140,23 @@ public class TransformTreeNode {
   }
 
   /**
-   * Returns true if this node represents a composite transform that does not perform
-   * processing of its own, but merely encapsulates a sub-pipeline (which may be empty).
+   * Returns true if this node represents a composite transform that does not perform processing of
+   * its own, but merely encapsulates a sub-pipeline (which may be empty).
    *
-   * <p>Note that a node may be composite with no sub-transforms if it  returns its input directly
+   * <p>Note that a node may be composite with no sub-transforms if it returns its input directly
    * extracts a component of a tuple, or other operations that occur at pipeline assembly time.
    */
   public boolean isCompositeNode() {
-    return !parts.isEmpty() || returnsOthersOutput() || isRootNode();
+    return !parts.isEmpty() || isRootNode() || returnsOthersOutput();
   }
 
   private boolean returnsOthersOutput() {
     PTransform<?, ?> transform = getTransform();
-    for (PValue output : getExpandedOutputs()) {
-      if (!output.getProducingTransformInternal().getTransform().equals(transform)) {
-        return true;
+    if (output != null) {
+      for (PValue outputValue : output.expand()) {
+        if (!hierarchy.getProducer(outputValue).getTransform().equals(transform)) {
+          return true;
+        }
       }
     }
     return false;
@@ -142,14 +171,6 @@ public class TransformTreeNode {
   }
 
   /**
-   * Adds an input to the transform node.
-   */
-  public void addInputProducer(PValue expandedInput, TransformTreeNode producer) {
-    checkState(!finishedSpecifying);
-    inputs.put(expandedInput, producer);
-  }
-
-  /**
    * Returns the transform input, in unexpanded form.
    */
   public PInput getInput() {
@@ -157,20 +178,37 @@ public class TransformTreeNode {
   }
 
   /**
-   * Returns a mapping of inputs to the producing nodes for all inputs to
-   * the transform.
-   */
-  public Map<PValue, TransformTreeNode> getInputs() {
-    return Collections.unmodifiableMap(inputs);
-  }
-
-  /**
    * Adds an output to the transform node.
    */
   public void setOutput(POutput output) {
     checkState(!finishedSpecifying);
-    checkState(this.output == null);
+    checkState(this.output == null, "Tried to specify more than one output for %s", getFullName());
+    checkNotNull(output, "Tried to set the output of %s to null", getFullName());
     this.output = output;
+
+    // Validate that a primitive transform produces only primitive output, and a composite transform
+    // does not produce primitive output.
+    Set<TransformTreeNode> outputProducers = new HashSet<>();
+    for (PValue outputValue : output.expand()) {
+      outputProducers.add(hierarchy.getProducer(outputValue));
+    }
+    if (outputProducers.contains(this) && outputProducers.size() != 1) {
+      Set<String> otherProducerNames = new HashSet<>();
+      for (TransformTreeNode outputProducer : outputProducers) {
+        if (outputProducer != this) {
+          otherProducerNames.add(outputProducer.getFullName());
+        }
+      }
+      throw new IllegalArgumentException(
+          String.format(
+              "Output of transform [%s] contains a %s produced by it as well as other Transforms. "
+                  + "A primitive transform must produce all of its outputs, and outputs of a "
+                  + "composite transform must be produced by a component transform or be part of"
+                  + "the input."
+                  + "%n    Other Outputs: %s"
+                  + "%n    Other Producers: %s",
+              getFullName(), POutput.class.getSimpleName(), output.expand(), otherProducerNames));
+    }
   }
 
   /**
@@ -180,17 +218,10 @@ public class TransformTreeNode {
     return output;
   }
 
-  /**
-   * Returns the transform outputs, in expanded form.
-   */
-  public Collection<? extends PValue> getExpandedOutputs() {
-    if (output != null) {
-      return output.expand();
-    } else {
-      return Collections.emptyList();
-    }
+  AppliedPTransform<?, ?, ?> toAppliedPTransform() {
+    return AppliedPTransform.of(
+        getFullName(), getInput(), getOutput(), (PTransform) getTransform());
   }
-
   /**
    * Visit the transform node.
    *
@@ -204,10 +235,12 @@ public class TransformTreeNode {
       finishSpecifying();
     }
 
-    // Visit inputs.
-    for (Map.Entry<PValue, TransformTreeNode> entry : inputs.entrySet()) {
-      if (visitedValues.add(entry.getKey())) {
-        visitor.visitValue(entry.getKey(), entry.getValue());
+    if (!isRootNode()) {
+      // Visit inputs.
+      for (PValue inputValue : input.expand()) {
+        if (visitedValues.add(inputValue)) {
+          visitor.visitValue(inputValue, hierarchy.getProducer(inputValue));
+        }
       }
     }
 
@@ -224,10 +257,12 @@ public class TransformTreeNode {
       visitor.visitPrimitiveTransform(this);
     }
 
-    // Visit outputs.
-    for (PValue pValue : getExpandedOutputs()) {
-      if (visitedValues.add(pValue)) {
-        visitor.visitValue(pValue, this);
+    if (!isRootNode()) {
+      // Visit outputs.
+      for (PValue pValue : output.expand()) {
+        if (visitedValues.add(pValue)) {
+          visitor.visitValue(pValue, this);
+        }
       }
     }
   }
@@ -243,15 +278,5 @@ public class TransformTreeNode {
       return;
     }
     finishedSpecifying = true;
-
-    for (TransformTreeNode input : inputs.values()) {
-      if (input != null) {
-        input.finishSpecifying();
-      }
-    }
-
-    if (output != null) {
-      output.finishSpecifyingOutput();
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ab1f1ad0/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
index c28f23e..3bf6d64 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
@@ -20,6 +20,7 @@ package org.apache.beam.sdk.runners;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.nullValue;
 import static org.junit.Assert.assertThat;
 
 import java.util.HashSet;
@@ -30,9 +31,15 @@ import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
 import org.hamcrest.Matchers;
 import org.junit.Before;
@@ -63,7 +70,7 @@ public class TransformHierarchyTest {
   }
 
   @Test
-  public void popWithoutPushThrows() {
+  public void pushWithoutPushFails() {
     thrown.expect(IllegalStateException.class);
     hierarchy.popNode();
   }
@@ -71,72 +78,153 @@ public class TransformHierarchyTest {
   @Test
   public void pushThenPopSucceeds() {
     TransformTreeNode root = hierarchy.getCurrent();
-    TransformTreeNode node =
-        new TransformTreeNode(hierarchy.getCurrent(), Create.of(1), "Create", PBegin.in(pipeline));
-    hierarchy.pushNode(node);
+    TransformTreeNode node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1));
     assertThat(hierarchy.getCurrent(), equalTo(node));
     hierarchy.popNode();
+    assertThat(node.finishedSpecifying, is(true));
     assertThat(hierarchy.getCurrent(), equalTo(root));
   }
 
   @Test
+  public void emptyCompositeSucceeds() {
+    PCollection<Long> created =
+        PCollection.createPrimitiveOutputInternal(
+            pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED);
+    TransformTreeNode node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1));
+    hierarchy.setOutput(created);
+    hierarchy.popNode();
+    PCollectionList<Long> pcList = PCollectionList.of(created);
+
+    TransformTreeNode emptyTransform =
+        hierarchy.pushNode(
+            "Extract",
+            pcList,
+            new PTransform<PCollectionList<Long>, PCollection<Long>>() {
+              @Override
+              public PCollection<Long> apply(PCollectionList<Long> input) {
+                return input.get(0);
+              }
+            });
+    hierarchy.setOutput(created);
+    hierarchy.popNode();
+    assertThat(hierarchy.getProducer(created), equalTo(node));
+    assertThat(
+        "A Transform that produces non-primtive output should be composite",
+        emptyTransform.isCompositeNode(),
+        is(true));
+  }
+
+  @Test
+  public void producingOwnAndOthersOutputsFails() {
+    PCollection<Long> created =
+        PCollection.createPrimitiveOutputInternal(
+            pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED);
+    hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1));
+    hierarchy.setOutput(created);
+    hierarchy.popNode();
+    PCollectionList<Long> pcList = PCollectionList.of(created);
+
+    final PCollectionList<Long> appended =
+        pcList.and(
+            PCollection.<Long>createPrimitiveOutputInternal(
+                pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED));
+    hierarchy.pushNode(
+        "AddPc",
+        pcList,
+        new PTransform<PCollectionList<Long>, PCollectionList<Long>>() {
+          @Override
+          public PCollectionList<Long> apply(PCollectionList<Long> input) {
+            return appended;
+          }
+        });
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("produced by it as well as other Transforms");
+    thrown.expectMessage("primitive transform must produce all of its outputs");
+    thrown.expectMessage("composite transform must be produced by a component transform");
+    thrown.expectMessage("AddPc");
+    thrown.expectMessage("Create");
+    thrown.expectMessage(appended.expand().toString());
+    hierarchy.setOutput(appended);
+  }
+
+  @Test
   public void visitVisitsAllPushed() {
     TransformTreeNode root = hierarchy.getCurrent();
-    Create.Values<Integer> create = Create.of(1);
-    PCollection<Integer> created = pipeline.apply(create);
     PBegin begin = PBegin.in(pipeline);
 
-    TransformTreeNode compositeNode =
-        new TransformTreeNode(root, create, "Create", begin);
-    root.addComposite(compositeNode);
-    TransformTreeNode primitiveNode =
-        new TransformTreeNode(
-            compositeNode, Read.from(CountingSource.upTo(1L)), "Create/Read", begin);
-    compositeNode.addComposite(primitiveNode);
-
-    TransformTreeNode otherPrimitive =
-        new TransformTreeNode(
-            root, MapElements.via(new SimpleFunction<Integer, Integer>() {
-          @Override
-          public Integer apply(Integer input) {
-            return input;
-          }
-        }), "ParDo", created);
-    root.addComposite(otherPrimitive);
-    otherPrimitive.addInputProducer(created, primitiveNode);
+    Create.Values<Long> create = Create.of(1L);
+    Read.Bounded<Long> read = Read.from(CountingSource.upTo(1L));
 
-    hierarchy.pushNode(compositeNode);
-    hierarchy.pushNode(primitiveNode);
+    PCollection<Long> created =
+        PCollection.createPrimitiveOutputInternal(
+            pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED);
+
+    MapElements<Long, Long> map = MapElements.via(new SimpleFunction<Long, Long>() {
+      @Override
+      public Long apply(Long input) {
+        return input;
+      }
+    });
+
+    PCollection<Long> mapped =
+        PCollection.createPrimitiveOutputInternal(
+            pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED);
+
+    TransformTreeNode compositeNode = hierarchy.pushNode("Create", begin, create);
+    assertThat(hierarchy.getCurrent(), equalTo(compositeNode));
+    assertThat(compositeNode.getInput(), Matchers.<PInput>equalTo(begin));
+    assertThat(compositeNode.getTransform(), Matchers.<PTransform<?, ?>>equalTo(create));
+    // Not yet set
+    assertThat(compositeNode.getOutput(), nullValue());
+    assertThat(compositeNode.getEnclosingNode().isRootNode(), is(true));
+
+    TransformTreeNode primitiveNode = hierarchy.pushNode("Create/Read", begin, read);
+    assertThat(hierarchy.getCurrent(), equalTo(primitiveNode));
+    hierarchy.setOutput(created);
     hierarchy.popNode();
+    assertThat(primitiveNode.getOutput(), Matchers.<POutput>equalTo(created));
+    assertThat(primitiveNode.getInput(), Matchers.<PInput>equalTo(begin));
+    assertThat(primitiveNode.getTransform(), Matchers.<PTransform<?, ?>>equalTo(read));
+    assertThat(primitiveNode.getEnclosingNode(), equalTo(compositeNode));
+
+    hierarchy.setOutput(created);
+    // The composite is listed as outputting a PValue created by the contained primitive
+    assertThat(compositeNode.getOutput(), Matchers.<POutput>equalTo(created));
+    // The producer of that PValue is still the primitive in which it is first output
+    assertThat(hierarchy.getProducer(created), equalTo(primitiveNode));
     hierarchy.popNode();
-    hierarchy.pushNode(otherPrimitive);
+
+    TransformTreeNode otherPrimitive = hierarchy.pushNode("ParDo", created, map);
+    hierarchy.setOutput(mapped);
     hierarchy.popNode();
 
     final Set<TransformTreeNode> visitedCompositeNodes = new HashSet<>();
     final Set<TransformTreeNode> visitedPrimitiveNodes = new HashSet<>();
     final Set<PValue> visitedValuesInVisitor = new HashSet<>();
 
-    Set<PValue> visitedValues = new HashSet<>();
-    hierarchy.visit(new PipelineVisitor.Defaults() {
-      @Override
-      public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
-        visitedCompositeNodes.add(node);
-        return CompositeBehavior.ENTER_TRANSFORM;
-      }
-
-      @Override
-      public void visitPrimitiveTransform(TransformTreeNode node) {
-        visitedPrimitiveNodes.add(node);
-      }
-
-      @Override
-      public void visitValue(PValue value, TransformTreeNode producer) {
-        visitedValuesInVisitor.add(value);
-      }
-    }, visitedValues);
+    Set<PValue> visitedValues =
+        hierarchy.visit(
+            new PipelineVisitor.Defaults() {
+              @Override
+              public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+                visitedCompositeNodes.add(node);
+                return CompositeBehavior.ENTER_TRANSFORM;
+              }
+
+              @Override
+              public void visitPrimitiveTransform(TransformTreeNode node) {
+                visitedPrimitiveNodes.add(node);
+              }
+
+              @Override
+              public void visitValue(PValue value, TransformTreeNode producer) {
+                visitedValuesInVisitor.add(value);
+              }
+            });
 
     assertThat(visitedCompositeNodes, containsInAnyOrder(root, compositeNode));
     assertThat(visitedPrimitiveNodes, containsInAnyOrder(primitiveNode, otherPrimitive));
-    assertThat(visitedValuesInVisitor, Matchers.<PValue>containsInAnyOrder(created));
+    assertThat(visitedValuesInVisitor, Matchers.<PValue>containsInAnyOrder(created, mapped));
+    assertThat(visitedValuesInVisitor, equalTo(visitedValues));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ab1f1ad0/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
index def3a02..b95fa70 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
@@ -22,7 +22,6 @@ import static org.hamcrest.Matchers.not;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.util.Arrays;
@@ -169,14 +168,13 @@ public class TransformTreeTest {
     assertTrue(left.equals(EnumSet.of(TransformsSeen.SAMPLE_ANY)));
   }
 
-  @Test(expected = IllegalStateException.class)
+  @Test(expected = IllegalArgumentException.class)
   public void testOutputChecking() throws Exception {
     Pipeline p = TestPipeline.create();
 
     p.apply(new InvalidCompositeTransform());
 
     p.traverseTopologically(new Pipeline.PipelineVisitor.Defaults() {});
-    fail("traversal should have failed with an IllegalStateException");
   }
 
   @Test



[33/50] [abbrv] incubator-beam git commit: Makes DoFnTester use new DoFn internally.

Posted by ke...@apache.org.
Makes DoFnTester use new DoFn internally.

There were 2 remaining users of DoFnTester.of(OldDoFn):
- SplittableParDo.ProcessElements: this is fixed in
  https://github.com/apache/incubator-beam/pull/1261
- GroupAlsoByWindowsProperties: this one is harder.
  Various GABWDoFn's use OldDoFn.windowingInternals,
  and we can't pass that through a new DoFn.
  So instead I removed usage of DoFnTester from
  GroupAlsoByWindowsProperties in favor of a tiny
  hand-coded solution.

So after #1261 DoFnTester.of(OldDoFn) can be deleted.


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/96455768
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/96455768
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/96455768

Branch: refs/heads/gearpump-runner
Commit: 96455768568616141a95833380f37c478a989397
Parents: e04cd47
Author: Eugene Kirpichov <ki...@google.com>
Authored: Fri Nov 18 13:10:22 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Dec 2 15:42:33 2016 -0800

----------------------------------------------------------------------
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |   6 +-
 .../core/GroupByKeyViaGroupByKeyOnly.java       |  22 +-
 .../core/GroupAlsoByWindowsProperties.java      | 590 +++++++++++--------
 .../beam/sdk/transforms/DoFnAdapters.java       |   2 +
 .../apache/beam/sdk/transforms/DoFnTester.java  | 130 ++--
 .../sdk/transforms/reflect/DoFnInvokers.java    |  11 -
 .../beam/sdk/transforms/DoFnTesterTest.java     |   4 +-
 7 files changed, 394 insertions(+), 371 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/96455768/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java
index f8f6207..b4b366c 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java
@@ -21,7 +21,6 @@ import com.google.common.collect.Iterables;
 import java.util.List;
 import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine;
 import org.apache.beam.runners.core.triggers.TriggerStateMachines;
-import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.SystemDoFnInternal;
 import org.apache.beam.sdk.util.WindowedValue;
@@ -30,7 +29,6 @@ import org.apache.beam.sdk.util.state.InMemoryTimerInternals;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.util.state.StateInternalsFactory;
 import org.apache.beam.sdk.util.state.TimerCallback;
-import org.apache.beam.sdk.values.KV;
 import org.joda.time.Instant;
 
 /**
@@ -55,9 +53,7 @@ public class GroupAlsoByWindowsViaOutputBufferDoFn<K, InputT, OutputT, W extends
   }
 
   @Override
-  public void processElement(
-      OldDoFn<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>>.ProcessContext c)
-          throws Exception {
+  public void processElement(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

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/96455768/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java
index 79d2252..43047ca 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java
@@ -26,15 +26,13 @@ import java.util.List;
 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.OldDoFn;
 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;
 import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
 import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.state.StateInternalsFactory;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 
@@ -135,10 +133,9 @@ public class GroupByKeyViaGroupByKeyOnly<K, V>
       return input
           .apply(
               ParDo.of(
-                  new OldDoFn<
-                      KV<K, Iterable<WindowedValue<V>>>,
-                      KV<K, Iterable<WindowedValue<V>>>>() {
-                    @Override
+                  new DoFn<KV<K, Iterable<WindowedValue<V>>>,
+                           KV<K, Iterable<WindowedValue<V>>>>() {
+                    @ProcessElement
                     public void processElement(ProcessContext c) {
                       KV<K, Iterable<WindowedValue<V>>> kvs = c.element();
                       K key = kvs.getKey();
@@ -251,16 +248,5 @@ public class GroupByKeyViaGroupByKeyOnly<K, V>
           input.getPipeline(), windowingStrategy, input.isBounded())
           .setCoder(outputKvCoder);
     }
-
-    private <W extends BoundedWindow>
-        GroupAlsoByWindowsViaOutputBufferDoFn<K, V, Iterable<V>, W> groupAlsoByWindowsFn(
-            WindowingStrategy<?, W> strategy,
-            StateInternalsFactory<K> stateInternalsFactory,
-            Coder<V> inputIterableElementValueCoder) {
-      return new GroupAlsoByWindowsViaOutputBufferDoFn<K, V, Iterable<V>, W>(
-          strategy,
-          stateInternalsFactory,
-          SystemReduceFn.<K, V, W>buffering(inputIterableElementValueCoder));
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/96455768/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java
index d1e0c68..97b67c6 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java
@@ -23,50 +23,60 @@ import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.hasSize;
 import static org.junit.Assert.assertThat;
 
+import com.google.common.base.Predicate;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
+import javax.annotation.Nullable;
+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.DoFnTester;
-import org.apache.beam.sdk.transforms.DoFnTester.CloningBehavior;
 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.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.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingInternals;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.util.state.InMemoryStateInternals;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.util.state.StateInternalsFactory;
 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 org.joda.time.Duration;
 import org.joda.time.Instant;
 
 /**
  * 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
+ * <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.
+   * 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, StateInternalsFactory<K> stateInternalsFactory);
+    <W extends BoundedWindow> GroupAlsoByWindowsDoFn<K, InputT, OutputT, W> forStrategy(
+        WindowingStrategy<?, W> strategy, StateInternalsFactory<K> stateInternalsFactory);
   }
 
   /**
@@ -76,8 +86,7 @@ public class GroupAlsoByWindowsProperties {
    * <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 {
+      GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> gabwFactory) throws Exception {
 
     WindowingStrategy<?, IntervalWindow> windowingStrategy =
         WindowingStrategy.of(FixedWindows.of(Duration.millis(10)));
@@ -87,13 +96,14 @@ public class GroupAlsoByWindowsProperties {
     @SuppressWarnings("unchecked")
     K fakeKey = (K) "this key should never be used";
 
-    DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> result = runGABW(
-        gabwFactory,
-        windowingStrategy,
-        fakeKey,
-        Collections.<WindowedValue<InputT>>emptyList());
+    List<WindowedValue<KV<K, OutputT>>> result =
+        runGABW(
+            gabwFactory,
+            windowingStrategy,
+            fakeKey,
+            Collections.<WindowedValue<InputT>>emptyList());
 
-    assertThat(result.peekOutputElements(), hasSize(0));
+    assertThat(result, hasSize(0));
   }
 
   /**
@@ -102,38 +112,32 @@ public class GroupAlsoByWindowsProperties {
    */
   public static void groupsElementsIntoFixedWindows(
       GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
-          throws Exception {
+      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",
+    List<WindowedValue<KV<String, Iterable<String>>>> result =
+        runGABW(
+            gabwFactory,
+            windowingStrategy,
+            "key",
             WindowedValue.of(
-                "v1",
-                new Instant(1),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
+                "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),
+                "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));
+                "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING));
 
-    assertThat(result.peekOutputElements(), hasSize(2));
+    assertThat(result, hasSize(2));
 
     TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
+        getOnlyElementInWindow(result, 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)));
+        getOnlyElementInWindow(result, window(10, 20));
     assertThat(item1.getValue().getValue(), contains("v3"));
     assertThat(item1.getTimestamp(), equalTo(window(10, 20).maxTimestamp()));
   }
@@ -146,14 +150,17 @@ public class GroupAlsoByWindowsProperties {
    */
   public static void groupsElementsIntoSlidingWindowsWithMinTimestamp(
       GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
-          throws Exception {
+      throws Exception {
 
-    WindowingStrategy<?, IntervalWindow> windowingStrategy = WindowingStrategy.of(
-        SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10)))
-        .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp());
+    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",
+    List<WindowedValue<KV<String, Iterable<String>>>> result =
+        runGABW(
+            gabwFactory,
+            windowingStrategy,
+            "key",
             WindowedValue.of(
                 "v1",
                 new Instant(5),
@@ -165,21 +172,21 @@ public class GroupAlsoByWindowsProperties {
                 Arrays.asList(window(0, 20), window(10, 30)),
                 PaneInfo.NO_FIRING));
 
-    assertThat(result.peekOutputElements(), hasSize(3));
+    assertThat(result, hasSize(3));
 
     TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(-10, 10)));
+        getOnlyElementInWindow(result, 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)));
+        getOnlyElementInWindow(result, window(0, 20));
     assertThat(item1.getValue().getValue(), containsInAnyOrder("v1", "v2"));
     // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
     assertThat(item1.getTimestamp(), equalTo(new Instant(10)));
 
     TimestampedValue<KV<String, Iterable<String>>> item2 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30)));
+        getOnlyElementInWindow(result, window(10, 30));
     assertThat(item2.getValue().getValue(), contains("v2"));
     // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
     assertThat(item2.getTimestamp(), equalTo(new Instant(20)));
@@ -194,14 +201,17 @@ public class GroupAlsoByWindowsProperties {
   public static void combinesElementsInSlidingWindows(
       GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
       CombineFn<Long, ?, Long> combineFn)
-          throws Exception {
+      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",
+    List<WindowedValue<KV<String, Long>>> result =
+        runGABW(
+            gabwFactory,
+            windowingStrategy,
+            "k",
             WindowedValue.of(
                 1L,
                 new Instant(5),
@@ -218,23 +228,20 @@ public class GroupAlsoByWindowsProperties {
                 Arrays.asList(window(0, 20), window(10, 30)),
                 PaneInfo.NO_FIRING));
 
-    assertThat(result.peekOutputElements(), hasSize(3));
+    assertThat(result, hasSize(3));
 
-    TimestampedValue<KV<String, Long>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(-10, 10)));
+    TimestampedValue<KV<String, Long>> item0 = getOnlyElementInWindow(result, 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)));
+    TimestampedValue<KV<String, Long>> item1 = getOnlyElementInWindow(result, window(0, 20));
     assertThat(item1.getValue().getKey(), equalTo("k"));
     assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L, 4L))));
     // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
     assertThat(item1.getTimestamp(), equalTo(new Instant(10L)));
 
-    TimestampedValue<KV<String, Long>> item2 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30)));
+    TimestampedValue<KV<String, Long>> item2 = getOnlyElementInWindow(result, window(10, 30));
     assertThat(item2.getValue().getKey(), equalTo("k"));
     assertThat(item2.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(2L, 4L))));
     // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
@@ -247,79 +254,63 @@ public class GroupAlsoByWindowsProperties {
    */
   public static void groupsIntoOverlappingNonmergingWindows(
       GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
-          throws Exception {
+      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),
+    List<WindowedValue<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));
+                "v3", new Instant(4), Arrays.asList(window(0, 5)), PaneInfo.NO_FIRING));
 
-    assertThat(result.peekOutputElements(), hasSize(2));
+    assertThat(result, hasSize(2));
 
     TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 5)));
+        getOnlyElementInWindow(result, 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)));
+        getOnlyElementInWindow(result, 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.
-   */
+  /** Tests that the given GABW implementation correctly groups elements into merged sessions. */
   public static void groupsElementsInMergedSessions(
       GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory)
-          throws Exception {
+      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",
+    List<WindowedValue<KV<String, Iterable<String>>>> result =
+        runGABW(
+            gabwFactory,
+            windowingStrategy,
+            "key",
             WindowedValue.of(
-                "v1",
-                new Instant(0),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
+                "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),
+                "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));
+                "v3", new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING));
 
-    assertThat(result.peekOutputElements(), hasSize(2));
+    assertThat(result, hasSize(2));
 
     TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
+        getOnlyElementInWindow(result, 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)));
+        getOnlyElementInWindow(result, window(15, 25));
     assertThat(item1.getValue().getValue(), contains("v3"));
     assertThat(item1.getTimestamp(), equalTo(window(15, 25).maxTimestamp()));
   }
@@ -331,39 +322,29 @@ public class GroupAlsoByWindowsProperties {
   public static void combinesElementsPerSession(
       GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
       CombineFn<Long, ?, Long> combineFn)
-          throws Exception {
+      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),
+    List<WindowedValue<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(
-                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));
+                4L, new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING));
 
-    assertThat(result.peekOutputElements(), hasSize(2));
+    assertThat(result, hasSize(2));
 
-    TimestampedValue<KV<String, Long>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
+    TimestampedValue<KV<String, Long>> item0 = getOnlyElementInWindow(result, 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)));
+    TimestampedValue<KV<String, Long>> item1 = getOnlyElementInWindow(result, 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()));
@@ -371,176 +352,152 @@ public class GroupAlsoByWindowsProperties {
 
   /**
    * 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()}.
+   * 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 {
+      throws Exception {
 
     WindowingStrategy<?, IntervalWindow> windowingStrategy =
         WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
-        .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
+            .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
 
-    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
-        runGABW(gabwFactory, windowingStrategy, "key",
+    List<WindowedValue<KV<String, Iterable<String>>>> result =
+        runGABW(
+            gabwFactory,
+            windowingStrategy,
+            "key",
             WindowedValue.of(
-                "v1",
-                new Instant(1),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
+                "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),
+                "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));
+                "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING));
 
-    assertThat(result.peekOutputElements(), hasSize(2));
+    assertThat(result, hasSize(2));
 
     TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
+        getOnlyElementInWindow(result, 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)));
+        getOnlyElementInWindow(result, 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()}.
+   * 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 {
+      throws Exception {
 
     WindowingStrategy<?, IntervalWindow> windowingStrategy =
         WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))
-        .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp());
+            .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp());
 
-    DoFnTester<KV<String, Iterable<WindowedValue<String>>>, KV<String, Iterable<String>>> result =
-        runGABW(gabwFactory, windowingStrategy, "k",
+    List<WindowedValue<KV<String, Iterable<String>>>> result =
+        runGABW(
+            gabwFactory,
+            windowingStrategy,
+            "k",
             WindowedValue.of(
-                "v1",
-                new Instant(1),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
+                "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),
+                "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));
+                "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING));
 
-    assertThat(result.peekOutputElements(), hasSize(2));
+    assertThat(result, hasSize(2));
 
     TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 10)));
+        getOnlyElementInWindow(result, 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)));
+        getOnlyElementInWindow(result, 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.
+   * 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 {
+      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",
+    List<WindowedValue<KV<String, Iterable<String>>>> result =
+        runGABW(
+            gabwFactory,
+            windowingStrategy,
+            "k",
             WindowedValue.of(
-                "v1",
-                new Instant(0),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
+                "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),
+                "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));
+                "v3", new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING));
 
-    assertThat(result.peekOutputElements(), hasSize(2));
+    assertThat(result, hasSize(2));
 
     TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 15)));
+        getOnlyElementInWindow(result, 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)));
+        getOnlyElementInWindow(result, 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.
+   * 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 {
+      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",
+    List<WindowedValue<KV<String, Iterable<String>>>> result =
+        runGABW(
+            gabwFactory,
+            windowingStrategy,
+            "k",
             WindowedValue.of(
-                "v1",
-                new Instant(0),
-                Arrays.asList(window(0, 10)),
-                PaneInfo.NO_FIRING),
+                "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),
+                "v2", new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING),
             WindowedValue.of(
-                "v3",
-                new Instant(15),
-                Arrays.asList(unmergedWindow),
-                PaneInfo.NO_FIRING));
+                "v3", new Instant(15), Arrays.asList(unmergedWindow), PaneInfo.NO_FIRING));
 
-    assertThat(result.peekOutputElements(), hasSize(2));
+    assertThat(result, hasSize(2));
 
     BoundedWindow mergedWindow = window(0, 15);
     TimestampedValue<KV<String, Iterable<String>>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(mergedWindow));
+        getOnlyElementInWindow(result, 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));
+        getOnlyElementInWindow(result, unmergedWindow);
     assertThat(item1.getValue().getValue(), contains("v3"));
     assertThat(item1.getTimestamp(), equalTo(new Instant(15)));
   }
@@ -552,81 +509,66 @@ public class GroupAlsoByWindowsProperties {
   public static void combinesElementsPerSessionWithEndOfWindowTimestamp(
       GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory,
       CombineFn<Long, ?, Long> combineFn)
-          throws Exception {
+      throws Exception {
 
     WindowingStrategy<?, IntervalWindow> windowingStrategy =
         WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10)))
-        .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow());
+            .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));
+    List<WindowedValue<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));
+    assertThat(result, hasSize(2));
 
     BoundedWindow firstResultWindow = window(0, 15);
-    TimestampedValue<KV<String, Long>> item0 =
-        Iterables.getOnlyElement(result.peekOutputElementsInWindow(firstResultWindow));
+    TimestampedValue<KV<String, Long>> item0 = getOnlyElementInWindow(result, 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));
+    TimestampedValue<KV<String, Long>> item1 = getOnlyElementInWindow(result, secondWindow);
     assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(4L))));
-    assertThat(item1.getTimestamp(),
-        equalTo(secondWindow.maxTimestamp()));
+    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 {
+      List<WindowedValue<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 {
+      List<WindowedValue<KV<K, OutputT>>> runGABW(
+          GroupAlsoByWindowsDoFnFactory<K, InputT, OutputT> gabwFactory,
+          WindowingStrategy<?, W> windowingStrategy,
+          K key,
+          Collection<WindowedValue<InputT>> values)
+          throws Exception {
 
     final StateInternalsFactory<K> stateInternalsCache = new CachingStateInternalsFactory<K>();
 
-    DoFnTester<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>> tester =
-        DoFnTester.of(gabwFactory.forStrategy(windowingStrategy, stateInternalsCache));
-
-    // Though we use a DoFnTester, the function itself is instantiated directly by the
-    // runner and should not be serialized; it may not even be serializable.
-    tester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE);
-    tester.startBundle();
-    tester.processElement(KV.<K, Iterable<WindowedValue<InputT>>>of(key, values));
-    tester.finishBundle();
+    List<WindowedValue<KV<K, OutputT>>> output =
+        processElement(
+            gabwFactory.forStrategy(windowingStrategy, stateInternalsCache),
+            KV.<K, Iterable<WindowedValue<InputT>>>of(key, values));
 
     // Sanity check for corruption
-    for (KV<K, OutputT> elem : tester.peekOutputElements()) {
-      assertThat(elem.getKey(), equalTo(key));
+    for (WindowedValue<KV<K, OutputT>> value : output) {
+      assertThat(value.getValue().getKey(), equalTo(key));
     }
 
-    return tester;
+    return output;
   }
 
   private static BoundedWindow window(long start, long end) {
@@ -657,4 +599,158 @@ public class GroupAlsoByWindowsProperties {
       return InMemoryStateInternals.forKey(key);
     }
   }
+
+  private static <K, InputT, OutputT, W extends BoundedWindow>
+      List<WindowedValue<KV<K, OutputT>>> processElement(
+          GroupAlsoByWindowsDoFn<K, InputT, OutputT, W> fn,
+          KV<K, Iterable<WindowedValue<InputT>>> element)
+          throws Exception {
+    TestProcessContext<K, InputT, OutputT, W> c = new TestProcessContext<>(fn, element);
+    fn.processElement(c);
+    return c.getOutput();
+  }
+
+  private static <K, OutputT> TimestampedValue<KV<K, OutputT>> getOnlyElementInWindow(
+      List<WindowedValue<KV<K, OutputT>>> output, final BoundedWindow window) {
+    WindowedValue<KV<K, OutputT>> res =
+        Iterables.getOnlyElement(
+            Iterables.filter(
+                output,
+                new Predicate<WindowedValue<KV<K, OutputT>>>() {
+                  @Override
+                  public boolean apply(@Nullable WindowedValue<KV<K, OutputT>> input) {
+                    return input.getWindows().contains(window);
+                  }
+                }));
+    return TimestampedValue.of(res.getValue(), res.getTimestamp());
+  }
+
+  /**
+   * A {@link GroupAlsoByWindowsDoFn.ProcessContext} providing just enough context for a {@link
+   * GroupAlsoByWindowsDoFn} - namely, information about the element and output via {@link
+   * WindowingInternals}, but no side inputs/outputs and no normal output.
+   */
+  private static class TestProcessContext<K, InputT, OutputT, W extends BoundedWindow>
+      extends GroupAlsoByWindowsDoFn<K, InputT, OutputT, W>.ProcessContext {
+    private final PipelineOptions options = PipelineOptionsFactory.create();
+    private final KV<K, Iterable<WindowedValue<InputT>>> element;
+    private final List<WindowedValue<KV<K, OutputT>>> output = new ArrayList<>();
+
+    private TestProcessContext(
+        GroupAlsoByWindowsDoFn<K, InputT, OutputT, W> fn,
+        KV<K, Iterable<WindowedValue<InputT>>> element) {
+      fn.super();
+      this.element = element;
+    }
+
+    @Override
+    public KV<K, Iterable<WindowedValue<InputT>>> element() {
+      return element;
+    }
+
+    @Override
+    public Instant timestamp() {
+      return BoundedWindow.TIMESTAMP_MIN_VALUE;
+    }
+
+    @Override
+    public BoundedWindow window() {
+      return GlobalWindow.INSTANCE;
+    }
+
+    @Override
+    public PaneInfo pane() {
+      return PaneInfo.NO_FIRING;
+    }
+
+    @Override
+    public <T> T sideInput(PCollectionView<T> view) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public WindowingInternals<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>>
+        windowingInternals() {
+      return new WindowingInternals<KV<K, Iterable<WindowedValue<InputT>>>, KV<K, OutputT>>() {
+        @Override
+        public void outputWindowedValue(
+            KV<K, OutputT> output,
+            Instant timestamp,
+            Collection<? extends BoundedWindow> windows,
+            PaneInfo pane) {
+          TestProcessContext.this.output.add(WindowedValue.of(output, timestamp, windows, pane));
+        }
+
+        @Override
+        public <SideOutputT> void sideOutputWindowedValue(
+            TupleTag<SideOutputT> tag,
+            SideOutputT output,
+            Instant timestamp,
+            Collection<? extends BoundedWindow> windows,
+            PaneInfo pane) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public StateInternals<?> stateInternals() {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public TimerInternals timerInternals() {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public Collection<? extends BoundedWindow> windows() {
+          return ImmutableList.of(GlobalWindow.INSTANCE);
+        }
+
+        @Override
+        public PaneInfo pane() {
+          return PaneInfo.NO_FIRING;
+        }
+
+        @Override
+        public <T> T sideInput(PCollectionView<T> view, BoundedWindow sideInputWindow) {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return options;
+    }
+
+    @Override
+    public void output(KV<K, OutputT> output) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void outputWithTimestamp(KV<K, OutputT> output, Instant timestamp) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+        String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+      throw new UnsupportedOperationException();
+    }
+
+    public List<WindowedValue<KV<K, OutputT>>> getOutput() {
+      return output;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/96455768/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
index 1a74ae7..6ee42e7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java
@@ -19,6 +19,7 @@ package org.apache.beam.sdk.transforms;
 
 import java.io.IOException;
 import java.util.Collection;
+import javax.annotation.Nullable;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.DoFn.Context;
@@ -185,6 +186,7 @@ public class DoFnAdapters {
    * If the fn was created using {@link #toOldDoFn}, returns the original {@link DoFn}. Otherwise,
    * returns {@code null}.
    */
+  @Nullable
   public static <InputT, OutputT> DoFn<InputT, OutputT> getDoFn(OldDoFn<InputT, OutputT> fn) {
     if (fn instanceof SimpleDoFnAdapter) {
       return ((SimpleDoFnAdapter<InputT, OutputT>) fn).fn;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/96455768/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
index 17fa612..a9f93dd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
@@ -23,10 +23,10 @@ import static com.google.common.base.Preconditions.checkState;
 import com.google.common.base.Function;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -36,6 +36,8 @@ import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.ValueInSingleWindow;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
@@ -43,7 +45,6 @@ import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.UserCodeException;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingInternals;
 import org.apache.beam.sdk.util.state.InMemoryStateInternals;
 import org.apache.beam.sdk.util.state.InMemoryTimerInternals;
 import org.apache.beam.sdk.util.state.StateInternals;
@@ -86,7 +87,8 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
    */
   @SuppressWarnings("unchecked")
   public static <InputT, OutputT> DoFnTester<InputT, OutputT> of(DoFn<InputT, OutputT> fn) {
-    return new DoFnTester<>(DoFnAdapters.toOldDoFn(fn));
+    checkNotNull(fn, "fn can't be null");
+    return new DoFnTester<>(fn);
   }
 
   /**
@@ -96,9 +98,11 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
    * @see #of(DoFn)
    */
   @SuppressWarnings("unchecked")
-   public static <InputT, OutputT> DoFnTester<InputT, OutputT>
+  @Deprecated
+  public static <InputT, OutputT> DoFnTester<InputT, OutputT>
       of(OldDoFn<InputT, OutputT> fn) {
-    return new DoFnTester<>(fn);
+    checkNotNull(fn, "fn can't be null");
+    return new DoFnTester<>(fn.toDoFn());
   }
 
   /**
@@ -238,7 +242,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     stateInternals = InMemoryStateInternals.forKey(new Object());
     timerInternals = new InMemoryTimerInternals();
     try {
-      fn.startBundle(context);
+      fnInvoker.invokeStartBundle(context);
     } catch (UserCodeException e) {
       unwrapUserCodeException(e);
     }
@@ -271,8 +275,8 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
   }
 
   /**
-   * Calls {@link OldDoFn#processElement} on the {@code OldDoFn} under test, in a
-   * context where {@link OldDoFn.ProcessContext#element} returns the
+   * Calls {@link DoFn.ProcessElement} on the {@code DoFn} under test, in a
+   * context where {@link DoFn.ProcessContext#element} returns the
    * given element and timestamp.
    *
    * <p>Will call {@link #startBundle} automatically, if it hasn't
@@ -286,7 +290,13 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
       startBundle();
     }
     try {
-      fn.processElement(createProcessContext(element));
+      final TestProcessContext processContext = createProcessContext(element);
+      fnInvoker.invokeProcessElement(new DoFnInvoker.FakeArgumentProvider<InputT, OutputT>() {
+        @Override
+        public DoFn<InputT, OutputT>.ProcessContext processContext(DoFn<InputT, OutputT> doFn) {
+          return processContext;
+        }
+      });
     } catch (UserCodeException e) {
       unwrapUserCodeException(e);
     }
@@ -308,13 +318,14 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
         "Must be inside bundle to call finishBundle, but was: %s",
         state);
     try {
-      fn.finishBundle(createContext(fn));
+      fnInvoker.invokeFinishBundle(createContext(fn));
     } catch (UserCodeException e) {
       unwrapUserCodeException(e);
     }
     if (cloningBehavior == CloningBehavior.CLONE_PER_BUNDLE) {
-      fn.teardown();
+      fnInvoker.invokeTeardown();
       fn = null;
+      fnInvoker = null;
       state = State.UNINITIALIZED;
     } else {
       state = State.BUNDLE_FINISHED;
@@ -532,11 +543,11 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     return mainOutputTag;
   }
 
-  private TestContext createContext(OldDoFn<InputT, OutputT> fn) {
+  private TestContext createContext(DoFn<InputT, OutputT> fn) {
     return new TestContext();
   }
 
-  private class TestContext extends OldDoFn<InputT, OutputT>.Context {
+  private class TestContext extends DoFn<InputT, OutputT>.Context {
     TestContext() {
       fn.super();
     }
@@ -557,7 +568,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     }
 
     @Override
-    protected <AggInT, AggOutT> Aggregator<AggInT, AggOutT> createAggregatorInternal(
+    protected <AggInT, AggOutT> Aggregator<AggInT, AggOutT> createAggregator(
         final String name, final CombineFn<AggInT, ?, AggOutT> combiner) {
       return aggregator(name, combiner);
     }
@@ -624,7 +635,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
             elem.getValue(), elem.getTimestamp(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
   }
 
-  private class TestProcessContext extends OldDoFn<InputT, OutputT>.ProcessContext {
+  private class TestProcessContext extends DoFn<InputT, OutputT>.ProcessContext {
     private final TestContext context;
     private final ValueInSingleWindow<InputT> element;
 
@@ -644,7 +655,9 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
       Map<BoundedWindow, ?> viewValues = sideInputs.get(view);
       if (viewValues != null) {
         BoundedWindow sideInputWindow =
-            view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(window());
+            view.getWindowingStrategyInternal()
+                .getWindowFn()
+                .getSideInputWindow(element.getWindow());
         @SuppressWarnings("unchecked")
         T windowValue = (T) viewValues.get(sideInputWindow);
         if (windowValue != null) {
@@ -660,73 +673,11 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     }
 
     @Override
-    public BoundedWindow window() {
-      return element.getWindow();
-    }
-
-    @Override
     public PaneInfo pane() {
       return element.getPane();
     }
 
     @Override
-    public WindowingInternals<InputT, OutputT> windowingInternals() {
-      return new WindowingInternals<InputT, OutputT>() {
-        @Override
-        public StateInternals<?> stateInternals() {
-          return stateInternals;
-        }
-
-        @Override
-        public void outputWindowedValue(
-            OutputT output,
-            Instant timestamp,
-            Collection<? extends BoundedWindow> windows,
-            PaneInfo pane) {
-          for (BoundedWindow window : windows) {
-            context.noteOutput(
-                mainOutputTag, ValueInSingleWindow.of(output, timestamp, window, pane));
-          }
-        }
-
-        @Override
-        public <SideOutputT> void sideOutputWindowedValue(
-            TupleTag<SideOutputT> tag,
-            SideOutputT output,
-            Instant timestamp,
-            Collection<? extends BoundedWindow> windows,
-            PaneInfo pane) {
-          for (BoundedWindow window : windows) {
-            context.noteOutput(
-                tag, ValueInSingleWindow.of(output, timestamp, window, pane));
-          }
-        }
-
-        @Override
-        public TimerInternals timerInternals() {
-          return timerInternals;
-        }
-
-        @Override
-        public Collection<? extends BoundedWindow> windows() {
-          return Collections.singleton(element.getWindow());
-        }
-
-        @Override
-        public PaneInfo pane() {
-          return element.getPane();
-        }
-
-        @Override
-        public <T> T sideInput(
-            PCollectionView<T> view, BoundedWindow sideInputWindow) {
-          throw new UnsupportedOperationException(
-              "SideInput from WindowingInternals is not supported in in the context of DoFnTester");
-        }
-      };
-    }
-
-    @Override
     public PipelineOptions getPipelineOptions() {
       return context.getPipelineOptions();
     }
@@ -753,10 +704,10 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     }
 
     @Override
-    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregator(
         String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
       throw new IllegalStateException("Aggregators should not be created within ProcessContext. "
-          + "Instead, create an aggregator at OldDoFn construction time with"
+          + "Instead, create an aggregator at DoFn construction time with"
           + " createAggregator, and ensure they are set up by the time startBundle is"
           + " called with setupDelegateAggregators.");
     }
@@ -768,8 +719,9 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
       finishBundle();
     }
     if (state == State.BUNDLE_FINISHED) {
-      fn.teardown();
+      fnInvoker.invokeTeardown();
       fn = null;
+      fnInvoker = null;
     }
     state = State.TORN_DOWN;
   }
@@ -786,8 +738,8 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
 
   private final PipelineOptions options = PipelineOptionsFactory.create();
 
-  /** The original {@link OldDoFn} under test. */
-  private final OldDoFn<InputT, OutputT> origFn;
+  /** The original {@link DoFn} under test. */
+  private final DoFn<InputT, OutputT> origFn;
 
   /**
    * Whether to clone the original {@link DoFn} or just use it as-is.
@@ -805,8 +757,9 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
   /** The output tags used by the {@link DoFn} under test. */
   private TupleTag<OutputT> mainOutputTag = new TupleTag<>();
 
-  /** The original OldDoFn under test, if started. */
-  OldDoFn<InputT, OutputT> fn;
+  /** The original DoFn under test, if started. */
+  private DoFn<InputT, OutputT> fn;
+  private DoFnInvoker<InputT, OutputT> fnInvoker;
 
   /** The outputs from the {@link DoFn} under test. */
   private Map<TupleTag<?>, List<ValueInSingleWindow<?>>> outputs;
@@ -817,7 +770,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
   /** The state of processing of the {@link DoFn} under test. */
   private State state = State.UNINITIALIZED;
 
-  private DoFnTester(OldDoFn<InputT, OutputT> origFn) {
+  private DoFnTester(DoFn<InputT, OutputT> origFn) {
     this.origFn = origFn;
   }
 
@@ -828,12 +781,13 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     if (cloningBehavior.equals(CloningBehavior.DO_NOT_CLONE)) {
       fn = origFn;
     } else {
-      fn = (OldDoFn<InputT, OutputT>)
+      fn = (DoFn<InputT, OutputT>)
           SerializableUtils.deserializeFromByteArray(
               SerializableUtils.serializeToByteArray(origFn),
               origFn.toString());
     }
-    fn.setup();
+    fnInvoker = DoFnInvokers.invokerFor(fn);
+    fnInvoker.invokeSetup();
     outputs = new HashMap<>();
     accumulators = new HashMap<>();
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/96455768/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java
index 4ad7dad..50a7082 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java
@@ -18,9 +18,6 @@
 package org.apache.beam.sdk.transforms.reflect;
 
 import java.io.Serializable;
-import java.lang.reflect.Constructor;
-import java.util.LinkedHashMap;
-import java.util.Map;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderRegistry;
 import org.apache.beam.sdk.transforms.DoFn;
@@ -45,14 +42,6 @@ public class DoFnInvokers {
     return ByteBuddyDoFnInvokerFactory.only().newByteBuddyInvoker(fn);
   }
 
-  /**
-   * A cache of constructors of generated {@link DoFnInvoker} classes, keyed by {@link DoFn} class.
-   * Needed because generating an invoker class is expensive, and to avoid generating an excessive
-   * number of classes consuming PermGen memory.
-   */
-  private final Map<Class<?>, Constructor<?>> byteBuddyInvokerConstructorCache =
-      new LinkedHashMap<>();
-
   private DoFnInvokers() {}
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/96455768/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
index ac76b2e..ff8a9bc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
@@ -350,14 +350,14 @@ public class DoFnTesterTest {
     }
   }
 
-  private static class SideInputDoFn extends OldDoFn<Integer, Integer> {
+  private static class SideInputDoFn extends DoFn<Integer, Integer> {
     private final PCollectionView<Integer> value;
 
     private SideInputDoFn(PCollectionView<Integer> value) {
       this.value = value;
     }
 
-    @Override
+    @ProcessElement
     public void processElement(ProcessContext c) throws Exception {
       c.output(c.sideInput(value));
     }


[48/50] [abbrv] incubator-beam git commit: Merge remote-tracking branch 'upstream/master' into gearpump-runner-sync

Posted by ke...@apache.org.
Merge remote-tracking branch 'upstream/master' into gearpump-runner-sync


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/86414c08
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/86414c08
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/86414c08

Branch: refs/heads/gearpump-runner
Commit: 86414c08eb70c8975dac7b4750ca02b8b9cf875b
Parents: 2812405 ef9871c
Author: manuzhang <ow...@gmail.com>
Authored: Mon Dec 5 11:30:31 2016 +0800
Committer: manuzhang <ow...@gmail.com>
Committed: Mon Dec 5 11:30:31 2016 +0800

----------------------------------------------------------------------
 .gitignore                                      |   4 +
 .jenkins/common_job_properties.groovy           | 166 ++++++
 ...job_beam_PostCommit_Java_MavenInstall.groovy |  42 ++
 ...ommit_Java_RunnableOnService_Dataflow.groovy |  39 ++
 ...stCommit_Java_RunnableOnService_Flink.groovy |  38 ++
 ...ommit_Java_RunnableOnService_Gearpump.groovy |  41 ++
 ...stCommit_Java_RunnableOnService_Spark.groovy |  38 ++
 .../job_beam_PostCommit_Python_Verify.groovy    |  37 ++
 .../job_beam_PreCommit_Java_MavenInstall.groovy |  42 ++
 .../job_beam_Release_NightlySnapshot.groovy     |  46 ++
 .jenkins/job_seed.groovy                        |  47 ++
 .travis.yml                                     |   2 +-
 .travis/README.md                               |   2 +-
 .../org/apache/beam/examples/complete/README.md |  14 +-
 .../beam/examples/cookbook/DeDupExample.java    |  96 ---
 .../beam/examples/cookbook/DistinctExample.java |  96 +++
 .../org/apache/beam/examples/cookbook/README.md |  16 +-
 .../org/apache/beam/examples/WordCountIT.java   |   2 +-
 pom.xml                                         |  10 +-
 runners/apex/pom.xml                            |  11 +-
 .../translation/ApexPipelineTranslator.java     |  12 +-
 .../apex/translation/TranslationContext.java    |   6 +-
 .../operators/ApexGroupByKeyOperator.java       |  14 +-
 .../core/ElementAndRestrictionCoder.java        |   8 +
 .../runners/core/GBKIntoKeyedWorkItems.java     |  55 --
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |   6 +-
 .../core/GroupByKeyViaGroupByKeyOnly.java       |  22 +-
 .../runners/core/PerKeyCombineFnRunners.java    | 263 +++++++++
 .../core/PushbackSideInputDoFnRunner.java       |  20 +-
 .../beam/runners/core/SimpleDoFnRunner.java     |  76 ++-
 .../beam/runners/core/SimpleOldDoFnRunner.java  |  16 -
 .../beam/runners/core/SplittableParDo.java      | 378 +++++++++---
 .../core/GroupAlsoByWindowsProperties.java      | 590 +++++++++++--------
 .../core/PushbackSideInputDoFnRunnerTest.java   |  18 +-
 .../beam/runners/core/ReduceFnRunnerTest.java   |   5 +-
 .../beam/runners/core/SplittableParDoTest.java  | 132 ++++-
 .../beam/runners/core/WindowMatchers.java       | 204 +++++++
 .../beam/runners/core/WindowMatchersTest.java   |  82 +++
 .../direct/AbstractModelEnforcement.java        |   2 +-
 .../direct/BoundedReadEvaluatorFactory.java     |   8 +-
 .../beam/runners/direct/CommittedResult.java    |   2 +-
 .../beam/runners/direct/CompletionCallback.java |   2 +-
 .../direct/ConsumerTrackingPipelineVisitor.java | 173 ------
 ...ectGBKIntoKeyedWorkItemsOverrideFactory.java |  41 +-
 .../apache/beam/runners/direct/DirectGraph.java |  89 +++
 .../beam/runners/direct/DirectGraphVisitor.java | 145 +++++
 .../beam/runners/direct/DirectGroupByKey.java   |   2 +-
 .../beam/runners/direct/DirectRunner.java       |  48 +-
 .../runners/direct/DirectTimerInternals.java    |   4 +-
 .../runners/direct/DisplayDataValidator.java    |   6 +-
 .../runners/direct/DoFnLifecycleManager.java    |   4 +-
 ...ecycleManagerRemovingTransformEvaluator.java |   2 +-
 .../runners/direct/EmptyTransformEvaluator.java |   4 +-
 .../beam/runners/direct/EvaluationContext.java  |  87 ++-
 .../direct/ExecutorServiceParallelExecutor.java |  17 +-
 .../runners/direct/FlattenEvaluatorFactory.java |  10 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |   5 +-
 .../direct/GroupByKeyOnlyEvaluatorFactory.java  |   2 +-
 .../ImmutabilityCheckingBundleFactory.java      |  21 +-
 .../direct/ImmutabilityEnforcementFactory.java  |   2 +-
 .../direct/KeyedPValueTrackingVisitor.java      |  12 +-
 .../beam/runners/direct/ModelEnforcement.java   |   2 +-
 .../beam/runners/direct/ParDoEvaluator.java     |  33 +-
 .../runners/direct/ParDoEvaluatorFactory.java   |  94 ++-
 .../direct/ParDoMultiOverrideFactory.java       |  76 ++-
 .../ParDoSingleViaMultiOverrideFactory.java     |   6 +-
 .../direct/PassthroughTransformEvaluator.java   |   4 +-
 ...littableProcessElementsEvaluatorFactory.java | 144 +++++
 .../direct/StatefulParDoEvaluatorFactory.java   | 256 ++++++++
 .../runners/direct/StepTransformResult.java     |  38 +-
 .../direct/TestStreamEvaluatorFactory.java      |   2 +-
 .../beam/runners/direct/TransformEvaluator.java |   2 +-
 .../direct/TransformEvaluatorRegistry.java      |   7 +
 .../beam/runners/direct/TransformExecutor.java  |   4 +-
 .../beam/runners/direct/TransformResult.java    |  16 +-
 .../direct/UnboundedReadEvaluatorFactory.java   |   3 +-
 .../runners/direct/ViewEvaluatorFactory.java    |   2 +-
 .../direct/WatermarkCallbackExecutor.java       |  34 ++
 .../beam/runners/direct/WatermarkManager.java   |  50 +-
 .../runners/direct/WindowEvaluatorFactory.java  |   6 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |  16 +-
 .../ConsumerTrackingPipelineVisitorTest.java    | 287 ---------
 .../runners/direct/DirectGraphVisitorTest.java  | 239 ++++++++
 ...leManagerRemovingTransformEvaluatorTest.java |   4 +-
 .../runners/direct/EvaluationContextTest.java   |  49 +-
 .../direct/FlattenEvaluatorFactoryTest.java     |   6 +-
 .../ImmutabilityCheckingBundleFactoryTest.java  |   6 +-
 .../ImmutabilityEnforcementFactoryTest.java     |   6 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |   2 +-
 .../beam/runners/direct/SplittableDoFnTest.java | 231 --------
 .../StatefulParDoEvaluatorFactoryTest.java      | 300 ++++++++++
 .../runners/direct/StepTransformResultTest.java |  25 +-
 .../direct/TestStreamEvaluatorFactoryTest.java  |  10 +-
 .../runners/direct/TransformExecutorTest.java   | 223 +++----
 .../UnboundedReadEvaluatorFactoryTest.java      |  20 +-
 .../runners/direct/WatermarkManagerTest.java    |  23 +-
 .../direct/WindowEvaluatorFactoryTest.java      |  16 +-
 runners/flink/runner/pom.xml                    |   9 +-
 .../apache/beam/runners/flink/FlinkRunner.java  |  12 +-
 .../FlinkBatchPipelineTranslator.java           |  14 +-
 .../FlinkStreamingPipelineTranslator.java       |  16 +-
 .../FlinkStreamingTransformTranslators.java     |  63 +-
 .../PipelineTranslationOptimizer.java           |  10 +-
 .../FlinkMergingNonShuffleReduceFunction.java   |   2 +-
 .../FlinkMergingPartialReduceFunction.java      |   2 +-
 .../functions/FlinkMergingReduceFunction.java   |   2 +-
 .../functions/FlinkPartialReduceFunction.java   |   2 +-
 .../functions/FlinkProcessContextBase.java      |   8 -
 .../functions/FlinkReduceFunction.java          |   2 +-
 runners/google-cloud-dataflow-java/pom.xml      |   4 +
 .../dataflow/DataflowPipelineTranslator.java    |  33 +-
 .../beam/runners/dataflow/DataflowRunner.java   |  53 +-
 .../DataflowPipelineTranslatorTest.java         |   2 +-
 .../runners/dataflow/DataflowRunnerTest.java    |   4 +-
 .../dataflow/RecordingPipelineVisitor.java      |   6 +-
 runners/spark/pom.xml                           |   7 +-
 .../runners/spark/SparkPipelineOptions.java     |   5 +
 .../apache/beam/runners/spark/SparkRunner.java  |  80 ++-
 .../spark/aggregators/AccumulatorSingleton.java |   2 +-
 .../beam/runners/spark/io/MicrobatchSource.java |  20 +-
 .../beam/runners/spark/io/SourceDStream.java    |   3 +-
 .../spark/stateful/StateSpecFunctions.java      |   2 +-
 .../spark/translation/BoundedDataset.java       |   5 +-
 .../beam/runners/spark/translation/Dataset.java |   2 +-
 .../spark/translation/EvaluationContext.java    |  10 +-
 .../spark/translation/SparkProcessContext.java  |   9 -
 .../translation/StorageLevelPTransform.java     |  43 ++
 .../spark/translation/TransformTranslator.java  |  27 +
 .../SparkRunnerStreamingContextFactory.java     |   2 +-
 .../streaming/StreamingTransformTranslator.java |   2 +-
 .../translation/streaming/UnboundedDataset.java |  13 +-
 .../spark/translation/StorageLevelTest.java     |  56 ++
 .../streaming/EmptyStreamAssertionTest.java     |   2 +
 .../streaming/FlattenStreamingTest.java         |   2 +
 .../streaming/SimpleStreamingWordCountTest.java |   1 +
 .../SparkTestPipelineOptionsForStreaming.java   |   6 -
 sdks/java/core/pom.xml                          |  29 +-
 .../beam/sdk/AggregatorPipelineExtractor.java   |   6 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java | 151 ++---
 .../org/apache/beam/sdk/io/FileBasedSink.java   |  22 +-
 .../java/org/apache/beam/sdk/io/PubsubIO.java   | 176 +++++-
 .../apache/beam/sdk/io/PubsubUnboundedSink.java |  23 +-
 .../beam/sdk/io/PubsubUnboundedSource.java      |  40 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |  28 +-
 .../java/org/apache/beam/sdk/io/XmlSink.java    |   4 +-
 .../beam/sdk/options/PipelineOptions.java       |   7 +
 .../sdk/options/PipelineOptionsFactory.java     |   1 +
 .../sdk/options/ProxyInvocationHandler.java     |  26 +
 .../apache/beam/sdk/options/ValueProvider.java  |   2 +-
 .../beam/sdk/runners/TransformHierarchy.java    | 340 +++++++++--
 .../beam/sdk/runners/TransformTreeNode.java     | 257 --------
 .../beam/sdk/testing/FileChecksumMatcher.java   | 168 +++++-
 .../apache/beam/sdk/testing/GatherAllPanes.java |  88 +++
 .../org/apache/beam/sdk/testing/PAssert.java    |  77 +--
 .../apache/beam/sdk/testing/PaneExtractors.java |  55 +-
 .../beam/sdk/testing/UsesSplittableParDo.java   |  25 +
 .../beam/sdk/testing/ValueInSingleWindow.java   | 134 +++++
 .../org/apache/beam/sdk/transforms/Combine.java |  37 +-
 .../org/apache/beam/sdk/transforms/DoFn.java    |  16 +-
 .../beam/sdk/transforms/DoFnAdapters.java       |   2 +
 .../apache/beam/sdk/transforms/DoFnTester.java  | 350 ++++++-----
 .../org/apache/beam/sdk/transforms/OldDoFn.java |   8 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   |  10 -
 .../reflect/ByteBuddyDoFnInvokerFactory.java    |  28 +-
 .../reflect/ByteBuddyOnTimerInvokerFactory.java |  10 +-
 .../sdk/transforms/reflect/DoFnInvoker.java     |   3 +
 .../sdk/transforms/reflect/DoFnInvokers.java    |  16 +-
 .../sdk/transforms/reflect/DoFnSignature.java   |   8 +
 .../apache/beam/sdk/util/GatherAllPanes.java    |  86 ---
 .../apache/beam/sdk/util/IdentityWindowFn.java  |   2 +-
 .../beam/sdk/util/PerKeyCombineFnRunners.java   | 258 --------
 .../apache/beam/sdk/util/TimerInternals.java    | 107 ++--
 .../beam/sdk/util/WindowingInternals.java       |  10 -
 .../beam/sdk/util/common/ReflectHelpers.java    |   3 +-
 .../sdk/util/state/InMemoryTimerInternals.java  |   8 +-
 .../sdk/util/state/TimerInternalsFactory.java   |  36 ++
 .../sdk/AggregatorPipelineExtractorTest.java    |  20 +-
 .../org/apache/beam/sdk/WindowMatchers.java     | 204 -------
 .../org/apache/beam/sdk/WindowMatchersTest.java |  82 ---
 .../org/apache/beam/sdk/io/PubsubIOTest.java    |  43 +-
 .../beam/sdk/io/PubsubUnboundedSinkTest.java    |  20 +-
 .../beam/sdk/io/PubsubUnboundedSourceTest.java  |  14 +-
 .../org/apache/beam/sdk/io/XmlSinkTest.java     |   6 +-
 .../beam/sdk/options/PipelineOptionsTest.java   |  24 +
 .../beam/sdk/options/ValueProviderTest.java     |  18 +
 .../sdk/runners/TransformHierarchyTest.java     | 230 ++++++++
 .../beam/sdk/runners/TransformTreeTest.java     |  12 +-
 .../sdk/testing/FileChecksumMatcherTest.java    | 131 +++-
 .../beam/sdk/testing/GatherAllPanesTest.java    | 140 +++++
 .../beam/sdk/testing/PaneExtractorsTest.java    | 133 ++---
 .../testing/ValueInSingleWindowCoderTest.java   |  51 ++
 .../beam/sdk/transforms/DoFnTesterTest.java     |  38 +-
 .../apache/beam/sdk/transforms/ParDoTest.java   | 177 +++++-
 .../beam/sdk/transforms/SplittableDoFnTest.java | 401 +++++++++++++
 .../display/DisplayDataEvaluator.java           |   8 +-
 .../beam/sdk/util/GatherAllPanesTest.java       | 143 -----
 .../apache/beam/sdk/util/ReleaseInfoTest.java   |  45 ++
 .../beam/sdk/util/TimerInternalsTest.java       |   4 +-
 .../sorter/BufferedExternalSorter.java          |   6 +-
 .../sdk/extensions/sorter/ExternalSorter.java   |   6 +-
 .../sorter/BufferedExternalSorterTest.java      |  58 +-
 .../extensions/sorter/ExternalSorterTest.java   |  53 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIO.java     |  52 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java |  26 +-
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   |   6 +-
 .../examples-java8/generate-sources.sh          |  82 +++
 .../maven-archetypes/examples-java8/pom.xml     | 177 ++++++
 .../META-INF/maven/archetype-metadata.xml       |  39 ++
 .../main/resources/archetype-resources/pom.xml  | 209 +++++++
 .../projects/basic/archetype.properties         |  21 +
 .../src/test/resources/projects/basic/goal.txt  |   1 +
 .../examples/generate-sources.sh                |  68 +++
 sdks/java/maven-archetypes/examples/pom.xml     |  27 +-
 .../main/resources/archetype-resources/pom.xml  | 123 +++-
 .../src/main/java/DebuggingWordCount.java       | 164 ------
 .../src/main/java/MinimalWordCount.java         | 118 ----
 .../src/main/java/WindowedWordCount.java        | 229 -------
 .../src/main/java/WordCount.java                | 186 ------
 .../common/ExampleBigQueryTableOptions.java     |  55 --
 .../src/main/java/common/ExampleOptions.java    |  37 --
 ...xamplePubsubTopicAndSubscriptionOptions.java |  45 --
 .../java/common/ExamplePubsubTopicOptions.java  |  45 --
 .../src/main/java/common/ExampleUtils.java      | 352 -----------
 .../src/test/java/DebuggingWordCountTest.java   |  52 --
 .../src/test/java/WordCountTest.java            |  85 ---
 sdks/java/maven-archetypes/pom.xml              |  38 +-
 .../main/resources/archetype-resources/pom.xml  |   8 +
 .../resources/projects/basic/reference/pom.xml  |   8 +
 .../update-examples-archetype.sh                |  59 --
 229 files changed, 7944 insertions(+), 5261 deletions(-)
----------------------------------------------------------------------



[18/50] [abbrv] incubator-beam git commit: Closes #1482

Posted by ke...@apache.org.
Closes #1482


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/0fb56106
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/0fb56106
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/0fb56106

Branch: refs/heads/gearpump-runner
Commit: 0fb561068a5420cc8ee668be498e53eb8665fe29
Parents: f70fc40 d6eb514
Author: Dan Halperin <dh...@google.com>
Authored: Fri Dec 2 12:52:59 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Dec 2 12:52:59 2016 -0800

----------------------------------------------------------------------
 .travis.yml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[46/50] [abbrv] incubator-beam git commit: [BEAM-1057] Fix JDBC test derby startup issues

Posted by ke...@apache.org.
[BEAM-1057] Fix JDBC test derby startup issues


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/58916b94
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/58916b94
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/58916b94

Branch: refs/heads/gearpump-runner
Commit: 58916b94ada142d63d16a315da58f88184995f10
Parents: c22b97d
Author: Daniel Kulp <dk...@apache.org>
Authored: Mon Nov 28 16:47:37 2016 -0500
Committer: Jean-Baptiste Onofr� <jb...@apache.org>
Committed: Sun Dec 4 07:17:42 2016 +0100

----------------------------------------------------------------------
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java | 26 +++++++++++++++++++-
 1 file changed, 25 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/58916b94/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
index fe574af..d09929d 100644
--- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
+++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
@@ -20,7 +20,9 @@ package org.apache.beam.sdk.io.jdbc;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.io.PrintWriter;
 import java.io.Serializable;
+import java.io.StringWriter;
 import java.net.InetAddress;
 import java.net.ServerSocket;
 import java.sql.Connection;
@@ -73,7 +75,29 @@ public class JdbcIOTest implements Serializable {
     System.setProperty("derby.stream.error.file", "target/derby.log");
 
     derbyServer = new NetworkServerControl(InetAddress.getByName("localhost"), port);
-    derbyServer.start(null);
+    StringWriter out = new StringWriter();
+    derbyServer.start(new PrintWriter(out));
+    boolean started = false;
+    int count = 0;
+    // Use two different methods to detect when server is started:
+    // 1) Check the server stdout for the "started" string
+    // 2) wait up to 15 seconds for the derby server to start based on a ping
+    // on faster machines and networks, this may return very quick, but on slower
+    // networks where the DNS lookups are slow, this may take a little time
+    while (!started && count < 30) {
+      if (out.toString().contains("started")) {
+        started = true;
+      } else {
+        count++;
+        Thread.sleep(500);
+        try {
+          derbyServer.ping();
+          started = true;
+        } catch (Throwable t) {
+          //ignore, still trying to start
+        }
+      }
+    }
 
     dataSource = new ClientDataSource();
     dataSource.setCreateDatabase("create");


[41/50] [abbrv] incubator-beam git commit: BEAM-1083: Removing the link for the DatastoreWordCount in the README

Posted by ke...@apache.org.
BEAM-1083: Removing the link for the DatastoreWordCount in the README


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/8f712fd6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/8f712fd6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/8f712fd6

Branch: refs/heads/gearpump-runner
Commit: 8f712fd6291803bfcda312ad7c31cb5c811c6508
Parents: a13024c
Author: Neelesh Srinivas Salian <ns...@cloudera.com>
Authored: Sat Dec 3 09:08:54 2016 -0800
Committer: Neelesh Srinivas Salian <ns...@cloudera.com>
Committed: Sat Dec 3 09:08:54 2016 -0800

----------------------------------------------------------------------
 .../java/src/main/java/org/apache/beam/examples/cookbook/README.md | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f712fd6/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md b/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md
index e709955..105fb4b 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md
@@ -37,8 +37,6 @@ larger Dataflow pipeline. They include:
   transform, which lets you combine the values in a key-grouped
   <code>PCollection</code>.
   </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/DatastoreWordCount.java">DatastoreWordCount</a>
-  &mdash; An example that shows you how to read from Google Cloud Datastore.</li>
   <li><a href="https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/cookbook/DistinctExample.java">DistinctExample</a>
   &mdash; An example that uses Shakespeare's plays as plain text files, and
   removes duplicate lines across all the files. Demonstrates the


[44/50] [abbrv] incubator-beam git commit: Remove global default execution of exec-maven-plugin

Posted by ke...@apache.org.
Remove global default execution of exec-maven-plugin


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/6489b6db
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/6489b6db
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/6489b6db

Branch: refs/heads/gearpump-runner
Commit: 6489b6dbbcc947e707994892cbe1b6edb0e5b396
Parents: 8f712fd
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Nov 18 09:54:36 2016 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Sat Dec 3 16:12:00 2016 -0800

----------------------------------------------------------------------
 pom.xml | 8 --------
 1 file changed, 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6489b6db/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4e60fab..35a055c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -988,14 +988,6 @@
           <groupId>org.codehaus.mojo</groupId>
           <artifactId>exec-maven-plugin</artifactId>
           <version>1.4.0</version>
-          <executions>
-            <execution>
-              <phase>verify</phase>
-              <goals>
-                <goal>java</goal>
-              </goals>
-            </execution>
-          </executions>
           <configuration>
             <cleanupDaemonThreads>false</cleanupDaemonThreads>
             <systemProperties>


[50/50] [abbrv] incubator-beam git commit: This closes #1507

Posted by ke...@apache.org.
This closes #1507


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/88de0cb2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/88de0cb2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/88de0cb2

Branch: refs/heads/gearpump-runner
Commit: 88de0cb2367327f0fe41282d64f02d761c3ce25e
Parents: 2812405 2afc0cd
Author: Kenneth Knowles <kl...@google.com>
Authored: Tue Dec 6 08:36:54 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Tue Dec 6 08:36:54 2016 -0800

----------------------------------------------------------------------
 .gitignore                                      |   4 +
 .jenkins/common_job_properties.groovy           | 166 ++++++
 ...job_beam_PostCommit_Java_MavenInstall.groovy |  42 ++
 ...ommit_Java_RunnableOnService_Dataflow.groovy |  39 ++
 ...stCommit_Java_RunnableOnService_Flink.groovy |  38 ++
 ...ommit_Java_RunnableOnService_Gearpump.groovy |  41 ++
 ...stCommit_Java_RunnableOnService_Spark.groovy |  38 ++
 .../job_beam_PostCommit_Python_Verify.groovy    |  37 ++
 .../job_beam_PreCommit_Java_MavenInstall.groovy |  42 ++
 .../job_beam_Release_NightlySnapshot.groovy     |  46 ++
 .jenkins/job_seed.groovy                        |  47 ++
 .travis.yml                                     |   2 +-
 .travis/README.md                               |   2 +-
 .../org/apache/beam/examples/complete/README.md |  14 +-
 .../beam/examples/cookbook/DeDupExample.java    |  96 ---
 .../beam/examples/cookbook/DistinctExample.java |  96 +++
 .../org/apache/beam/examples/cookbook/README.md |  16 +-
 .../org/apache/beam/examples/WordCountIT.java   |   2 +-
 pom.xml                                         |  10 +-
 runners/apex/pom.xml                            |  11 +-
 .../translation/ApexPipelineTranslator.java     |  12 +-
 .../apex/translation/TranslationContext.java    |   6 +-
 .../operators/ApexGroupByKeyOperator.java       |  14 +-
 .../core/ElementAndRestrictionCoder.java        |   8 +
 .../runners/core/GBKIntoKeyedWorkItems.java     |  55 --
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |   6 +-
 .../core/GroupByKeyViaGroupByKeyOnly.java       |  22 +-
 .../runners/core/PerKeyCombineFnRunners.java    | 263 +++++++++
 .../core/PushbackSideInputDoFnRunner.java       |  20 +-
 .../beam/runners/core/SimpleDoFnRunner.java     |  76 ++-
 .../beam/runners/core/SimpleOldDoFnRunner.java  |  16 -
 .../beam/runners/core/SplittableParDo.java      | 378 +++++++++---
 .../core/GroupAlsoByWindowsProperties.java      | 590 +++++++++++--------
 .../core/PushbackSideInputDoFnRunnerTest.java   |  18 +-
 .../beam/runners/core/ReduceFnRunnerTest.java   |   5 +-
 .../beam/runners/core/SplittableParDoTest.java  | 132 ++++-
 .../beam/runners/core/WindowMatchers.java       | 204 +++++++
 .../beam/runners/core/WindowMatchersTest.java   |  82 +++
 .../direct/AbstractModelEnforcement.java        |   2 +-
 .../direct/BoundedReadEvaluatorFactory.java     |   8 +-
 .../beam/runners/direct/CommittedResult.java    |   2 +-
 .../beam/runners/direct/CompletionCallback.java |   2 +-
 .../direct/ConsumerTrackingPipelineVisitor.java | 173 ------
 ...ectGBKIntoKeyedWorkItemsOverrideFactory.java |  41 +-
 .../apache/beam/runners/direct/DirectGraph.java |  89 +++
 .../beam/runners/direct/DirectGraphVisitor.java | 145 +++++
 .../beam/runners/direct/DirectGroupByKey.java   |   2 +-
 .../beam/runners/direct/DirectRunner.java       |  48 +-
 .../runners/direct/DirectTimerInternals.java    |   4 +-
 .../runners/direct/DisplayDataValidator.java    |   6 +-
 .../runners/direct/DoFnLifecycleManager.java    |   4 +-
 ...ecycleManagerRemovingTransformEvaluator.java |   2 +-
 .../runners/direct/EmptyTransformEvaluator.java |   4 +-
 .../beam/runners/direct/EvaluationContext.java  |  87 ++-
 .../direct/ExecutorServiceParallelExecutor.java |  17 +-
 .../runners/direct/FlattenEvaluatorFactory.java |  10 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |   5 +-
 .../direct/GroupByKeyOnlyEvaluatorFactory.java  |   2 +-
 .../ImmutabilityCheckingBundleFactory.java      |  21 +-
 .../direct/ImmutabilityEnforcementFactory.java  |   2 +-
 .../direct/KeyedPValueTrackingVisitor.java      |  12 +-
 .../beam/runners/direct/ModelEnforcement.java   |   2 +-
 .../beam/runners/direct/ParDoEvaluator.java     |  33 +-
 .../runners/direct/ParDoEvaluatorFactory.java   |  94 ++-
 .../direct/ParDoMultiOverrideFactory.java       |  76 ++-
 .../ParDoSingleViaMultiOverrideFactory.java     |   6 +-
 .../direct/PassthroughTransformEvaluator.java   |   4 +-
 ...littableProcessElementsEvaluatorFactory.java | 144 +++++
 .../direct/StatefulParDoEvaluatorFactory.java   | 256 ++++++++
 .../runners/direct/StepTransformResult.java     |  38 +-
 .../direct/TestStreamEvaluatorFactory.java      |   2 +-
 .../beam/runners/direct/TransformEvaluator.java |   2 +-
 .../direct/TransformEvaluatorRegistry.java      |   7 +
 .../beam/runners/direct/TransformExecutor.java  |   4 +-
 .../beam/runners/direct/TransformResult.java    |  16 +-
 .../direct/UnboundedReadEvaluatorFactory.java   |   3 +-
 .../runners/direct/ViewEvaluatorFactory.java    |   2 +-
 .../direct/WatermarkCallbackExecutor.java       |  34 ++
 .../beam/runners/direct/WatermarkManager.java   |  50 +-
 .../runners/direct/WindowEvaluatorFactory.java  |   6 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |  16 +-
 .../ConsumerTrackingPipelineVisitorTest.java    | 287 ---------
 .../runners/direct/DirectGraphVisitorTest.java  | 239 ++++++++
 ...leManagerRemovingTransformEvaluatorTest.java |   4 +-
 .../runners/direct/EvaluationContextTest.java   |  49 +-
 .../direct/FlattenEvaluatorFactoryTest.java     |   6 +-
 .../ImmutabilityCheckingBundleFactoryTest.java  |   6 +-
 .../ImmutabilityEnforcementFactoryTest.java     |   6 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |   2 +-
 .../beam/runners/direct/SplittableDoFnTest.java | 231 --------
 .../StatefulParDoEvaluatorFactoryTest.java      | 300 ++++++++++
 .../runners/direct/StepTransformResultTest.java |  25 +-
 .../direct/TestStreamEvaluatorFactoryTest.java  |  10 +-
 .../runners/direct/TransformExecutorTest.java   | 223 +++----
 .../UnboundedReadEvaluatorFactoryTest.java      |  20 +-
 .../runners/direct/WatermarkManagerTest.java    |  23 +-
 .../direct/WindowEvaluatorFactoryTest.java      |  16 +-
 runners/flink/runner/pom.xml                    |   9 +-
 .../apache/beam/runners/flink/FlinkRunner.java  |  12 +-
 .../FlinkBatchPipelineTranslator.java           |  14 +-
 .../FlinkStreamingPipelineTranslator.java       |  16 +-
 .../FlinkStreamingTransformTranslators.java     |  63 +-
 .../PipelineTranslationOptimizer.java           |  10 +-
 .../FlinkMergingNonShuffleReduceFunction.java   |   2 +-
 .../FlinkMergingPartialReduceFunction.java      |   2 +-
 .../functions/FlinkMergingReduceFunction.java   |   2 +-
 .../functions/FlinkPartialReduceFunction.java   |   2 +-
 .../functions/FlinkProcessContextBase.java      |   8 -
 .../functions/FlinkReduceFunction.java          |   2 +-
 runners/gearpump/pom.xml                        |   9 +-
 .../gearpump/GearpumpPipelineTranslator.java    |  12 +-
 .../translators/TranslationContext.java         |   4 +-
 .../gearpump/translators/io/ValuesSource.java   |  71 ++-
 runners/google-cloud-dataflow-java/pom.xml      |   4 +
 .../dataflow/DataflowPipelineTranslator.java    |  33 +-
 .../beam/runners/dataflow/DataflowRunner.java   |  53 +-
 .../DataflowPipelineTranslatorTest.java         |   2 +-
 .../runners/dataflow/DataflowRunnerTest.java    |   4 +-
 .../dataflow/RecordingPipelineVisitor.java      |   6 +-
 runners/spark/pom.xml                           |   7 +-
 .../runners/spark/SparkPipelineOptions.java     |   5 +
 .../apache/beam/runners/spark/SparkRunner.java  |  80 ++-
 .../spark/aggregators/AccumulatorSingleton.java |   2 +-
 .../beam/runners/spark/io/MicrobatchSource.java |  20 +-
 .../beam/runners/spark/io/SourceDStream.java    |   3 +-
 .../spark/stateful/StateSpecFunctions.java      |   2 +-
 .../spark/translation/BoundedDataset.java       |   5 +-
 .../beam/runners/spark/translation/Dataset.java |   2 +-
 .../spark/translation/EvaluationContext.java    |  10 +-
 .../spark/translation/SparkProcessContext.java  |   9 -
 .../translation/StorageLevelPTransform.java     |  43 ++
 .../spark/translation/TransformTranslator.java  |  27 +
 .../SparkRunnerStreamingContextFactory.java     |   2 +-
 .../streaming/StreamingTransformTranslator.java |   2 +-
 .../translation/streaming/UnboundedDataset.java |  13 +-
 .../spark/translation/StorageLevelTest.java     |  56 ++
 .../streaming/EmptyStreamAssertionTest.java     |   2 +
 .../streaming/FlattenStreamingTest.java         |   2 +
 .../streaming/SimpleStreamingWordCountTest.java |   1 +
 .../SparkTestPipelineOptionsForStreaming.java   |   6 -
 sdks/java/core/pom.xml                          |  29 +-
 .../beam/sdk/AggregatorPipelineExtractor.java   |   6 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java | 151 ++---
 .../org/apache/beam/sdk/io/FileBasedSink.java   |  22 +-
 .../java/org/apache/beam/sdk/io/PubsubIO.java   | 176 +++++-
 .../apache/beam/sdk/io/PubsubUnboundedSink.java |  23 +-
 .../beam/sdk/io/PubsubUnboundedSource.java      |  40 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |  28 +-
 .../java/org/apache/beam/sdk/io/XmlSink.java    |   4 +-
 .../beam/sdk/options/PipelineOptions.java       |   7 +
 .../sdk/options/PipelineOptionsFactory.java     |   1 +
 .../sdk/options/ProxyInvocationHandler.java     |  26 +
 .../apache/beam/sdk/options/ValueProvider.java  |   2 +-
 .../beam/sdk/runners/TransformHierarchy.java    | 340 +++++++++--
 .../beam/sdk/runners/TransformTreeNode.java     | 257 --------
 .../beam/sdk/testing/FileChecksumMatcher.java   | 168 +++++-
 .../apache/beam/sdk/testing/GatherAllPanes.java |  88 +++
 .../org/apache/beam/sdk/testing/PAssert.java    |  77 +--
 .../apache/beam/sdk/testing/PaneExtractors.java |  55 +-
 .../beam/sdk/testing/UsesSplittableParDo.java   |  25 +
 .../beam/sdk/testing/ValueInSingleWindow.java   | 134 +++++
 .../org/apache/beam/sdk/transforms/Combine.java |  37 +-
 .../org/apache/beam/sdk/transforms/DoFn.java    |  16 +-
 .../beam/sdk/transforms/DoFnAdapters.java       |   2 +
 .../apache/beam/sdk/transforms/DoFnTester.java  | 350 ++++++-----
 .../org/apache/beam/sdk/transforms/OldDoFn.java |   8 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   |  10 -
 .../reflect/ByteBuddyDoFnInvokerFactory.java    |  28 +-
 .../reflect/ByteBuddyOnTimerInvokerFactory.java |  10 +-
 .../sdk/transforms/reflect/DoFnInvoker.java     |   3 +
 .../sdk/transforms/reflect/DoFnInvokers.java    |  16 +-
 .../sdk/transforms/reflect/DoFnSignature.java   |   8 +
 .../apache/beam/sdk/util/GatherAllPanes.java    |  86 ---
 .../apache/beam/sdk/util/IdentityWindowFn.java  |   2 +-
 .../beam/sdk/util/PerKeyCombineFnRunners.java   | 258 --------
 .../apache/beam/sdk/util/TimerInternals.java    | 107 ++--
 .../beam/sdk/util/WindowingInternals.java       |  10 -
 .../beam/sdk/util/common/ReflectHelpers.java    |   3 +-
 .../sdk/util/state/InMemoryTimerInternals.java  |   8 +-
 .../sdk/util/state/TimerInternalsFactory.java   |  36 ++
 .../sdk/AggregatorPipelineExtractorTest.java    |  20 +-
 .../org/apache/beam/sdk/WindowMatchers.java     | 204 -------
 .../org/apache/beam/sdk/WindowMatchersTest.java |  82 ---
 .../org/apache/beam/sdk/io/PubsubIOTest.java    |  43 +-
 .../beam/sdk/io/PubsubUnboundedSinkTest.java    |  20 +-
 .../beam/sdk/io/PubsubUnboundedSourceTest.java  |  14 +-
 .../org/apache/beam/sdk/io/XmlSinkTest.java     |   6 +-
 .../beam/sdk/options/PipelineOptionsTest.java   |  24 +
 .../beam/sdk/options/ValueProviderTest.java     |  18 +
 .../sdk/runners/TransformHierarchyTest.java     | 230 ++++++++
 .../beam/sdk/runners/TransformTreeTest.java     |  12 +-
 .../sdk/testing/FileChecksumMatcherTest.java    | 131 +++-
 .../beam/sdk/testing/GatherAllPanesTest.java    | 140 +++++
 .../beam/sdk/testing/PaneExtractorsTest.java    | 133 ++---
 .../testing/ValueInSingleWindowCoderTest.java   |  51 ++
 .../beam/sdk/transforms/DoFnTesterTest.java     |  38 +-
 .../apache/beam/sdk/transforms/ParDoTest.java   | 177 +++++-
 .../beam/sdk/transforms/SplittableDoFnTest.java | 401 +++++++++++++
 .../display/DisplayDataEvaluator.java           |   8 +-
 .../beam/sdk/util/GatherAllPanesTest.java       | 143 -----
 .../apache/beam/sdk/util/ReleaseInfoTest.java   |  45 ++
 .../beam/sdk/util/TimerInternalsTest.java       |   4 +-
 .../sorter/BufferedExternalSorter.java          |   6 +-
 .../sdk/extensions/sorter/ExternalSorter.java   |   6 +-
 .../sorter/BufferedExternalSorterTest.java      |  58 +-
 .../extensions/sorter/ExternalSorterTest.java   |  53 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIO.java     |  52 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java |  26 +-
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   |   6 +-
 .../examples-java8/generate-sources.sh          |  82 +++
 .../maven-archetypes/examples-java8/pom.xml     | 177 ++++++
 .../META-INF/maven/archetype-metadata.xml       |  39 ++
 .../main/resources/archetype-resources/pom.xml  | 209 +++++++
 .../projects/basic/archetype.properties         |  21 +
 .../src/test/resources/projects/basic/goal.txt  |   1 +
 .../examples/generate-sources.sh                |  68 +++
 sdks/java/maven-archetypes/examples/pom.xml     |  27 +-
 .../main/resources/archetype-resources/pom.xml  | 123 +++-
 .../src/main/java/DebuggingWordCount.java       | 164 ------
 .../src/main/java/MinimalWordCount.java         | 118 ----
 .../src/main/java/WindowedWordCount.java        | 229 -------
 .../src/main/java/WordCount.java                | 186 ------
 .../common/ExampleBigQueryTableOptions.java     |  55 --
 .../src/main/java/common/ExampleOptions.java    |  37 --
 ...xamplePubsubTopicAndSubscriptionOptions.java |  45 --
 .../java/common/ExamplePubsubTopicOptions.java  |  45 --
 .../src/main/java/common/ExampleUtils.java      | 352 -----------
 .../src/test/java/DebuggingWordCountTest.java   |  52 --
 .../src/test/java/WordCountTest.java            |  85 ---
 sdks/java/maven-archetypes/pom.xml              |  38 +-
 .../main/resources/archetype-resources/pom.xml  |   8 +
 .../resources/projects/basic/reference/pom.xml  |   8 +
 .../update-examples-archetype.sh                |  59 --
 233 files changed, 7991 insertions(+), 5310 deletions(-)
----------------------------------------------------------------------



[28/50] [abbrv] incubator-beam git commit: Rename ConsumerTrackingPipelineVisitor to DirectGraphVisitor

Posted by ke...@apache.org.
Rename ConsumerTrackingPipelineVisitor to DirectGraphVisitor

Reduce visibility of Visitor.


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/662416a4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/662416a4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/662416a4

Branch: refs/heads/gearpump-runner
Commit: 662416a4e176cca252c0d6fde1bf4252aeaa56c0
Parents: 8162cd2
Author: Thomas Groh <tg...@google.com>
Authored: Fri Dec 2 10:07:05 2016 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Fri Dec 2 14:02:25 2016 -0800

----------------------------------------------------------------------
 .../direct/ConsumerTrackingPipelineVisitor.java | 145 -----------
 .../beam/runners/direct/DirectGraphVisitor.java | 145 +++++++++++
 .../beam/runners/direct/DirectRunner.java       |   8 +-
 .../ConsumerTrackingPipelineVisitorTest.java    | 239 -------------------
 .../runners/direct/DirectGraphVisitorTest.java  | 239 +++++++++++++++++++
 .../runners/direct/EvaluationContextTest.java   |   6 +-
 .../ImmutabilityCheckingBundleFactoryTest.java  |   2 +-
 .../runners/direct/WatermarkManagerTest.java    |   8 +-
 8 files changed, 396 insertions(+), 396 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/662416a4/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
deleted file mode 100644
index b9e77c5..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
+++ /dev/null
@@ -1,145 +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.runners.direct;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.ListMultimap;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.Pipeline.PipelineVisitor;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.runners.TransformHierarchy;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-import org.apache.beam.sdk.values.PValue;
-
-/**
- * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the
- * {@link Pipeline}. This is used to schedule consuming {@link PTransform PTransforms} to consume
- * input after the upstream transform has produced and committed output.
- */
-public class ConsumerTrackingPipelineVisitor extends PipelineVisitor.Defaults {
-  private Map<POutput, AppliedPTransform<?, ?, ?>> producers = new HashMap<>();
-
-  private ListMultimap<PInput, AppliedPTransform<?, ?, ?>> primitiveConsumers =
-      ArrayListMultimap.create();
-
-  private Set<PCollectionView<?>> views = new HashSet<>();
-  private Set<AppliedPTransform<?, ?, ?>> rootTransforms = new HashSet<>();
-  private Map<AppliedPTransform<?, ?, ?>, String> stepNames = new HashMap<>();
-  private Set<PValue> toFinalize = new HashSet<>();
-  private int numTransforms = 0;
-  private boolean finalized = false;
-
-  @Override
-  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
-    checkState(
-        !finalized,
-        "Attempting to traverse a pipeline (node %s) with a %s "
-            + "which has already visited a Pipeline and is finalized",
-        node.getFullName(),
-        ConsumerTrackingPipelineVisitor.class.getSimpleName());
-    return CompositeBehavior.ENTER_TRANSFORM;
-  }
-
-  @Override
-  public void leaveCompositeTransform(TransformHierarchy.Node node) {
-    checkState(
-        !finalized,
-        "Attempting to traverse a pipeline (node %s) with a %s which is already finalized",
-        node.getFullName(),
-        ConsumerTrackingPipelineVisitor.class.getSimpleName());
-    if (node.isRootNode()) {
-      finalized = true;
-    }
-  }
-
-  @Override
-  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
-    toFinalize.removeAll(node.getInput().expand());
-    AppliedPTransform<?, ?, ?> appliedTransform = getAppliedTransform(node);
-    stepNames.put(appliedTransform, genStepName());
-    if (node.getInput().expand().isEmpty()) {
-      rootTransforms.add(appliedTransform);
-    } else {
-      for (PValue value : node.getInput().expand()) {
-        primitiveConsumers.put(value, appliedTransform);
-      }
-    }
-  }
-
- @Override
-  public void visitValue(PValue value, TransformHierarchy.Node producer) {
-    toFinalize.add(value);
-
-    AppliedPTransform<?, ?, ?> appliedTransform = getAppliedTransform(producer);
-    if (!producers.containsKey(value)) {
-      producers.put(value, appliedTransform);
-    }
-    for (PValue expandedValue : value.expand()) {
-      if (expandedValue instanceof PCollectionView) {
-        views.add((PCollectionView<?>) expandedValue);
-      }
-      if (!producers.containsKey(expandedValue)) {
-        producers.put(value, appliedTransform);
-      }
-    }
-  }
-
-  private AppliedPTransform<?, ?, ?> getAppliedTransform(TransformHierarchy.Node node) {
-    @SuppressWarnings({"rawtypes", "unchecked"})
-    AppliedPTransform<?, ?, ?> application = AppliedPTransform.of(
-        node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform());
-    return application;
-  }
-
-  private String genStepName() {
-    return String.format("s%s", numTransforms++);
-  }
-
-  /**
-   * Returns all of the {@link PValue PValues} that have been produced but not consumed. These
-   * {@link PValue PValues} should be finalized by the {@link PipelineRunner} before the
-   * {@link Pipeline} is executed.
-   */
-  public void finishSpecifyingRemainder() {
-    checkState(
-        finalized,
-        "Can't call finishSpecifyingRemainder before the Pipeline has been completely traversed");
-    for (PValue unfinalized : toFinalize) {
-      unfinalized.finishSpecifying();
-    }
-  }
-
-  /**
-   * Get the graph constructed by this {@link ConsumerTrackingPipelineVisitor}, which provides
-   * lookups for producers and consumers of {@link PValue PValues}.
-   */
-  public DirectGraph getGraph() {
-    checkState(finalized, "Can't get a graph before the Pipeline has been completely traversed");
-    return DirectGraph.create(producers, primitiveConsumers, views, rootTransforms, stepNames);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/662416a4/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java
new file mode 100644
index 0000000..cd9d120
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java
@@ -0,0 +1,145 @@
+/*
+ * 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.runners.direct;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ListMultimap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
+
+/**
+ * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the
+ * {@link Pipeline}. This is used to schedule consuming {@link PTransform PTransforms} to consume
+ * input after the upstream transform has produced and committed output.
+ */
+class DirectGraphVisitor extends PipelineVisitor.Defaults {
+  private Map<POutput, AppliedPTransform<?, ?, ?>> producers = new HashMap<>();
+
+  private ListMultimap<PInput, AppliedPTransform<?, ?, ?>> primitiveConsumers =
+      ArrayListMultimap.create();
+
+  private Set<PCollectionView<?>> views = new HashSet<>();
+  private Set<AppliedPTransform<?, ?, ?>> rootTransforms = new HashSet<>();
+  private Map<AppliedPTransform<?, ?, ?>, String> stepNames = new HashMap<>();
+  private Set<PValue> toFinalize = new HashSet<>();
+  private int numTransforms = 0;
+  private boolean finalized = false;
+
+  @Override
+  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
+    checkState(
+        !finalized,
+        "Attempting to traverse a pipeline (node %s) with a %s "
+            + "which has already visited a Pipeline and is finalized",
+        node.getFullName(),
+        getClass().getSimpleName());
+    return CompositeBehavior.ENTER_TRANSFORM;
+  }
+
+  @Override
+  public void leaveCompositeTransform(TransformHierarchy.Node node) {
+    checkState(
+        !finalized,
+        "Attempting to traverse a pipeline (node %s) with a %s which is already finalized",
+        node.getFullName(),
+        getClass().getSimpleName());
+    if (node.isRootNode()) {
+      finalized = true;
+    }
+  }
+
+  @Override
+  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
+    toFinalize.removeAll(node.getInput().expand());
+    AppliedPTransform<?, ?, ?> appliedTransform = getAppliedTransform(node);
+    stepNames.put(appliedTransform, genStepName());
+    if (node.getInput().expand().isEmpty()) {
+      rootTransforms.add(appliedTransform);
+    } else {
+      for (PValue value : node.getInput().expand()) {
+        primitiveConsumers.put(value, appliedTransform);
+      }
+    }
+  }
+
+ @Override
+  public void visitValue(PValue value, TransformHierarchy.Node producer) {
+    toFinalize.add(value);
+
+    AppliedPTransform<?, ?, ?> appliedTransform = getAppliedTransform(producer);
+    if (!producers.containsKey(value)) {
+      producers.put(value, appliedTransform);
+    }
+    for (PValue expandedValue : value.expand()) {
+      if (expandedValue instanceof PCollectionView) {
+        views.add((PCollectionView<?>) expandedValue);
+      }
+      if (!producers.containsKey(expandedValue)) {
+        producers.put(value, appliedTransform);
+      }
+    }
+  }
+
+  private AppliedPTransform<?, ?, ?> getAppliedTransform(TransformHierarchy.Node node) {
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    AppliedPTransform<?, ?, ?> application = AppliedPTransform.of(
+        node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform());
+    return application;
+  }
+
+  private String genStepName() {
+    return String.format("s%s", numTransforms++);
+  }
+
+  /**
+   * Returns all of the {@link PValue PValues} that have been produced but not consumed. These
+   * {@link PValue PValues} should be finalized by the {@link PipelineRunner} before the
+   * {@link Pipeline} is executed.
+   */
+  public void finishSpecifyingRemainder() {
+    checkState(
+        finalized,
+        "Can't call finishSpecifyingRemainder before the Pipeline has been completely traversed");
+    for (PValue unfinalized : toFinalize) {
+      unfinalized.finishSpecifying();
+    }
+  }
+
+  /**
+   * Get the graph constructed by this {@link DirectGraphVisitor}, which provides
+   * lookups for producers and consumers of {@link PValue PValues}.
+   */
+  public DirectGraph getGraph() {
+    checkState(finalized, "Can't get a graph before the Pipeline has been completely traversed");
+    return DirectGraph.create(producers, primitiveConsumers, views, rootTransforms, stepNames);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/662416a4/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index 0ad5836..2f84356 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -298,9 +298,9 @@ public class DirectRunner extends PipelineRunner<DirectPipelineResult> {
   @Override
   public DirectPipelineResult run(Pipeline pipeline) {
     MetricsEnvironment.setMetricsSupported(true);
-    ConsumerTrackingPipelineVisitor consumerTrackingVisitor = new ConsumerTrackingPipelineVisitor();
-    pipeline.traverseTopologically(consumerTrackingVisitor);
-    consumerTrackingVisitor.finishSpecifyingRemainder();
+    DirectGraphVisitor graphVisitor = new DirectGraphVisitor();
+    pipeline.traverseTopologically(graphVisitor);
+    graphVisitor.finishSpecifyingRemainder();
 
     @SuppressWarnings("rawtypes")
     KeyedPValueTrackingVisitor keyedPValueVisitor =
@@ -313,7 +313,7 @@ public class DirectRunner extends PipelineRunner<DirectPipelineResult> {
 
     DisplayDataValidator.validatePipeline(pipeline);
 
-    DirectGraph graph = consumerTrackingVisitor.getGraph();
+    DirectGraph graph = graphVisitor.getGraph();
     EvaluationContext context =
         EvaluationContext.create(
             getPipelineOptions(),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/662416a4/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
deleted file mode 100644
index 02fe007..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
+++ /dev/null
@@ -1,239 +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.runners.direct;
-
-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 java.io.Serializable;
-import java.util.List;
-import org.apache.beam.sdk.io.CountingInput;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
-import org.hamcrest.Matchers;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link ConsumerTrackingPipelineVisitor}.
- */
-@RunWith(JUnit4.class)
-public class ConsumerTrackingPipelineVisitorTest implements Serializable {
-  @Rule public transient ExpectedException thrown = ExpectedException.none();
-
-  private transient TestPipeline p = TestPipeline.create();
-  private transient ConsumerTrackingPipelineVisitor visitor = new ConsumerTrackingPipelineVisitor();
-
-  @Test
-  public void getViewsReturnsViews() {
-    PCollectionView<List<String>> listView =
-        p.apply("listCreate", Create.of("foo", "bar"))
-            .apply(
-                ParDo.of(
-                    new DoFn<String, String>() {
-                      @ProcessElement
-                      public void processElement(DoFn<String, String>.ProcessContext c)
-                          throws Exception {
-                        c.output(Integer.toString(c.element().length()));
-                      }
-                    }))
-            .apply(View.<String>asList());
-    PCollectionView<Object> singletonView =
-        p.apply("singletonCreate", Create.<Object>of(1, 2, 3)).apply(View.<Object>asSingleton());
-    p.traverseTopologically(visitor);
-    assertThat(
-        visitor.getGraph().getViews(),
-        Matchers.<PCollectionView<?>>containsInAnyOrder(listView, singletonView));
-  }
-
-  @Test
-  public void getRootTransformsContainsPBegins() {
-    PCollection<String> created = p.apply(Create.of("foo", "bar"));
-    PCollection<Long> counted = p.apply(CountingInput.upTo(1234L));
-    PCollection<Long> unCounted = p.apply(CountingInput.unbounded());
-    p.traverseTopologically(visitor);
-    assertThat(
-        visitor.getGraph().getRootTransforms(),
-        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
-            created.getProducingTransformInternal(),
-            counted.getProducingTransformInternal(),
-            unCounted.getProducingTransformInternal()));
-  }
-
-  @Test
-  public void getRootTransformsContainsEmptyFlatten() {
-    PCollection<String> empty =
-        PCollectionList.<String>empty(p).apply(Flatten.<String>pCollections());
-    p.traverseTopologically(visitor);
-    assertThat(
-        visitor.getGraph().getRootTransforms(),
-        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
-            empty.getProducingTransformInternal()));
-  }
-
-  @Test
-  public void getValueToConsumersSucceeds() {
-    PCollection<String> created = p.apply(Create.of("1", "2", "3"));
-    PCollection<String> transformed =
-        created.apply(
-            ParDo.of(
-                new DoFn<String, String>() {
-                  @ProcessElement
-                  public void processElement(DoFn<String, String>.ProcessContext c)
-                      throws Exception {
-                    c.output(Integer.toString(c.element().length()));
-                  }
-                }));
-
-    PCollection<String> flattened =
-        PCollectionList.of(created).and(transformed).apply(Flatten.<String>pCollections());
-
-    p.traverseTopologically(visitor);
-
-    assertThat(
-        visitor.getGraph().getPrimitiveConsumers(created),
-        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
-            transformed.getProducingTransformInternal(),
-            flattened.getProducingTransformInternal()));
-    assertThat(
-        visitor.getGraph().getPrimitiveConsumers(transformed),
-        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
-            flattened.getProducingTransformInternal()));
-    assertThat(visitor.getGraph().getPrimitiveConsumers(flattened), emptyIterable());
-  }
-
-  @Test
-  public void getValueToConsumersWithDuplicateInputSucceeds() {
-    PCollection<String> created = p.apply(Create.of("1", "2", "3"));
-
-    PCollection<String> flattened =
-        PCollectionList.of(created).and(created).apply(Flatten.<String>pCollections());
-
-    p.traverseTopologically(visitor);
-
-    assertThat(
-        visitor.getGraph().getPrimitiveConsumers(created),
-        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
-            flattened.getProducingTransformInternal(),
-            flattened.getProducingTransformInternal()));
-    assertThat(visitor.getGraph().getPrimitiveConsumers(flattened), emptyIterable());
-  }
-
-  @Test
-  public void getUnfinalizedPValuesContainsDanglingOutputs() {
-    PCollection<String> created = p.apply(Create.of("1", "2", "3"));
-    PCollection<String> transformed =
-        created.apply(
-            ParDo.of(
-                new DoFn<String, String>() {
-                  @ProcessElement
-                  public void processElement(DoFn<String, String>.ProcessContext c)
-                      throws Exception {
-                    c.output(Integer.toString(c.element().length()));
-                  }
-                }));
-
-    assertThat(transformed.isFinishedSpecifyingInternal(), is(false));
-
-    p.traverseTopologically(visitor);
-    visitor.finishSpecifyingRemainder();
-    assertThat(transformed.isFinishedSpecifyingInternal(), is(true));
-  }
-
-  @Test
-  public void getStepNamesContainsAllTransforms() {
-    PCollection<String> created = p.apply(Create.of("1", "2", "3"));
-    PCollection<String> transformed =
-        created.apply(
-            ParDo.of(
-                new DoFn<String, String>() {
-                  @ProcessElement
-                  public void processElement(DoFn<String, String>.ProcessContext c)
-                      throws Exception {
-                    c.output(Integer.toString(c.element().length()));
-                  }
-                }));
-    PDone finished =
-        transformed.apply(
-            new PTransform<PInput, PDone>() {
-              @Override
-              public PDone apply(PInput input) {
-                return PDone.in(input.getPipeline());
-              }
-            });
-
-    p.traverseTopologically(visitor);
-    DirectGraph graph = visitor.getGraph();
-    assertThat(graph.getStepName(graph.getProducer(created)), equalTo("s0"));
-    assertThat(graph.getStepName(graph.getProducer(transformed)), equalTo("s1"));
-    // finished doesn't have a producer, because it's not a PValue.
-    // TODO: Demonstrate that PCollectionList/Tuple and other composite PValues are either safe to
-    // use, or make them so.
-  }
-
-  @Test
-  public void traverseMultipleTimesThrows() {
-    p.apply(Create.of(1, 2, 3));
-
-    p.traverseTopologically(visitor);
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage(ConsumerTrackingPipelineVisitor.class.getSimpleName());
-    thrown.expectMessage("is finalized");
-    p.traverseTopologically(visitor);
-  }
-
-  @Test
-  public void traverseIndependentPathsSucceeds() {
-    p.apply("left", Create.of(1, 2, 3));
-    p.apply("right", Create.of("foo", "bar", "baz"));
-
-    p.traverseTopologically(visitor);
-  }
-
-  @Test
-  public void getGraphWithoutVisitingThrows() {
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("completely traversed");
-    thrown.expectMessage("get a graph");
-    visitor.getGraph();
-  }
-
-  @Test
-  public void finishSpecifyingRemainderWithoutVisitingThrows() {
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("completely traversed");
-    thrown.expectMessage("finishSpecifyingRemainder");
-    visitor.finishSpecifyingRemainder();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/662416a4/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java
new file mode 100644
index 0000000..d218a81
--- /dev/null
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java
@@ -0,0 +1,239 @@
+/*
+ * 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.runners.direct;
+
+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 java.io.Serializable;
+import java.util.List;
+import org.apache.beam.sdk.io.CountingInput;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.hamcrest.Matchers;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link DirectGraphVisitor}.
+ */
+@RunWith(JUnit4.class)
+public class DirectGraphVisitorTest implements Serializable {
+  @Rule public transient ExpectedException thrown = ExpectedException.none();
+
+  private transient TestPipeline p = TestPipeline.create();
+  private transient DirectGraphVisitor visitor = new DirectGraphVisitor();
+
+  @Test
+  public void getViewsReturnsViews() {
+    PCollectionView<List<String>> listView =
+        p.apply("listCreate", Create.of("foo", "bar"))
+            .apply(
+                ParDo.of(
+                    new DoFn<String, String>() {
+                      @ProcessElement
+                      public void processElement(DoFn<String, String>.ProcessContext c)
+                          throws Exception {
+                        c.output(Integer.toString(c.element().length()));
+                      }
+                    }))
+            .apply(View.<String>asList());
+    PCollectionView<Object> singletonView =
+        p.apply("singletonCreate", Create.<Object>of(1, 2, 3)).apply(View.<Object>asSingleton());
+    p.traverseTopologically(visitor);
+    assertThat(
+        visitor.getGraph().getViews(),
+        Matchers.<PCollectionView<?>>containsInAnyOrder(listView, singletonView));
+  }
+
+  @Test
+  public void getRootTransformsContainsPBegins() {
+    PCollection<String> created = p.apply(Create.of("foo", "bar"));
+    PCollection<Long> counted = p.apply(CountingInput.upTo(1234L));
+    PCollection<Long> unCounted = p.apply(CountingInput.unbounded());
+    p.traverseTopologically(visitor);
+    assertThat(
+        visitor.getGraph().getRootTransforms(),
+        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
+            created.getProducingTransformInternal(),
+            counted.getProducingTransformInternal(),
+            unCounted.getProducingTransformInternal()));
+  }
+
+  @Test
+  public void getRootTransformsContainsEmptyFlatten() {
+    PCollection<String> empty =
+        PCollectionList.<String>empty(p).apply(Flatten.<String>pCollections());
+    p.traverseTopologically(visitor);
+    assertThat(
+        visitor.getGraph().getRootTransforms(),
+        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
+            empty.getProducingTransformInternal()));
+  }
+
+  @Test
+  public void getValueToConsumersSucceeds() {
+    PCollection<String> created = p.apply(Create.of("1", "2", "3"));
+    PCollection<String> transformed =
+        created.apply(
+            ParDo.of(
+                new DoFn<String, String>() {
+                  @ProcessElement
+                  public void processElement(DoFn<String, String>.ProcessContext c)
+                      throws Exception {
+                    c.output(Integer.toString(c.element().length()));
+                  }
+                }));
+
+    PCollection<String> flattened =
+        PCollectionList.of(created).and(transformed).apply(Flatten.<String>pCollections());
+
+    p.traverseTopologically(visitor);
+
+    assertThat(
+        visitor.getGraph().getPrimitiveConsumers(created),
+        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
+            transformed.getProducingTransformInternal(),
+            flattened.getProducingTransformInternal()));
+    assertThat(
+        visitor.getGraph().getPrimitiveConsumers(transformed),
+        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
+            flattened.getProducingTransformInternal()));
+    assertThat(visitor.getGraph().getPrimitiveConsumers(flattened), emptyIterable());
+  }
+
+  @Test
+  public void getValueToConsumersWithDuplicateInputSucceeds() {
+    PCollection<String> created = p.apply(Create.of("1", "2", "3"));
+
+    PCollection<String> flattened =
+        PCollectionList.of(created).and(created).apply(Flatten.<String>pCollections());
+
+    p.traverseTopologically(visitor);
+
+    assertThat(
+        visitor.getGraph().getPrimitiveConsumers(created),
+        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
+            flattened.getProducingTransformInternal(),
+            flattened.getProducingTransformInternal()));
+    assertThat(visitor.getGraph().getPrimitiveConsumers(flattened), emptyIterable());
+  }
+
+  @Test
+  public void getUnfinalizedPValuesContainsDanglingOutputs() {
+    PCollection<String> created = p.apply(Create.of("1", "2", "3"));
+    PCollection<String> transformed =
+        created.apply(
+            ParDo.of(
+                new DoFn<String, String>() {
+                  @ProcessElement
+                  public void processElement(DoFn<String, String>.ProcessContext c)
+                      throws Exception {
+                    c.output(Integer.toString(c.element().length()));
+                  }
+                }));
+
+    assertThat(transformed.isFinishedSpecifyingInternal(), is(false));
+
+    p.traverseTopologically(visitor);
+    visitor.finishSpecifyingRemainder();
+    assertThat(transformed.isFinishedSpecifyingInternal(), is(true));
+  }
+
+  @Test
+  public void getStepNamesContainsAllTransforms() {
+    PCollection<String> created = p.apply(Create.of("1", "2", "3"));
+    PCollection<String> transformed =
+        created.apply(
+            ParDo.of(
+                new DoFn<String, String>() {
+                  @ProcessElement
+                  public void processElement(DoFn<String, String>.ProcessContext c)
+                      throws Exception {
+                    c.output(Integer.toString(c.element().length()));
+                  }
+                }));
+    PDone finished =
+        transformed.apply(
+            new PTransform<PInput, PDone>() {
+              @Override
+              public PDone apply(PInput input) {
+                return PDone.in(input.getPipeline());
+              }
+            });
+
+    p.traverseTopologically(visitor);
+    DirectGraph graph = visitor.getGraph();
+    assertThat(graph.getStepName(graph.getProducer(created)), equalTo("s0"));
+    assertThat(graph.getStepName(graph.getProducer(transformed)), equalTo("s1"));
+    // finished doesn't have a producer, because it's not a PValue.
+    // TODO: Demonstrate that PCollectionList/Tuple and other composite PValues are either safe to
+    // use, or make them so.
+  }
+
+  @Test
+  public void traverseMultipleTimesThrows() {
+    p.apply(Create.of(1, 2, 3));
+
+    p.traverseTopologically(visitor);
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage(DirectGraphVisitor.class.getSimpleName());
+    thrown.expectMessage("is finalized");
+    p.traverseTopologically(visitor);
+  }
+
+  @Test
+  public void traverseIndependentPathsSucceeds() {
+    p.apply("left", Create.of(1, 2, 3));
+    p.apply("right", Create.of("foo", "bar", "baz"));
+
+    p.traverseTopologically(visitor);
+  }
+
+  @Test
+  public void getGraphWithoutVisitingThrows() {
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("completely traversed");
+    thrown.expectMessage("get a graph");
+    visitor.getGraph();
+  }
+
+  @Test
+  public void finishSpecifyingRemainderWithoutVisitingThrows() {
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("completely traversed");
+    thrown.expectMessage("finishSpecifyingRemainder");
+    visitor.finishSpecifyingRemainder();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/662416a4/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
index 1c2bf14..17cdea1 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
@@ -101,11 +101,11 @@ public class EvaluationContextTest {
     view = created.apply(View.<Integer>asIterable());
     unbounded = p.apply(CountingInput.unbounded());
 
-    ConsumerTrackingPipelineVisitor cVis = new ConsumerTrackingPipelineVisitor();
-    p.traverseTopologically(cVis);
+    DirectGraphVisitor graphVisitor = new DirectGraphVisitor();
+    p.traverseTopologically(graphVisitor);
 
     bundleFactory = ImmutableListBundleFactory.create();
-    graph = cVis.getGraph();
+    graph = graphVisitor.getGraph();
     context =
         EvaluationContext.create(
             runner.getPipelineOptions(), NanosOffsetClock.create(), bundleFactory, graph);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/662416a4/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
index e7e1e62..6ab8aea 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
@@ -56,7 +56,7 @@ public class ImmutabilityCheckingBundleFactoryTest {
     TestPipeline p = TestPipeline.create();
     created = p.apply(Create.<byte[]>of().withCoder(ByteArrayCoder.of()));
     transformed = created.apply(ParDo.of(new IdentityDoFn<byte[]>()));
-    ConsumerTrackingPipelineVisitor visitor = new ConsumerTrackingPipelineVisitor();
+    DirectGraphVisitor visitor = new DirectGraphVisitor();
     p.traverseTopologically(visitor);
     factory =
         ImmutabilityCheckingBundleFactory.create(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/662416a4/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
index 5cde4d6..076e0fb 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
@@ -140,7 +140,7 @@ public class WatermarkManagerTest implements Serializable {
     consumers.put(flattened, Collections.<AppliedPTransform<?, ?, ?>>emptyList());
 
     clock = MockClock.fromInstant(new Instant(1000));
-    ConsumerTrackingPipelineVisitor visitor = new ConsumerTrackingPipelineVisitor();
+    DirectGraphVisitor visitor = new DirectGraphVisitor();
     p.traverseTopologically(visitor);
     graph = visitor.getGraph();
 
@@ -309,9 +309,9 @@ public class WatermarkManagerTest implements Serializable {
     PCollection<Integer> created = p.apply(Create.of(1, 2, 3));
     PCollection<Integer> multiConsumer =
         PCollectionList.of(created).and(created).apply(Flatten.<Integer>pCollections());
-    ConsumerTrackingPipelineVisitor trackingVisitor = new ConsumerTrackingPipelineVisitor();
-    p.traverseTopologically(trackingVisitor);
-    DirectGraph graph = trackingVisitor.getGraph();
+    DirectGraphVisitor graphVisitor = new DirectGraphVisitor();
+    p.traverseTopologically(graphVisitor);
+    DirectGraph graph = graphVisitor.getGraph();
 
     AppliedPTransform<?, ?, ?> theFlatten = graph.getProducer(multiConsumer);
 



[06/50] [abbrv] incubator-beam git commit: Improve Splittable DoFn

Posted by ke...@apache.org.
Improve Splittable DoFn

Makes Splittable DoFn be more like a real DoFn:
- Adds support for side inputs and outputs to SDF
- Teaches `ProcessFn` to work with exploded windows inside the
  `KeyedWorkItem`. It works with them by un-exploding the windows
  in the `Iterable<WindowedValue<ElementAndRestriction>>` into a
  single `WindowedValue`, since the values and timestamps are
  guaranteed to be the same.

Makes SplittableParDo.ProcessFn not use the (now unavailable)
OldDoFn state and timers API:
- Makes `ProcessFn` be a primitive transform with its own
  `ParDoEvaluator`. As a nice side effect, this enables the runner to
  provide additional hooks into it - e.g. for giving the runner access
  to the restriction tracker (in later PRs)
- For consistency, moves declaration of `GBKIntoKeyedWorkItems`
  primitive transform into `SplittableParDo`, alongside the
  `SplittableProcessElements` transform
- Preserves compressed representation of `WindowedValue`'s in
  `PushbackSideInputDoFnRunner`
- Uses OutputWindowedValue in SplittableParDo.ProcessFn

Proper lifecycle management for wrapped fn.

- Caches underlying fn using DoFnLifecycleManager, so its
  @Setup and @Teardown methods are called.
- Calls @StartBundle and @FinishBundle methods on the underlying
  fn explicitly. Output from them is prohibited, since an SDF
  is only allowed to output after a successful RestrictionTracker.tryClaim.
  It's possible that an SDF should not be allowed to have
  StartBundle/FinishBundle methods at all, but I'm not sure.


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/87ff5ac3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/87ff5ac3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/87ff5ac3

Branch: refs/heads/gearpump-runner
Commit: 87ff5ac36bb9cc62fa4864ffa7b5a5e495b9a4a1
Parents: fd4b631
Author: Eugene Kirpichov <ki...@google.com>
Authored: Wed Oct 26 16:05:01 2016 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Thu Dec 1 14:15:55 2016 -0800

----------------------------------------------------------------------
 .../core/ElementAndRestrictionCoder.java        |   8 +
 .../runners/core/GBKIntoKeyedWorkItems.java     |  55 ---
 .../beam/runners/core/SplittableParDo.java      | 378 +++++++++++++++----
 .../beam/runners/core/SplittableParDoTest.java  | 134 +++++--
 ...ectGBKIntoKeyedWorkItemsOverrideFactory.java |  41 +-
 .../beam/runners/direct/DirectGroupByKey.java   |   2 +-
 .../beam/runners/direct/DirectRunner.java       |   8 +-
 .../runners/direct/DoFnLifecycleManager.java    |   4 +-
 .../beam/runners/direct/ParDoEvaluator.java     |  26 +-
 .../runners/direct/ParDoEvaluatorFactory.java   |  63 +++-
 .../direct/ParDoMultiOverrideFactory.java       |   2 +-
 ...littableProcessElementsEvaluatorFactory.java | 144 +++++++
 .../direct/TransformEvaluatorRegistry.java      |   5 +
 .../beam/runners/direct/SplittableDoFnTest.java | 194 +++++++++-
 .../org/apache/beam/sdk/transforms/DoFn.java    |  12 +
 .../apache/beam/sdk/transforms/DoFnTester.java  |  51 ++-
 .../sdk/util/state/TimerInternalsFactory.java   |  36 ++
 17 files changed, 905 insertions(+), 258 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java
index 6dec8e2..64c1e14 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java
@@ -64,4 +64,12 @@ public class ElementAndRestrictionCoder<ElementT, RestrictionT>
     RestrictionT value = restrictionCoder.decode(inStream, context);
     return ElementAndRestriction.of(key, value);
   }
+
+  public Coder<ElementT> getElementCoder() {
+    return elementCoder;
+  }
+
+  public Coder<RestrictionT> getRestrictionCoder() {
+    return restrictionCoder;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/runners/core-java/src/main/java/org/apache/beam/runners/core/GBKIntoKeyedWorkItems.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GBKIntoKeyedWorkItems.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GBKIntoKeyedWorkItems.java
deleted file mode 100644
index 304e349..0000000
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GBKIntoKeyedWorkItems.java
+++ /dev/null
@@ -1,55 +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.runners.core;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.sdk.annotations.Experimental;
-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.util.KeyedWorkItem;
-import org.apache.beam.sdk.util.KeyedWorkItemCoder;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Interface for creating a runner-specific {@link GroupByKey GroupByKey-like} {@link PTransform}
- * that produces {@link KeyedWorkItem KeyedWorkItems} so that downstream transforms can access state
- * and timers.
- */
-@Experimental(Experimental.Kind.SPLITTABLE_DO_FN)
-public class GBKIntoKeyedWorkItems<KeyT, InputT>
-    extends PTransform<PCollection<KV<KeyT, InputT>>, PCollection<KeyedWorkItem<KeyT, InputT>>> {
-  @Override
-  public PCollection<KeyedWorkItem<KeyT, InputT>> apply(PCollection<KV<KeyT, InputT>> input) {
-    checkArgument(input.getCoder() instanceof KvCoder,
-        "Expected input coder to be KvCoder, but was %s",
-        input.getCoder().getClass().getSimpleName());
-
-    KvCoder<KeyT, InputT> kvCoder = (KvCoder<KeyT, InputT>) input.getCoder();
-    Coder<KeyedWorkItem<KeyT, InputT>> coder = KeyedWorkItemCoder.of(
-        kvCoder.getKeyCoder(), kvCoder.getValueCoder(),
-        input.getWindowingStrategy().getWindowFn().windowCoder());
-    PCollection<KeyedWorkItem<KeyT, InputT>> collection = PCollection.createPrimitiveOutputInternal(
-        input.getPipeline(), input.getWindowingStrategy(), input.isBounded());
-    collection.setCoder((Coder) coder);
-    return collection;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
index c38ab2f..80fd17b 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java
@@ -19,17 +19,22 @@ package org.apache.beam.runners.core;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
+import java.util.List;
 import java.util.UUID;
 import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.OldDoFn;
+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.SerializableFunction;
@@ -45,21 +50,30 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.KeyedWorkItemCoder;
 import org.apache.beam.sdk.util.TimeDomain;
 import org.apache.beam.sdk.util.Timer;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.util.state.State;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateInternalsFactory;
 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.StateTags;
+import org.apache.beam.sdk.util.state.TimerInternalsFactory;
 import org.apache.beam.sdk.util.state.ValueState;
 import org.apache.beam.sdk.util.state.WatermarkHoldState;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.apache.beam.sdk.values.TypedPValue;
 import org.joda.time.Instant;
 
 /**
@@ -80,31 +94,53 @@ import org.joda.time.Instant;
  * ParDo.of(splittable DoFn)}, but not for direct use by pipeline writers.
  */
 @Experimental(Experimental.Kind.SPLITTABLE_DO_FN)
-public class SplittableParDo<
-        InputT, OutputT, RestrictionT, TrackerT extends RestrictionTracker<RestrictionT>>
-    extends PTransform<PCollection<InputT>, PCollection<OutputT>> {
-  private final DoFn<InputT, OutputT> fn;
-  private final DoFnSignature signature;
+public class SplittableParDo<InputT, OutputT, RestrictionT>
+    extends PTransform<PCollection<InputT>, PCollectionTuple> {
+  private final ParDo.BoundMulti<InputT, OutputT> parDo;
 
   /**
-   * Creates the transform for the given original {@link ParDo} and {@link DoFn}.
+   * Creates the transform for the given original multi-output {@link ParDo}.
    *
-   * @param fn The splittable {@link DoFn} inside the original {@link ParDo} transform.
+   * @param parDo The splittable {@link ParDo} transform.
    */
-  public SplittableParDo(DoFn<InputT, OutputT> fn) {
-    checkNotNull(fn, "fn must not be null");
-    this.fn = fn;
-    this.signature = DoFnSignatures.getSignature(fn.getClass());
-    checkArgument(signature.processElement().isSplittable(), "fn must be a splittable DoFn");
+  public SplittableParDo(ParDo.BoundMulti<InputT, OutputT> parDo) {
+    checkNotNull(parDo, "parDo must not be null");
+    this.parDo = parDo;
+    checkArgument(
+        DoFnSignatures.getSignature(parDo.getNewFn().getClass()).processElement().isSplittable(),
+        "fn must be a splittable DoFn");
   }
 
   @Override
-  public PCollection<OutputT> apply(PCollection<InputT> input) {
-    PCollection.IsBounded isFnBounded = signature.isBoundedPerElement();
+  public PCollectionTuple apply(PCollection<InputT> input) {
+    return applyTyped(input);
+  }
+
+  private PCollectionTuple applyTyped(PCollection<InputT> input) {
+    DoFn<InputT, OutputT> fn = parDo.getNewFn();
     Coder<RestrictionT> restrictionCoder =
-        DoFnInvokers
-            .invokerFor(fn)
+        DoFnInvokers.invokerFor(fn)
             .invokeGetRestrictionCoder(input.getPipeline().getCoderRegistry());
+    PCollection<KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>> keyedWorkItems =
+        applySplitIntoKeyedWorkItems(input, fn, restrictionCoder);
+    return keyedWorkItems.apply(
+        "Process",
+        new ProcessElements<>(
+            fn,
+            input.getCoder(),
+            restrictionCoder,
+            input.getWindowingStrategy(),
+            parDo.getSideInputs(),
+            parDo.getMainOutputTag(),
+            parDo.getSideOutputTags()));
+  }
+
+  private static <InputT, OutputT, RestrictionT>
+      PCollection<KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>>
+          applySplitIntoKeyedWorkItems(
+              PCollection<InputT> input,
+              DoFn<InputT, OutputT> fn,
+              Coder<RestrictionT> restrictionCoder) {
     Coder<ElementAndRestriction<InputT, RestrictionT>> splitCoder =
         ElementAndRestrictionCoder.of(input.getCoder(), restrictionCoder);
 
@@ -121,23 +157,133 @@ public class SplittableParDo<
                 WithKeys.of(new RandomUniqueKeyFn<ElementAndRestriction<InputT, RestrictionT>>()))
             .apply(
                 "Group by key",
-                new GBKIntoKeyedWorkItems<String, ElementAndRestriction<InputT, RestrictionT>>());
+                new GBKIntoKeyedWorkItems<String, ElementAndRestriction<InputT, RestrictionT>>())
+            .setCoder(
+                KeyedWorkItemCoder.of(
+                    StringUtf8Coder.of(),
+                    splitCoder,
+                    input.getWindowingStrategy().getWindowFn().windowCoder()));
     checkArgument(
         keyedWorkItems.getWindowingStrategy().getWindowFn() instanceof GlobalWindows,
         "GBKIntoKeyedWorkItems must produce a globally windowed collection, "
             + "but windowing strategy was: %s",
         keyedWorkItems.getWindowingStrategy());
-    return keyedWorkItems
-        .apply(
-            "Process",
-            ParDo.of(
-                new ProcessFn<InputT, OutputT, RestrictionT, TrackerT>(
-                    fn,
-                    input.getCoder(),
-                    restrictionCoder,
-                    input.getWindowingStrategy().getWindowFn().windowCoder())))
-        .setIsBoundedInternal(input.isBounded().and(isFnBounded))
-        .setWindowingStrategyInternal(input.getWindowingStrategy());
+    return keyedWorkItems;
+  }
+
+  /**
+   * Runner-specific primitive {@link GroupByKey GroupByKey-like} {@link PTransform} that produces
+   * {@link KeyedWorkItem KeyedWorkItems} so that downstream transforms can access state and timers.
+   *
+   * <p>Unlike a real {@link GroupByKey}, ignores the input's windowing and triggering strategy and
+   * emits output immediately.
+   */
+  public static class GBKIntoKeyedWorkItems<KeyT, InputT>
+      extends PTransform<PCollection<KV<KeyT, InputT>>, PCollection<KeyedWorkItem<KeyT, InputT>>> {
+    @Override
+    public PCollection<KeyedWorkItem<KeyT, InputT>> apply(PCollection<KV<KeyT, InputT>> input) {
+      return PCollection.createPrimitiveOutputInternal(
+          input.getPipeline(), WindowingStrategy.globalDefault(), input.isBounded());
+    }
+  }
+
+  /**
+   * Runner-specific primitive {@link PTransform} that invokes the {@link DoFn.ProcessElement}
+   * method for a splittable {@link DoFn}.
+   */
+  public static class ProcessElements<InputT, OutputT, RestrictionT>
+      extends PTransform<
+          PCollection<? extends KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>>,
+          PCollectionTuple> {
+    private final DoFn<InputT, OutputT> fn;
+    private final Coder<InputT> elementCoder;
+    private final Coder<RestrictionT> restrictionCoder;
+    private final WindowingStrategy<?, ?> windowingStrategy;
+    private final List<PCollectionView<?>> sideInputs;
+    private final TupleTag<OutputT> mainOutputTag;
+    private final TupleTagList sideOutputTags;
+
+    /**
+     * @param fn the splittable {@link DoFn}.
+     * @param windowingStrategy the {@link WindowingStrategy} of the input collection.
+     * @param sideInputs list of side inputs that should be available to the {@link DoFn}.
+     * @param mainOutputTag {@link TupleTag Tag} of the {@link DoFn DoFn's} main output.
+     * @param sideOutputTags {@link TupleTagList Tags} of the {@link DoFn DoFn's} side outputs.
+     */
+    public ProcessElements(
+        DoFn<InputT, OutputT> fn,
+        Coder<InputT> elementCoder,
+        Coder<RestrictionT> restrictionCoder,
+        WindowingStrategy<?, ?> windowingStrategy,
+        List<PCollectionView<?>> sideInputs,
+        TupleTag<OutputT> mainOutputTag,
+        TupleTagList sideOutputTags) {
+      this.fn = fn;
+      this.elementCoder = elementCoder;
+      this.restrictionCoder = restrictionCoder;
+      this.windowingStrategy = windowingStrategy;
+      this.sideInputs = sideInputs;
+      this.mainOutputTag = mainOutputTag;
+      this.sideOutputTags = sideOutputTags;
+    }
+
+    public DoFn<InputT, OutputT> getFn() {
+      return fn;
+    }
+
+    public List<PCollectionView<?>> getSideInputs() {
+      return sideInputs;
+    }
+
+    public TupleTag<OutputT> getMainOutputTag() {
+      return mainOutputTag;
+    }
+
+    public TupleTagList getSideOutputTags() {
+      return sideOutputTags;
+    }
+
+    public ProcessFn<InputT, OutputT, RestrictionT, ?> newProcessFn(DoFn<InputT, OutputT> fn) {
+      return new SplittableParDo.ProcessFn<>(
+          fn, elementCoder, restrictionCoder, windowingStrategy.getWindowFn().windowCoder());
+    }
+
+    @Override
+    public PCollectionTuple apply(
+        PCollection<? extends KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>>
+            input) {
+      DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass());
+      PCollectionTuple outputs =
+          PCollectionTuple.ofPrimitiveOutputsInternal(
+              input.getPipeline(),
+              TupleTagList.of(mainOutputTag).and(sideOutputTags.getAll()),
+              windowingStrategy,
+              input.isBounded().and(signature.isBoundedPerElement()));
+
+      // Set output type descriptor similarly to how ParDo.BoundMulti does it.
+      outputs.get(mainOutputTag).setTypeDescriptorInternal(fn.getOutputTypeDescriptor());
+
+      return outputs;
+    }
+
+    @Override
+    public <T> Coder<T> getDefaultOutputCoder(
+        PCollection<? extends KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>>
+            input,
+        TypedPValue<T> output)
+        throws CannotProvideCoderException {
+      // Similar logic to ParDo.BoundMulti.getDefaultOutputCoder.
+      @SuppressWarnings("unchecked")
+      KeyedWorkItemCoder<String, ElementAndRestriction<InputT, RestrictionT>> kwiCoder =
+          (KeyedWorkItemCoder) input.getCoder();
+      Coder<InputT> inputCoder =
+          ((ElementAndRestrictionCoder<InputT, RestrictionT>) kwiCoder.getElementCoder())
+              .getElementCoder();
+      return input
+          .getPipeline()
+          .getCoderRegistry()
+          .getDefaultCoder(output.getTypeDescriptor(), fn.getInputTypeDescriptor(), inputCoder);
+    }
   }
 
   /**
@@ -182,15 +328,11 @@ public class SplittableParDo<
    * The heart of splittable {@link DoFn} execution: processes a single (element, restriction) pair
    * by creating a tracker for the restriction and checkpointing/resuming processing later if
    * necessary.
-   *
-   * <p>TODO: This uses deprecated OldDoFn since DoFn does not provide access to state/timer
-   * internals. This should be rewritten to use the <a href="https://s.apache.org/beam-state">State
-   * and Timers API</a> once it is available.
    */
   @VisibleForTesting
-  static class ProcessFn<
+  public static class ProcessFn<
           InputT, OutputT, RestrictionT, TrackerT extends RestrictionTracker<RestrictionT>>
-      extends OldDoFn<KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>, OutputT> {
+      extends DoFn<KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>, OutputT> {
     // Commit at least once every 10k output records.  This keeps the watermark advancing
     // smoothly, and ensures that not too much work will have to be reprocessed in the event of
     // a crash.
@@ -227,30 +369,56 @@ public class SplittableParDo<
      */
     private StateTag<Object, ValueState<RestrictionT>> restrictionTag;
 
+    private transient StateInternalsFactory<String> stateInternalsFactory;
+    private transient TimerInternalsFactory<String> timerInternalsFactory;
+    private transient OutputWindowedValue<OutputT> outputWindowedValue;
+
     private final DoFn<InputT, OutputT> fn;
     private final Coder<? extends BoundedWindow> windowCoder;
 
     private transient DoFnInvoker<InputT, OutputT> invoker;
 
-    ProcessFn(
+    public ProcessFn(
         DoFn<InputT, OutputT> fn,
         Coder<InputT> elementCoder,
         Coder<RestrictionT> restrictionCoder,
         Coder<? extends BoundedWindow> windowCoder) {
       this.fn = fn;
+      this.invoker = DoFnInvokers.invokerFor(fn);
       this.windowCoder = windowCoder;
-      elementTag =
+      this.elementTag =
           StateTags.value("element", WindowedValue.getFullCoder(elementCoder, this.windowCoder));
-      restrictionTag = StateTags.value("restriction", restrictionCoder);
+      this.restrictionTag = StateTags.value("restriction", restrictionCoder);
     }
 
-    @Override
-    public void setup() throws Exception {
-      invoker = DoFnInvokers.invokerFor(fn);
+    public void setStateInternalsFactory(StateInternalsFactory<String> stateInternalsFactory) {
+      this.stateInternalsFactory = stateInternalsFactory;
     }
 
-    @Override
+    public void setTimerInternalsFactory(TimerInternalsFactory<String> timerInternalsFactory) {
+      this.timerInternalsFactory = timerInternalsFactory;
+    }
+
+    public void setOutputWindowedValue(OutputWindowedValue<OutputT> outputWindowedValue) {
+      this.outputWindowedValue = outputWindowedValue;
+    }
+
+    @StartBundle
+    public void startBundle(Context c) throws Exception {
+      invoker.invokeStartBundle(wrapContext(c));
+    }
+
+    @FinishBundle
+    public void finishBundle(Context c) throws Exception {
+      invoker.invokeFinishBundle(wrapContext(c));
+    }
+
+    @ProcessElement
     public void processElement(final ProcessContext c) {
+      StateInternals<String> stateInternals =
+          stateInternalsFactory.stateInternalsForKey(c.element().key());
+      TimerInternals timerInternals = timerInternalsFactory.timerInternalsForKey(c.element().key());
+
       // Initialize state (element and restriction) depending on whether this is the seed call.
       // The seed call is the first call for this element, which actually has the element.
       // Subsequent calls are timer firings and the element has to be retrieved from the state.
@@ -258,17 +426,23 @@ public class SplittableParDo<
       boolean isSeedCall = (timer == null);
       StateNamespace stateNamespace = isSeedCall ? StateNamespaces.global() : timer.getNamespace();
       ValueState<WindowedValue<InputT>> elementState =
-          c.windowingInternals().stateInternals().state(stateNamespace, elementTag);
+          stateInternals.state(stateNamespace, elementTag);
       ValueState<RestrictionT> restrictionState =
-          c.windowingInternals().stateInternals().state(stateNamespace, restrictionTag);
+          stateInternals.state(stateNamespace, restrictionTag);
       WatermarkHoldState<GlobalWindow> holdState =
-          c.windowingInternals().stateInternals().state(stateNamespace, watermarkHoldTag);
+          stateInternals.state(stateNamespace, watermarkHoldTag);
 
       ElementAndRestriction<WindowedValue<InputT>, RestrictionT> elementAndRestriction;
       if (isSeedCall) {
         // The element and restriction are available in c.element().
+        // elementsIterable() will, by construction of SplittableParDo, contain the same value
+        // potentially in several different windows. We implode this into a single WindowedValue
+        // in order to simplify the rest of the code and avoid iterating over elementsIterable()
+        // explicitly. The windows of this WindowedValue will be propagated to windows of the
+        // output. This is correct because a splittable DoFn is not allowed to inspect the window
+        // of its element.
         WindowedValue<ElementAndRestriction<InputT, RestrictionT>> windowedValue =
-            Iterables.getOnlyElement(c.element().elementsIterable());
+            implodeWindows(c.element().elementsIterable());
         WindowedValue<InputT> element = windowedValue.withValue(windowedValue.getValue().element());
         elementState.write(element);
         elementAndRestriction =
@@ -290,7 +464,7 @@ public class SplittableParDo<
       DoFn.ProcessContinuation cont =
           invoker.invokeProcessElement(
               wrapTracker(
-                  tracker, makeContext(c, elementAndRestriction.element(), tracker, residual)));
+                  tracker, wrapContext(c, elementAndRestriction.element(), tracker, residual)));
       if (residual[0] == null) {
         // This means the call completed unsolicited, and the context produced by makeContext()
         // did not take a checkpoint. Take one now.
@@ -307,19 +481,85 @@ public class SplittableParDo<
       }
       restrictionState.write(residual[0]);
       Instant futureOutputWatermark = cont.getWatermark();
-      if (futureOutputWatermark != null) {
-        holdState.add(futureOutputWatermark);
+      if (futureOutputWatermark == null) {
+        futureOutputWatermark = elementAndRestriction.element().getTimestamp();
       }
+      Instant wakeupTime = timerInternals.currentProcessingTime().plus(cont.resumeDelay());
+      holdState.add(futureOutputWatermark);
       // Set a timer to continue processing this element.
-      TimerInternals timerInternals = c.windowingInternals().timerInternals();
       timerInternals.setTimer(
-          TimerInternals.TimerData.of(
-              stateNamespace,
-              timerInternals.currentProcessingTime().plus(cont.resumeDelay()),
-              TimeDomain.PROCESSING_TIME));
+          TimerInternals.TimerData.of(stateNamespace, wakeupTime, TimeDomain.PROCESSING_TIME));
+    }
+
+    /**
+     * Does the opposite of {@link WindowedValue#explodeWindows()} - creates a single {@link
+     * WindowedValue} from a collection of {@link WindowedValue}'s that is known to contain copies
+     * of the same value with the same timestamp, but different window sets.
+     *
+     * <p>This is only legal to do because we know that {@link RandomUniqueKeyFn} created unique
+     * keys for every {@link ElementAndRestriction}, so if there's multiple {@link WindowedValue}'s
+     * for the same key, that means only that the windows of that {@link ElementAndRestriction} are
+     * being delivered separately rather than all at once. It is also legal to do because splittable
+     * {@link DoFn} is not allowed to access the window of its element, so we can propagate the full
+     * set of windows of its input to its output.
+     */
+    private static <InputT, RestrictionT>
+        WindowedValue<ElementAndRestriction<InputT, RestrictionT>> implodeWindows(
+            Iterable<WindowedValue<ElementAndRestriction<InputT, RestrictionT>>> values) {
+      WindowedValue<ElementAndRestriction<InputT, RestrictionT>> first =
+          Iterables.getFirst(values, null);
+      checkState(first != null, "Got a KeyedWorkItem with no elements and no timers");
+      ImmutableList.Builder<BoundedWindow> windows = ImmutableList.builder();
+      for (WindowedValue<ElementAndRestriction<InputT, RestrictionT>> value : values) {
+        windows.addAll(value.getWindows());
+      }
+      return WindowedValue.of(
+          first.getValue(), first.getTimestamp(), windows.build(), first.getPane());
+    }
+
+    private DoFn<InputT, OutputT>.Context wrapContext(final Context baseContext) {
+      return fn.new Context() {
+        @Override
+        public PipelineOptions getPipelineOptions() {
+          return baseContext.getPipelineOptions();
+        }
+
+        @Override
+        public void output(OutputT output) {
+          throwUnsupportedOutput();
+        }
+
+        @Override
+        public void outputWithTimestamp(OutputT output, Instant timestamp) {
+          throwUnsupportedOutput();
+        }
+
+        @Override
+        public <T> void sideOutput(TupleTag<T> tag, T output) {
+          throwUnsupportedOutput();
+        }
+
+        @Override
+        public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+          throwUnsupportedOutput();
+        }
+
+        @Override
+        protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregator(
+            String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
+          return fn.createAggregator(name, combiner);
+        }
+
+        private void throwUnsupportedOutput() {
+          throw new UnsupportedOperationException(
+              String.format(
+                  "Splittable DoFn can only output from @%s",
+                  ProcessElement.class.getSimpleName()));
+        }
+      };
     }
 
-    private DoFn<InputT, OutputT>.ProcessContext makeContext(
+    private DoFn<InputT, OutputT>.ProcessContext wrapContext(
         final ProcessContext baseContext,
         final WindowedValue<InputT> element,
         final TrackerT tracker,
@@ -340,17 +580,14 @@ public class SplittableParDo<
         }
 
         public void output(OutputT output) {
-          baseContext
-              .windowingInternals()
-              .outputWindowedValue(
-                  output, element.getTimestamp(), element.getWindows(), element.getPane());
+          outputWindowedValue.outputWindowedValue(
+              output, element.getTimestamp(), element.getWindows(), element.getPane());
           noteOutput();
         }
 
         public void outputWithTimestamp(OutputT output, Instant timestamp) {
-          baseContext
-              .windowingInternals()
-              .outputWindowedValue(output, timestamp, element.getWindows(), element.getPane());
+          outputWindowedValue.outputWindowedValue(
+              output, timestamp, element.getWindows(), element.getPane());
           noteOutput();
         }
 
@@ -370,17 +607,15 @@ public class SplittableParDo<
         }
 
         public <T> void sideOutput(TupleTag<T> tag, T output) {
-          // TODO: I'm not sure how to implement this correctly: there's no
-          // "internals.sideOutputWindowedValue".
-          throw new UnsupportedOperationException(
-              "Side outputs not yet supported by splittable DoFn");
+          outputWindowedValue.sideOutputWindowedValue(
+              tag, output, element.getTimestamp(), element.getWindows(), element.getPane());
+          noteOutput();
         }
 
         public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-          // TODO: I'm not sure how to implement this correctly: there's no
-          // "internals.sideOutputWindowedValue".
-          throw new UnsupportedOperationException(
-              "Side outputs not yet supported by splittable DoFn");
+          outputWindowedValue.sideOutputWindowedValue(
+              tag, output, timestamp, element.getWindows(), element.getPane());
+          noteOutput();
         }
 
         @Override
@@ -393,8 +628,7 @@ public class SplittableParDo<
 
     /**
      * Creates an {@link DoFnInvoker.ArgumentProvider} that provides the given tracker as well as
-     * the given
-     * {@link ProcessContext} (which is also provided when a {@link Context} is requested.
+     * the given {@link ProcessContext} (which is also provided when a {@link Context} is requested.
      */
     private DoFnInvoker.ArgumentProvider<InputT, OutputT> wrapTracker(
         TrackerT tracker, DoFn<InputT, OutputT>.ProcessContext processContext) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
index 29ff838..990d892 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
@@ -29,6 +29,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.Serializable;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
@@ -38,6 +39,7 @@ import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
@@ -47,8 +49,13 @@ import org.apache.beam.sdk.util.KeyedWorkItem;
 import org.apache.beam.sdk.util.KeyedWorkItems;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateInternalsFactory;
+import org.apache.beam.sdk.util.state.TimerInternalsFactory;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Test;
@@ -120,6 +127,12 @@ public class SplittableParDoTest {
         .setIsBoundedInternal(PCollection.IsBounded.BOUNDED);
   }
 
+  private static final TupleTag<String> MAIN_OUTPUT_TAG = new TupleTag<String>() {};
+
+  private ParDo.BoundMulti<Integer, String> makeParDo(DoFn<Integer, String> fn) {
+    return ParDo.of(fn).withOutputTags(MAIN_OUTPUT_TAG, TupleTagList.empty());
+  }
+
   @Test
   public void testBoundednessForBoundedFn() {
     Pipeline pipeline = TestPipeline.create();
@@ -128,14 +141,15 @@ public class SplittableParDoTest {
         "Applying a bounded SDF to a bounded collection produces a bounded collection",
         PCollection.IsBounded.BOUNDED,
         makeBoundedCollection(pipeline)
-            .apply("bounded to bounded", new SplittableParDo<>(boundedFn))
-            .isBounded());
+            .apply("bounded to bounded", new SplittableParDo<>(makeParDo(boundedFn)))
+            .get(MAIN_OUTPUT_TAG).isBounded());
     assertEquals(
         "Applying a bounded SDF to an unbounded collection produces an unbounded collection",
         PCollection.IsBounded.UNBOUNDED,
         makeUnboundedCollection(pipeline)
-            .apply("bounded to unbounded", new SplittableParDo<>(boundedFn))
-            .isBounded());
+            .apply(
+                "bounded to unbounded", new SplittableParDo<>(makeParDo(boundedFn)))
+            .get(MAIN_OUTPUT_TAG).isBounded());
   }
 
   @Test
@@ -146,18 +160,27 @@ public class SplittableParDoTest {
         "Applying an unbounded SDF to a bounded collection produces a bounded collection",
         PCollection.IsBounded.UNBOUNDED,
         makeBoundedCollection(pipeline)
-            .apply("unbounded to bounded", new SplittableParDo<>(unboundedFn))
-            .isBounded());
+            .apply(
+                "unbounded to bounded",
+                new SplittableParDo<>(makeParDo(unboundedFn)))
+            .get(MAIN_OUTPUT_TAG).isBounded());
     assertEquals(
         "Applying an unbounded SDF to an unbounded collection produces an unbounded collection",
         PCollection.IsBounded.UNBOUNDED,
         makeUnboundedCollection(pipeline)
-            .apply("unbounded to unbounded", new SplittableParDo<>(unboundedFn))
-            .isBounded());
+            .apply(
+                "unbounded to unbounded",
+                new SplittableParDo<>(makeParDo(unboundedFn)))
+            .get(MAIN_OUTPUT_TAG).isBounded());
   }
 
   // ------------------------------- Tests for ProcessFn ---------------------------------
 
+  enum WindowExplosion {
+    EXPLODE_WINDOWS,
+    DO_NOT_EXPLODE_WINDOWS
+  }
+
   /**
    * A helper for testing {@link SplittableParDo.ProcessFn} on 1 element (but possibly over multiple
    * {@link DoFn.ProcessElement} calls).
@@ -179,6 +202,46 @@ public class SplittableParDoTest {
           new SplittableParDo.ProcessFn<>(
               fn, inputCoder, restrictionCoder, IntervalWindow.getCoder());
       this.tester = DoFnTester.of(processFn);
+      processFn.setStateInternalsFactory(
+          new StateInternalsFactory<String>() {
+            @Override
+            public StateInternals<String> stateInternalsForKey(String key) {
+              return tester.getStateInternals();
+            }
+          });
+      processFn.setTimerInternalsFactory(
+          new TimerInternalsFactory<String>() {
+            @Override
+            public TimerInternals timerInternalsForKey(String key) {
+              return tester.getTimerInternals();
+            }
+          });
+      processFn.setOutputWindowedValue(
+          new OutputWindowedValue<OutputT>() {
+            @Override
+            public void outputWindowedValue(
+                OutputT output,
+                Instant timestamp,
+                Collection<? extends BoundedWindow> windows,
+                PaneInfo pane) {
+              tester
+                  .getMutableOutput(tester.getMainOutputTag())
+                  .add(WindowedValue.of(output, timestamp, windows, pane));
+            }
+
+            @Override
+            public <SideOutputT> void sideOutputWindowedValue(
+                TupleTag<SideOutputT> tag,
+                SideOutputT output,
+                Instant timestamp,
+                Collection<? extends BoundedWindow> windows,
+                PaneInfo pane) {
+              tester.getMutableOutput(tag).add(WindowedValue.of(output, timestamp, windows, pane));
+            }
+          });
+      // Do not clone since ProcessFn references non-serializable DoFnTester itself
+      // through the state/timer/output callbacks.
+      this.tester.setCloningBehavior(DoFnTester.CloningBehavior.DO_NOT_CLONE);
       this.tester.startBundle();
       this.tester.advanceProcessingTime(currentProcessingTime);
 
@@ -192,12 +255,24 @@ public class SplittableParDoTest {
               ElementAndRestriction.of(element, restriction),
               currentProcessingTime,
               GlobalWindow.INSTANCE,
-              PaneInfo.ON_TIME_AND_ONLY_FIRING));
+              PaneInfo.ON_TIME_AND_ONLY_FIRING),
+          WindowExplosion.DO_NOT_EXPLODE_WINDOWS);
     }
 
-    void startElement(WindowedValue<ElementAndRestriction<InputT, RestrictionT>> windowedValue)
+    void startElement(
+        WindowedValue<ElementAndRestriction<InputT, RestrictionT>> windowedValue,
+        WindowExplosion explosion)
         throws Exception {
-      tester.processElement(KeyedWorkItems.elementsWorkItem("key", Arrays.asList(windowedValue)));
+      switch (explosion) {
+        case EXPLODE_WINDOWS:
+          tester.processElement(
+              KeyedWorkItems.elementsWorkItem("key", windowedValue.explodeWindows()));
+          break;
+        case DO_NOT_EXPLODE_WINDOWS:
+          tester.processElement(
+              KeyedWorkItems.elementsWorkItem("key", Arrays.asList(windowedValue)));
+          break;
+      }
     }
 
     /**
@@ -253,9 +328,6 @@ public class SplittableParDoTest {
     DoFn<Integer, String> fn = new ToStringFn();
 
     Instant base = Instant.now();
-    ProcessFnTester<Integer, String, SomeRestriction, SomeRestrictionTracker> tester =
-        new ProcessFnTester<>(
-            base, fn, BigEndianIntegerCoder.of(), SerializableCoder.of(SomeRestriction.class));
 
     IntervalWindow w1 =
         new IntervalWindow(
@@ -267,20 +339,26 @@ public class SplittableParDoTest {
         new IntervalWindow(
             base.minus(Duration.standardMinutes(3)), base.plus(Duration.standardMinutes(3)));
 
-    tester.startElement(
-        WindowedValue.of(
-            ElementAndRestriction.of(42, new SomeRestriction()),
-            base,
-            Arrays.asList(w1, w2, w3),
-            PaneInfo.ON_TIME_AND_ONLY_FIRING));
-
-    for (IntervalWindow w : new IntervalWindow[] {w1, w2, w3}) {
-      assertEquals(
-          Arrays.asList(
-              TimestampedValue.of("42a", base),
-              TimestampedValue.of("42b", base),
-              TimestampedValue.of("42c", base)),
-          tester.peekOutputElementsInWindow(w));
+    for (WindowExplosion explosion : WindowExplosion.values()) {
+      ProcessFnTester<Integer, String, SomeRestriction, SomeRestrictionTracker> tester =
+          new ProcessFnTester<>(
+              base, fn, BigEndianIntegerCoder.of(), SerializableCoder.of(SomeRestriction.class));
+      tester.startElement(
+          WindowedValue.of(
+              ElementAndRestriction.of(42, new SomeRestriction()),
+              base,
+              Arrays.asList(w1, w2, w3),
+              PaneInfo.ON_TIME_AND_ONLY_FIRING),
+          explosion);
+
+      for (IntervalWindow w : new IntervalWindow[] {w1, w2, w3}) {
+        assertEquals(
+            Arrays.asList(
+                TimestampedValue.of("42a", base),
+                TimestampedValue.of("42b", base),
+                TimestampedValue.of("42c", base)),
+            tester.peekOutputElementsInWindow(w));
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java
index 680a971..04becd7 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java
@@ -17,48 +17,23 @@
  */
 package org.apache.beam.runners.direct;
 
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.runners.core.GBKIntoKeyedWorkItems;
-import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.runners.core.SplittableParDo;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.util.KeyedWorkItem;
-import org.apache.beam.sdk.util.KeyedWorkItemCoder;
-import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 
-/** Provides an implementation of {@link GBKIntoKeyedWorkItems} for the Direct Runner. */
+/**
+ * Provides an implementation of {@link SplittableParDo.GBKIntoKeyedWorkItems} for the Direct
+ * Runner.
+ */
 class DirectGBKIntoKeyedWorkItemsOverrideFactory<KeyT, InputT>
     implements PTransformOverrideFactory<
         PCollection<KV<KeyT, InputT>>, PCollection<KeyedWorkItem<KeyT, InputT>>,
-        GBKIntoKeyedWorkItems<KeyT, InputT>> {
+        SplittableParDo.GBKIntoKeyedWorkItems<KeyT, InputT>> {
   @Override
   public PTransform<PCollection<KV<KeyT, InputT>>, PCollection<KeyedWorkItem<KeyT, InputT>>>
-      override(GBKIntoKeyedWorkItems<KeyT, InputT> transform) {
-    return new DirectGBKIntoKeyedWorkItems<>(transform.getName());
-  }
-
-  /** The Direct Runner specific implementation of {@link GBKIntoKeyedWorkItems}. */
-  private static class DirectGBKIntoKeyedWorkItems<KeyT, InputT>
-      extends PTransform<PCollection<KV<KeyT, InputT>>, PCollection<KeyedWorkItem<KeyT, InputT>>> {
-    DirectGBKIntoKeyedWorkItems(String name) {
-      super(name);
-    }
-
-    @Override
-    public PCollection<KeyedWorkItem<KeyT, InputT>> apply(PCollection<KV<KeyT, InputT>> input) {
-      checkArgument(input.getCoder() instanceof KvCoder);
-      KvCoder<KeyT, InputT> kvCoder = (KvCoder<KeyT, InputT>) input.getCoder();
-      return input
-          // TODO: Perhaps windowing strategy should instead be set by ReifyTAW, or by DGBKO
-          .setWindowingStrategyInternal(WindowingStrategy.globalDefault())
-          .apply(new DirectGroupByKey.DirectGroupByKeyOnly<KeyT, InputT>())
-          .setCoder(
-              KeyedWorkItemCoder.of(
-                  kvCoder.getKeyCoder(),
-                  kvCoder.getValueCoder(),
-                  input.getWindowingStrategy().getWindowFn().windowCoder()));
-    }
+      override(SplittableParDo.GBKIntoKeyedWorkItems<KeyT, InputT> transform) {
+    return new DirectGroupByKey.DirectGroupByKeyOnly<>();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/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 219314a..efee801 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
@@ -65,7 +65,7 @@ class DirectGroupByKey<K, V>
             KeyedWorkItemCoder.of(
                 inputCoder.getKeyCoder(),
                 inputCoder.getValueCoder(),
-                input.getWindowingStrategy().getWindowFn().windowCoder()))
+                inputWindowingStrategy.getWindowFn().windowCoder()))
 
         // Group each key's values by window, merging windows as needed.
         .apply(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index f71e109..82de9ab 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -30,7 +30,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 import javax.annotation.Nullable;
-import org.apache.beam.runners.core.GBKIntoKeyedWorkItems;
+import org.apache.beam.runners.core.SplittableParDo;
 import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow;
 import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly;
 import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult;
@@ -88,7 +88,7 @@ public class DirectRunner extends PipelineRunner<DirectPipelineResult> {
               .put(ParDo.Bound.class, new ParDoSingleViaMultiOverrideFactory())
               .put(ParDo.BoundMulti.class, new ParDoMultiOverrideFactory())
               .put(
-                  GBKIntoKeyedWorkItems.class,
+                  SplittableParDo.GBKIntoKeyedWorkItems.class,
                   new DirectGBKIntoKeyedWorkItemsOverrideFactory())
               .build();
 
@@ -307,8 +307,8 @@ public class DirectRunner extends PipelineRunner<DirectPipelineResult> {
     @SuppressWarnings("rawtypes")
     KeyedPValueTrackingVisitor keyedPValueVisitor =
         KeyedPValueTrackingVisitor.create(
-            ImmutableSet.<Class<? extends PTransform>>of(
-                GBKIntoKeyedWorkItems.class,
+            ImmutableSet.of(
+                SplittableParDo.GBKIntoKeyedWorkItems.class,
                 DirectGroupByKeyOnly.class,
                 DirectGroupAlsoByWindow.class));
     pipeline.traverseTopologically(keyedPValueVisitor);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
index 67d957c..cd644a6 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java
@@ -56,9 +56,9 @@ class DoFnLifecycleManager {
     thrownOnTeardown = new ConcurrentHashMap<>();
   }
 
-  public DoFn<?, ?> get() throws Exception {
+  public <InputT, OutputT> DoFn<InputT, OutputT> get() throws Exception {
     Thread currentThread = Thread.currentThread();
-    return outstanding.get(currentThread);
+    return (DoFn<InputT, OutputT>) outstanding.get(currentThread);
   }
 
   public void remove() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
index 750e5f1..504ddc4 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
@@ -20,7 +20,6 @@ package org.apache.beam.runners.direct;
 import com.google.common.collect.ImmutableList;
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -58,9 +57,9 @@ class ParDoEvaluator<InputT, OutputT> implements TransformEvaluator<InputT> {
     Map<TupleTag<?>, UncommittedBundle<?>> outputBundles = new HashMap<>();
     for (Map.Entry<TupleTag<?>, PCollection<?>> outputEntry : outputs.entrySet()) {
       outputBundles.put(
-          outputEntry.getKey(),
-          evaluationContext.createBundle(outputEntry.getValue()));
+          outputEntry.getKey(), evaluationContext.createBundle(outputEntry.getValue()));
     }
+    BundleOutputManager outputManager = BundleOutputManager.create(outputBundles);
 
     ReadyCheckingSideInputReader sideInputReader =
         evaluationContext.createSideInputReader(sideInputs);
@@ -69,7 +68,7 @@ class ParDoEvaluator<InputT, OutputT> implements TransformEvaluator<InputT> {
             evaluationContext.getPipelineOptions(),
             fn,
             sideInputReader,
-            BundleOutputManager.create(outputBundles),
+            outputManager,
             mainOutputTag,
             sideOutputTags,
             stepContext,
@@ -85,12 +84,7 @@ class ParDoEvaluator<InputT, OutputT> implements TransformEvaluator<InputT> {
     }
 
     return new ParDoEvaluator<>(
-        evaluationContext,
-        runner,
-        application,
-        aggregatorChanges,
-        outputBundles.values(),
-        stepContext);
+        evaluationContext, runner, application, aggregatorChanges, outputManager, stepContext);
   }
 
   ////////////////////////////////////////////////////////////////////////////////////////////////
@@ -99,7 +93,7 @@ class ParDoEvaluator<InputT, OutputT> implements TransformEvaluator<InputT> {
   private final PushbackSideInputDoFnRunner<InputT, ?> fnRunner;
   private final AppliedPTransform<?, ?, ?> transform;
   private final AggregatorContainer.Mutator aggregatorChanges;
-  private final Collection<UncommittedBundle<?>> outputBundles;
+  private final BundleOutputManager outputManager;
   private final DirectStepContext stepContext;
 
   private final ImmutableList.Builder<WindowedValue<InputT>> unprocessedElements;
@@ -109,17 +103,21 @@ class ParDoEvaluator<InputT, OutputT> implements TransformEvaluator<InputT> {
       PushbackSideInputDoFnRunner<InputT, ?> fnRunner,
       AppliedPTransform<?, ?, ?> transform,
       AggregatorContainer.Mutator aggregatorChanges,
-      Collection<UncommittedBundle<?>> outputBundles,
+      BundleOutputManager outputManager,
       DirectStepContext stepContext) {
     this.evaluationContext = evaluationContext;
     this.fnRunner = fnRunner;
     this.transform = transform;
-    this.outputBundles = outputBundles;
+    this.outputManager = outputManager;
     this.stepContext = stepContext;
     this.aggregatorChanges = aggregatorChanges;
     this.unprocessedElements = ImmutableList.builder();
   }
 
+  public BundleOutputManager getOutputManager() {
+    return outputManager;
+  }
+
   @Override
   public void processElement(WindowedValue<InputT> element) {
     try {
@@ -147,7 +145,7 @@ class ParDoEvaluator<InputT, OutputT> implements TransformEvaluator<InputT> {
       resultBuilder = StepTransformResult.withoutHold(transform);
     }
     return resultBuilder
-        .addOutput(outputBundles)
+        .addOutput(outputManager.bundles.values())
         .withTimerUpdate(stepContext.getTimerUpdate())
         .withAggregatorChanges(aggregatorChanges)
         .addUnprocessedElements(unprocessedElements.build())

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java
index 02e034a..ec5dc2c 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java
@@ -57,6 +57,7 @@ final class ParDoEvaluatorFactory<InputT, OutputT> implements TransformEvaluator
   public <T> TransformEvaluator<T> forApplication(
       AppliedPTransform<?, ?, ?> application, CommittedBundle<?> inputBundle) throws Exception {
 
+    @SuppressWarnings("unchecked")
     AppliedPTransform<PCollection<InputT>, PCollectionTuple, ParDo.BoundMulti<InputT, OutputT>>
         parDoApplication =
             (AppliedPTransform<
@@ -93,13 +94,12 @@ final class ParDoEvaluatorFactory<InputT, OutputT> implements TransformEvaluator
    */
   @SuppressWarnings({"unchecked", "rawtypes"})
   TransformEvaluator<InputT> createEvaluator(
-        AppliedPTransform<PCollection<?>, PCollectionTuple, ?>
-        application,
-        StructuralKey<?> inputBundleKey,
-        DoFn<InputT, OutputT> doFn,
-        List<PCollectionView<?>> sideInputs,
-        TupleTag<OutputT> mainOutputTag,
-        List<TupleTag<?>> sideOutputTags)
+      AppliedPTransform<PCollection<InputT>, PCollectionTuple, ?> application,
+      StructuralKey<?> inputBundleKey,
+      DoFn<InputT, OutputT> doFn,
+      List<PCollectionView<?>> sideInputs,
+      TupleTag<OutputT> mainOutputTag,
+      List<TupleTag<?>> sideOutputTags)
       throws Exception {
     String stepName = evaluationContext.getStepName(application);
     DirectStepContext stepContext =
@@ -107,21 +107,40 @@ final class ParDoEvaluatorFactory<InputT, OutputT> implements TransformEvaluator
             .getExecutionContext(application, inputBundleKey)
             .getOrCreateStepContext(stepName, stepName);
 
-    DoFnLifecycleManager fnManager = fnClones.getUnchecked(doFn);
+    DoFnLifecycleManager fnManager = getManagerForCloneOf(doFn);
 
+    return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(
+        createParDoEvaluator(
+            application,
+            sideInputs,
+            mainOutputTag,
+            sideOutputTags,
+            stepContext,
+            fnManager.<InputT, OutputT>get(),
+            fnManager),
+        fnManager);
+  }
+
+  ParDoEvaluator<InputT, OutputT> createParDoEvaluator(
+      AppliedPTransform<PCollection<InputT>, PCollectionTuple, ?> application,
+      List<PCollectionView<?>> sideInputs,
+      TupleTag<OutputT> mainOutputTag,
+      List<TupleTag<?>> sideOutputTags,
+      DirectStepContext stepContext,
+      DoFn<InputT, OutputT> fn,
+      DoFnLifecycleManager fnManager)
+      throws Exception {
     try {
-      return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(
-          ParDoEvaluator.<InputT, OutputT>create(
-              evaluationContext,
-              stepContext,
-              application,
-              application.getInput().getWindowingStrategy(),
-              fnManager.get(),
-              sideInputs,
-              mainOutputTag,
-              sideOutputTags,
-              application.getOutput().getAll()),
-          fnManager);
+      return ParDoEvaluator.create(
+          evaluationContext,
+          stepContext,
+          application,
+          application.getInput().getWindowingStrategy(),
+          fn,
+          sideInputs,
+          mainOutputTag,
+          sideOutputTags,
+          application.getOutput().getAll());
     } catch (Exception e) {
       try {
         fnManager.remove();
@@ -134,4 +153,8 @@ final class ParDoEvaluatorFactory<InputT, OutputT> implements TransformEvaluator
       throw e;
     }
   }
+
+  public DoFnLifecycleManager getManagerForCloneOf(DoFn<?, ?> fn) {
+    return fnClones.getUnchecked(fn);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
index 8db5159..9c9256d 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
@@ -49,7 +49,7 @@ class ParDoMultiOverrideFactory<InputT, OutputT>
     DoFn<InputT, OutputT> fn = transform.getNewFn();
     DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass());
     if (signature.processElement().isSplittable()) {
-      return new SplittableParDo(fn);
+      return new SplittableParDo(transform);
     } else if (signature.stateDeclarations().size() > 0
         || signature.timerDeclarations().size() > 0) {
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java
new file mode 100644
index 0000000..0eca710
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.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.runners.direct;
+
+import java.util.Collection;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+import org.apache.beam.runners.core.ElementAndRestriction;
+import org.apache.beam.runners.core.OutputWindowedValue;
+import org.apache.beam.runners.core.SplittableParDo;
+import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateInternalsFactory;
+import org.apache.beam.sdk.util.state.TimerInternalsFactory;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TupleTag;
+import org.joda.time.Instant;
+
+class SplittableProcessElementsEvaluatorFactory<InputT, OutputT, RestrictionT>
+    implements TransformEvaluatorFactory {
+  private final ParDoEvaluatorFactory<
+          KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>, OutputT>
+      delegateFactory;
+  private final EvaluationContext evaluationContext;
+
+  SplittableProcessElementsEvaluatorFactory(EvaluationContext evaluationContext) {
+    this.evaluationContext = evaluationContext;
+    this.delegateFactory = new ParDoEvaluatorFactory<>(evaluationContext);
+  }
+
+  @Override
+  public <T> TransformEvaluator<T> forApplication(
+      AppliedPTransform<?, ?, ?> application, CommittedBundle<?> inputBundle) throws Exception {
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    TransformEvaluator<T> evaluator =
+        (TransformEvaluator<T>)
+            createEvaluator((AppliedPTransform) application, (CommittedBundle) inputBundle);
+    return evaluator;
+  }
+
+  @Override
+  public void cleanup() throws Exception {
+    delegateFactory.cleanup();
+  }
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  private TransformEvaluator<KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>>
+      createEvaluator(
+          AppliedPTransform<
+                  PCollection<KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>>,
+                  PCollectionTuple, SplittableParDo.ProcessElements<InputT, OutputT, RestrictionT>>
+              application,
+          CommittedBundle<InputT> inputBundle)
+          throws Exception {
+    final SplittableParDo.ProcessElements<InputT, OutputT, RestrictionT> transform =
+        application.getTransform();
+
+    DoFnLifecycleManager fnManager = delegateFactory.getManagerForCloneOf(transform.getFn());
+
+    SplittableParDo.ProcessFn<InputT, OutputT, RestrictionT, ?> processFn =
+        transform.newProcessFn(fnManager.<InputT, OutputT>get());
+
+    String stepName = evaluationContext.getStepName(application);
+    final DirectExecutionContext.DirectStepContext stepContext =
+        evaluationContext
+            .getExecutionContext(application, inputBundle.getKey())
+            .getOrCreateStepContext(stepName, stepName);
+
+    ParDoEvaluator<KeyedWorkItem<String, ElementAndRestriction<InputT, RestrictionT>>, OutputT>
+        parDoEvaluator =
+            delegateFactory.createParDoEvaluator(
+                application,
+                transform.getSideInputs(),
+                transform.getMainOutputTag(),
+                transform.getSideOutputTags().getAll(),
+                stepContext,
+                processFn,
+                fnManager);
+
+    processFn.setStateInternalsFactory(
+        new StateInternalsFactory<String>() {
+          @SuppressWarnings({"unchecked", "rawtypes"})
+          @Override
+          public StateInternals<String> stateInternalsForKey(String key) {
+            return (StateInternals) stepContext.stateInternals();
+          }
+        });
+
+    processFn.setTimerInternalsFactory(
+        new TimerInternalsFactory<String>() {
+          @Override
+          public TimerInternals timerInternalsForKey(String key) {
+            return stepContext.timerInternals();
+          }
+        });
+
+    final OutputManager outputManager = parDoEvaluator.getOutputManager();
+    processFn.setOutputWindowedValue(
+        new OutputWindowedValue<OutputT>() {
+          @Override
+          public void outputWindowedValue(
+              OutputT output,
+              Instant timestamp,
+              Collection<? extends BoundedWindow> windows,
+              PaneInfo pane) {
+            outputManager.output(
+                transform.getMainOutputTag(), WindowedValue.of(output, timestamp, windows, pane));
+          }
+
+          @Override
+          public <SideOutputT> void sideOutputWindowedValue(
+              TupleTag<SideOutputT> tag,
+              SideOutputT output,
+              Instant timestamp,
+              Collection<? extends BoundedWindow> windows,
+              PaneInfo pane) {
+            outputManager.output(tag, WindowedValue.of(output, timestamp, windows, pane));
+          }
+        });
+
+    return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(parDoEvaluator, fnManager);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
index a4c462a..1ddf9f4 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
@@ -25,6 +25,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.beam.runners.core.SplittableParDo;
 import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow;
 import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
@@ -61,6 +62,10 @@ class TransformEvaluatorRegistry implements TransformEvaluatorFactory {
             .put(
                 TestStreamEvaluatorFactory.DirectTestStreamFactory.DirectTestStream.class,
                 new TestStreamEvaluatorFactory(ctxt))
+            // Runner-specific primitive used in expansion of SplittableParDo
+            .put(
+                SplittableParDo.ProcessElements.class,
+                new SplittableProcessElementsEvaluatorFactory<>(ctxt))
             .build();
     return new TransformEvaluatorRegistry(primitives);
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
index c164ce6..f9e833f 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
@@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -32,20 +33,28 @@ import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestStream;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.Keys;
 import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.View;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.joda.time.MutableDateTime;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -66,6 +75,11 @@ public class SplittableDoFnTest {
       this.from = from;
       this.to = to;
     }
+
+    @Override
+    public String toString() {
+      return "OffsetRange{" + "from=" + from + ", to=" + to + '}';
+    }
   }
 
   private static class OffsetRangeTracker implements RestrictionTracker<OffsetRange> {
@@ -140,11 +154,8 @@ public class SplittableDoFnTest {
     }
   }
 
-  @Ignore(
-      "BEAM-801: SplittableParDo uses unsupported OldDoFn features that are not available in DoFn; "
-          + "It must be implemented as a primitive.")
   @Test
-  public void testPairWithIndexBasic() throws ClassNotFoundException {
+  public void testPairWithIndexBasic() {
     Pipeline p = TestPipeline.create();
     p.getOptions().setRunner(DirectRunner.class);
     PCollection<KV<String, Integer>> res =
@@ -167,11 +178,8 @@ public class SplittableDoFnTest {
     p.run();
   }
 
-  @Ignore(
-      "BEAM-801: SplittableParDo uses unsupported OldDoFn features that are not available in DoFn; "
-          + "It must be implemented as a primitive.")
   @Test
-  public void testPairWithIndexWindowedTimestamped() throws ClassNotFoundException {
+  public void testPairWithIndexWindowedTimestamped() {
     // Tests that Splittable DoFn correctly propagates windowing strategy, windows and timestamps
     // of elements in the input collection.
     Pipeline p = TestPipeline.create();
@@ -228,4 +236,172 @@ public class SplittableDoFnTest {
     }
     p.run();
   }
+
+  private static class SDFWithSideInputsAndOutputs extends DoFn<Integer, String> {
+    private final PCollectionView<String> sideInput;
+    private final TupleTag<String> sideOutput;
+
+    private SDFWithSideInputsAndOutputs(
+        PCollectionView<String> sideInput, TupleTag<String> sideOutput) {
+      this.sideInput = sideInput;
+      this.sideOutput = sideOutput;
+    }
+
+    @ProcessElement
+    public void process(ProcessContext c, OffsetRangeTracker tracker) {
+      checkState(tracker.tryClaim(tracker.currentRestriction().from));
+      String side = c.sideInput(sideInput);
+      c.output("main:" + side + ":" + c.element());
+      c.sideOutput(sideOutput, "side:" + side + ":" + c.element());
+    }
+
+    @GetInitialRestriction
+    public OffsetRange getInitialRestriction(Integer value) {
+      return new OffsetRange(0, 1);
+    }
+
+    @NewTracker
+    public OffsetRangeTracker newTracker(OffsetRange range) {
+      return new OffsetRangeTracker(range);
+    }
+  }
+
+  @Test
+  public void testSideInputsAndOutputs() throws Exception {
+    Pipeline p = TestPipeline.create();
+    p.getOptions().setRunner(DirectRunner.class);
+
+    PCollectionView<String> sideInput =
+        p.apply("side input", Create.of("foo")).apply(View.<String>asSingleton());
+    TupleTag<String> mainOutputTag = new TupleTag<>("main");
+    TupleTag<String> sideOutputTag = new TupleTag<>("side");
+
+    PCollectionTuple res =
+        p.apply("input", Create.of(0, 1, 2))
+            .apply(
+                ParDo.of(new SDFWithSideInputsAndOutputs(sideInput, sideOutputTag))
+                    .withSideInputs(sideInput)
+                    .withOutputTags(mainOutputTag, TupleTagList.of(sideOutputTag)));
+    res.get(mainOutputTag).setCoder(StringUtf8Coder.of());
+    res.get(sideOutputTag).setCoder(StringUtf8Coder.of());
+
+    PAssert.that(res.get(mainOutputTag))
+        .containsInAnyOrder(Arrays.asList("main:foo:0", "main:foo:1", "main:foo:2"));
+    PAssert.that(res.get(sideOutputTag))
+        .containsInAnyOrder(Arrays.asList("side:foo:0", "side:foo:1", "side:foo:2"));
+
+    p.run();
+  }
+
+  @Test
+  public void testLateData() throws Exception {
+    Pipeline p = TestPipeline.create();
+    p.getOptions().setRunner(DirectRunner.class);
+
+    Instant base = Instant.now();
+
+    TestStream<String> stream =
+        TestStream.create(StringUtf8Coder.of())
+            .advanceWatermarkTo(base)
+            .addElements("aa")
+            .advanceWatermarkTo(base.plus(Duration.standardSeconds(5)))
+            .addElements(TimestampedValue.of("bb", base.minus(Duration.standardHours(1))))
+            .advanceProcessingTime(Duration.standardHours(1))
+            .advanceWatermarkToInfinity();
+
+    PCollection<String> input =
+        p.apply(stream)
+            .apply(
+                Window.<String>into(FixedWindows.of(Duration.standardMinutes(1)))
+                    .withAllowedLateness(Duration.standardMinutes(1)));
+
+    PCollection<KV<String, Integer>> afterSDF =
+        input
+            .apply(ParDo.of(new PairStringWithIndexToLength()))
+            .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()));
+
+    PCollection<String> nonLate =
+        afterSDF.apply(GroupByKey.<String, Integer>create()).apply(Keys.<String>create());
+
+    // The splittable DoFn itself should not drop any data and act as pass-through.
+    PAssert.that(afterSDF)
+        .containsInAnyOrder(
+            Arrays.asList(KV.of("aa", 0), KV.of("aa", 1), KV.of("bb", 0), KV.of("bb", 1)));
+
+    // But it should preserve the windowing strategy of the data, including allowed lateness:
+    // the follow-up GBK should drop the late data.
+    assertEquals(afterSDF.getWindowingStrategy(), input.getWindowingStrategy());
+    PAssert.that(nonLate).containsInAnyOrder("aa");
+
+    p.run();
+  }
+
+  private static class SDFWithLifecycle extends DoFn<String, String> {
+    private enum State {
+      BEFORE_SETUP,
+      OUTSIDE_BUNDLE,
+      INSIDE_BUNDLE,
+      TORN_DOWN
+    }
+
+    private State state = State.BEFORE_SETUP;
+
+    @ProcessElement
+    public void processElement(ProcessContext c, OffsetRangeTracker tracker) {
+      assertEquals(State.INSIDE_BUNDLE, state);
+      assertTrue(tracker.tryClaim(0));
+      c.output(c.element());
+    }
+
+    @GetInitialRestriction
+    public OffsetRange getInitialRestriction(String value) {
+      return new OffsetRange(0, 1);
+    }
+
+    @NewTracker
+    public OffsetRangeTracker newTracker(OffsetRange range) {
+      return new OffsetRangeTracker(range);
+    }
+
+    @Setup
+    public void setUp() {
+      assertEquals(State.BEFORE_SETUP, state);
+      state = State.OUTSIDE_BUNDLE;
+    }
+
+    @StartBundle
+    public void startBundle(Context c) {
+      assertEquals(State.OUTSIDE_BUNDLE, state);
+      state = State.INSIDE_BUNDLE;
+    }
+
+    @FinishBundle
+    public void finishBundle(Context c) {
+      assertEquals(State.INSIDE_BUNDLE, state);
+      state = State.OUTSIDE_BUNDLE;
+    }
+
+    @Teardown
+    public void tearDown() {
+      assertEquals(State.OUTSIDE_BUNDLE, state);
+      state = State.TORN_DOWN;
+    }
+  }
+
+  @Test
+  public void testLifecycleMethods() throws Exception {
+    Pipeline p = TestPipeline.create();
+    p.getOptions().setRunner(DirectRunner.class);
+
+    PCollection<String> res =
+        p.apply(Create.of("a", "b", "c")).apply(ParDo.of(new SDFWithLifecycle()));
+
+    PAssert.that(res).containsInAnyOrder("a", "b", "c");
+
+    p.run();
+  }
+
+  // TODO (https://issues.apache.org/jira/browse/BEAM-988): Test that Splittable DoFn
+  // emits output immediately (i.e. has a pass-through trigger) regardless of input's
+  // windowing/triggering strategy.
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
index 3f1a3f9..7aabec9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
@@ -120,6 +120,9 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
      * should be in, throwing an exception if the {@code WindowFn} attempts
      * to access any information about the input element. The output element
      * will have a timestamp of negative infinity.
+     *
+     * <p><i>Note:</i> A splittable {@link DoFn} is not allowed to output from
+     * {@link StartBundle} or {@link FinishBundle} methods.
      */
     public abstract void output(OutputT output);
 
@@ -142,6 +145,9 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
      * should be in, throwing an exception if the {@code WindowFn} attempts
      * to access any information about the input element except for the
      * timestamp.
+     *
+     * <p><i>Note:</i> A splittable {@link DoFn} is not allowed to output from
+     * {@link StartBundle} or {@link FinishBundle} methods.
      */
     public abstract void outputWithTimestamp(OutputT output, Instant timestamp);
 
@@ -168,6 +174,9 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
      * to access any information about the input element. The output element
      * will have a timestamp of negative infinity.
      *
+     * <p><i>Note:</i> A splittable {@link DoFn} is not allowed to output from
+     * {@link StartBundle} or {@link FinishBundle} methods.
+     *
      * @see ParDo#withOutputTags
      */
     public abstract <T> void sideOutput(TupleTag<T> tag, T output);
@@ -192,6 +201,9 @@ public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayD
      * to access any information about the input element except for the
      * timestamp.
      *
+     * <p><i>Note:</i> A splittable {@link DoFn} is not allowed to output from
+     * {@link StartBundle} or {@link FinishBundle} methods.
+     *
      * @see ParDo#withOutputTags
      */
     public abstract <T> void sideOutputWithTimestamp(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
index daa8a06..0c6043f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
@@ -140,6 +140,15 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     windowValues.put(window, value);
   }
 
+  @SuppressWarnings("unchecked")
+  public <K> StateInternals<K> getStateInternals() {
+    return (StateInternals<K>) stateInternals;
+  }
+
+  public TimerInternals getTimerInternals() {
+    return timerInternals;
+  }
+
   /**
    * When a {@link DoFnTester} should clone the {@link DoFn} under test and how it should manage
    * the lifecycle of the {@link DoFn}.
@@ -321,7 +330,6 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
    *
    */
   public List<OutputT> peekOutputElements() {
-    // TODO: Should we return an unmodifiable list?
     return Lists.transform(
         peekOutputElementsWithTimestamp(),
         new Function<TimestampedValue<OutputT>, OutputT>() {
@@ -344,7 +352,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
   @Experimental
   public List<TimestampedValue<OutputT>> peekOutputElementsWithTimestamp() {
     // TODO: Should we return an unmodifiable list?
-    return Lists.transform(getOutput(mainOutputTag),
+    return Lists.transform(getImmutableOutput(mainOutputTag),
         new Function<WindowedValue<OutputT>, TimestampedValue<OutputT>>() {
           @Override
           @SuppressWarnings("unchecked")
@@ -370,7 +378,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
       TupleTag<OutputT> tag,
       BoundedWindow window) {
     ImmutableList.Builder<TimestampedValue<OutputT>> valuesBuilder = ImmutableList.builder();
-    for (WindowedValue<OutputT> value : getOutput(tag)) {
+    for (WindowedValue<OutputT> value : getImmutableOutput(tag)) {
       if (value.getWindows().contains(window)) {
         valuesBuilder.add(TimestampedValue.of(value.getValue(), value.getTimestamp()));
       }
@@ -384,7 +392,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
    * @see #peekOutputElements
    */
   public void clearOutputElements() {
-    peekOutputElements().clear();
+    getMutableOutput(mainOutputTag).clear();
   }
 
   /**
@@ -425,7 +433,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
    */
   public <T> List<T> peekSideOutputElements(TupleTag<T> tag) {
     // TODO: Should we return an unmodifiable list?
-    return Lists.transform(getOutput(tag),
+    return Lists.transform(getImmutableOutput(tag),
         new Function<WindowedValue<T>, T>() {
           @SuppressWarnings("unchecked")
           @Override
@@ -441,7 +449,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
    * @see #peekSideOutputElements
    */
   public <T> void clearSideOutputElements(TupleTag<T> tag) {
-    peekSideOutputElements(tag).clear();
+    getMutableOutput(tag).clear();
   }
 
   /**
@@ -502,10 +510,25 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     return combiner.extractOutput(accumulator);
   }
 
-  private <T> List<WindowedValue<T>> getOutput(TupleTag<T> tag) {
+  private <T> List<WindowedValue<T>> getImmutableOutput(TupleTag<T> tag) {
     @SuppressWarnings({"unchecked", "rawtypes"})
     List<WindowedValue<T>> elems = (List) outputs.get(tag);
-    return MoreObjects.firstNonNull(elems, Collections.<WindowedValue<T>>emptyList());
+    return ImmutableList.copyOf(
+        MoreObjects.firstNonNull(elems, Collections.<WindowedValue<T>>emptyList()));
+  }
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  public <T> List<WindowedValue<T>> getMutableOutput(TupleTag<T> tag) {
+    List<WindowedValue<T>> outputList = (List) outputs.get(tag);
+    if (outputList == null) {
+      outputList = new ArrayList<>();
+      outputs.put(tag, (List) outputList);
+    }
+    return outputList;
+  }
+
+  public TupleTag<OutputT> getMainOutputTag() {
+    return mainOutputTag;
   }
 
   private TestContext createContext(OldDoFn<InputT, OutputT> fn) {
@@ -590,17 +613,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     }
 
     public <T> void noteOutput(TupleTag<T> tag, WindowedValue<T> output) {
-      getOutputList(tag).add(output);
-    }
-
-    private <T> List<WindowedValue<T>> getOutputList(TupleTag<T> tag) {
-      @SuppressWarnings({"unchecked", "rawtypes"})
-      List<WindowedValue<T>> outputList = (List) outputs.get(tag);
-      if (outputList == null) {
-        outputList = new ArrayList<>();
-        outputs.put(tag, (List) outputList);
-      }
-      return outputList;
+      getMutableOutput(tag).add(output);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/87ff5ac3/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerInternalsFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerInternalsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerInternalsFactory.java
new file mode 100644
index 0000000..b9c3d5e
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/TimerInternalsFactory.java
@@ -0,0 +1,36 @@
+/*
+ * 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.state;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.util.TimerInternals;
+
+/**
+ * A factory for providing {@link TimerInternals} for a particular key.
+ *
+ * <p>Because it will generally be embedded in a {@link org.apache.beam.sdk.transforms.DoFn DoFn},
+ * albeit at execution time, it is marked {@link Serializable}.
+ */
+@Experimental(Kind.STATE)
+public interface TimerInternalsFactory<K> {
+
+  /** Returns {@link TimerInternals} for the provided key. */
+  TimerInternals timerInternalsForKey(K key);
+}



[13/50] [abbrv] incubator-beam git commit: Fix pom syntax for excludedGroups for SplittableParDo

Posted by ke...@apache.org.
Fix pom syntax for excludedGroups for SplittableParDo


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/ffa81edd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/ffa81edd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/ffa81edd

Branch: refs/heads/gearpump-runner
Commit: ffa81edd0ec4d9a8150280efdb6a6de412114743
Parents: c0c5802
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Dec 1 21:03:04 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Dec 1 21:03:04 2016 -0800

----------------------------------------------------------------------
 runners/apex/pom.xml                       |  6 ++++--
 runners/flink/runner/pom.xml               | 11 ++++++++---
 runners/google-cloud-dataflow-java/pom.xml |  6 ++++--
 runners/spark/pom.xml                      |  6 ++++--
 4 files changed, 20 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ffa81edd/runners/apex/pom.xml
----------------------------------------------------------------------
diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml
index 983781d..629e890 100644
--- a/runners/apex/pom.xml
+++ b/runners/apex/pom.xml
@@ -185,8 +185,10 @@
             </goals>
             <configuration>
               <groups>org.apache.beam.sdk.testing.RunnableOnService</groups>
-              <excludedGroups>org.apache.beam.sdk.testing.UsesStatefulParDo</excludedGroups>
-              <excludedGroups>org.apache.beam.sdk.testing.UsesSplittableParDo</excludedGroups>
+              <excludedGroups>
+                org.apache.beam.sdk.testing.UsesStatefulParDo,
+                org.apache.beam.sdk.testing.UsesSplittableParDo
+              </excludedGroups>
               <parallel>none</parallel>
               <failIfNoTests>true</failIfNoTests>
               <dependenciesToScan>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ffa81edd/runners/flink/runner/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml
index 3e3dd7e..615d5f1 100644
--- a/runners/flink/runner/pom.xml
+++ b/runners/flink/runner/pom.xml
@@ -53,8 +53,10 @@
                 </goals>
                 <configuration>
                   <groups>org.apache.beam.sdk.testing.RunnableOnService</groups>
-                  <excludedGroups>org.apache.beam.sdk.testing.UsesStatefulParDo</excludedGroups>
-                  <excludedGroups>org.apache.beam.sdk.testing.UsesSplittableParDo</excludedGroups>
+                  <excludedGroups>
+                    org.apache.beam.sdk.testing.UsesStatefulParDo,
+                    org.apache.beam.sdk.testing.UsesSplittableParDo
+                  </excludedGroups>
                   <parallel>none</parallel>
                   <failIfNoTests>true</failIfNoTests>
                   <dependenciesToScan>
@@ -80,7 +82,10 @@
                 </goals>
                 <configuration>
                   <groups>org.apache.beam.sdk.testing.RunnableOnService</groups>
-                  <excludedGroups>org.apache.beam.sdk.testing.UsesStatefulParDo</excludedGroups>
+                  <excludedGroups>
+                    org.apache.beam.sdk.testing.UsesStatefulParDo,
+                    org.apache.beam.sdk.testing.UsesSplittableParDo
+                  </excludedGroups>
                   <parallel>none</parallel>
                   <failIfNoTests>true</failIfNoTests>
                   <dependenciesToScan>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ffa81edd/runners/google-cloud-dataflow-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml
index 8547499..adebb2a 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -77,8 +77,10 @@
           <execution>
             <id>runnable-on-service-tests</id>
             <configuration>
-              <excludedGroups>org.apache.beam.sdk.testing.UsesStatefulParDo</excludedGroups>
-              <excludedGroups>org.apache.beam.sdk.testing.UsesSplittableParDo</excludedGroups>
+              <excludedGroups>
+                org.apache.beam.sdk.testing.UsesStatefulParDo,
+                org.apache.beam.sdk.testing.UsesSplittableParDo
+              </excludedGroups>
               <excludes>
                 <exclude>org.apache.beam.sdk.transforms.FlattenTest</exclude>
               </excludes>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ffa81edd/runners/spark/pom.xml
----------------------------------------------------------------------
diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
index dc000bf..e34af15 100644
--- a/runners/spark/pom.xml
+++ b/runners/spark/pom.xml
@@ -72,8 +72,10 @@
                 </goals>
                 <configuration>
                   <groups>org.apache.beam.sdk.testing.RunnableOnService</groups>
-                  <excludedGroups>org.apache.beam.sdk.testing.UsesStatefulParDo</excludedGroups>
-                  <excludedGroups>org.apache.beam.sdk.testing.UsesSplittableParDo</excludedGroups>
+                  <excludedGroups>
+                    org.apache.beam.sdk.testing.UsesStatefulParDo,
+                    org.apache.beam.sdk.testing.UsesSplittableParDo
+                  </excludedGroups>
                   <forkCount>1</forkCount>
                   <reuseForks>false</reuseForks>
                   <failIfNoTests>true</failIfNoTests>


[23/50] [abbrv] incubator-beam git commit: Add timerId to TimerData

Posted by ke...@apache.org.
Add timerId to TimerData

This timerId is generated to be identical to historical behavior, and
to be unique per time domain and timestamp.


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/840fb3b9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/840fb3b9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/840fb3b9

Branch: refs/heads/gearpump-runner
Commit: 840fb3b9030908ef50937cc2e4498a2cdcb7b680
Parents: a088449
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Nov 23 14:30:57 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Dec 2 13:28:57 2016 -0800

----------------------------------------------------------------------
 .../operators/ApexGroupByKeyOperator.java       |   4 +-
 .../runners/direct/DirectTimerInternals.java    |   4 +-
 .../apache/beam/sdk/util/TimerInternals.java    | 107 +++++++------------
 .../sdk/util/state/InMemoryTimerInternals.java  |   8 +-
 .../beam/sdk/util/TimerInternalsTest.java       |   4 +-
 5 files changed, 49 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/840fb3b9/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java
----------------------------------------------------------------------
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java
index 3b0e4f2..f49c785 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java
@@ -426,8 +426,8 @@ public class ApexGroupByKeyOperator<K, V> implements Operator {
   public class ApexTimerInternals implements TimerInternals {
 
     @Override
-    public void setTimer(TimerData timerKey) {
-      registerActiveTimer(context.element().key(), timerKey);
+    public void setTimer(TimerData timerData) {
+      registerActiveTimer(context.element().key(), timerData);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/840fb3b9/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
index 4245a87..8970b4b 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
@@ -53,8 +53,8 @@ class DirectTimerInternals implements TimerInternals {
   }
 
   @Override
-  public void setTimer(TimerData timerKey) {
-    timerUpdateBuilder.setTimer(timerKey);
+  public void setTimer(TimerData timerData) {
+    timerUpdateBuilder.setTimer(timerData);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/840fb3b9/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java
index 5d4a72d..c3e498e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java
@@ -18,18 +18,16 @@
 package org.apache.beam.sdk.util;
 
 import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.MoreObjects;
+import com.google.auto.value.AutoValue;
 import com.google.common.collect.ComparisonChain;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.Arrays;
 import java.util.List;
-import java.util.Objects;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
@@ -50,7 +48,7 @@ import org.joda.time.Instant;
 public interface TimerInternals {
 
   /**
-   * Writes out a timer to be fired when the current time in the specified time domain reaches the
+   * Sets a timer to be fired when the current time in the specified time domain reaches the
    * target timestamp.
    *
    * <p>The combination of {@code namespace} and {@code timerId} uniquely identify a timer.
@@ -63,14 +61,9 @@ public interface TimerInternals {
   void setTimer(StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain);
 
   /**
-   * Writes out a timer to be fired when the watermark reaches the given timestamp, automatically
-   * generating an id for it from the provided {@link TimerData}.
-   *
-   * <p>The {@link TimerData} contains all the fields necessary to set the timer. The timer's ID
-   * is determinstically generated from the {@link TimerData}, so it may be canceled using
-   * the same {@link TimerData}.
+   * Sets the timer described by {@code timerData}.
    */
-  void setTimer(TimerData timerKey);
+  void setTimer(TimerData timerData);
 
   /**
    * Deletes the given timer.
@@ -78,7 +71,7 @@ public interface TimerInternals {
   void deleteTimer(StateNamespace namespace, String timerId);
 
   /**
-   * Deletes the given timer, automatically inferring its ID from the {@link TimerData}.
+   * Deletes the timer with the ID contained in the provided {@link TimerData}.
    */
   void deleteTimer(TimerData timerKey);
 
@@ -163,64 +156,38 @@ public interface TimerInternals {
   /**
    * Data about a timer as represented within {@link TimerInternals}.
    */
-  class TimerData implements Comparable<TimerData> {
-    private final StateNamespace namespace;
-    private final Instant timestamp;
-    private final TimeDomain domain;
+  @AutoValue
+  abstract class TimerData implements Comparable<TimerData> {
 
-    private TimerData(StateNamespace namespace, Instant timestamp, TimeDomain domain) {
-      this.namespace = checkNotNull(namespace);
-      this.timestamp = checkNotNull(timestamp);
-      this.domain = checkNotNull(domain);
-    }
+    public abstract String getTimerId();
 
-    public StateNamespace getNamespace() {
-      return namespace;
-    }
+    public abstract StateNamespace getNamespace();
 
-    public Instant getTimestamp() {
-      return timestamp;
-    }
+    public abstract Instant getTimestamp();
 
-    public TimeDomain getDomain() {
-      return domain;
-    }
+    public abstract TimeDomain getDomain();
 
     /**
-     * Construct the {@code TimerKey} for the given parameters.
+     * Construct a {@link TimerData} for the given parameters, where the timer ID is automatically
+     * generated.
      */
-    public static TimerData of(StateNamespace namespace, Instant timestamp, TimeDomain domain) {
-      return new TimerData(namespace, timestamp, domain);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (this == obj) {
-        return true;
-      }
-
-      if (!(obj instanceof TimerData)) {
-        return false;
-      }
-
-      TimerData that = (TimerData) obj;
-      return Objects.equals(this.domain, that.domain)
-          && this.timestamp.isEqual(that.timestamp)
-          && Objects.equals(this.namespace, that.namespace);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(domain, timestamp, namespace);
+    public static TimerData of(
+        String timerId, StateNamespace namespace, Instant timestamp, TimeDomain domain) {
+      return new AutoValue_TimerInternals_TimerData(timerId, namespace, timestamp, domain);
     }
 
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(getClass())
-          .add("namespace", namespace)
-          .add("timestamp", timestamp)
-          .add("domain", domain)
-          .toString();
+    /**
+     * Construct a {@link TimerData} for the given parameters, where the timer ID is
+     * deterministically generated from the {@code timestamp} and {@code domain}.
+     */
+    public static TimerData of(StateNamespace namespace, Instant timestamp, TimeDomain domain) {
+      String timerId =
+          new StringBuilder()
+              .append(domain.ordinal())
+              .append(':')
+              .append(timestamp.getMillis())
+              .toString();
+      return of(timerId, namespace, timestamp, domain);
     }
 
     /**
@@ -236,11 +203,11 @@ public interface TimerInternals {
       }
       ComparisonChain chain =
           ComparisonChain.start()
-              .compare(this.timestamp, that.getTimestamp())
-              .compare(this.domain, that.domain);
-      if (chain.result() == 0 && !this.namespace.equals(that.namespace)) {
+              .compare(this.getTimestamp(), that.getTimestamp())
+              .compare(this.getDomain(), that.getDomain());
+      if (chain.result() == 0 && !this.getNamespace().equals(that.getNamespace())) {
         // Obtaining the stringKey may be expensive; only do so if required
-        chain = chain.compare(namespace.stringKey(), that.namespace.stringKey());
+        chain = chain.compare(getNamespace().stringKey(), that.getNamespace().stringKey());
       }
       return chain.result();
     }
@@ -275,20 +242,22 @@ public interface TimerInternals {
     public void encode(TimerData timer, OutputStream outStream, Context context)
         throws CoderException, IOException {
       Context nestedContext = context.nested();
-      STRING_CODER.encode(timer.namespace.stringKey(), outStream, nestedContext);
-      INSTANT_CODER.encode(timer.timestamp, outStream, nestedContext);
-      STRING_CODER.encode(timer.domain.name(), outStream, nestedContext);
+      STRING_CODER.encode(timer.getTimerId(), outStream, nestedContext);
+      STRING_CODER.encode(timer.getNamespace().stringKey(), outStream, nestedContext);
+      INSTANT_CODER.encode(timer.getTimestamp(), outStream, nestedContext);
+      STRING_CODER.encode(timer.getDomain().name(), outStream, nestedContext);
     }
 
     @Override
     public TimerData decode(InputStream inStream, Context context)
         throws CoderException, IOException {
       Context nestedContext = context.nested();
+      String timerId = STRING_CODER.decode(inStream, nestedContext);
       StateNamespace namespace =
           StateNamespaces.fromString(STRING_CODER.decode(inStream, nestedContext), windowCoder);
       Instant timestamp = INSTANT_CODER.decode(inStream, nestedContext);
       TimeDomain domain = TimeDomain.valueOf(STRING_CODER.decode(inStream, nestedContext));
-      return TimerData.of(namespace, timestamp, domain);
+      return TimerData.of(timerId, namespace, timestamp, domain);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/840fb3b9/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java
index a3bb45a..60a90f5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/InMemoryTimerInternals.java
@@ -104,10 +104,10 @@ public class InMemoryTimerInternals implements TimerInternals {
   }
 
   @Override
-  public void setTimer(TimerData timer) {
-    WindowTracing.trace("TestTimerInternals.setTimer: {}", timer);
-    if (existingTimers.add(timer)) {
-      queue(timer.getDomain()).add(timer);
+  public void setTimer(TimerData timerData) {
+    WindowTracing.trace("TestTimerInternals.setTimer: {}", timerData);
+    if (existingTimers.add(timerData)) {
+      queue(timerData.getDomain()).add(timerData);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/840fb3b9/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java
index e8ffdb3..7b56f1c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TimerInternalsTest.java
@@ -45,12 +45,14 @@ public class TimerInternalsTest {
   public void testTimerDataCoder() throws Exception {
     CoderProperties.coderDecodeEncodeEqual(
         TimerDataCoder.of(GlobalWindow.Coder.INSTANCE),
-        TimerData.of(StateNamespaces.global(), new Instant(0), TimeDomain.EVENT_TIME));
+        TimerData.of(
+            "arbitrary-id", StateNamespaces.global(), new Instant(0), TimeDomain.EVENT_TIME));
 
     Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
     CoderProperties.coderDecodeEncodeEqual(
         TimerDataCoder.of(windowCoder),
         TimerData.of(
+            "another-id",
             StateNamespaces.window(
                 windowCoder, new IntervalWindow(new Instant(0), new Instant(100))),
             new Instant(99), TimeDomain.PROCESSING_TIME));


[19/50] [abbrv] incubator-beam git commit: Move Towards removing WindowedValue from SDK

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java
index ef501d4..7df2f89 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java
@@ -23,10 +23,10 @@ import static org.junit.Assert.assertThat;
 
 import com.google.common.collect.ImmutableList;
 import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
-import org.apache.beam.sdk.util.WindowedValue;
 import org.joda.time.Instant;
 import org.junit.Rule;
 import org.junit.Test;
@@ -34,32 +34,33 @@ import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * Tests for {@link PaneExtractors}.
- */
+/** Tests for {@link PaneExtractors}. */
 @RunWith(JUnit4.class)
 public class PaneExtractorsTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
   @Test
   public void onlyPaneNoFiring() {
-    SerializableFunction<Iterable<WindowedValue<Integer>>, Iterable<Integer>> extractor =
+    SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor =
         PaneExtractors.onlyPane();
-    Iterable<WindowedValue<Integer>> noFiring =
+    Iterable<ValueInSingleWindow<Integer>> noFiring =
         ImmutableList.of(
-            WindowedValue.valueInGlobalWindow(9), WindowedValue.valueInEmptyWindows(19));
+            ValueInSingleWindow.of(
+                9, BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING),
+            ValueInSingleWindow.of(
+                19, BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
     assertThat(extractor.apply(noFiring), containsInAnyOrder(9, 19));
   }
 
   @Test
   public void onlyPaneOnlyOneFiring() {
-    SerializableFunction<Iterable<WindowedValue<Integer>>, Iterable<Integer>> extractor =
+    SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor =
         PaneExtractors.onlyPane();
-    Iterable<WindowedValue<Integer>> onlyFiring =
+    Iterable<ValueInSingleWindow<Integer>> onlyFiring =
         ImmutableList.of(
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 1, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING));
 
     assertThat(extractor.apply(onlyFiring), containsInAnyOrder(2, 1));
@@ -67,21 +68,21 @@ public class PaneExtractorsTest {
 
   @Test
   public void onlyPaneMultiplePanesFails() {
-    SerializableFunction<Iterable<WindowedValue<Integer>>, Iterable<Integer>> extractor =
+    SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor =
         PaneExtractors.onlyPane();
-    Iterable<WindowedValue<Integer>> multipleFiring =
+    Iterable<ValueInSingleWindow<Integer>> multipleFiring =
         ImmutableList.of(
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 4,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(true, false, Timing.EARLY)),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 2,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 1,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
@@ -94,16 +95,16 @@ public class PaneExtractorsTest {
 
   @Test
   public void onTimePane() {
-    SerializableFunction<Iterable<WindowedValue<Integer>>, Iterable<Integer>> extractor =
+    SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor =
         PaneExtractors.onTimePane();
-    Iterable<WindowedValue<Integer>> onlyOnTime =
+    Iterable<ValueInSingleWindow<Integer>> onlyOnTime =
         ImmutableList.of(
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 4,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 2,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
@@ -114,26 +115,26 @@ public class PaneExtractorsTest {
 
   @Test
   public void onTimePaneOnlyEarlyAndLate() {
-    SerializableFunction<Iterable<WindowedValue<Integer>>, Iterable<Integer>> extractor =
+    SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor =
         PaneExtractors.onTimePane();
-    Iterable<WindowedValue<Integer>> onlyOnTime =
+    Iterable<ValueInSingleWindow<Integer>> onlyOnTime =
         ImmutableList.of(
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 8,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L)),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 4,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 2,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 1,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
@@ -144,21 +145,21 @@ public class PaneExtractorsTest {
 
   @Test
   public void finalPane() {
-    SerializableFunction<Iterable<WindowedValue<Integer>>, Iterable<Integer>> extractor =
+    SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor =
         PaneExtractors.finalPane();
-    Iterable<WindowedValue<Integer>> onlyOnTime =
+    Iterable<ValueInSingleWindow<Integer>> onlyOnTime =
         ImmutableList.of(
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 8,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(false, true, Timing.LATE, 2L, 1L)),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 4,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 1,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
@@ -169,21 +170,21 @@ public class PaneExtractorsTest {
 
   @Test
   public void finalPaneNoExplicitFinalEmpty() {
-    SerializableFunction<Iterable<WindowedValue<Integer>>, Iterable<Integer>> extractor =
+    SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor =
         PaneExtractors.finalPane();
-    Iterable<WindowedValue<Integer>> onlyOnTime =
+    Iterable<ValueInSingleWindow<Integer>> onlyOnTime =
         ImmutableList.of(
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 8,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L)),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 4,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 1,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
@@ -194,15 +195,15 @@ public class PaneExtractorsTest {
 
   @Test
   public void nonLatePanesSingleOnTime() {
-    SerializableFunction<Iterable<WindowedValue<Integer>>, Iterable<Integer>> extractor =
+    SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor =
         PaneExtractors.nonLatePanes();
-    Iterable<WindowedValue<Integer>> onlyOnTime =
+    Iterable<ValueInSingleWindow<Integer>> onlyOnTime =
         ImmutableList.of(
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING));
 
     assertThat(extractor.apply(onlyOnTime), containsInAnyOrder(2, 4, 8));
@@ -210,16 +211,16 @@ public class PaneExtractorsTest {
 
   @Test
   public void nonLatePanesSingleEarly() {
-    SerializableFunction<Iterable<WindowedValue<Integer>>, Iterable<Integer>> extractor =
+    SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor =
         PaneExtractors.nonLatePanes();
-    Iterable<WindowedValue<Integer>> onlyOnTime =
+    Iterable<ValueInSingleWindow<Integer>> onlyOnTime =
         ImmutableList.of(
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 8,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(true, false, Timing.EARLY)),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 4,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
@@ -230,11 +231,11 @@ public class PaneExtractorsTest {
 
   @Test
   public void allPanesSingleLate() {
-    SerializableFunction<Iterable<WindowedValue<Integer>>, Iterable<Integer>> extractor =
+    SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor =
         PaneExtractors.nonLatePanes();
-    Iterable<WindowedValue<Integer>> onlyOnTime =
+    Iterable<ValueInSingleWindow<Integer>> onlyOnTime =
         ImmutableList.of(
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 8,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
@@ -245,22 +246,22 @@ public class PaneExtractorsTest {
 
   @Test
   public void nonLatePanesMultiplePanes() {
-    SerializableFunction<Iterable<WindowedValue<Integer>>, Iterable<Integer>> extractor =
+    SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor =
         PaneExtractors.nonLatePanes();
-    Iterable<WindowedValue<Integer>> onlyOnTime =
+    Iterable<ValueInSingleWindow<Integer>> onlyOnTime =
         ImmutableList.of(
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 8,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L)),
-            WindowedValue.of(7, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING),
-            WindowedValue.of(
+            ValueInSingleWindow.of(7, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING),
+            ValueInSingleWindow.of(
                 4,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 1,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
@@ -271,15 +272,15 @@ public class PaneExtractorsTest {
 
   @Test
   public void allPanesSinglePane() {
-    SerializableFunction<Iterable<WindowedValue<Integer>>, Iterable<Integer>> extractor =
+    SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor =
         PaneExtractors.allPanes();
-    Iterable<WindowedValue<Integer>> onlyOnTime =
+    Iterable<ValueInSingleWindow<Integer>> onlyOnTime =
         ImmutableList.of(
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING));
 
     assertThat(extractor.apply(onlyOnTime), containsInAnyOrder(2, 4, 8));
@@ -287,21 +288,21 @@ public class PaneExtractorsTest {
 
   @Test
   public void allPanesMultiplePanes() {
-    SerializableFunction<Iterable<WindowedValue<Integer>>, Iterable<Integer>> extractor =
+    SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor =
         PaneExtractors.allPanes();
-    Iterable<WindowedValue<Integer>> onlyOnTime =
+    Iterable<ValueInSingleWindow<Integer>> onlyOnTime =
         ImmutableList.of(
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 8,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L)),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 4,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
                 PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)),
-            WindowedValue.of(
+            ValueInSingleWindow.of(
                 1,
                 new Instant(0L),
                 GlobalWindow.INSTANCE,
@@ -312,9 +313,9 @@ public class PaneExtractorsTest {
 
   @Test
   public void allPanesEmpty() {
-    SerializableFunction<Iterable<WindowedValue<Integer>>, Iterable<Integer>> extractor =
+    SerializableFunction<Iterable<ValueInSingleWindow<Integer>>, Iterable<Integer>> extractor =
         PaneExtractors.allPanes();
-    Iterable<WindowedValue<Integer>> noPanes = ImmutableList.of();
+    Iterable<ValueInSingleWindow<Integer>> noPanes = ImmutableList.of();
 
     assertThat(extractor.apply(noPanes), emptyIterable());
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ValueInSingleWindowCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ValueInSingleWindowCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ValueInSingleWindowCoderTest.java
new file mode 100644
index 0000000..daf73b6
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ValueInSingleWindowCoderTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.testing;
+
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link ValueInSingleWindow.Coder}. */
+@RunWith(JUnit4.class)
+public class ValueInSingleWindowCoderTest {
+  @Test
+  public void testDecodeEncodeEqual() throws Exception {
+    Instant now = Instant.now();
+    ValueInSingleWindow<String> value =
+        ValueInSingleWindow.of(
+            "foo",
+            now,
+            new IntervalWindow(now, now.plus(Duration.standardSeconds(10))),
+            PaneInfo.NO_FIRING);
+
+    CoderProperties.coderDecodeEncodeEqual(
+        ValueInSingleWindow.Coder.of(StringUtf8Coder.of(), IntervalWindow.getCoder()), value);
+  }
+
+  @Test
+  public void testCoderSerializable() throws Exception {
+    CoderProperties.coderSerializable(
+        ValueInSingleWindow.Coder.of(StringUtf8Coder.of(), IntervalWindow.getCoder()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java
deleted file mode 100644
index d195623..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java
+++ /dev/null
@@ -1,143 +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.fail;
-
-import com.google.common.collect.Iterables;
-import java.io.Serializable;
-import org.apache.beam.sdk.io.CountingInput;
-import org.apache.beam.sdk.testing.NeedsRunner;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.transforms.Values;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.transforms.WithTimestamps;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.TypeDescriptor;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link GatherAllPanes}.
- */
-@RunWith(JUnit4.class)
-public class GatherAllPanesTest implements Serializable {
-  @Test
-  @Category(NeedsRunner.class)
-  public void singlePaneSingleReifiedPane() {
-    TestPipeline p = TestPipeline.create();
-    PCollection<Iterable<WindowedValue<Iterable<Long>>>> accumulatedPanes =
-        p.apply(CountingInput.upTo(20000))
-            .apply(
-                WithTimestamps.of(
-                    new SerializableFunction<Long, Instant>() {
-                      @Override
-                      public Instant apply(Long input) {
-                        return new Instant(input * 10);
-                      }
-                    }))
-            .apply(
-                Window.<Long>into(FixedWindows.of(Duration.standardMinutes(1)))
-                    .triggering(AfterWatermark.pastEndOfWindow())
-                    .withAllowedLateness(Duration.ZERO)
-                    .discardingFiredPanes())
-            .apply(WithKeys.<Void, Long>of((Void) null).withKeyType(new TypeDescriptor<Void>() {}))
-            .apply(GroupByKey.<Void, Long>create())
-            .apply(Values.<Iterable<Long>>create())
-            .apply(GatherAllPanes.<Iterable<Long>>globally());
-
-    PAssert.that(accumulatedPanes)
-        .satisfies(
-            new SerializableFunction<Iterable<Iterable<WindowedValue<Iterable<Long>>>>, Void>() {
-              @Override
-              public Void apply(Iterable<Iterable<WindowedValue<Iterable<Long>>>> input) {
-                for (Iterable<WindowedValue<Iterable<Long>>> windowedInput : input) {
-                  if (Iterables.size(windowedInput) > 1) {
-                    fail("Expected all windows to have exactly one pane, got " + windowedInput);
-                    return null;
-                  }
-                }
-                return null;
-              }
-            });
-
-    p.run();
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void multiplePanesMultipleReifiedPane() {
-    TestPipeline p = TestPipeline.create();
-
-    PCollection<Long> someElems = p.apply("someLongs", CountingInput.upTo(20000));
-    PCollection<Long> otherElems = p.apply("otherLongs", CountingInput.upTo(20000));
-    PCollection<Iterable<WindowedValue<Iterable<Long>>>> accumulatedPanes =
-        PCollectionList.of(someElems)
-            .and(otherElems)
-            .apply(Flatten.<Long>pCollections())
-            .apply(
-                WithTimestamps.of(
-                    new SerializableFunction<Long, Instant>() {
-                      @Override
-                      public Instant apply(Long input) {
-                        return new Instant(input * 10);
-                      }
-                    }))
-            .apply(
-                Window.<Long>into(FixedWindows.of(Duration.standardMinutes(1)))
-                    .triggering(
-                        AfterWatermark.pastEndOfWindow()
-                            .withEarlyFirings(AfterPane.elementCountAtLeast(1)))
-                    .withAllowedLateness(Duration.ZERO)
-                    .discardingFiredPanes())
-            .apply(WithKeys.<Void, Long>of((Void) null).withKeyType(new TypeDescriptor<Void>() {}))
-            .apply(GroupByKey.<Void, Long>create())
-            .apply(Values.<Iterable<Long>>create())
-            .apply(GatherAllPanes.<Iterable<Long>>globally());
-
-    PAssert.that(accumulatedPanes)
-        .satisfies(
-            new SerializableFunction<Iterable<Iterable<WindowedValue<Iterable<Long>>>>, Void>() {
-              @Override
-              public Void apply(Iterable<Iterable<WindowedValue<Iterable<Long>>>> input) {
-                for (Iterable<WindowedValue<Iterable<Long>>> windowedInput : input) {
-                  if (Iterables.size(windowedInput) > 1) {
-                    return null;
-                  }
-                }
-                fail("Expected at least one window to have multiple panes");
-                return null;
-              }
-            });
-
-    p.run();
-  }
-}


[27/50] [abbrv] incubator-beam git commit: This closes #1487

Posted by ke...@apache.org.
This closes #1487


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/1abbb900
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/1abbb900
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/1abbb900

Branch: refs/heads/gearpump-runner
Commit: 1abbb9007e83fc64f1bb61ff4593f37c6c386545
Parents: 8cb2689 662416a
Author: Thomas Groh <tg...@google.com>
Authored: Fri Dec 2 14:02:25 2016 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Fri Dec 2 14:02:25 2016 -0800

----------------------------------------------------------------------
 .../direct/ConsumerTrackingPipelineVisitor.java | 173 -----------
 .../apache/beam/runners/direct/DirectGraph.java |  89 ++++++
 .../beam/runners/direct/DirectGraphVisitor.java | 145 ++++++++++
 .../beam/runners/direct/DirectRunner.java       |  35 +--
 .../beam/runners/direct/EvaluationContext.java  |  76 ++---
 .../direct/ExecutorServiceParallelExecutor.java |  15 +-
 .../ImmutabilityCheckingBundleFactory.java      |  21 +-
 .../beam/runners/direct/WatermarkManager.java   |  50 ++--
 .../ConsumerTrackingPipelineVisitorTest.java    | 287 -------------------
 .../runners/direct/DirectGraphVisitorTest.java  | 239 +++++++++++++++
 .../runners/direct/EvaluationContextTest.java   |  29 +-
 .../ImmutabilityCheckingBundleFactoryTest.java  |   6 +-
 .../runners/direct/WatermarkManagerTest.java    |  23 +-
 13 files changed, 575 insertions(+), 613 deletions(-)
----------------------------------------------------------------------



[17/50] [abbrv] incubator-beam git commit: travis.yml: disable skipping things that no longer run

Posted by ke...@apache.org.
travis.yml: disable skipping things that no longer run


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/d6eb5143
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/d6eb5143
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/d6eb5143

Branch: refs/heads/gearpump-runner
Commit: d6eb5143b17eca9e5a59eaf6d2e3cd696e8bb38c
Parents: f70fc40
Author: Dan Halperin <dh...@google.com>
Authored: Thu Dec 1 10:04:38 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Dec 2 12:52:58 2016 -0800

----------------------------------------------------------------------
 .travis.yml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d6eb5143/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 9e1406c..a806477 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -30,7 +30,7 @@ notifications:
 
 env:
   global:
-   - MAVEN_OVERRIDE="--settings=.travis/settings.xml -Dmaven.javadoc.skip=true -Dcheckstyle.skip=true -Dfindbugs.skip=true"
+   - MAVEN_OVERRIDE="--settings=.travis/settings.xml"
    - MAVEN_CONTAINER_OVERRIDE="-DbeamSurefireArgline='-Xmx512m'"
 
 matrix:


[02/50] [abbrv] incubator-beam git commit: This closes #1469

Posted by ke...@apache.org.
This closes #1469


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/48130f71
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/48130f71
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/48130f71

Branch: refs/heads/gearpump-runner
Commit: 48130f718d019d6928c464e6f7ad90cd510b62d2
Parents: 0c875ba ab1f1ad
Author: Thomas Groh <tg...@google.com>
Authored: Thu Dec 1 12:55:26 2016 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Thu Dec 1 12:55:26 2016 -0800

----------------------------------------------------------------------
 .../direct/KeyedPValueTrackingVisitor.java      |   2 +-
 .../DataflowPipelineTranslatorTest.java         |   2 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java | 117 +++---------
 .../beam/sdk/runners/TransformHierarchy.java    | 126 ++++++++-----
 .../beam/sdk/runners/TransformTreeNode.java     | 165 +++++++++--------
 .../sdk/runners/TransformHierarchyTest.java     | 180 ++++++++++++++-----
 .../beam/sdk/runners/TransformTreeTest.java     |   4 +-
 7 files changed, 340 insertions(+), 256 deletions(-)
----------------------------------------------------------------------



[32/50] [abbrv] incubator-beam git commit: This closes #1461

Posted by ke...@apache.org.
This closes #1461


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/26eb4354
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/26eb4354
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/26eb4354

Branch: refs/heads/gearpump-runner
Commit: 26eb4354cab72b7c482d8868c99eeb5933fd797e
Parents: e04cd47 86173a8
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Dec 2 15:42:33 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Dec 2 15:42:33 2016 -0800

----------------------------------------------------------------------
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |   6 +-
 .../core/GroupByKeyViaGroupByKeyOnly.java       |  22 +-
 .../core/GroupAlsoByWindowsProperties.java      | 590 +++++++++++--------
 .../beam/sdk/transforms/DoFnAdapters.java       |   2 +
 .../apache/beam/sdk/transforms/DoFnTester.java  | 279 +++++----
 .../sdk/transforms/reflect/DoFnInvokers.java    |  11 -
 .../beam/sdk/transforms/DoFnTesterTest.java     |  38 +-
 7 files changed, 536 insertions(+), 412 deletions(-)
----------------------------------------------------------------------



[14/50] [abbrv] incubator-beam git commit: Closes #1488

Posted by ke...@apache.org.
Closes #1488


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/7ad78779
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/7ad78779
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/7ad78779

Branch: refs/heads/gearpump-runner
Commit: 7ad7877978e94c2b167f12010842e36374400775
Parents: c0c5802 ffa81ed
Author: Dan Halperin <dh...@google.com>
Authored: Thu Dec 1 23:17:33 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Thu Dec 1 23:17:33 2016 -0800

----------------------------------------------------------------------
 runners/apex/pom.xml                       |  6 ++++--
 runners/flink/runner/pom.xml               | 11 ++++++++---
 runners/google-cloud-dataflow-java/pom.xml |  6 ++++--
 runners/spark/pom.xml                      |  6 ++++--
 4 files changed, 20 insertions(+), 9 deletions(-)
----------------------------------------------------------------------



[09/50] [abbrv] incubator-beam git commit: Reduce the visibility of TransformHierarchy Node Mutators

Posted by ke...@apache.org.
Reduce the visibility of TransformHierarchy Node Mutators

These mutators should not be accessible when visiting the nodes.


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/7c5c7910
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/7c5c7910
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/7c5c7910

Branch: refs/heads/gearpump-runner
Commit: 7c5c7910c0b8cba8623a1c49fc24c51ea691dac3
Parents: 569e8d7
Author: Thomas Groh <tg...@google.com>
Authored: Thu Dec 1 13:22:11 2016 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Thu Dec 1 14:34:21 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/runners/TransformHierarchy.java  | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/7c5c7910/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
index 662acc1..e9829cc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
@@ -263,7 +263,7 @@ public class TransformHierarchy {
     /**
      * Adds an output to the transform node.
      */
-    public void setOutput(POutput output) {
+    private void setOutput(POutput output) {
       checkState(!finishedSpecifying);
       checkState(
           this.output == null, "Tried to specify more than one output for %s", getFullName());
@@ -304,13 +304,14 @@ public class TransformHierarchy {
       return AppliedPTransform.of(
           getFullName(), getInput(), getOutput(), (PTransform) getTransform());
     }
+
     /**
      * Visit the transform node.
      *
      * <p>Provides an ordered visit of the input values, the primitive transform (or child nodes for
      * composite transforms), then the output values.
      */
-    public void visit(PipelineVisitor visitor, Set<PValue> visitedValues) {
+    private void visit(PipelineVisitor visitor, Set<PValue> visitedValues) {
       if (!finishedSpecifying) {
         finishSpecifying();
       }
@@ -352,7 +353,7 @@ public class TransformHierarchy {
      *
      * <p>All inputs are finished first, then the transform, then all outputs.
      */
-    public void finishSpecifying() {
+    private void finishSpecifying() {
       if (finishedSpecifying) {
         return;
       }


[11/50] [abbrv] incubator-beam git commit: This closes #1486

Posted by ke...@apache.org.
This closes #1486


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/c0c58022
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/c0c58022
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/c0c58022

Branch: refs/heads/gearpump-runner
Commit: c0c580227c0cd7cecf1d6aff3e2baec2c775a46d
Parents: 63491bf 887b357
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Dec 1 19:26:37 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Dec 1 19:26:37 2016 -0800

----------------------------------------------------------------------
 runners/apex/pom.xml                            |   1 +
 .../beam/runners/direct/SplittableDoFnTest.java | 407 -------------------
 runners/flink/runner/pom.xml                    |   1 +
 runners/google-cloud-dataflow-java/pom.xml      |   1 +
 runners/spark/pom.xml                           |   1 +
 .../beam/sdk/testing/UsesSplittableParDo.java   |  25 ++
 .../beam/sdk/transforms/SplittableDoFnTest.java | 401 ++++++++++++++++++
 7 files changed, 430 insertions(+), 407 deletions(-)
----------------------------------------------------------------------



[26/50] [abbrv] incubator-beam git commit: Stop using Maps of Transforms in the DirectRunner

Posted by ke...@apache.org.
Stop using Maps of Transforms in the DirectRunner

Instead, add a "DirectGraph" class, which adds a layer of indirection to
all lookup methods.

Remove all remaining uses of getProducingTransformInternal, and instead
use DirectGraph methods to obtain the producing transform.


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/8162cd29
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/8162cd29
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/8162cd29

Branch: refs/heads/gearpump-runner
Commit: 8162cd29d97ef307b6fac588f453e4e39d70fca7
Parents: 8cb2689
Author: Thomas Groh <tg...@google.com>
Authored: Thu Dec 1 15:39:30 2016 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Fri Dec 2 14:02:24 2016 -0800

----------------------------------------------------------------------
 .../direct/ConsumerTrackingPipelineVisitor.java | 108 +++++++------------
 .../apache/beam/runners/direct/DirectGraph.java |  89 +++++++++++++++
 .../beam/runners/direct/DirectRunner.java       |  31 +++---
 .../beam/runners/direct/EvaluationContext.java  |  76 ++++---------
 .../direct/ExecutorServiceParallelExecutor.java |  15 +--
 .../ImmutabilityCheckingBundleFactory.java      |  21 ++--
 .../beam/runners/direct/WatermarkManager.java   |  50 ++++-----
 .../ConsumerTrackingPipelineVisitorTest.java    |  98 +++++------------
 .../runners/direct/EvaluationContextTest.java   |  25 ++---
 .../ImmutabilityCheckingBundleFactoryTest.java  |   6 +-
 .../runners/direct/WatermarkManagerTest.java    |  23 ++--
 11 files changed, 252 insertions(+), 290 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8162cd29/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
index acfad16..b9e77c5 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
@@ -19,8 +19,8 @@ package org.apache.beam.runners.direct;
 
 import static com.google.common.base.Preconditions.checkState;
 
-import java.util.ArrayList;
-import java.util.Collection;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ListMultimap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -33,6 +33,7 @@ import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
 
 /**
@@ -41,9 +42,13 @@ import org.apache.beam.sdk.values.PValue;
  * input after the upstream transform has produced and committed output.
  */
 public class ConsumerTrackingPipelineVisitor extends PipelineVisitor.Defaults {
-  private Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers = new HashMap<>();
-  private Collection<AppliedPTransform<?, ?, ?>> rootTransforms = new ArrayList<>();
-  private Collection<PCollectionView<?>> views = new ArrayList<>();
+  private Map<POutput, AppliedPTransform<?, ?, ?>> producers = new HashMap<>();
+
+  private ListMultimap<PInput, AppliedPTransform<?, ?, ?>> primitiveConsumers =
+      ArrayListMultimap.create();
+
+  private Set<PCollectionView<?>> views = new HashSet<>();
+  private Set<AppliedPTransform<?, ?, ?>> rootTransforms = new HashSet<>();
   private Map<AppliedPTransform<?, ?, ?>, String> stepNames = new HashMap<>();
   private Set<PValue> toFinalize = new HashSet<>();
   private int numTransforms = 0;
@@ -81,81 +86,38 @@ public class ConsumerTrackingPipelineVisitor extends PipelineVisitor.Defaults {
       rootTransforms.add(appliedTransform);
     } else {
       for (PValue value : node.getInput().expand()) {
-        valueToConsumers.get(value).add(appliedTransform);
+        primitiveConsumers.put(value, appliedTransform);
       }
     }
   }
 
-  private AppliedPTransform<?, ?, ?> getAppliedTransform(TransformHierarchy.Node node) {
-    @SuppressWarnings({"rawtypes", "unchecked"})
-    AppliedPTransform<?, ?, ?> application = AppliedPTransform.of(
-        node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform());
-    return application;
-  }
-
-  @Override
+ @Override
   public void visitValue(PValue value, TransformHierarchy.Node producer) {
     toFinalize.add(value);
+
+    AppliedPTransform<?, ?, ?> appliedTransform = getAppliedTransform(producer);
+    if (!producers.containsKey(value)) {
+      producers.put(value, appliedTransform);
+    }
     for (PValue expandedValue : value.expand()) {
-      valueToConsumers.put(expandedValue, new ArrayList<AppliedPTransform<?, ?, ?>>());
       if (expandedValue instanceof PCollectionView) {
         views.add((PCollectionView<?>) expandedValue);
       }
-      expandedValue.recordAsOutput(getAppliedTransform(producer));
+      if (!producers.containsKey(expandedValue)) {
+        producers.put(value, appliedTransform);
+      }
     }
-    value.recordAsOutput(getAppliedTransform(producer));
-  }
-
-  private String genStepName() {
-    return String.format("s%s", numTransforms++);
-  }
-
-
-  /**
-   * Returns a mapping of each fully-expanded {@link PValue} to each
-   * {@link AppliedPTransform} that consumes it. For each AppliedPTransform in the collection
-   * returned from {@code getValueToCustomers().get(PValue)},
-   * {@code AppliedPTransform#getInput().expand()} will contain the argument {@link PValue}.
-   */
-  public Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> getValueToConsumers() {
-    checkState(
-        finalized,
-        "Can't call getValueToConsumers before the Pipeline has been completely traversed");
-
-    return valueToConsumers;
   }
 
-  /**
-   * Returns the mapping for each {@link AppliedPTransform} in the {@link Pipeline} to a unique step
-   * name.
-   */
-  public Map<AppliedPTransform<?, ?, ?>, String> getStepNames() {
-    checkState(
-        finalized, "Can't call getStepNames before the Pipeline has been completely traversed");
-
-    return stepNames;
-  }
-
-  /**
-   * Returns the root transforms of the {@link Pipeline}. A root {@link AppliedPTransform} consumes
-   * a {@link PInput} where the {@link PInput#expand()} returns an empty collection.
-   */
-  public Collection<AppliedPTransform<?, ?, ?>> getRootTransforms() {
-    checkState(
-        finalized,
-        "Can't call getRootTransforms before the Pipeline has been completely traversed");
-
-    return rootTransforms;
+  private AppliedPTransform<?, ?, ?> getAppliedTransform(TransformHierarchy.Node node) {
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    AppliedPTransform<?, ?, ?> application = AppliedPTransform.of(
+        node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform());
+    return application;
   }
 
-  /**
-   * Returns all of the {@link PCollectionView PCollectionViews} contained in the visited
-   * {@link Pipeline}.
-   */
-  public Collection<PCollectionView<?>> getViews() {
-    checkState(finalized, "Can't call getViews before the Pipeline has been completely traversed");
-
-    return views;
+  private String genStepName() {
+    return String.format("s%s", numTransforms++);
   }
 
   /**
@@ -163,11 +125,21 @@ public class ConsumerTrackingPipelineVisitor extends PipelineVisitor.Defaults {
    * {@link PValue PValues} should be finalized by the {@link PipelineRunner} before the
    * {@link Pipeline} is executed.
    */
-  public Set<PValue> getUnfinalizedPValues() {
+  public void finishSpecifyingRemainder() {
     checkState(
         finalized,
-        "Can't call getUnfinalizedPValues before the Pipeline has been completely traversed");
+        "Can't call finishSpecifyingRemainder before the Pipeline has been completely traversed");
+    for (PValue unfinalized : toFinalize) {
+      unfinalized.finishSpecifying();
+    }
+  }
 
-    return toFinalize;
+  /**
+   * Get the graph constructed by this {@link ConsumerTrackingPipelineVisitor}, which provides
+   * lookups for producers and consumers of {@link PValue PValues}.
+   */
+  public DirectGraph getGraph() {
+    checkState(finalized, "Can't get a graph before the Pipeline has been completely traversed");
+    return DirectGraph.create(producers, primitiveConsumers, views, rootTransforms, stepNames);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8162cd29/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java
new file mode 100644
index 0000000..f208f6e
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java
@@ -0,0 +1,89 @@
+/*
+ * 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.runners.direct;
+
+import com.google.common.collect.ListMultimap;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.PValue;
+
+/**
+ * Methods for interacting with the underlying structure of a {@link Pipeline} that is being
+ * executed with the {@link DirectRunner}.
+ */
+class DirectGraph {
+  private final Map<POutput, AppliedPTransform<?, ?, ?>> producers;
+  private final ListMultimap<PInput, AppliedPTransform<?, ?, ?>> primitiveConsumers;
+  private final Set<PCollectionView<?>> views;
+
+  private final Set<AppliedPTransform<?, ?, ?>> rootTransforms;
+  private final Map<AppliedPTransform<?, ?, ?>, String> stepNames;
+
+  public static DirectGraph create(
+      Map<POutput, AppliedPTransform<?, ?, ?>> producers,
+      ListMultimap<PInput, AppliedPTransform<?, ?, ?>> primitiveConsumers,
+      Set<PCollectionView<?>> views,
+      Set<AppliedPTransform<?, ?, ?>> rootTransforms,
+      Map<AppliedPTransform<?, ?, ?>, String> stepNames) {
+    return new DirectGraph(producers, primitiveConsumers, views, rootTransforms, stepNames);
+  }
+
+  private DirectGraph(
+      Map<POutput, AppliedPTransform<?, ?, ?>> producers,
+      ListMultimap<PInput, AppliedPTransform<?, ?, ?>> primitiveConsumers,
+      Set<PCollectionView<?>> views,
+      Set<AppliedPTransform<?, ?, ?>> rootTransforms,
+      Map<AppliedPTransform<?, ?, ?>, String> stepNames) {
+    this.producers = producers;
+    this.primitiveConsumers = primitiveConsumers;
+    this.views = views;
+    this.rootTransforms = rootTransforms;
+    this.stepNames = stepNames;
+  }
+
+  public AppliedPTransform<?, ?, ?> getProducer(PValue produced) {
+    return producers.get(produced);
+  }
+
+  public List<AppliedPTransform<?, ?, ?>> getPrimitiveConsumers(PValue consumed) {
+    return primitiveConsumers.get(consumed);
+  }
+
+  public Set<AppliedPTransform<?, ?, ?>> getRootTransforms() {
+    return rootTransforms;
+  }
+
+  public Set<PCollectionView<?>> getViews() {
+    return views;
+  }
+
+  public String getStepName(AppliedPTransform<?, ?, ?> step) {
+    return stepNames.get(step);
+  }
+
+  public Collection<AppliedPTransform<?, ?, ?>> getPrimitiveTransforms() {
+    return stepNames.keySet();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8162cd29/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index 82de9ab..0ad5836 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -62,7 +62,6 @@ import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
-import org.apache.beam.sdk.values.PValue;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 
@@ -198,18 +197,18 @@ public class DirectRunner extends PipelineRunner<DirectPipelineResult> {
   enum Enforcement {
     ENCODABILITY {
       @Override
-      public boolean appliesTo(PTransform<?, ?> transform) {
+      public boolean appliesTo(PCollection<?> collection, DirectGraph graph) {
         return true;
       }
     },
     IMMUTABILITY {
       @Override
-      public boolean appliesTo(PTransform<?, ?> transform) {
-        return CONTAINS_UDF.contains(transform.getClass());
+      public boolean appliesTo(PCollection<?> collection, DirectGraph graph) {
+        return CONTAINS_UDF.contains(graph.getProducer(collection).getTransform().getClass());
       }
     };
 
-    public abstract boolean appliesTo(PTransform<?, ?> transform);
+    public abstract boolean appliesTo(PCollection<?> collection, DirectGraph graph);
 
     ////////////////////////////////////////////////////////////////////////////////////////////////
     // Utilities for creating enforcements
@@ -224,13 +223,13 @@ public class DirectRunner extends PipelineRunner<DirectPipelineResult> {
       return Collections.unmodifiableSet(enabled);
     }
 
-    public static BundleFactory bundleFactoryFor(Set<Enforcement> enforcements) {
+    public static BundleFactory bundleFactoryFor(Set<Enforcement> enforcements, DirectGraph graph) {
       BundleFactory bundleFactory =
           enforcements.contains(Enforcement.ENCODABILITY)
               ? CloningBundleFactory.create()
               : ImmutableListBundleFactory.create();
       if (enforcements.contains(Enforcement.IMMUTABILITY)) {
-        bundleFactory = ImmutabilityCheckingBundleFactory.create(bundleFactory);
+        bundleFactory = ImmutabilityCheckingBundleFactory.create(bundleFactory, graph);
       }
       return bundleFactory;
     }
@@ -301,9 +300,8 @@ public class DirectRunner extends PipelineRunner<DirectPipelineResult> {
     MetricsEnvironment.setMetricsSupported(true);
     ConsumerTrackingPipelineVisitor consumerTrackingVisitor = new ConsumerTrackingPipelineVisitor();
     pipeline.traverseTopologically(consumerTrackingVisitor);
-    for (PValue unfinalized : consumerTrackingVisitor.getUnfinalizedPValues()) {
-      unfinalized.finishSpecifying();
-    }
+    consumerTrackingVisitor.finishSpecifyingRemainder();
+
     @SuppressWarnings("rawtypes")
     KeyedPValueTrackingVisitor keyedPValueVisitor =
         KeyedPValueTrackingVisitor.create(
@@ -315,28 +313,25 @@ public class DirectRunner extends PipelineRunner<DirectPipelineResult> {
 
     DisplayDataValidator.validatePipeline(pipeline);
 
+    DirectGraph graph = consumerTrackingVisitor.getGraph();
     EvaluationContext context =
         EvaluationContext.create(
             getPipelineOptions(),
             clockSupplier.get(),
-            Enforcement.bundleFactoryFor(enabledEnforcements),
-            consumerTrackingVisitor.getRootTransforms(),
-            consumerTrackingVisitor.getValueToConsumers(),
-            consumerTrackingVisitor.getStepNames(),
-            consumerTrackingVisitor.getViews());
+            Enforcement.bundleFactoryFor(enabledEnforcements, graph),
+            graph);
 
     RootProviderRegistry rootInputProvider = RootProviderRegistry.defaultRegistry(context);
     TransformEvaluatorRegistry registry = TransformEvaluatorRegistry.defaultRegistry(context);
     PipelineExecutor executor =
         ExecutorServiceParallelExecutor.create(
-            options.getTargetParallelism(),
-            consumerTrackingVisitor.getValueToConsumers(),
+            options.getTargetParallelism(), graph,
             keyedPValueVisitor.getKeyedPValues(),
             rootInputProvider,
             registry,
             Enforcement.defaultModelEnforcements(enabledEnforcements),
             context);
-    executor.start(consumerTrackingVisitor.getRootTransforms());
+    executor.start(graph.getRootTransforms());
 
     Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps =
         pipeline.getAggregatorSteps();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8162cd29/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
index 201aaed..b5a23d7 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
@@ -74,8 +74,10 @@ import org.joda.time.Instant;
  * can be executed.
  */
 class EvaluationContext {
-  /** The step name for each {@link AppliedPTransform} in the {@link Pipeline}. */
-  private final Map<AppliedPTransform<?, ?, ?>, String> stepNames;
+  /**
+   * The graph representing this {@link Pipeline}.
+   */
+  private final DirectGraph graph;
 
   /** The options that were used to create this {@link Pipeline}. */
   private final DirectOptions options;
@@ -99,36 +101,19 @@ class EvaluationContext {
   private final DirectMetrics metrics;
 
   public static EvaluationContext create(
-      DirectOptions options,
-      Clock clock,
-      BundleFactory bundleFactory,
-      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
-      Map<AppliedPTransform<?, ?, ?>, String> stepNames,
-      Collection<PCollectionView<?>> views) {
-    return new EvaluationContext(
-        options, clock, bundleFactory, rootTransforms, valueToConsumers, stepNames, views);
+      DirectOptions options, Clock clock, BundleFactory bundleFactory, DirectGraph graph) {
+    return new EvaluationContext(options, clock, bundleFactory, graph);
   }
 
   private EvaluationContext(
-      DirectOptions options,
-      Clock clock,
-      BundleFactory bundleFactory,
-      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
-      Map<AppliedPTransform<?, ?, ?>, String> stepNames,
-      Collection<PCollectionView<?>> views) {
+      DirectOptions options, Clock clock, BundleFactory bundleFactory, DirectGraph graph) {
     this.options = checkNotNull(options);
     this.clock = clock;
     this.bundleFactory = checkNotNull(bundleFactory);
-    checkNotNull(rootTransforms);
-    checkNotNull(valueToConsumers);
-    checkNotNull(stepNames);
-    checkNotNull(views);
-    this.stepNames = stepNames;
+    this.graph = checkNotNull(graph);
 
-    this.watermarkManager = WatermarkManager.create(clock, rootTransforms, valueToConsumers);
-    this.sideInputContainer = SideInputContainer.create(this, views);
+    this.watermarkManager = WatermarkManager.create(clock, graph);
+    this.sideInputContainer = SideInputContainer.create(this, graph.getViews());
 
     this.applicationStateInternals = new ConcurrentHashMap<>();
     this.mergedAggregators = AggregatorContainer.create();
@@ -211,7 +196,7 @@ class EvaluationContext {
     ImmutableList.Builder<CommittedBundle<?>> completed = ImmutableList.builder();
     for (UncommittedBundle<?> inProgress : bundles) {
       AppliedPTransform<?, ?, ?> producing =
-          inProgress.getPCollection().getProducingTransformInternal();
+          graph.getProducer(inProgress.getPCollection());
       TransformWatermarks watermarks = watermarkManager.getWatermarks(producing);
       CommittedBundle<?> committed =
           inProgress.commit(watermarks.getSynchronizedProcessingOutputTime());
@@ -225,7 +210,7 @@ class EvaluationContext {
   }
 
   private void fireAllAvailableCallbacks() {
-    for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
+    for (AppliedPTransform<?, ?, ?> transform : graph.getPrimitiveTransforms()) {
       fireAvailableCallbacks(transform);
     }
   }
@@ -290,10 +275,10 @@ class EvaluationContext {
       BoundedWindow window,
       WindowingStrategy<?, ?> windowingStrategy,
       Runnable runnable) {
-    AppliedPTransform<?, ?, ?> producing = getProducing(value);
+    AppliedPTransform<?, ?, ?> producing = graph.getProducer(value);
     callbackExecutor.callOnGuaranteedFiring(producing, window, windowingStrategy, runnable);
 
-    fireAvailableCallbacks(lookupProducing(value));
+    fireAvailableCallbacks(producing);
   }
 
   /**
@@ -311,22 +296,6 @@ class EvaluationContext {
     fireAvailableCallbacks(producing);
   }
 
-  private AppliedPTransform<?, ?, ?> getProducing(PValue value) {
-    if (value.getProducingTransformInternal() != null) {
-      return value.getProducingTransformInternal();
-    }
-    return lookupProducing(value);
-  }
-
-  private AppliedPTransform<?, ?, ?> lookupProducing(PValue value) {
-    for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
-      if (transform.getOutput().equals(value) || transform.getOutput().expand().contains(value)) {
-        return transform;
-      }
-    }
-    return null;
-  }
-
   /**
    * Get the options used by this {@link Pipeline}.
    */
@@ -347,18 +316,17 @@ class EvaluationContext {
         watermarkManager.getWatermarks(application));
   }
 
-  /**
-   * Get all of the steps used in this {@link Pipeline}.
-   */
-  public Collection<AppliedPTransform<?, ?, ?>> getSteps() {
-    return stepNames.keySet();
-  }
 
   /**
    * Get the Step Name for the provided application.
    */
-  public String getStepName(AppliedPTransform<?, ?, ?> application) {
-    return stepNames.get(application);
+  String getStepName(AppliedPTransform<?, ?, ?> application) {
+    return graph.getStepName(application);
+  }
+
+  /** Returns all of the steps in this {@link Pipeline}. */
+  Collection<AppliedPTransform<?, ?, ?>> getSteps() {
+    return graph.getPrimitiveTransforms();
   }
 
   /**
@@ -450,7 +418,7 @@ class EvaluationContext {
    * Returns true if all steps are done.
    */
   public boolean isDone() {
-    for (AppliedPTransform<?, ?, ?> transform : stepNames.keySet()) {
+    for (AppliedPTransform<?, ?, ?> transform : graph.getPrimitiveTransforms()) {
       if (!isDone(transform)) {
         return false;
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8162cd29/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
index b7908c5..929d09d 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
@@ -69,7 +69,7 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor {
   private final int targetParallelism;
   private final ExecutorService executorService;
 
-  private final Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers;
+  private final DirectGraph graph;
   private final Set<PValue> keyedPValues;
   private final RootProviderRegistry rootProviderRegistry;
   private final TransformEvaluatorRegistry registry;
@@ -104,7 +104,7 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor {
 
   public static ExecutorServiceParallelExecutor create(
       int targetParallelism,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
+      DirectGraph graph,
       Set<PValue> keyedPValues,
       RootProviderRegistry rootProviderRegistry,
       TransformEvaluatorRegistry registry,
@@ -114,7 +114,7 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor {
       EvaluationContext context) {
     return new ExecutorServiceParallelExecutor(
         targetParallelism,
-        valueToConsumers,
+        graph,
         keyedPValues,
         rootProviderRegistry,
         registry,
@@ -124,7 +124,7 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor {
 
   private ExecutorServiceParallelExecutor(
       int targetParallelism,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers,
+      DirectGraph graph,
       Set<PValue> keyedPValues,
       RootProviderRegistry rootProviderRegistry,
       TransformEvaluatorRegistry registry,
@@ -133,7 +133,7 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor {
       EvaluationContext context) {
     this.targetParallelism = targetParallelism;
     this.executorService = Executors.newFixedThreadPool(targetParallelism);
-    this.valueToConsumers = valueToConsumers;
+    this.graph = graph;
     this.keyedPValues = keyedPValues;
     this.rootProviderRegistry = rootProviderRegistry;
     this.registry = registry;
@@ -273,8 +273,9 @@ final class ExecutorServiceParallelExecutor implements PipelineExecutor {
         CommittedBundle<?> inputBundle, TransformResult<?> result) {
       CommittedResult committedResult = evaluationContext.handleResult(inputBundle, timers, result);
       for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
-        allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle,
-            valueToConsumers.get(outputBundle.getPCollection())));
+        allUpdates.offer(
+            ExecutorUpdate.fromBundle(
+                outputBundle, graph.getPrimitiveConsumers(outputBundle.getPCollection())));
       }
       CommittedBundle<?> unprocessedInputs = committedResult.getUnprocessedInputs();
       if (unprocessedInputs != null && !Iterables.isEmpty(unprocessedInputs.getElements())) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8162cd29/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
index 4f72f68..8d77e25 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
@@ -46,17 +46,20 @@ import org.joda.time.Instant;
  */
 class ImmutabilityCheckingBundleFactory implements BundleFactory {
   /**
-   * Create a new {@link ImmutabilityCheckingBundleFactory} that uses the underlying
-   * {@link BundleFactory} to create the output bundle.
+   * Create a new {@link ImmutabilityCheckingBundleFactory} that uses the underlying {@link
+   * BundleFactory} to create the output bundle.
    */
-  public static ImmutabilityCheckingBundleFactory create(BundleFactory underlying) {
-    return new ImmutabilityCheckingBundleFactory(underlying);
+  public static ImmutabilityCheckingBundleFactory create(
+      BundleFactory underlying, DirectGraph graph) {
+    return new ImmutabilityCheckingBundleFactory(underlying, graph);
   }
 
   private final BundleFactory underlying;
+  private final DirectGraph graph;
 
-  private ImmutabilityCheckingBundleFactory(BundleFactory underlying) {
+  private ImmutabilityCheckingBundleFactory(BundleFactory underlying, DirectGraph graph) {
     this.underlying = checkNotNull(underlying);
+    this.graph = graph;
   }
 
   /**
@@ -72,7 +75,7 @@ class ImmutabilityCheckingBundleFactory implements BundleFactory {
 
   @Override
   public <T> UncommittedBundle<T> createBundle(PCollection<T> output) {
-    if (Enforcement.IMMUTABILITY.appliesTo(output.getProducingTransformInternal().getTransform())) {
+    if (Enforcement.IMMUTABILITY.appliesTo(output, graph)) {
       return new ImmutabilityEnforcingBundle<>(underlying.createBundle(output));
     }
     return underlying.createBundle(output);
@@ -81,13 +84,13 @@ class ImmutabilityCheckingBundleFactory implements BundleFactory {
   @Override
   public <K, T> UncommittedBundle<T> createKeyedBundle(
       StructuralKey<K> key, PCollection<T> output) {
-    if (Enforcement.IMMUTABILITY.appliesTo(output.getProducingTransformInternal().getTransform())) {
+    if (Enforcement.IMMUTABILITY.appliesTo(output, graph)) {
       return new ImmutabilityEnforcingBundle<>(underlying.createKeyedBundle(key, output));
     }
     return underlying.createKeyedBundle(key, output);
   }
 
-  private static class ImmutabilityEnforcingBundle<T> implements UncommittedBundle<T> {
+  private class ImmutabilityEnforcingBundle<T> implements UncommittedBundle<T> {
     private final UncommittedBundle<T> underlying;
     private final SetMultimap<WindowedValue<T>, MutationDetector> mutationDetectors;
     private Coder<T> coder;
@@ -125,7 +128,7 @@ class ImmutabilityCheckingBundleFactory implements BundleFactory {
                 String.format(
                     "PTransform %s mutated value %s after it was output (new value was %s)."
                         + " Values must not be mutated in any way after being output.",
-                    underlying.getPCollection().getProducingTransformInternal().getFullName(),
+                    graph.getProducer(underlying.getPCollection()).getFullName(),
                     exn.getSavedValue(),
                     exn.getNewValue()),
                 exn.getSavedValue(),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8162cd29/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
index a53c11c..247b1cc 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
@@ -669,10 +669,10 @@ public class WatermarkManager {
   private final Clock clock;
 
   /**
-   * A map from each {@link PCollection} to all {@link AppliedPTransform PTransform applications}
-   * that consume that {@link PCollection}.
+   * The {@link DirectGraph} representing the {@link Pipeline} this {@link WatermarkManager} tracks
+   * watermarks for.
    */
-  private final Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers;
+  private final DirectGraph graph;
 
   /**
    * The input and output watermark of each {@link AppliedPTransform}.
@@ -697,27 +697,21 @@ public class WatermarkManager {
   private final Set<AppliedPTransform<?, ?, ?>> pendingRefreshes;
 
   /**
-   * Creates a new {@link WatermarkManager}. All watermarks within the newly created
-   * {@link WatermarkManager} start at {@link BoundedWindow#TIMESTAMP_MIN_VALUE}, the
-   * minimum watermark, with no watermark holds or pending elements.
+   * Creates a new {@link WatermarkManager}. All watermarks within the newly created {@link
+   * WatermarkManager} start at {@link BoundedWindow#TIMESTAMP_MIN_VALUE}, the minimum watermark,
+   * with no watermark holds or pending elements.
    *
-   * @param rootTransforms the root-level transforms of the {@link Pipeline}
-   * @param consumers a mapping between each {@link PCollection} in the {@link Pipeline} to the
-   *                  transforms that consume it as a part of their input
+   * @param clock the clock to use to determine processing time
+   * @param graph the graph representing this pipeline
    */
-  public static WatermarkManager create(
-      Clock clock,
-      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers) {
-    return new WatermarkManager(clock, rootTransforms, consumers);
+  public static WatermarkManager create(Clock clock, DirectGraph graph) {
+    return new WatermarkManager(clock, graph);
   }
 
-  private WatermarkManager(
-      Clock clock,
-      Collection<AppliedPTransform<?, ?, ?>> rootTransforms,
-      Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> consumers) {
+  private WatermarkManager(Clock clock, DirectGraph graph) {
     this.clock = clock;
-    this.consumers = consumers;
+    this.graph = graph;
+
     this.pendingUpdates = new ConcurrentLinkedQueue<>();
 
     this.refreshLock = new ReentrantLock();
@@ -725,13 +719,11 @@ public class WatermarkManager {
 
     transformToWatermarks = new HashMap<>();
 
-    for (AppliedPTransform<?, ?, ?> rootTransform : rootTransforms) {
+    for (AppliedPTransform<?, ?, ?> rootTransform : graph.getRootTransforms()) {
       getTransformWatermark(rootTransform);
     }
-    for (Collection<AppliedPTransform<?, ?, ?>> intermediateTransforms : consumers.values()) {
-      for (AppliedPTransform<?, ?, ?> transform : intermediateTransforms) {
-        getTransformWatermark(transform);
-      }
+    for (AppliedPTransform<?, ?, ?> primitiveTransform : graph.getPrimitiveTransforms()) {
+      getTransformWatermark(primitiveTransform);
     }
   }
 
@@ -769,8 +761,7 @@ public class WatermarkManager {
     }
     for (PValue pvalue : inputs) {
       Watermark producerOutputWatermark =
-          getTransformWatermark(pvalue.getProducingTransformInternal())
-              .synchronizedProcessingOutputWatermark;
+          getTransformWatermark(graph.getProducer(pvalue)).synchronizedProcessingOutputWatermark;
       inputWmsBuilder.add(producerOutputWatermark);
     }
     return inputWmsBuilder.build();
@@ -784,7 +775,7 @@ public class WatermarkManager {
     }
     for (PValue pvalue : inputs) {
       Watermark producerOutputWatermark =
-          getTransformWatermark(pvalue.getProducingTransformInternal()).outputWatermark;
+          getTransformWatermark(graph.getProducer(pvalue)).outputWatermark;
       inputWatermarksBuilder.add(producerOutputWatermark);
     }
     List<Watermark> inputCollectionWatermarks = inputWatermarksBuilder.build();
@@ -920,7 +911,8 @@ public class WatermarkManager {
     // do not share a Mutex within this call and thus can be interleaved with external calls to
     // refresh.
     for (CommittedBundle<?> bundle : result.getOutputs()) {
-      for (AppliedPTransform<?, ?, ?> consumer : consumers.get(bundle.getPCollection())) {
+      for (AppliedPTransform<?, ?, ?> consumer :
+          graph.getPrimitiveConsumers(bundle.getPCollection())) {
         TransformWatermarks watermarks = transformToWatermarks.get(consumer);
         watermarks.addPending(bundle);
       }
@@ -968,7 +960,7 @@ public class WatermarkManager {
     if (updateResult.isAdvanced()) {
       Set<AppliedPTransform<?, ?, ?>> additionalRefreshes = new HashSet<>();
       for (PValue outputPValue : toRefresh.getOutput().expand()) {
-        additionalRefreshes.addAll(consumers.get(outputPValue));
+        additionalRefreshes.addAll(graph.getPrimitiveConsumers(outputPValue));
       }
       return additionalRefreshes;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8162cd29/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
index f7f4b71..02fe007 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitorTest.java
@@ -18,6 +18,8 @@
 package org.apache.beam.runners.direct;
 
 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 java.io.Serializable;
@@ -36,7 +38,6 @@ import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PDone;
 import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.PValue;
 import org.hamcrest.Matchers;
 import org.junit.Rule;
 import org.junit.Test;
@@ -72,7 +73,7 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
         p.apply("singletonCreate", Create.<Object>of(1, 2, 3)).apply(View.<Object>asSingleton());
     p.traverseTopologically(visitor);
     assertThat(
-        visitor.getViews(),
+        visitor.getGraph().getViews(),
         Matchers.<PCollectionView<?>>containsInAnyOrder(listView, singletonView));
   }
 
@@ -83,7 +84,7 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
     PCollection<Long> unCounted = p.apply(CountingInput.unbounded());
     p.traverseTopologically(visitor);
     assertThat(
-        visitor.getRootTransforms(),
+        visitor.getGraph().getRootTransforms(),
         Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
             created.getProducingTransformInternal(),
             counted.getProducingTransformInternal(),
@@ -96,7 +97,7 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
         PCollectionList.<String>empty(p).apply(Flatten.<String>pCollections());
     p.traverseTopologically(visitor);
     assertThat(
-        visitor.getRootTransforms(),
+        visitor.getGraph().getRootTransforms(),
         Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
             empty.getProducingTransformInternal()));
   }
@@ -121,15 +122,15 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
     p.traverseTopologically(visitor);
 
     assertThat(
-        visitor.getValueToConsumers().get(created),
+        visitor.getGraph().getPrimitiveConsumers(created),
         Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
             transformed.getProducingTransformInternal(),
             flattened.getProducingTransformInternal()));
     assertThat(
-        visitor.getValueToConsumers().get(transformed),
+        visitor.getGraph().getPrimitiveConsumers(transformed),
         Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
             flattened.getProducingTransformInternal()));
-    assertThat(visitor.getValueToConsumers().get(flattened), emptyIterable());
+    assertThat(visitor.getGraph().getPrimitiveConsumers(flattened), emptyIterable());
   }
 
   @Test
@@ -142,11 +143,11 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
     p.traverseTopologically(visitor);
 
     assertThat(
-        visitor.getValueToConsumers().get(created),
+        visitor.getGraph().getPrimitiveConsumers(created),
         Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
             flattened.getProducingTransformInternal(),
             flattened.getProducingTransformInternal()));
-    assertThat(visitor.getValueToConsumers().get(flattened), emptyIterable());
+    assertThat(visitor.getGraph().getPrimitiveConsumers(flattened), emptyIterable());
   }
 
   @Test
@@ -163,32 +164,11 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
                   }
                 }));
 
-    p.traverseTopologically(visitor);
-    assertThat(visitor.getUnfinalizedPValues(), Matchers.<PValue>contains(transformed));
-  }
-
-  @Test
-  public void getUnfinalizedPValuesEmpty() {
-    p.apply(Create.of("1", "2", "3"))
-        .apply(
-            ParDo.of(
-                new DoFn<String, String>() {
-                  @ProcessElement
-                  public void processElement(DoFn<String, String>.ProcessContext c)
-                      throws Exception {
-                    c.output(Integer.toString(c.element().length()));
-                  }
-                }))
-        .apply(
-            new PTransform<PInput, PDone>() {
-              @Override
-              public PDone apply(PInput input) {
-                return PDone.in(input.getPipeline());
-              }
-            });
+    assertThat(transformed.isFinishedSpecifyingInternal(), is(false));
 
     p.traverseTopologically(visitor);
-    assertThat(visitor.getUnfinalizedPValues(), emptyIterable());
+    visitor.finishSpecifyingRemainder();
+    assertThat(transformed.isFinishedSpecifyingInternal(), is(true));
   }
 
   @Test
@@ -214,18 +194,12 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
             });
 
     p.traverseTopologically(visitor);
-    assertThat(
-        visitor.getStepNames(),
-        Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
-            created.getProducingTransformInternal(), "s0"));
-    assertThat(
-        visitor.getStepNames(),
-        Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
-            transformed.getProducingTransformInternal(), "s1"));
-    assertThat(
-        visitor.getStepNames(),
-        Matchers.<AppliedPTransform<?, ?, ?>, String>hasEntry(
-            finished.getProducingTransformInternal(), "s2"));
+    DirectGraph graph = visitor.getGraph();
+    assertThat(graph.getStepName(graph.getProducer(created)), equalTo("s0"));
+    assertThat(graph.getStepName(graph.getProducer(transformed)), equalTo("s1"));
+    // finished doesn't have a producer, because it's not a PValue.
+    // TODO: Demonstrate that PCollectionList/Tuple and other composite PValues are either safe to
+    // use, or make them so.
   }
 
   @Test
@@ -248,40 +222,18 @@ public class ConsumerTrackingPipelineVisitorTest implements Serializable {
   }
 
   @Test
-  public void getRootTransformsWithoutVisitingThrows() {
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("completely traversed");
-    thrown.expectMessage("getRootTransforms");
-    visitor.getRootTransforms();
-  }
-  @Test
-  public void getStepNamesWithoutVisitingThrows() {
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("completely traversed");
-    thrown.expectMessage("getStepNames");
-    visitor.getStepNames();
-  }
-  @Test
-  public void getUnfinalizedPValuesWithoutVisitingThrows() {
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage("completely traversed");
-    thrown.expectMessage("getUnfinalizedPValues");
-    visitor.getUnfinalizedPValues();
-  }
-
-  @Test
-  public void getValueToConsumersWithoutVisitingThrows() {
+  public void getGraphWithoutVisitingThrows() {
     thrown.expect(IllegalStateException.class);
     thrown.expectMessage("completely traversed");
-    thrown.expectMessage("getValueToConsumers");
-    visitor.getValueToConsumers();
+    thrown.expectMessage("get a graph");
+    visitor.getGraph();
   }
 
   @Test
-  public void getViewsWithoutVisitingThrows() {
+  public void finishSpecifyingRemainderWithoutVisitingThrows() {
     thrown.expect(IllegalStateException.class);
     thrown.expectMessage("completely traversed");
-    thrown.expectMessage("getViews");
-    visitor.getViews();
+    thrown.expectMessage("finishSpecifyingRemainder");
+    visitor.finishSpecifyingRemainder();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8162cd29/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
index 9a3959d..1c2bf14 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
@@ -29,7 +29,6 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Map;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext;
@@ -67,7 +66,6 @@ import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;
 import org.junit.Before;
@@ -87,10 +85,9 @@ public class EvaluationContextTest {
   private PCollection<KV<String, Integer>> downstream;
   private PCollectionView<Iterable<Integer>> view;
   private PCollection<Long> unbounded;
-  private Collection<AppliedPTransform<?, ?, ?>> rootTransforms;
-  private Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers;
 
   private BundleFactory bundleFactory;
+  private DirectGraph graph;
 
   @Before
   public void setup() {
@@ -106,20 +103,12 @@ public class EvaluationContextTest {
 
     ConsumerTrackingPipelineVisitor cVis = new ConsumerTrackingPipelineVisitor();
     p.traverseTopologically(cVis);
-    rootTransforms = cVis.getRootTransforms();
-    valueToConsumers = cVis.getValueToConsumers();
 
     bundleFactory = ImmutableListBundleFactory.create();
-
+    graph = cVis.getGraph();
     context =
         EvaluationContext.create(
-            runner.getPipelineOptions(),
-            NanosOffsetClock.create(),
-            ImmutableListBundleFactory.create(),
-            rootTransforms,
-            valueToConsumers,
-            cVis.getStepNames(),
-            cVis.getViews());
+            runner.getPipelineOptions(), NanosOffsetClock.create(), bundleFactory, graph);
   }
 
   @Test
@@ -427,13 +416,13 @@ public class EvaluationContextTest {
   @Test
   public void isDoneWithUnboundedPCollectionAndNotShutdown() {
     context.getPipelineOptions().setShutdownUnboundedProducersWithMaxWatermark(false);
-    assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false));
+    assertThat(context.isDone(graph.getProducer(unbounded)), is(false));
 
     context.handleResult(
         null,
         ImmutableList.<TimerData>of(),
-        StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
-    assertThat(context.isDone(unbounded.getProducingTransformInternal()), is(false));
+        StepTransformResult.withoutHold(graph.getProducer(unbounded)).build());
+    assertThat(context.isDone(graph.getProducer(unbounded)), is(false));
   }
 
   @Test
@@ -472,7 +461,7 @@ public class EvaluationContextTest {
         StepTransformResult.withoutHold(unbounded.getProducingTransformInternal()).build());
     assertThat(context.isDone(), is(false));
 
-    for (AppliedPTransform<?, ?, ?> consumers : valueToConsumers.get(created)) {
+    for (AppliedPTransform<?, ?, ?> consumers : graph.getPrimitiveConsumers(created)) {
       context.handleResult(
           committedBundle,
           ImmutableList.<TimerData>of(),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8162cd29/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
index ea44125..e7e1e62 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
@@ -56,7 +56,11 @@ public class ImmutabilityCheckingBundleFactoryTest {
     TestPipeline p = TestPipeline.create();
     created = p.apply(Create.<byte[]>of().withCoder(ByteArrayCoder.of()));
     transformed = created.apply(ParDo.of(new IdentityDoFn<byte[]>()));
-    factory = ImmutabilityCheckingBundleFactory.create(ImmutableListBundleFactory.create());
+    ConsumerTrackingPipelineVisitor visitor = new ConsumerTrackingPipelineVisitor();
+    p.traverseTopologically(visitor);
+    factory =
+        ImmutabilityCheckingBundleFactory.create(
+            ImmutableListBundleFactory.create(), visitor.getGraph());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8162cd29/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
index 2e8ab84..5cde4d6 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
@@ -94,6 +94,7 @@ public class WatermarkManagerTest implements Serializable {
 
   private transient WatermarkManager manager;
   private transient BundleFactory bundleFactory;
+  private DirectGraph graph;
 
   @Before
   public void setup() {
@@ -139,8 +140,11 @@ public class WatermarkManagerTest implements Serializable {
     consumers.put(flattened, Collections.<AppliedPTransform<?, ?, ?>>emptyList());
 
     clock = MockClock.fromInstant(new Instant(1000));
+    ConsumerTrackingPipelineVisitor visitor = new ConsumerTrackingPipelineVisitor();
+    p.traverseTopologically(visitor);
+    graph = visitor.getGraph();
 
-    manager = WatermarkManager.create(clock, rootTransforms, consumers);
+    manager = WatermarkManager.create(clock, graph);
     bundleFactory = ImmutableListBundleFactory.create();
   }
 
@@ -305,20 +309,13 @@ public class WatermarkManagerTest implements Serializable {
     PCollection<Integer> created = p.apply(Create.of(1, 2, 3));
     PCollection<Integer> multiConsumer =
         PCollectionList.of(created).and(created).apply(Flatten.<Integer>pCollections());
-    AppliedPTransform<?, ?, ?> theFlatten = multiConsumer.getProducingTransformInternal();
+    ConsumerTrackingPipelineVisitor trackingVisitor = new ConsumerTrackingPipelineVisitor();
+    p.traverseTopologically(trackingVisitor);
+    DirectGraph graph = trackingVisitor.getGraph();
 
-    Map<PValue, Collection<AppliedPTransform<?, ?, ?>>> valueToConsumers =
-        ImmutableMap.<PValue, Collection<AppliedPTransform<?, ?, ?>>>builder()
-            .put(created, ImmutableList.<AppliedPTransform<?, ?, ?>>of(theFlatten, theFlatten))
-            .put(multiConsumer, Collections.<AppliedPTransform<?, ?, ?>>emptyList())
-            .build();
+    AppliedPTransform<?, ?, ?> theFlatten = graph.getProducer(multiConsumer);
 
-    WatermarkManager tstMgr =
-        WatermarkManager.create(
-            clock,
-            Collections.<AppliedPTransform<?, ?, ?>>singleton(
-                created.getProducingTransformInternal()),
-            valueToConsumers);
+    WatermarkManager tstMgr = WatermarkManager.create(clock, graph);
     CommittedBundle<Void> root =
         bundleFactory
             .<Void>createRootBundle()


[42/50] [abbrv] incubator-beam git commit: Create java8-examples archetype module

Posted by ke...@apache.org.
Create java8-examples archetype module

This archetype module is automatically generated during
the build process, and its dependencies and tests verified.


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/9cb8b5fa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/9cb8b5fa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/9cb8b5fa

Branch: refs/heads/gearpump-runner
Commit: 9cb8b5fa4c2275b2dda94483575a3aaf04dae34d
Parents: 6489b6d
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Nov 10 14:47:56 2016 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Sat Dec 3 16:12:00 2016 -0800

----------------------------------------------------------------------
 .gitignore                                      |   3 +
 .../examples-java8/generate-sources.sh          |  82 ++++++++
 .../maven-archetypes/examples-java8/pom.xml     | 177 ++++++++++++++++
 .../META-INF/maven/archetype-metadata.xml       |  39 ++++
 .../main/resources/archetype-resources/pom.xml  | 209 +++++++++++++++++++
 .../projects/basic/archetype.properties         |  21 ++
 .../src/test/resources/projects/basic/goal.txt  |   1 +
 .../main/resources/archetype-resources/pom.xml  |  19 +-
 sdks/java/maven-archetypes/pom.xml              |  17 ++
 9 files changed, 567 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9cb8b5fa/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 0340d8a..fcfeafc 100644
--- a/.gitignore
+++ b/.gitignore
@@ -6,6 +6,9 @@
 target/
 bin/
 
+# Ignore generated archetypes
+sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/src/
+
 # Ignore IntelliJ files.
 .idea/
 *.iml

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9cb8b5fa/sdks/java/maven-archetypes/examples-java8/generate-sources.sh
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples-java8/generate-sources.sh b/sdks/java/maven-archetypes/examples-java8/generate-sources.sh
new file mode 100755
index 0000000..7b85e85
--- /dev/null
+++ b/sdks/java/maven-archetypes/examples-java8/generate-sources.sh
@@ -0,0 +1,82 @@
+#!/bin/bash -ex
+#
+#    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.
+#
+
+# Updates the examples-java8 archetype to have selection of our examples
+# for use in walkthroughs, etc.
+#
+# Usage: Invoke with no arguments from any working directory.
+
+# The directory of this script. Assumes root of the maven-archetypes module.
+HERE="$(dirname $0)"
+
+# The directory of the examples-java and examples-java8 modules
+EXAMPLES_ROOT="${HERE}/../../../../examples/java"
+JAVA8_EXAMPLES_ROOT="${HERE}/../../../../examples/java8"
+
+# The root of the examples archetype
+ARCHETYPE_ROOT="${HERE}/src/main/resources/archetype-resources"
+
+mkdir -p "$ARCHETYPE_ROOT/src/main/java"
+mkdir -p "$ARCHETYPE_ROOT/src/test/java"
+
+#
+# Copy the Java 7 subset of the examples project verbatim. 
+#
+rsync -a --exclude cookbook --exclude complete                  \
+    "${EXAMPLES_ROOT}"/src/main/java/org/apache/beam/examples/  \
+    "${ARCHETYPE_ROOT}/src/main/java"
+
+rsync -a --exclude cookbook --exclude complete --exclude '*IT.java'  \
+    "${EXAMPLES_ROOT}"/src/test/java/org/apache/beam/examples/        \
+    "${ARCHETYPE_ROOT}/src/test/java"
+
+#
+# Copy in MinimalWordCountJava8 and mobile gaming example
+#
+rsync -a                                                              \
+    "${JAVA8_EXAMPLES_ROOT}"/src/main/java/org/apache/beam/examples/  \
+    "${ARCHETYPE_ROOT}/src/main/java"
+
+rsync -a                                                              \
+    "${JAVA8_EXAMPLES_ROOT}"/src/test/java/org/apache/beam/examples/  \
+    "${ARCHETYPE_ROOT}/src/test/java"
+
+
+#
+# Replace 'package org.apache.beam.examples' with 'package ${package}' in all Java code
+#
+find "${ARCHETYPE_ROOT}/src/main/java" -name '*.java' -print0 \
+    | xargs -0 sed -i.bak 's/^package org\.apache\.beam\.examples/package ${package}/g'
+
+find "${ARCHETYPE_ROOT}/src/test/java" -name '*.java' -print0 \
+    | xargs -0 sed -i.bak 's/^package org\.apache\.beam\.examples/package ${package}/g'
+
+#
+# Replace 'import org.apache.beam.examples.' with 'import ${package}.' in all Java code
+#
+find "${ARCHETYPE_ROOT}/src/main/java" -name '*.java' -print0 \
+    | xargs -0 sed -i.bak 's/^import org\.apache\.beam\.examples/import ${package}/g'
+
+find "${ARCHETYPE_ROOT}/src/test/java" -name '*.java' -print0 \
+    | xargs -0 sed -i.bak 's/^import org\.apache\.beam\.examples/import ${package}/g'
+
+#
+# The use of -i.bak is necessary for the above to work with both GNU and BSD sed. 
+# Delete the files now.
+#
+find "${ARCHETYPE_ROOT}/src" -name '*.bak' -delete

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9cb8b5fa/sdks/java/maven-archetypes/examples-java8/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples-java8/pom.xml b/sdks/java/maven-archetypes/examples-java8/pom.xml
new file mode 100644
index 0000000..5cf094e
--- /dev/null
+++ b/sdks/java/maven-archetypes/examples-java8/pom.xml
@@ -0,0 +1,177 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-sdks-java-maven-archetypes-parent</artifactId>
+    <version>0.4.0-incubating-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-sdks-java-maven-archetypes-examples-java8</artifactId>
+  <name>Apache Beam :: SDKs :: Java :: Maven Archetypes :: Examples - Java 8</name>
+  <description>A Maven Archetype to create a project containing
+    example pipelines from the Apache Beam Java SDK, targeting Java 8.
+  </description>
+
+  <packaging>maven-archetype</packaging>
+
+  <build>
+    <extensions>
+      <extension>
+        <groupId>org.apache.maven.archetype</groupId>
+        <artifactId>archetype-packaging</artifactId>
+        <version>2.4</version>
+      </extension>
+    </extensions>
+
+    <pluginManagement>
+      <plugins>
+        <plugin>
+          <artifactId>maven-archetype-plugin</artifactId>
+          <version>2.4</version>
+          <executions>
+            <!-- archetype-packaging above binds this plugin's goals as follows:
+                     archetype:jar to the package phase,
+                     archetype:integration-test to the integration-test phase,
+                     archetype:update-local-catalog to the install phase.
+                 We defer the integration-test goal to the install phase, since
+                 this test actually depends on the core SDK to be installed. -->
+            <execution>
+              <id>default-integration-test</id>
+              <phase>install</phase>
+              <goals>
+                <goal>integration-test</goal>
+              </goals>
+            </execution>
+          </executions>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+
+    <plugins>
+      <plugin>
+        <artifactId>exec-maven-plugin</artifactId>
+        <groupId>org.codehaus.mojo</groupId>
+        <executions>
+          <execution>
+            <id>generate-archetype-contents</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <executable>${project.basedir}/generate-sources.sh</executable>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <!--
+    These are the dependencies of the generated code. This enables
+    the include-what-you-use analysis to work on the archetype.
+
+    In alphabetical order by groupId, then artifactId.
+  -->
+  <dependencies>
+    <dependency>
+      <groupId>com.google.api-client</groupId>
+      <artifactId>google-api-client</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-bigquery</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-pubsub</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.http-client</groupId>
+      <artifactId>google-http-client</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.oauth-client</groupId>
+      <artifactId>google-oauth-client</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-core</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-direct-java</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-google-cloud-dataflow-java</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9cb8b5fa/sdks/java/maven-archetypes/examples-java8/src/main/resources/META-INF/maven/archetype-metadata.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples-java8/src/main/resources/META-INF/maven/archetype-metadata.xml b/sdks/java/maven-archetypes/examples-java8/src/main/resources/META-INF/maven/archetype-metadata.xml
new file mode 100644
index 0000000..dbdd614
--- /dev/null
+++ b/sdks/java/maven-archetypes/examples-java8/src/main/resources/META-INF/maven/archetype-metadata.xml
@@ -0,0 +1,39 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<archetype-descriptor
+    xsi:schemaLocation="http://maven.apache.org/plugins/maven-archetype-plugin/archetype-descriptor/1.0.0 http://maven.apache.org/xsd/archetype-descriptor-1.0.0.xsd"
+    name="Google Cloud Dataflow Example Pipelines Archetype"
+    xmlns="http://maven.apache.org/plugins/maven-archetype-plugin/archetype-descriptor/1.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+  <fileSets>
+    <fileSet filtered="true" packaged="true" encoding="UTF-8">
+      <directory>src/main/java</directory>
+      <includes>
+        <include>**/*.java</include>
+      </includes>
+    </fileSet>
+
+    <fileSet filtered="true" packaged="true" encoding="UTF-8">
+      <directory>src/test/java</directory>
+      <includes>
+        <include>**/*.java</include>
+      </includes>
+    </fileSet>
+  </fileSets>
+</archetype-descriptor>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9cb8b5fa/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml
new file mode 100644
index 0000000..ece8412
--- /dev/null
+++ b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml
@@ -0,0 +1,209 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <groupId>${groupId}</groupId>
+  <artifactId>${artifactId}</artifactId>
+  <version>${version}</version>
+
+  <packaging>jar</packaging>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>3.5.1</version>
+        <configuration>
+          <source>1.8</source>
+          <target>1.8</target>
+        </configuration>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>2.19.1</version>
+        <configuration>
+          <parallel>all</parallel>
+          <threadCount>4</threadCount>
+          <redirectTestOutputToFile>true</redirectTestOutputToFile>
+        </configuration>
+        <dependencies>
+          <dependency>
+            <groupId>org.apache.maven.surefire</groupId>
+            <artifactId>surefire-junit47</artifactId>
+            <version>2.19.1</version>
+          </dependency>
+        </dependencies>
+      </plugin>
+    </plugins>
+
+    <pluginManagement>
+      <plugins>
+        <plugin>
+          <groupId>org.codehaus.mojo</groupId>
+          <artifactId>exec-maven-plugin</artifactId>
+          <version>1.4.0</version>
+          <configuration>
+            <cleanupDaemonThreads>false</cleanupDaemonThreads>
+          </configuration>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+  </build>
+
+  <dependencies>
+    <!-- Adds a dependency on a specific version of the Beam SDK. -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-core</artifactId>
+      <version>0.4.0-incubating-SNAPSHOT</version>
+    </dependency>
+
+    <!-- Adds a dependency on a specific version of the Dataflow runnner. -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-direct-java</artifactId>
+      <version>0.4.0-incubating-SNAPSHOT</version>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-google-cloud-dataflow-java</artifactId>
+      <version>0.4.0-incubating-SNAPSHOT</version>
+      <scope>runtime</scope>
+    </dependency>
+
+    <!-- Adds a dependency on a specific version of the Beam Google Cloud Platform IO module. -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
+      <version>0.4.0-incubating-SNAPSHOT</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.api-client</groupId>
+      <artifactId>google-api-client</artifactId>
+      <version>1.22.0</version>
+      <exclusions>
+        <!-- Exclude an old version of guava that is being pulled
+             in by a transitive dependency of google-api-client -->
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava-jdk5</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!-- Dependencies below this line are specific dependencies needed by the examples code. -->
+    <dependency>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-bigquery</artifactId>
+      <version>v2-rev295-1.22.0</version>
+      <exclusions>
+        <!-- Exclude an old version of guava that is being pulled
+             in by a transitive dependency of google-api-client -->
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava-jdk5</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.http-client</groupId>
+      <artifactId>google-http-client</artifactId>
+      <version>1.22.0</version>
+      <exclusions>
+        <!-- Exclude an old version of guava that is being pulled
+             in by a transitive dependency of google-api-client -->
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava-jdk5</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-pubsub</artifactId>
+      <version>v1-rev10-1.22.0</version>
+      <exclusions>
+        <!-- Exclude an old version of guava that is being pulled
+             in by a transitive dependency of google-api-client -->
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava-jdk5</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+      <version>2.4</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <version>19.0</version>
+    </dependency>
+
+    <!-- Add slf4j API frontend binding with JUL backend -->
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+      <version>1.7.14</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-jdk14</artifactId>
+      <version>1.7.14</version>
+      <!-- When loaded at runtime this will wire up slf4j to the JUL backend -->
+      <scope>runtime</scope>
+    </dependency>
+
+    <!-- Hamcrest and JUnit are required dependencies of PAssert,
+         which is used in the main code of DebuggingWordCount example. -->
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+      <version>1.3</version>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>4.11</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>1.9.5</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9cb8b5fa/sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/archetype.properties
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/archetype.properties b/sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/archetype.properties
new file mode 100644
index 0000000..ee81f8f
--- /dev/null
+++ b/sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/archetype.properties
@@ -0,0 +1,21 @@
+#
+#    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=it.pkg
+version=0.1
+groupId=archetype.it
+artifactId=basic
+targetPlatform=1.8

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9cb8b5fa/sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/goal.txt
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/goal.txt b/sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/goal.txt
new file mode 100644
index 0000000..0b59873
--- /dev/null
+++ b/sdks/java/maven-archetypes/examples-java8/src/test/resources/projects/basic/goal.txt
@@ -0,0 +1 @@
+verify

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9cb8b5fa/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
index df2e9f3..48f56fd 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
@@ -85,7 +85,24 @@
             <cleanupDaemonThreads>false</cleanupDaemonThreads>
           </configuration>
         </plugin>
-      </plugins>
+ 
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <version>2.10</version>
+        <executions>
+          <execution>
+            <goals><goal>analyze-only</goal></goals>
+            <configuration>
+              <!-- Ignore runtime-only dependencies in analysis -->
+              <ignoreNonCompile>true</ignoreNonCompile>
+              <failOnWarning>true</failOnWarning>
+            </configuration>
+          </execution>
+        </executions>
+       </plugin>
+     </plugins>
+
     </pluginManagement>
   </build>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9cb8b5fa/sdks/java/maven-archetypes/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/pom.xml b/sdks/java/maven-archetypes/pom.xml
index 02bb150..1b1c319 100644
--- a/sdks/java/maven-archetypes/pom.xml
+++ b/sdks/java/maven-archetypes/pom.xml
@@ -31,6 +31,23 @@
 
   <name>Apache Beam :: SDKs :: Java :: Maven Archetypes</name>
 
+  <profiles>
+    <profile>
+      <id>beam-sdks-java-archetypes-examples-java8</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+        <os>
+          <family>unix</family>
+        </os>
+        <jdk>[1.8,)</jdk>
+        <property><name>!beam.sdks.java.archetypes.examples.skip</name></property>
+      </activation>
+      <modules>
+        <module>examples-java8</module>
+      </modules>
+    </profile>
+  </profiles>
+
   <modules>
     <module>starter</module>
     <module>examples</module>


[30/50] [abbrv] incubator-beam git commit: Closes #1489

Posted by ke...@apache.org.
Closes #1489


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/e04cd47d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/e04cd47d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/e04cd47d

Branch: refs/heads/gearpump-runner
Commit: e04cd47ddf8fb5f04f1f684219724031179a55ec
Parents: 1abbb90 e3dca4c
Author: Dan Halperin <dh...@google.com>
Authored: Fri Dec 2 15:20:17 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Dec 2 15:20:17 2016 -0800

----------------------------------------------------------------------
 .../beam/examples/cookbook/DeDupExample.java    | 96 --------------------
 .../beam/examples/cookbook/DistinctExample.java | 96 ++++++++++++++++++++
 2 files changed, 96 insertions(+), 96 deletions(-)
----------------------------------------------------------------------



[47/50] [abbrv] incubator-beam git commit: [BEAM-1057] This closes #1444

Posted by ke...@apache.org.
[BEAM-1057] This closes #1444


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/ef9871c3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/ef9871c3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/ef9871c3

Branch: refs/heads/gearpump-runner
Commit: ef9871c36f24cde537c4067357f534afa4a920a9
Parents: c22b97d 58916b9
Author: Jean-Baptiste Onofr� <jb...@apache.org>
Authored: Sun Dec 4 07:21:28 2016 +0100
Committer: Jean-Baptiste Onofr� <jb...@apache.org>
Committed: Sun Dec 4 07:21:28 2016 +0100

----------------------------------------------------------------------
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java | 26 +++++++++++++++++++-
 1 file changed, 25 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[22/50] [abbrv] incubator-beam git commit: This closes #1477

Posted by ke...@apache.org.
This closes #1477


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/37e891fe
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/37e891fe
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/37e891fe

Branch: refs/heads/gearpump-runner
Commit: 37e891fe92e51477c4d92a2781a9475f13873a7f
Parents: a088449 840fb3b
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Dec 2 13:28:57 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Dec 2 13:28:57 2016 -0800

----------------------------------------------------------------------
 .../operators/ApexGroupByKeyOperator.java       |   4 +-
 .../runners/direct/DirectTimerInternals.java    |   4 +-
 .../apache/beam/sdk/util/TimerInternals.java    | 107 +++++++------------
 .../sdk/util/state/InMemoryTimerInternals.java  |   8 +-
 .../beam/sdk/util/TimerInternalsTest.java       |   4 +-
 5 files changed, 49 insertions(+), 78 deletions(-)
----------------------------------------------------------------------



[15/50] [abbrv] incubator-beam git commit: Closes #1483

Posted by ke...@apache.org.
Closes #1483


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/f70fc409
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/f70fc409
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/f70fc409

Branch: refs/heads/gearpump-runner
Commit: f70fc40992b4ded37ca77c44dc2569666936b30d
Parents: 7ad7877 8fd520c
Author: Dan Halperin <dh...@google.com>
Authored: Fri Dec 2 09:17:11 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Dec 2 09:17:11 2016 -0800

----------------------------------------------------------------------
 .../org/apache/beam/runners/dataflow/DataflowRunner.java     | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[25/50] [abbrv] incubator-beam git commit: This closes #1490

Posted by ke...@apache.org.
This closes #1490


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/8cb2689f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/8cb2689f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/8cb2689f

Branch: refs/heads/gearpump-runner
Commit: 8cb2689f8952a73a4e855a03f98c1d5bec8181fb
Parents: 37e891f b4ee8b7
Author: Thomas Groh <tg...@google.com>
Authored: Fri Dec 2 13:58:39 2016 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Fri Dec 2 13:58:39 2016 -0800

----------------------------------------------------------------------
 .../runners/direct/TransformExecutorTest.java   | 184 ++++++++++---------
 1 file changed, 97 insertions(+), 87 deletions(-)
----------------------------------------------------------------------



[08/50] [abbrv] incubator-beam git commit: This closes #1484

Posted by ke...@apache.org.
This closes #1484


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/63491bf2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/63491bf2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/63491bf2

Branch: refs/heads/gearpump-runner
Commit: 63491bf211d8b2f23f6b7db1375a733c4332b850
Parents: 24fab9f 7c5c791
Author: Thomas Groh <tg...@google.com>
Authored: Thu Dec 1 14:34:21 2016 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Thu Dec 1 14:34:21 2016 -0800

----------------------------------------------------------------------
 .../translation/ApexPipelineTranslator.java     |  12 +-
 .../apex/translation/TranslationContext.java    |   6 +-
 .../direct/ConsumerTrackingPipelineVisitor.java |  12 +-
 .../runners/direct/DisplayDataValidator.java    |   6 +-
 .../direct/KeyedPValueTrackingVisitor.java      |  10 +-
 .../apache/beam/runners/flink/FlinkRunner.java  |  12 +-
 .../FlinkBatchPipelineTranslator.java           |  14 +-
 .../FlinkStreamingPipelineTranslator.java       |  16 +-
 .../PipelineTranslationOptimizer.java           |  10 +-
 .../dataflow/DataflowPipelineTranslator.java    |   8 +-
 .../beam/runners/dataflow/DataflowRunner.java   |  10 +-
 .../runners/dataflow/DataflowRunnerTest.java    |   4 +-
 .../dataflow/RecordingPipelineVisitor.java      |   6 +-
 .../apache/beam/runners/spark/SparkRunner.java  |  21 +-
 .../beam/sdk/AggregatorPipelineExtractor.java   |   6 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java |  17 +-
 .../beam/sdk/runners/TransformHierarchy.java    | 244 +++++++++++++++-
 .../beam/sdk/runners/TransformTreeNode.java     | 282 -------------------
 .../sdk/AggregatorPipelineExtractorTest.java    |  20 +-
 .../sdk/runners/TransformHierarchyTest.java     |  26 +-
 .../beam/sdk/runners/TransformTreeTest.java     |   8 +-
 .../display/DisplayDataEvaluator.java           |   8 +-
 22 files changed, 344 insertions(+), 414 deletions(-)
----------------------------------------------------------------------



[21/50] [abbrv] incubator-beam git commit: This closes #1260

Posted by ke...@apache.org.
This closes #1260


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/a0884492
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/a0884492
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/a0884492

Branch: refs/heads/gearpump-runner
Commit: a0884492a5c037a7fe58a1f011db560f9c228ae0
Parents: 0fb5610 d989123
Author: Thomas Groh <tg...@google.com>
Authored: Fri Dec 2 13:16:22 2016 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Fri Dec 2 13:16:22 2016 -0800

----------------------------------------------------------------------
 .../beam/runners/core/ReduceFnRunnerTest.java   |   5 +-
 .../beam/runners/core/SplittableParDoTest.java  |  38 ++--
 .../beam/runners/core/WindowMatchers.java       | 204 +++++++++++++++++++
 .../beam/runners/core/WindowMatchersTest.java   |  82 ++++++++
 .../direct/WindowEvaluatorFactoryTest.java      |   4 +-
 .../apache/beam/sdk/testing/GatherAllPanes.java |  88 ++++++++
 .../org/apache/beam/sdk/testing/PAssert.java    |  77 +++----
 .../apache/beam/sdk/testing/PaneExtractors.java |  55 +++--
 .../beam/sdk/testing/ValueInSingleWindow.java   | 134 ++++++++++++
 .../apache/beam/sdk/transforms/DoFnTester.java  |  58 +++---
 .../apache/beam/sdk/util/GatherAllPanes.java    |  86 --------
 .../apache/beam/sdk/util/IdentityWindowFn.java  |   2 +-
 .../org/apache/beam/sdk/WindowMatchers.java     | 204 -------------------
 .../org/apache/beam/sdk/WindowMatchersTest.java |  82 --------
 .../beam/sdk/testing/GatherAllPanesTest.java    | 140 +++++++++++++
 .../beam/sdk/testing/PaneExtractorsTest.java    | 133 ++++++------
 .../testing/ValueInSingleWindowCoderTest.java   |  51 +++++
 .../beam/sdk/util/GatherAllPanesTest.java       | 143 -------------
 18 files changed, 893 insertions(+), 693 deletions(-)
----------------------------------------------------------------------



[49/50] [abbrv] incubator-beam git commit: [BEAM-79] fix gearpump runner build failure

Posted by ke...@apache.org.
[BEAM-79] fix gearpump runner build failure


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/2afc0cd9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/2afc0cd9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/2afc0cd9

Branch: refs/heads/gearpump-runner
Commit: 2afc0cd99e33bc724345a2e5b0498820d05b460c
Parents: 86414c0
Author: manuzhang <ow...@gmail.com>
Authored: Tue Dec 6 11:28:24 2016 +0800
Committer: manuzhang <ow...@gmail.com>
Committed: Tue Dec 6 13:04:01 2016 +0800

----------------------------------------------------------------------
 runners/gearpump/pom.xml                        |  9 ++-
 .../gearpump/GearpumpPipelineTranslator.java    | 12 ++--
 .../translators/TranslationContext.java         |  4 +-
 .../gearpump/translators/io/ValuesSource.java   | 71 ++++++++++----------
 4 files changed, 47 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2afc0cd9/runners/gearpump/pom.xml
----------------------------------------------------------------------
diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml
index abd135f..04bd724 100644
--- a/runners/gearpump/pom.xml
+++ b/runners/gearpump/pom.xml
@@ -170,11 +170,6 @@
       <artifactId>beam-runners-core-java</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.google.code.findbugs</groupId>
-      <artifactId>annotations</artifactId>
-      <version>3.0.1</version>
-    </dependency>
-    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
     </dependency>
@@ -199,6 +194,10 @@
       <artifactId>guava</artifactId>
     </dependency>
     <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.gearpump</groupId>
       <artifactId>gearpump-shaded-metrics-graphite_2.11</artifactId>
       <version>${gearpump.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2afc0cd9/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java
index 8588fff..84dfeec 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java
@@ -31,7 +31,7 @@ import org.apache.beam.runners.gearpump.translators.TransformTranslator;
 import org.apache.beam.runners.gearpump.translators.TranslationContext;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.Flatten;
 import org.apache.beam.sdk.transforms.GroupByKey;
@@ -48,7 +48,7 @@ import org.slf4j.LoggerFactory;
  * into Gearpump {@link Graph}.
  */
 @SuppressWarnings({"rawtypes", "unchecked"})
-public class GearpumpPipelineTranslator implements Pipeline.PipelineVisitor {
+public class GearpumpPipelineTranslator extends Pipeline.PipelineVisitor.Defaults {
 
   private static final Logger LOG = LoggerFactory.getLogger(
       GearpumpPipelineTranslator.class);
@@ -83,18 +83,18 @@ public class GearpumpPipelineTranslator implements Pipeline.PipelineVisitor {
   }
 
   @Override
-  public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
     LOG.debug("entering composite transform {}", node.getTransform());
     return CompositeBehavior.ENTER_TRANSFORM;
   }
 
   @Override
-  public void leaveCompositeTransform(TransformTreeNode node) {
+  public void leaveCompositeTransform(TransformHierarchy.Node node) {
     LOG.debug("leaving composite transform {}", node.getTransform());
   }
 
   @Override
-  public void visitPrimitiveTransform(TransformTreeNode node) {
+  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
     LOG.debug("visiting transform {}", node.getTransform());
     PTransform transform = node.getTransform();
     TransformTranslator translator = getTransformTranslator(transform.getClass());
@@ -107,7 +107,7 @@ public class GearpumpPipelineTranslator implements Pipeline.PipelineVisitor {
   }
 
   @Override
-  public void visitValue(PValue value, TransformTreeNode producer) {
+  public void visitValue(PValue value, TransformHierarchy.Node producer) {
     LOG.debug("visiting value {}", value);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2afc0cd9/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java
index d3bc75d..d9d6a8e 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java
@@ -24,7 +24,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.beam.runners.gearpump.GearpumpPipelineOptions;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PInput;
@@ -53,7 +53,7 @@ public class TranslationContext {
 
   }
 
-  public void setCurrentTransform(TransformTreeNode treeNode) {
+  public void setCurrentTransform(TransformHierarchy.Node treeNode) {
     this.currentTransform = AppliedPTransform.of(treeNode.getFullName(),
         treeNode.getInput(), treeNode.getOutput(), (PTransform) treeNode.getTransform());
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2afc0cd9/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java
----------------------------------------------------------------------
diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java
index 9359e35..3b67f09 100644
--- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java
+++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java
@@ -21,16 +21,14 @@ package org.apache.beam.runners.gearpump.translators.io;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.Serializable;
 import java.util.Collections;
 import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
 import java.util.NoSuchElementException;
 
 import javax.annotation.Nullable;
 
 import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
 import org.apache.beam.sdk.io.UnboundedSource;
 import org.apache.beam.sdk.options.PipelineOptions;
 
@@ -41,26 +39,33 @@ import org.joda.time.Instant;
  */
 public class ValuesSource<T> extends UnboundedSource<T, UnboundedSource.CheckpointMark> {
 
-  private final Iterable<byte[]> values;
-  private final Coder<T> coder;
+  private final byte[] values;
+  private final IterableCoder<T> iterableCoder;
 
   public ValuesSource(Iterable<T> values, Coder<T> coder) {
-    this.values = encode(values, coder);
-    this.coder = coder;
+    this.iterableCoder = IterableCoder.of(coder);
+    this.values = encode(values, iterableCoder);
   }
 
-  private Iterable<byte[]> encode(Iterable<T> values, Coder<T> coder) {
-    List<byte[]> bytes = new LinkedList<>();
-    for (T t: values) {
-      try {
-        ByteArrayOutputStream stream = new ByteArrayOutputStream();
-        coder.encode(t, stream, Coder.Context.OUTER);
-        bytes.add(stream.toByteArray());
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
+  private byte[] encode(Iterable<T> values, IterableCoder<T> coder) {
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    try {
+      coder.encode(values, stream, Coder.Context.OUTER);
+    } catch (IOException ex) {
+      throw new RuntimeException(ex);
+    }
+    return stream.toByteArray();
+  }
+
+  private Iterable<T> decode(byte[] bytes) throws IOException{
+    ByteArrayInputStream inputStream = new ByteArrayInputStream(bytes);
+    try {
+      return iterableCoder.decode(inputStream, Coder.Context.OUTER);
+    } catch (IOException ex) {
+      throw new RuntimeException(ex);
+    } finally {
+      inputStream.close();
     }
-    return bytes;
   }
 
   @Override
@@ -72,7 +77,11 @@ public class ValuesSource<T> extends UnboundedSource<T, UnboundedSource.Checkpoi
   @Override
   public UnboundedReader<T> createReader(PipelineOptions options,
       @Nullable CheckpointMark checkpointMark) {
-    return new ValuesReader<>(values, coder, this);
+    try {
+      return new ValuesReader<>(decode(values), iterableCoder, this);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
   }
 
   @Nullable
@@ -87,32 +96,22 @@ public class ValuesSource<T> extends UnboundedSource<T, UnboundedSource.Checkpoi
 
   @Override
   public Coder<T> getDefaultOutputCoder() {
-    return coder;
+    return iterableCoder.getElemCoder();
   }
 
-  private static class ValuesReader<T> extends UnboundedReader<T> implements Serializable {
-
-    private final Iterable<byte[]> values;
-    private final Coder<T> coder;
+  private static class ValuesReader<T> extends UnboundedReader<T> {
     private final UnboundedSource<T, CheckpointMark> source;
-    private transient Iterator<byte[]> iterator;
+    private final Iterable<T> values;
+    private transient Iterator<T> iterator;
     private T current;
 
-    public ValuesReader(Iterable<byte[]> values, Coder<T> coder,
+    public ValuesReader(Iterable<T> values, IterableCoder<T> coder,
         UnboundedSource<T, CheckpointMark> source) {
       this.values = values;
-      this.coder = coder;
       this.source = source;
     }
 
-    private T decode(byte[] bytes) throws IOException {
-      ByteArrayInputStream inputStream = new ByteArrayInputStream(bytes);
-      try {
-        return coder.decode(inputStream, Coder.Context.OUTER);
-      } finally {
-        inputStream.close();
-      }
-    }
+
 
     @Override
     public boolean start() throws IOException {
@@ -125,7 +124,7 @@ public class ValuesSource<T> extends UnboundedSource<T, UnboundedSource.Checkpoi
     @Override
     public boolean advance() throws IOException {
       if (iterator.hasNext()) {
-        current = decode(iterator.next());
+        current = iterator.next();
         return true;
       } else {
         return false;


[16/50] [abbrv] incubator-beam git commit: DataflowRunner: reject job submission when the version has not been properly set

Posted by ke...@apache.org.
DataflowRunner: reject job submission when the version has not been properly set


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/8fd520c0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/8fd520c0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/8fd520c0

Branch: refs/heads/gearpump-runner
Commit: 8fd520c07e464c4308d8d32cc0e88e2ecd96c8d2
Parents: 7ad7877
Author: Dan Halperin <dh...@google.com>
Authored: Thu Dec 1 11:21:30 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Dec 2 09:17:11 2016 -0800

----------------------------------------------------------------------
 .../org/apache/beam/runners/dataflow/DataflowRunner.java     | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8fd520c0/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index 6ed386a..0357b46 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -513,10 +513,14 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
     Job newJob = jobSpecification.getJob();
     newJob.setClientRequestId(requestId);
 
-    String version = ReleaseInfo.getReleaseInfo().getVersion();
+    ReleaseInfo releaseInfo = ReleaseInfo.getReleaseInfo();
+    String version = releaseInfo.getVersion();
+    checkState(
+        !version.equals("${pom.version}"),
+        "Unable to submit a job to the Dataflow service with unset version ${pom.version}");
     System.out.println("Dataflow SDK version: " + version);
 
-    newJob.getEnvironment().setUserAgent(ReleaseInfo.getReleaseInfo());
+    newJob.getEnvironment().setUserAgent(releaseInfo);
     // The Dataflow Service may write to the temporary directory directly, so
     // must be verified.
     if (!isNullOrEmpty(options.getGcpTempLocation())) {


[12/50] [abbrv] incubator-beam git commit: Category for tests using splittable DoFn

Posted by ke...@apache.org.
Category for tests using splittable DoFn


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/887b357f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/887b357f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/887b357f

Branch: refs/heads/gearpump-runner
Commit: 887b357f7dff9fe10914b32dc69d32f0716fa237
Parents: 63491bf
Author: Eugene Kirpichov <ki...@google.com>
Authored: Wed Nov 30 12:55:45 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Dec 1 19:26:37 2016 -0800

----------------------------------------------------------------------
 runners/apex/pom.xml                            |   1 +
 .../beam/runners/direct/SplittableDoFnTest.java | 407 -------------------
 runners/flink/runner/pom.xml                    |   1 +
 runners/google-cloud-dataflow-java/pom.xml      |   1 +
 runners/spark/pom.xml                           |   1 +
 .../beam/sdk/testing/UsesSplittableParDo.java   |  25 ++
 .../beam/sdk/transforms/SplittableDoFnTest.java | 401 ++++++++++++++++++
 7 files changed, 430 insertions(+), 407 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/887b357f/runners/apex/pom.xml
----------------------------------------------------------------------
diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml
index 84185b8..983781d 100644
--- a/runners/apex/pom.xml
+++ b/runners/apex/pom.xml
@@ -186,6 +186,7 @@
             <configuration>
               <groups>org.apache.beam.sdk.testing.RunnableOnService</groups>
               <excludedGroups>org.apache.beam.sdk.testing.UsesStatefulParDo</excludedGroups>
+              <excludedGroups>org.apache.beam.sdk.testing.UsesSplittableParDo</excludedGroups>
               <parallel>none</parallel>
               <failIfNoTests>true</failIfNoTests>
               <dependenciesToScan>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/887b357f/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
deleted file mode 100644
index f9e833f..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SplittableDoFnTest.java
+++ /dev/null
@@ -1,407 +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.runners.direct;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.testing.TestStream;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.Keys;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.TupleTagList;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.joda.time.MutableDateTime;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for <a href="https://s.apache.org/splittable-do-fn>splittable</a> {@link DoFn} behavior
- * using the direct runner.
- *
- * <p>TODO: make this use @RunnableOnService.
- */
-@RunWith(JUnit4.class)
-public class SplittableDoFnTest {
-  static class OffsetRange implements Serializable {
-    public final int from;
-    public final int to;
-
-    OffsetRange(int from, int to) {
-      this.from = from;
-      this.to = to;
-    }
-
-    @Override
-    public String toString() {
-      return "OffsetRange{" + "from=" + from + ", to=" + to + '}';
-    }
-  }
-
-  private static class OffsetRangeTracker implements RestrictionTracker<OffsetRange> {
-    private OffsetRange range;
-    private Integer lastClaimedIndex = null;
-
-    OffsetRangeTracker(OffsetRange range) {
-      this.range = checkNotNull(range);
-    }
-
-    @Override
-    public OffsetRange currentRestriction() {
-      return range;
-    }
-
-    @Override
-    public OffsetRange checkpoint() {
-      if (lastClaimedIndex == null) {
-        OffsetRange res = range;
-        range = new OffsetRange(range.from, range.from);
-        return res;
-      }
-      OffsetRange res = new OffsetRange(lastClaimedIndex + 1, range.to);
-      this.range = new OffsetRange(range.from, lastClaimedIndex + 1);
-      return res;
-    }
-
-    boolean tryClaim(int i) {
-      checkState(lastClaimedIndex == null || i > lastClaimedIndex);
-      if (i >= range.to) {
-        return false;
-      }
-      lastClaimedIndex = i;
-      return true;
-    }
-  }
-
-  static class PairStringWithIndexToLength extends DoFn<String, KV<String, Integer>> {
-    @ProcessElement
-    public ProcessContinuation process(ProcessContext c, OffsetRangeTracker tracker) {
-      for (int i = tracker.currentRestriction().from; tracker.tryClaim(i); ++i) {
-        c.output(KV.of(c.element(), i));
-        if (i % 3 == 0) {
-          return ProcessContinuation.resume();
-        }
-      }
-      return ProcessContinuation.stop();
-    }
-
-    @GetInitialRestriction
-    public OffsetRange getInitialRange(String element) {
-      return new OffsetRange(0, element.length());
-    }
-
-    @SplitRestriction
-    public void splitRange(
-        String element, OffsetRange range, OutputReceiver<OffsetRange> receiver) {
-      receiver.output(new OffsetRange(range.from, (range.from + range.to) / 2));
-      receiver.output(new OffsetRange((range.from + range.to) / 2, range.to));
-    }
-
-    @NewTracker
-    public OffsetRangeTracker newTracker(OffsetRange range) {
-      return new OffsetRangeTracker(range);
-    }
-  }
-
-  private static class ReifyTimestampsFn<T> extends DoFn<T, TimestampedValue<T>> {
-    @ProcessElement
-    public void process(ProcessContext c) {
-      c.output(TimestampedValue.of(c.element(), c.timestamp()));
-    }
-  }
-
-  @Test
-  public void testPairWithIndexBasic() {
-    Pipeline p = TestPipeline.create();
-    p.getOptions().setRunner(DirectRunner.class);
-    PCollection<KV<String, Integer>> res =
-        p.apply(Create.of("a", "bb", "ccccc"))
-            .apply(ParDo.of(new PairStringWithIndexToLength()))
-            .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()));
-
-    PAssert.that(res)
-        .containsInAnyOrder(
-            Arrays.asList(
-                KV.of("a", 0),
-                KV.of("bb", 0),
-                KV.of("bb", 1),
-                KV.of("ccccc", 0),
-                KV.of("ccccc", 1),
-                KV.of("ccccc", 2),
-                KV.of("ccccc", 3),
-                KV.of("ccccc", 4)));
-
-    p.run();
-  }
-
-  @Test
-  public void testPairWithIndexWindowedTimestamped() {
-    // Tests that Splittable DoFn correctly propagates windowing strategy, windows and timestamps
-    // of elements in the input collection.
-    Pipeline p = TestPipeline.create();
-    p.getOptions().setRunner(DirectRunner.class);
-
-    MutableDateTime mutableNow = Instant.now().toMutableDateTime();
-    mutableNow.setMillisOfSecond(0);
-    Instant now = mutableNow.toInstant();
-    Instant nowP1 = now.plus(Duration.standardSeconds(1));
-    Instant nowP2 = now.plus(Duration.standardSeconds(2));
-
-    SlidingWindows windowFn =
-        SlidingWindows.of(Duration.standardSeconds(5)).every(Duration.standardSeconds(1));
-    PCollection<KV<String, Integer>> res =
-        p.apply(
-                Create.timestamped(
-                    TimestampedValue.of("a", now),
-                    TimestampedValue.of("bb", nowP1),
-                    TimestampedValue.of("ccccc", nowP2)))
-            .apply(Window.<String>into(windowFn))
-            .apply(ParDo.of(new PairStringWithIndexToLength()))
-            .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()));
-
-    assertEquals(windowFn, res.getWindowingStrategy().getWindowFn());
-
-    PCollection<TimestampedValue<KV<String, Integer>>> timestamped =
-        res.apply("Reify timestamps", ParDo.of(new ReifyTimestampsFn<KV<String, Integer>>()));
-
-    for (int i = 0; i < 4; ++i) {
-      Instant base = now.minus(Duration.standardSeconds(i));
-      IntervalWindow window = new IntervalWindow(base, base.plus(Duration.standardSeconds(5)));
-
-      List<TimestampedValue<KV<String, Integer>>> expectedUnfiltered =
-          Arrays.asList(
-              TimestampedValue.of(KV.of("a", 0), now),
-              TimestampedValue.of(KV.of("bb", 0), nowP1),
-              TimestampedValue.of(KV.of("bb", 1), nowP1),
-              TimestampedValue.of(KV.of("ccccc", 0), nowP2),
-              TimestampedValue.of(KV.of("ccccc", 1), nowP2),
-              TimestampedValue.of(KV.of("ccccc", 2), nowP2),
-              TimestampedValue.of(KV.of("ccccc", 3), nowP2),
-              TimestampedValue.of(KV.of("ccccc", 4), nowP2));
-
-      List<TimestampedValue<KV<String, Integer>>> expected = new ArrayList<>();
-      for (TimestampedValue<KV<String, Integer>> tv : expectedUnfiltered) {
-        if (!window.start().isAfter(tv.getTimestamp())
-            && !tv.getTimestamp().isAfter(window.maxTimestamp())) {
-          expected.add(tv);
-        }
-      }
-      assertFalse(expected.isEmpty());
-
-      PAssert.that(timestamped).inWindow(window).containsInAnyOrder(expected);
-    }
-    p.run();
-  }
-
-  private static class SDFWithSideInputsAndOutputs extends DoFn<Integer, String> {
-    private final PCollectionView<String> sideInput;
-    private final TupleTag<String> sideOutput;
-
-    private SDFWithSideInputsAndOutputs(
-        PCollectionView<String> sideInput, TupleTag<String> sideOutput) {
-      this.sideInput = sideInput;
-      this.sideOutput = sideOutput;
-    }
-
-    @ProcessElement
-    public void process(ProcessContext c, OffsetRangeTracker tracker) {
-      checkState(tracker.tryClaim(tracker.currentRestriction().from));
-      String side = c.sideInput(sideInput);
-      c.output("main:" + side + ":" + c.element());
-      c.sideOutput(sideOutput, "side:" + side + ":" + c.element());
-    }
-
-    @GetInitialRestriction
-    public OffsetRange getInitialRestriction(Integer value) {
-      return new OffsetRange(0, 1);
-    }
-
-    @NewTracker
-    public OffsetRangeTracker newTracker(OffsetRange range) {
-      return new OffsetRangeTracker(range);
-    }
-  }
-
-  @Test
-  public void testSideInputsAndOutputs() throws Exception {
-    Pipeline p = TestPipeline.create();
-    p.getOptions().setRunner(DirectRunner.class);
-
-    PCollectionView<String> sideInput =
-        p.apply("side input", Create.of("foo")).apply(View.<String>asSingleton());
-    TupleTag<String> mainOutputTag = new TupleTag<>("main");
-    TupleTag<String> sideOutputTag = new TupleTag<>("side");
-
-    PCollectionTuple res =
-        p.apply("input", Create.of(0, 1, 2))
-            .apply(
-                ParDo.of(new SDFWithSideInputsAndOutputs(sideInput, sideOutputTag))
-                    .withSideInputs(sideInput)
-                    .withOutputTags(mainOutputTag, TupleTagList.of(sideOutputTag)));
-    res.get(mainOutputTag).setCoder(StringUtf8Coder.of());
-    res.get(sideOutputTag).setCoder(StringUtf8Coder.of());
-
-    PAssert.that(res.get(mainOutputTag))
-        .containsInAnyOrder(Arrays.asList("main:foo:0", "main:foo:1", "main:foo:2"));
-    PAssert.that(res.get(sideOutputTag))
-        .containsInAnyOrder(Arrays.asList("side:foo:0", "side:foo:1", "side:foo:2"));
-
-    p.run();
-  }
-
-  @Test
-  public void testLateData() throws Exception {
-    Pipeline p = TestPipeline.create();
-    p.getOptions().setRunner(DirectRunner.class);
-
-    Instant base = Instant.now();
-
-    TestStream<String> stream =
-        TestStream.create(StringUtf8Coder.of())
-            .advanceWatermarkTo(base)
-            .addElements("aa")
-            .advanceWatermarkTo(base.plus(Duration.standardSeconds(5)))
-            .addElements(TimestampedValue.of("bb", base.minus(Duration.standardHours(1))))
-            .advanceProcessingTime(Duration.standardHours(1))
-            .advanceWatermarkToInfinity();
-
-    PCollection<String> input =
-        p.apply(stream)
-            .apply(
-                Window.<String>into(FixedWindows.of(Duration.standardMinutes(1)))
-                    .withAllowedLateness(Duration.standardMinutes(1)));
-
-    PCollection<KV<String, Integer>> afterSDF =
-        input
-            .apply(ParDo.of(new PairStringWithIndexToLength()))
-            .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()));
-
-    PCollection<String> nonLate =
-        afterSDF.apply(GroupByKey.<String, Integer>create()).apply(Keys.<String>create());
-
-    // The splittable DoFn itself should not drop any data and act as pass-through.
-    PAssert.that(afterSDF)
-        .containsInAnyOrder(
-            Arrays.asList(KV.of("aa", 0), KV.of("aa", 1), KV.of("bb", 0), KV.of("bb", 1)));
-
-    // But it should preserve the windowing strategy of the data, including allowed lateness:
-    // the follow-up GBK should drop the late data.
-    assertEquals(afterSDF.getWindowingStrategy(), input.getWindowingStrategy());
-    PAssert.that(nonLate).containsInAnyOrder("aa");
-
-    p.run();
-  }
-
-  private static class SDFWithLifecycle extends DoFn<String, String> {
-    private enum State {
-      BEFORE_SETUP,
-      OUTSIDE_BUNDLE,
-      INSIDE_BUNDLE,
-      TORN_DOWN
-    }
-
-    private State state = State.BEFORE_SETUP;
-
-    @ProcessElement
-    public void processElement(ProcessContext c, OffsetRangeTracker tracker) {
-      assertEquals(State.INSIDE_BUNDLE, state);
-      assertTrue(tracker.tryClaim(0));
-      c.output(c.element());
-    }
-
-    @GetInitialRestriction
-    public OffsetRange getInitialRestriction(String value) {
-      return new OffsetRange(0, 1);
-    }
-
-    @NewTracker
-    public OffsetRangeTracker newTracker(OffsetRange range) {
-      return new OffsetRangeTracker(range);
-    }
-
-    @Setup
-    public void setUp() {
-      assertEquals(State.BEFORE_SETUP, state);
-      state = State.OUTSIDE_BUNDLE;
-    }
-
-    @StartBundle
-    public void startBundle(Context c) {
-      assertEquals(State.OUTSIDE_BUNDLE, state);
-      state = State.INSIDE_BUNDLE;
-    }
-
-    @FinishBundle
-    public void finishBundle(Context c) {
-      assertEquals(State.INSIDE_BUNDLE, state);
-      state = State.OUTSIDE_BUNDLE;
-    }
-
-    @Teardown
-    public void tearDown() {
-      assertEquals(State.OUTSIDE_BUNDLE, state);
-      state = State.TORN_DOWN;
-    }
-  }
-
-  @Test
-  public void testLifecycleMethods() throws Exception {
-    Pipeline p = TestPipeline.create();
-    p.getOptions().setRunner(DirectRunner.class);
-
-    PCollection<String> res =
-        p.apply(Create.of("a", "b", "c")).apply(ParDo.of(new SDFWithLifecycle()));
-
-    PAssert.that(res).containsInAnyOrder("a", "b", "c");
-
-    p.run();
-  }
-
-  // TODO (https://issues.apache.org/jira/browse/BEAM-988): Test that Splittable DoFn
-  // emits output immediately (i.e. has a pass-through trigger) regardless of input's
-  // windowing/triggering strategy.
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/887b357f/runners/flink/runner/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml
index 18bf6a7..3e3dd7e 100644
--- a/runners/flink/runner/pom.xml
+++ b/runners/flink/runner/pom.xml
@@ -54,6 +54,7 @@
                 <configuration>
                   <groups>org.apache.beam.sdk.testing.RunnableOnService</groups>
                   <excludedGroups>org.apache.beam.sdk.testing.UsesStatefulParDo</excludedGroups>
+                  <excludedGroups>org.apache.beam.sdk.testing.UsesSplittableParDo</excludedGroups>
                   <parallel>none</parallel>
                   <failIfNoTests>true</failIfNoTests>
                   <dependenciesToScan>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/887b357f/runners/google-cloud-dataflow-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml
index 59276e4..8547499 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -78,6 +78,7 @@
             <id>runnable-on-service-tests</id>
             <configuration>
               <excludedGroups>org.apache.beam.sdk.testing.UsesStatefulParDo</excludedGroups>
+              <excludedGroups>org.apache.beam.sdk.testing.UsesSplittableParDo</excludedGroups>
               <excludes>
                 <exclude>org.apache.beam.sdk.transforms.FlattenTest</exclude>
               </excludes>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/887b357f/runners/spark/pom.xml
----------------------------------------------------------------------
diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
index da7a72a..dc000bf 100644
--- a/runners/spark/pom.xml
+++ b/runners/spark/pom.xml
@@ -73,6 +73,7 @@
                 <configuration>
                   <groups>org.apache.beam.sdk.testing.RunnableOnService</groups>
                   <excludedGroups>org.apache.beam.sdk.testing.UsesStatefulParDo</excludedGroups>
+                  <excludedGroups>org.apache.beam.sdk.testing.UsesSplittableParDo</excludedGroups>
                   <forkCount>1</forkCount>
                   <reuseForks>false</reuseForks>
                   <failIfNoTests>true</failIfNoTests>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/887b357f/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSplittableParDo.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSplittableParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSplittableParDo.java
new file mode 100644
index 0000000..209936f
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSplittableParDo.java
@@ -0,0 +1,25 @@
+/*
+ * 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.testing;
+
+import org.apache.beam.sdk.transforms.ParDo;
+
+/**
+ * Category tag for validation tests which utilize splittable {@link ParDo}.
+ */
+public interface UsesSplittableParDo {}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/887b357f/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java
new file mode 100644
index 0000000..82bd3a3
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java
@@ -0,0 +1,401 @@
+/*
+ * 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.transforms;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.RunnableOnService;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestStream;
+import org.apache.beam.sdk.testing.UsesSplittableParDo;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.joda.time.MutableDateTime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for <a href="https://s.apache.org/splittable-do-fn>splittable</a> {@link DoFn} behavior.
+ */
+@RunWith(JUnit4.class)
+public class SplittableDoFnTest {
+  static class OffsetRange implements Serializable {
+    public final int from;
+    public final int to;
+
+    OffsetRange(int from, int to) {
+      this.from = from;
+      this.to = to;
+    }
+
+    @Override
+    public String toString() {
+      return "OffsetRange{" + "from=" + from + ", to=" + to + '}';
+    }
+  }
+
+  private static class OffsetRangeTracker implements RestrictionTracker<OffsetRange> {
+    private OffsetRange range;
+    private Integer lastClaimedIndex = null;
+
+    OffsetRangeTracker(OffsetRange range) {
+      this.range = checkNotNull(range);
+    }
+
+    @Override
+    public OffsetRange currentRestriction() {
+      return range;
+    }
+
+    @Override
+    public OffsetRange checkpoint() {
+      if (lastClaimedIndex == null) {
+        OffsetRange res = range;
+        range = new OffsetRange(range.from, range.from);
+        return res;
+      }
+      OffsetRange res = new OffsetRange(lastClaimedIndex + 1, range.to);
+      this.range = new OffsetRange(range.from, lastClaimedIndex + 1);
+      return res;
+    }
+
+    boolean tryClaim(int i) {
+      checkState(lastClaimedIndex == null || i > lastClaimedIndex);
+      if (i >= range.to) {
+        return false;
+      }
+      lastClaimedIndex = i;
+      return true;
+    }
+  }
+
+  static class PairStringWithIndexToLength extends DoFn<String, KV<String, Integer>> {
+    @ProcessElement
+    public ProcessContinuation process(ProcessContext c, OffsetRangeTracker tracker) {
+      for (int i = tracker.currentRestriction().from; tracker.tryClaim(i); ++i) {
+        c.output(KV.of(c.element(), i));
+        if (i % 3 == 0) {
+          return ProcessContinuation.resume();
+        }
+      }
+      return ProcessContinuation.stop();
+    }
+
+    @GetInitialRestriction
+    public OffsetRange getInitialRange(String element) {
+      return new OffsetRange(0, element.length());
+    }
+
+    @SplitRestriction
+    public void splitRange(
+        String element, OffsetRange range, OutputReceiver<OffsetRange> receiver) {
+      receiver.output(new OffsetRange(range.from, (range.from + range.to) / 2));
+      receiver.output(new OffsetRange((range.from + range.to) / 2, range.to));
+    }
+
+    @NewTracker
+    public OffsetRangeTracker newTracker(OffsetRange range) {
+      return new OffsetRangeTracker(range);
+    }
+  }
+
+  private static class ReifyTimestampsFn<T> extends DoFn<T, TimestampedValue<T>> {
+    @ProcessElement
+    public void process(ProcessContext c) {
+      c.output(TimestampedValue.of(c.element(), c.timestamp()));
+    }
+  }
+
+  @Test
+  @Category({RunnableOnService.class, UsesSplittableParDo.class})
+  public void testPairWithIndexBasic() {
+    Pipeline p = TestPipeline.create();
+    PCollection<KV<String, Integer>> res =
+        p.apply(Create.of("a", "bb", "ccccc"))
+            .apply(ParDo.of(new PairStringWithIndexToLength()))
+            .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()));
+
+    PAssert.that(res)
+        .containsInAnyOrder(
+            Arrays.asList(
+                KV.of("a", 0),
+                KV.of("bb", 0),
+                KV.of("bb", 1),
+                KV.of("ccccc", 0),
+                KV.of("ccccc", 1),
+                KV.of("ccccc", 2),
+                KV.of("ccccc", 3),
+                KV.of("ccccc", 4)));
+
+    p.run();
+  }
+
+  @Test
+  @Category({RunnableOnService.class, UsesSplittableParDo.class})
+  public void testPairWithIndexWindowedTimestamped() {
+    // Tests that Splittable DoFn correctly propagates windowing strategy, windows and timestamps
+    // of elements in the input collection.
+    Pipeline p = TestPipeline.create();
+
+    MutableDateTime mutableNow = Instant.now().toMutableDateTime();
+    mutableNow.setMillisOfSecond(0);
+    Instant now = mutableNow.toInstant();
+    Instant nowP1 = now.plus(Duration.standardSeconds(1));
+    Instant nowP2 = now.plus(Duration.standardSeconds(2));
+
+    SlidingWindows windowFn =
+        SlidingWindows.of(Duration.standardSeconds(5)).every(Duration.standardSeconds(1));
+    PCollection<KV<String, Integer>> res =
+        p.apply(
+                Create.timestamped(
+                    TimestampedValue.of("a", now),
+                    TimestampedValue.of("bb", nowP1),
+                    TimestampedValue.of("ccccc", nowP2)))
+            .apply(Window.<String>into(windowFn))
+            .apply(ParDo.of(new PairStringWithIndexToLength()))
+            .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()));
+
+    assertEquals(windowFn, res.getWindowingStrategy().getWindowFn());
+
+    PCollection<TimestampedValue<KV<String, Integer>>> timestamped =
+        res.apply("Reify timestamps", ParDo.of(new ReifyTimestampsFn<KV<String, Integer>>()));
+
+    for (int i = 0; i < 4; ++i) {
+      Instant base = now.minus(Duration.standardSeconds(i));
+      IntervalWindow window = new IntervalWindow(base, base.plus(Duration.standardSeconds(5)));
+
+      List<TimestampedValue<KV<String, Integer>>> expectedUnfiltered =
+          Arrays.asList(
+              TimestampedValue.of(KV.of("a", 0), now),
+              TimestampedValue.of(KV.of("bb", 0), nowP1),
+              TimestampedValue.of(KV.of("bb", 1), nowP1),
+              TimestampedValue.of(KV.of("ccccc", 0), nowP2),
+              TimestampedValue.of(KV.of("ccccc", 1), nowP2),
+              TimestampedValue.of(KV.of("ccccc", 2), nowP2),
+              TimestampedValue.of(KV.of("ccccc", 3), nowP2),
+              TimestampedValue.of(KV.of("ccccc", 4), nowP2));
+
+      List<TimestampedValue<KV<String, Integer>>> expected = new ArrayList<>();
+      for (TimestampedValue<KV<String, Integer>> tv : expectedUnfiltered) {
+        if (!window.start().isAfter(tv.getTimestamp())
+            && !tv.getTimestamp().isAfter(window.maxTimestamp())) {
+          expected.add(tv);
+        }
+      }
+      assertFalse(expected.isEmpty());
+
+      PAssert.that(timestamped).inWindow(window).containsInAnyOrder(expected);
+    }
+    p.run();
+  }
+
+  private static class SDFWithSideInputsAndOutputs extends DoFn<Integer, String> {
+    private final PCollectionView<String> sideInput;
+    private final TupleTag<String> sideOutput;
+
+    private SDFWithSideInputsAndOutputs(
+        PCollectionView<String> sideInput, TupleTag<String> sideOutput) {
+      this.sideInput = sideInput;
+      this.sideOutput = sideOutput;
+    }
+
+    @ProcessElement
+    public void process(ProcessContext c, OffsetRangeTracker tracker) {
+      checkState(tracker.tryClaim(tracker.currentRestriction().from));
+      String side = c.sideInput(sideInput);
+      c.output("main:" + side + ":" + c.element());
+      c.sideOutput(sideOutput, "side:" + side + ":" + c.element());
+    }
+
+    @GetInitialRestriction
+    public OffsetRange getInitialRestriction(Integer value) {
+      return new OffsetRange(0, 1);
+    }
+
+    @NewTracker
+    public OffsetRangeTracker newTracker(OffsetRange range) {
+      return new OffsetRangeTracker(range);
+    }
+  }
+
+  @Test
+  @Category({RunnableOnService.class, UsesSplittableParDo.class})
+  public void testSideInputsAndOutputs() throws Exception {
+    Pipeline p = TestPipeline.create();
+
+    PCollectionView<String> sideInput =
+        p.apply("side input", Create.of("foo")).apply(View.<String>asSingleton());
+    TupleTag<String> mainOutputTag = new TupleTag<>("main");
+    TupleTag<String> sideOutputTag = new TupleTag<>("side");
+
+    PCollectionTuple res =
+        p.apply("input", Create.of(0, 1, 2))
+            .apply(
+                ParDo.of(new SDFWithSideInputsAndOutputs(sideInput, sideOutputTag))
+                    .withSideInputs(sideInput)
+                    .withOutputTags(mainOutputTag, TupleTagList.of(sideOutputTag)));
+    res.get(mainOutputTag).setCoder(StringUtf8Coder.of());
+    res.get(sideOutputTag).setCoder(StringUtf8Coder.of());
+
+    PAssert.that(res.get(mainOutputTag))
+        .containsInAnyOrder(Arrays.asList("main:foo:0", "main:foo:1", "main:foo:2"));
+    PAssert.that(res.get(sideOutputTag))
+        .containsInAnyOrder(Arrays.asList("side:foo:0", "side:foo:1", "side:foo:2"));
+
+    p.run();
+  }
+
+  @Test
+  @Category({RunnableOnService.class, UsesSplittableParDo.class})
+  public void testLateData() throws Exception {
+    Pipeline p = TestPipeline.create();
+
+    Instant base = Instant.now();
+
+    TestStream<String> stream =
+        TestStream.create(StringUtf8Coder.of())
+            .advanceWatermarkTo(base)
+            .addElements("aa")
+            .advanceWatermarkTo(base.plus(Duration.standardSeconds(5)))
+            .addElements(TimestampedValue.of("bb", base.minus(Duration.standardHours(1))))
+            .advanceProcessingTime(Duration.standardHours(1))
+            .advanceWatermarkToInfinity();
+
+    PCollection<String> input =
+        p.apply(stream)
+            .apply(
+                Window.<String>into(FixedWindows.of(Duration.standardMinutes(1)))
+                    .withAllowedLateness(Duration.standardMinutes(1)));
+
+    PCollection<KV<String, Integer>> afterSDF =
+        input
+            .apply(ParDo.of(new PairStringWithIndexToLength()))
+            .setCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()));
+
+    PCollection<String> nonLate =
+        afterSDF.apply(GroupByKey.<String, Integer>create()).apply(Keys.<String>create());
+
+    // The splittable DoFn itself should not drop any data and act as pass-through.
+    PAssert.that(afterSDF)
+        .containsInAnyOrder(
+            Arrays.asList(KV.of("aa", 0), KV.of("aa", 1), KV.of("bb", 0), KV.of("bb", 1)));
+
+    // But it should preserve the windowing strategy of the data, including allowed lateness:
+    // the follow-up GBK should drop the late data.
+    assertEquals(afterSDF.getWindowingStrategy(), input.getWindowingStrategy());
+    PAssert.that(nonLate).containsInAnyOrder("aa");
+
+    p.run();
+  }
+
+  private static class SDFWithLifecycle extends DoFn<String, String> {
+    private enum State {
+      BEFORE_SETUP,
+      OUTSIDE_BUNDLE,
+      INSIDE_BUNDLE,
+      TORN_DOWN
+    }
+
+    private State state = State.BEFORE_SETUP;
+
+    @ProcessElement
+    public void processElement(ProcessContext c, OffsetRangeTracker tracker) {
+      assertEquals(State.INSIDE_BUNDLE, state);
+      assertTrue(tracker.tryClaim(0));
+      c.output(c.element());
+    }
+
+    @GetInitialRestriction
+    public OffsetRange getInitialRestriction(String value) {
+      return new OffsetRange(0, 1);
+    }
+
+    @NewTracker
+    public OffsetRangeTracker newTracker(OffsetRange range) {
+      return new OffsetRangeTracker(range);
+    }
+
+    @Setup
+    public void setUp() {
+      assertEquals(State.BEFORE_SETUP, state);
+      state = State.OUTSIDE_BUNDLE;
+    }
+
+    @StartBundle
+    public void startBundle(Context c) {
+      assertEquals(State.OUTSIDE_BUNDLE, state);
+      state = State.INSIDE_BUNDLE;
+    }
+
+    @FinishBundle
+    public void finishBundle(Context c) {
+      assertEquals(State.INSIDE_BUNDLE, state);
+      state = State.OUTSIDE_BUNDLE;
+    }
+
+    @Teardown
+    public void tearDown() {
+      assertEquals(State.OUTSIDE_BUNDLE, state);
+      state = State.TORN_DOWN;
+    }
+  }
+
+  @Test
+  @Category({RunnableOnService.class, UsesSplittableParDo.class})
+  public void testLifecycleMethods() throws Exception {
+    Pipeline p = TestPipeline.create();
+
+    PCollection<String> res =
+        p.apply(Create.of("a", "b", "c")).apply(ParDo.of(new SDFWithLifecycle()));
+
+    PAssert.that(res).containsInAnyOrder("a", "b", "c");
+
+    p.run();
+  }
+
+  // TODO (https://issues.apache.org/jira/browse/BEAM-988): Test that Splittable DoFn
+  // emits output immediately (i.e. has a pass-through trigger) regardless of input's
+  // windowing/triggering strategy.
+}


[37/50] [abbrv] incubator-beam git commit: Initial commit of jobs

Posted by ke...@apache.org.
Initial commit of jobs

Signed-off-by: Jason Kuster <ja...@google.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/ad9ca455
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/ad9ca455
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/ad9ca455

Branch: refs/heads/gearpump-runner
Commit: ad9ca455218f6dda32e31ee97fe721e8b4ad6c2a
Parents: c840455
Author: Jason Kuster <ja...@google.com>
Authored: Mon Nov 14 15:35:40 2016 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Fri Dec 2 17:40:12 2016 -0800

----------------------------------------------------------------------
 .jenkins/common_job_properties.groovy           | 166 +++++++++++++++++++
 ...job_beam_PostCommit_Java_MavenInstall.groovy |  42 +++++
 ...ommit_Java_RunnableOnService_Dataflow.groovy |  39 +++++
 ...stCommit_Java_RunnableOnService_Flink.groovy |  38 +++++
 ...ommit_Java_RunnableOnService_Gearpump.groovy |  41 +++++
 ...stCommit_Java_RunnableOnService_Spark.groovy |  38 +++++
 .../job_beam_PostCommit_Python_Verify.groovy    |  37 +++++
 .../job_beam_PreCommit_Java_MavenInstall.groovy |  42 +++++
 .../job_beam_Release_NightlySnapshot.groovy     |  46 +++++
 .jenkins/job_seed.groovy                        |  47 ++++++
 10 files changed, 536 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad9ca455/.jenkins/common_job_properties.groovy
----------------------------------------------------------------------
diff --git a/.jenkins/common_job_properties.groovy b/.jenkins/common_job_properties.groovy
new file mode 100644
index 0000000..f3a8a07
--- /dev/null
+++ b/.jenkins/common_job_properties.groovy
@@ -0,0 +1,166 @@
+/*
+ * 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.
+ */
+
+// Contains functions that help build Jenkins projects. Functions typically set
+// common properties that are shared among all Jenkins projects.
+class common_job_properties {
+
+  // Sets common top-level job properties.
+  static def setTopLevelJobProperties(def context,
+                                      def default_branch = 'master',
+                                      def default_timeout = 100) {
+    // GitHub project.
+    context.properties {
+      githubProjectUrl('https://github.com/apache/incubator-beam/')
+    }
+
+    // Set JDK version.
+    context.jdk('JDK 1.8 (latest)')
+
+    // Restrict this project to run only on Jenkins executors dedicated to the
+    // Apache Beam project.
+    context.label('beam')
+
+    // Discard old builds. Build records are only kept up to this number of days.
+    context.logRotator {
+      daysToKeep(14)
+    }
+
+    // Source code management.
+    context.scm {
+      git {
+        remote {
+          url('https://github.com/apache/incubator-beam.git')
+          refspec('+refs/heads/*:refs/remotes/origin/* ' +
+                  '+refs/pull/*:refs/remotes/origin/pr/*')
+        }
+        branch('${sha1}')
+        extensions {
+          cleanAfterCheckout()
+        }
+      }
+    }
+
+    context.parameters {
+      // This is a recommended setup if you want to run the job manually. The
+      // ${sha1} parameter needs to be provided, and defaults to the main branch.
+      stringParam(
+          'sha1',
+          default_branch,
+          'Commit id or refname (eg: origin/pr/9/head) you want to build.')
+    }
+
+    context.wrappers {
+      // Abort the build if it's stuck for more minutes than specified.
+      timeout {
+        absolute(default_timeout)
+        abortBuild()
+      }
+
+      // Set SPARK_LOCAL_IP for spark tests.
+      environmentVariables {
+        env('SPARK_LOCAL_IP', '127.0.0.1')
+      }
+    }
+  }
+
+  // Sets the pull request build trigger.
+  static def setPullRequestBuildTrigger(def context,
+                                        def commitStatusContext,
+                                        def successComment = '--none--') {
+    context.triggers {
+      githubPullRequest {
+        admins(['asfbot'])
+        useGitHubHooks()
+        orgWhitelist(['apache'])
+        allowMembersOfWhitelistedOrgsAsAdmin()
+        permitAll()
+
+        extensions {
+          commitStatus {
+            // This is the name that will show up in the GitHub pull request UI
+            // for this Jenkins project.
+            delegate.context(commitStatusContext)
+          }
+
+          /*
+            This section is disabled, because of jenkinsci/ghprb-plugin#417 issue.
+            For the time being, an equivalent configure section below is added.
+
+          // Comment messages after build completes.
+          buildStatus {
+            completedStatus('SUCCESS', successComment)
+            completedStatus('FAILURE', '--none--')
+            completedStatus('ERROR', '--none--')
+          }
+          */
+        }
+      }
+    }
+
+    // Comment messages after build completes.
+    context.configure {
+      def messages = it / triggers / 'org.jenkinsci.plugins.ghprb.GhprbTrigger' / extensions / 'org.jenkinsci.plugins.ghprb.extensions.comments.GhprbBuildStatus' / messages
+      messages << 'org.jenkinsci.plugins.ghprb.extensions.comments.GhprbBuildResultMessage' {
+        message(successComment)
+        result('SUCCESS')
+      }
+      messages << 'org.jenkinsci.plugins.ghprb.extensions.comments.GhprbBuildResultMessage' {
+        message('--none--')
+        result('ERROR')
+      }
+      messages << 'org.jenkinsci.plugins.ghprb.extensions.comments.GhprbBuildResultMessage' {
+        message('--none--')
+        result('FAILURE')
+      }
+    }
+  }
+
+  // Sets common config for Maven jobs.
+  static def setMavenConfig(def context) {
+    context.mavenInstallation('Maven 3.3.3')
+    context.rootPOM('pom.xml')
+    // Use a repository local to the workspace for better isolation of jobs.
+    context.localRepository(LocalRepositoryLocation.LOCAL_TO_WORKSPACE)
+  }
+
+  // Sets common config for PreCommit jobs.
+  static def setPreCommit(def context, comment) {
+    // Set pull request build trigger.
+    setPullRequestBuildTrigger(context, comment)
+  }
+
+  // Sets common config for PostCommit jobs.
+  static def setPostCommit(def context,
+                           def build_schedule = '0 */6 * * *',
+                           def scm_schedule = '* * * * *',
+                           def notify_address = 'commits@beam.incubator.apache.org') {
+    // Set build triggers
+    context.triggers {
+      // By default runs every 6 hours.
+      cron(build_schedule)
+      // Also polls SCM every minute.
+      scm(scm_schedule)
+    }
+
+    context.publishers {
+      // Notify an email address for each failed build (defaults to commits@).
+      mailer(notify_address, false, true)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad9ca455/.jenkins/job_beam_PostCommit_Java_MavenInstall.groovy
----------------------------------------------------------------------
diff --git a/.jenkins/job_beam_PostCommit_Java_MavenInstall.groovy b/.jenkins/job_beam_PostCommit_Java_MavenInstall.groovy
new file mode 100644
index 0000000..7fc6c3b
--- /dev/null
+++ b/.jenkins/job_beam_PostCommit_Java_MavenInstall.groovy
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+import common_job_properties
+
+// This job runs the Java postcommit tests, including the suite of integration
+// tests.
+mavenJob('beam_PostCommit_Java_MavenInstall') {
+  description('Runs postcommit tests on the Java SDK.')
+
+  previousNames('beam_PostCommit_MavenVerify')
+
+  // Execute concurrent builds if necessary.
+  concurrentBuild()
+
+  // Set common parameters.
+  common_job_properties.setTopLevelJobProperties(delegate)
+
+  // Set maven parameters.
+  common_job_properties.setMavenConfig(delegate)
+
+  // Sets that this is a PostCommit job.
+  common_job_properties.setPostCommit(delegate)
+
+  // Maven goals for this job.
+  goals('-B -e -P release,dataflow-runner clean install -DskipITs=false -DintegrationTestPipelineOptions=\'[ "--project=apache-beam-testing", "--tempRoot=gs://temp-storage-for-end-to-end-tests", "--runner=org.apache.beam.runners.dataflow.testing.TestDataflowRunner" ]\'')
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad9ca455/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Dataflow.groovy
----------------------------------------------------------------------
diff --git a/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Dataflow.groovy b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Dataflow.groovy
new file mode 100644
index 0000000..44136ad
--- /dev/null
+++ b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Dataflow.groovy
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+import common_job_properties
+
+// This job runs the suite of RunnableOnService tests against the Dataflow
+// runner.
+mavenJob('beam_PostCommit_Java_RunnableOnService_Dataflow') {
+  description('Runs the RunnableOnService suite on the Dataflow runner.')
+
+  previousNames('beam_PostCommit_RunnableOnService_GoogleCloudDataflow')
+
+  // Set common parameters.
+  common_job_properties.setTopLevelJobProperties(delegate)
+
+  // Set maven parameters.
+  common_job_properties.setMavenConfig(delegate)
+
+  // Sets that this is a PostCommit job.
+  common_job_properties.setPostCommit(delegate)
+
+  // Maven goals for this job.
+  goals('-B -e clean verify -am -pl runners/google-cloud-dataflow-java -DforkCount=0 -DrunnableOnServicePipelineOptions=\'[ "--runner=org.apache.beam.runners.dataflow.testing.TestDataflowRunner", "--project=apache-beam-testing", "--tempRoot=gs://temp-storage-for-runnable-on-service-tests/" ]\'')
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad9ca455/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Flink.groovy
----------------------------------------------------------------------
diff --git a/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Flink.groovy b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Flink.groovy
new file mode 100644
index 0000000..5a5863f
--- /dev/null
+++ b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Flink.groovy
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+import common_job_properties
+
+// This job runs the suite of RunnableOnService tests against the Flink runner.
+mavenJob('beam_PostCommit_Java_RunnableOnService_Flink') {
+  description('Runs the RunnableOnService suite on the Flink runner.')
+
+  previousNames('beam_PostCommit_RunnableOnService_FlinkLocal')
+
+  // Set common parameters.
+  common_job_properties.setTopLevelJobProperties(delegate)
+
+  // Set maven parameters.
+  common_job_properties.setMavenConfig(delegate)
+
+  // Sets that this is a PostCommit job.
+  common_job_properties.setPostCommit(delegate)
+
+  // Maven goals for this job.
+  goals('-B -e clean verify -am -pl runners/flink/runner -Plocal-runnable-on-service-tests -Prunnable-on-service-tests')
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad9ca455/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Gearpump.groovy
----------------------------------------------------------------------
diff --git a/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Gearpump.groovy b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Gearpump.groovy
new file mode 100644
index 0000000..f07bbc6
--- /dev/null
+++ b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Gearpump.groovy
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+import common_job_properties
+
+// This job runs the suite of RunnableOnService tests against the Gearpump
+// runner.
+mavenJob('beam_PostCommit_Java_RunnableOnService_Gearpump') {
+  description('Runs the RunnableOnService suite on the Gearpump runner.')
+
+  previousNames('beam_PostCommit_RunnableOnService_GearpumpLocal')
+
+  // Set common parameters.
+  common_job_properties.setTopLevelJobProperties(delegate, 'gearpump-runner')
+
+  // Set maven parameters.
+  common_job_properties.setMavenConfig(delegate)
+
+  // Sets that this is a PostCommit job.
+  // 0 5 31 2 * will run on Feb 31 (i.e. never) according to job properties.
+  // This job triggers only on SCM.
+  common_job_properties.setPostCommit(delegate, '0 5 31 2 *')
+
+  // Maven goals for this job.
+  goals('-B -e clean verify -am -pl runners/gearpump -DforkCount=0 -DrunnableOnServicePipelineOptions=\'[ "--runner=TestGearpumpRunner", "--streaming=false" ]\'')
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad9ca455/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Spark.groovy
----------------------------------------------------------------------
diff --git a/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Spark.groovy b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Spark.groovy
new file mode 100644
index 0000000..875c6e6
--- /dev/null
+++ b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Spark.groovy
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+import common_job_properties
+
+// This job runs the suite of RunnableOnService tests against the Spark runner.
+mavenJob('beam_PostCommit_Java_RunnableOnService_Spark') {
+  description('Runs the RunnableOnService suite on the Spark runner.')
+
+  previousNames('beam_PostCommit_RunnableOnService_SparkLocal')
+
+  // Set common parameters.
+  common_job_properties.setTopLevelJobProperties(delegate)
+
+  // Set maven parameters.
+  common_job_properties.setMavenConfig(delegate)
+
+  // Sets that this is a PostCommit job.
+  common_job_properties.setPostCommit(delegate)
+
+  // Maven goals for this job.
+  goals('-B -e clean verify -am -pl runners/spark -Prunnable-on-service-tests -Plocal-runnable-on-service-tests -Dspark.port.maxRetries=64 -Dspark.ui.enabled=false')
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad9ca455/.jenkins/job_beam_PostCommit_Python_Verify.groovy
----------------------------------------------------------------------
diff --git a/.jenkins/job_beam_PostCommit_Python_Verify.groovy b/.jenkins/job_beam_PostCommit_Python_Verify.groovy
new file mode 100644
index 0000000..79e173b
--- /dev/null
+++ b/.jenkins/job_beam_PostCommit_Python_Verify.groovy
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+import common_job_properties
+
+// This job defines the Python postcommit tests.
+job('beam_PostCommit_Python_Verify') {
+  description('Runs postcommit tests on the Python SDK.')
+
+  previousNames('beam_PostCommit_PythonVerify')
+
+  // Set common parameters.
+  common_job_properties.setTopLevelJobProperties(delegate, 'python-sdk')
+
+  // Sets that this is a PostCommit job.
+  common_job_properties.setPostCommit(delegate, '0 3-22/6 * * *')
+
+  // Execute shell command to test Python SDK.
+  steps {
+    shell('bash sdks/python/run_postcommit.sh')
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad9ca455/.jenkins/job_beam_PreCommit_Java_MavenInstall.groovy
----------------------------------------------------------------------
diff --git a/.jenkins/job_beam_PreCommit_Java_MavenInstall.groovy b/.jenkins/job_beam_PreCommit_Java_MavenInstall.groovy
new file mode 100644
index 0000000..8a8dea5
--- /dev/null
+++ b/.jenkins/job_beam_PreCommit_Java_MavenInstall.groovy
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+import common_job_properties
+
+// This is the Java precommit which runs a maven install, and the current set
+// of precommit tests.
+mavenJob('beam_PreCommit_Java_MavenInstall') {
+  description('Runs an install of the current GitHub Pull Request.')
+
+  previousNames('beam_PreCommit_MavenVerify')
+
+  // Execute concurrent builds if necessary.
+  concurrentBuild()
+
+  // Set common parameters.
+  common_job_properties.setTopLevelJobProperties(delegate)
+
+  // Set Maven parameters.
+  common_job_properties.setMavenConfig(delegate)
+
+  // Sets that this is a PreCommit job.
+  common_job_properties.setPreCommit(delegate, 'Jenkins: Maven clean install')
+
+  // Maven goals for this job.
+  goals('-B -e -Prelease,include-runners,jenkins-precommit,direct-runner,dataflow-runner,spark-runner,flink-runner,apex-runner help:effective-settings clean install')
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad9ca455/.jenkins/job_beam_Release_NightlySnapshot.groovy
----------------------------------------------------------------------
diff --git a/.jenkins/job_beam_Release_NightlySnapshot.groovy b/.jenkins/job_beam_Release_NightlySnapshot.groovy
new file mode 100644
index 0000000..496a0c5
--- /dev/null
+++ b/.jenkins/job_beam_Release_NightlySnapshot.groovy
@@ -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.
+ */
+
+import common_job_properties
+
+// This is the nightly snapshot build -- we use this to deploy a daily snapshot
+// to https://repository.apache.org/content/groups/snapshots/org/apache/beam.
+// Runs the postsubmit suite before deploying.
+mavenJob('beam_Release_NightlySnapshot') {
+  description('Runs a mvn clean deploy of the nightly snapshot.')
+
+  // Execute concurrent builds if necessary.
+  concurrentBuild()
+
+  // Set common parameters.
+  common_job_properties.setTopLevelJobProperties(delegate)
+
+  // Set maven paramaters.
+  common_job_properties.setMavenConfig(delegate)
+
+  // Set that this is a PostCommit job.
+  // Polls SCM on Feb 31st, i.e. never.
+  common_job_properties.setPostCommit(
+      delegate,
+      '0 7 * * *',
+      '0 5 31 2 *',
+      'dev@beam.incubator.apache.org')
+
+  // Maven goals for this job.
+  goals('-B -e clean deploy -P release,dataflow-runner -DskipITs=false -DintegrationTestPipelineOptions=\'[ "--project=apache-beam-testing", "--tempRoot=gs://temp-storage-for-end-to-end-tests", "--runner=org.apache.beam.runners.dataflow.testing.TestDataflowRunner" ]\'')
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad9ca455/.jenkins/job_seed.groovy
----------------------------------------------------------------------
diff --git a/.jenkins/job_seed.groovy b/.jenkins/job_seed.groovy
new file mode 100644
index 0000000..e46e5a4
--- /dev/null
+++ b/.jenkins/job_seed.groovy
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+
+import common_job_properties
+
+// Defines the seed job, which creates or updates all other Jenkins projects.
+job('beam_SeedJob_Main') {
+  description('Automatically configures all Apache Beam main repo Jenkins ' +
+              'projects based on Jenkins DSL groovy files checked into the ' +
+              'code repository.')
+
+  // Set common parameters.
+  common_job_properties.setTopLevelJobProperties(delegate)
+
+  // Set that this is a PostCommit job.
+  // Polls SCM on Feb 31st, i.e. never.
+  common_job_properties.setPostCommit(
+      delegate,
+      '0 6 * * *',
+      '0 5 31 2 *',
+      'dev@beam.incubator.apache.org')
+
+  steps {
+    dsl {
+      // A list or a glob of other groovy files to process.
+      external('.jenkins/job_*.groovy')
+
+      // If a job is removed from the script, disable it (rather than deleting).
+      removeAction('DISABLE')
+    }
+  }
+}


[40/50] [abbrv] incubator-beam git commit: [BEAM-1078] Closes #1498

Posted by ke...@apache.org.
[BEAM-1078] Closes #1498


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/a13024c4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/a13024c4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/a13024c4

Branch: refs/heads/gearpump-runner
Commit: a13024c40f73b6065ea4094d6e750b50c5027bb2
Parents: 8a7919b 5a997a1
Author: Dan Halperin <dh...@google.com>
Authored: Fri Dec 2 21:45:31 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Dec 2 21:45:31 2016 -0800

----------------------------------------------------------------------
 .travis/README.md                                     |  2 +-
 .../java/org/apache/beam/examples/complete/README.md  | 14 +++++++-------
 .../java/org/apache/beam/examples/cookbook/README.md  | 14 +++++++-------
 3 files changed, 15 insertions(+), 15 deletions(-)
----------------------------------------------------------------------



[38/50] [abbrv] incubator-beam git commit: This closes #1390

Posted by ke...@apache.org.
This closes #1390


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/8a7919b5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/8a7919b5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/8a7919b5

Branch: refs/heads/gearpump-runner
Commit: 8a7919b5a5faa23f85397be33495924130bdbca0
Parents: c840455 ad9ca45
Author: Davor Bonaci <da...@google.com>
Authored: Fri Dec 2 17:40:20 2016 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Fri Dec 2 17:40:20 2016 -0800

----------------------------------------------------------------------
 .jenkins/common_job_properties.groovy           | 166 +++++++++++++++++++
 ...job_beam_PostCommit_Java_MavenInstall.groovy |  42 +++++
 ...ommit_Java_RunnableOnService_Dataflow.groovy |  39 +++++
 ...stCommit_Java_RunnableOnService_Flink.groovy |  38 +++++
 ...ommit_Java_RunnableOnService_Gearpump.groovy |  41 +++++
 ...stCommit_Java_RunnableOnService_Spark.groovy |  38 +++++
 .../job_beam_PostCommit_Python_Verify.groovy    |  37 +++++
 .../job_beam_PreCommit_Java_MavenInstall.groovy |  42 +++++
 .../job_beam_Release_NightlySnapshot.groovy     |  46 +++++
 .jenkins/job_seed.groovy                        |  47 ++++++
 10 files changed, 536 insertions(+)
----------------------------------------------------------------------



[05/50] [abbrv] incubator-beam git commit: Revert "Move resource filtering later to avoid spurious rebuilds"

Posted by ke...@apache.org.
Revert "Move resource filtering later to avoid spurious rebuilds"

This reverts commit 2422365719c71cade97e1e74f1fb7f42b264244f.


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/b36048bd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/b36048bd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/b36048bd

Branch: refs/heads/gearpump-runner
Commit: b36048bd0e558fea281a1ec42aa8435db09dbe64
Parents: 1094fa6
Author: Dan Halperin <dh...@google.com>
Authored: Thu Dec 1 10:22:15 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Thu Dec 1 13:10:56 2016 -0800

----------------------------------------------------------------------
 sdks/java/core/pom.xml | 29 +++++++----------------------
 1 file changed, 7 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b36048bd/sdks/java/core/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml
index f842be7..ad84846 100644
--- a/sdks/java/core/pom.xml
+++ b/sdks/java/core/pom.xml
@@ -40,6 +40,13 @@
   </properties>
 
   <build>
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+        <filtering>true</filtering>
+      </resource>
+    </resources>
+
     <pluginManagement>
       <plugins>
         <plugin>
@@ -74,28 +81,6 @@
 
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-resources-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>resources</id>
-            <phase>compile</phase>
-            <goals>
-              <goal>resources</goal>
-            </goals>
-            <configuration>
-              <resources>
-                <resource>
-                  <directory>src/main/resources</directory>
-                  <filtering>true</filtering>
-                </resource>
-              </resources>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
       </plugin>
 


[34/50] [abbrv] incubator-beam git commit: Supports window parameter in DoFnTester

Posted by ke...@apache.org.
Supports window parameter in DoFnTester

Also prohibits other parameters, and prohibits output from bundle
methods (whereas previously it was silently dropped).


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/78ac009b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/78ac009b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/78ac009b

Branch: refs/heads/gearpump-runner
Commit: 78ac009be743a2e053580e9966f841174b636e88
Parents: 9645576
Author: Eugene Kirpichov <ki...@google.com>
Authored: Fri Dec 2 11:39:48 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Dec 2 15:42:33 2016 -0800

----------------------------------------------------------------------
 .../apache/beam/sdk/transforms/DoFnTester.java  | 166 ++++++++++++++-----
 .../beam/sdk/transforms/DoFnTesterTest.java     |  34 ++++
 2 files changed, 158 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/78ac009b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
index a9f93dd..7c1abef 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
@@ -38,13 +38,18 @@ import org.apache.beam.sdk.testing.ValueInSingleWindow;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
 import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.Timer;
 import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.UserCodeException;
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingInternals;
 import org.apache.beam.sdk.util.state.InMemoryStateInternals;
 import org.apache.beam.sdk.util.state.InMemoryTimerInternals;
 import org.apache.beam.sdk.util.state.StateInternals;
@@ -84,6 +89,9 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
   /**
    * Returns a {@code DoFnTester} supporting unit-testing of the given
    * {@link DoFn}. By default, uses {@link CloningBehavior#CLONE_ONCE}.
+   *
+   * <p>The only supported extra parameter of the {@link DoFn.ProcessElement} method is
+   * {@link BoundedWindow}.
    */
   @SuppressWarnings("unchecked")
   public static <InputT, OutputT> DoFnTester<InputT, OutputT> of(DoFn<InputT, OutputT> fn) {
@@ -236,7 +244,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     if (state == State.UNINITIALIZED) {
       initializeState();
     }
-    TestContext context = createContext(fn);
+    TestContext context = new TestContext();
     context.setupDelegateAggregators();
     // State and timer internals are per-bundle.
     stateInternals = InMemoryStateInternals.forKey(new Object());
@@ -262,7 +270,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
   /**
    * Calls the {@link DoFn.ProcessElement} method on the {@link DoFn} under test, in a
    * context where {@link DoFn.ProcessContext#element} returns the
-   * given element.
+   * given element and the element is in the global window.
    *
    * <p>Will call {@link #startBundle} automatically, if it hasn't
    * already been called.
@@ -277,26 +285,86 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
   /**
    * Calls {@link DoFn.ProcessElement} on the {@code DoFn} under test, in a
    * context where {@link DoFn.ProcessContext#element} returns the
-   * given element and timestamp.
+   * given element and timestamp and the element is in the global window.
    *
    * <p>Will call {@link #startBundle} automatically, if it hasn't
    * already been called.
-   *
-   * <p>If the input timestamp is {@literal null}, the minimum timestamp will be used.
    */
   public void processTimestampedElement(TimestampedValue<InputT> element) throws Exception {
     checkNotNull(element, "Timestamped element cannot be null");
+    processWindowedElement(
+        element.getValue(), element.getTimestamp(), GlobalWindow.INSTANCE);
+  }
+
+  /**
+   * Calls {@link DoFn.ProcessElement} on the {@code DoFn} under test, in a
+   * context where {@link DoFn.ProcessContext#element} returns the
+   * given element and timestamp and the element is in the given window.
+   *
+   * <p>Will call {@link #startBundle} automatically, if it hasn't
+   * already been called.
+   */
+  public void processWindowedElement(
+      InputT element, Instant timestamp, final BoundedWindow window) throws Exception {
     if (state != State.BUNDLE_STARTED) {
       startBundle();
     }
     try {
-      final TestProcessContext processContext = createProcessContext(element);
-      fnInvoker.invokeProcessElement(new DoFnInvoker.FakeArgumentProvider<InputT, OutputT>() {
-        @Override
-        public DoFn<InputT, OutputT>.ProcessContext processContext(DoFn<InputT, OutputT> doFn) {
-          return processContext;
-        }
-      });
+      final TestProcessContext processContext =
+          new TestProcessContext(
+              ValueInSingleWindow.of(element, timestamp, window, PaneInfo.NO_FIRING));
+      fnInvoker.invokeProcessElement(
+          new DoFnInvoker.ArgumentProvider<InputT, OutputT>() {
+            @Override
+            public BoundedWindow window() {
+              return window;
+            }
+
+            @Override
+            public DoFn<InputT, OutputT>.Context context(DoFn<InputT, OutputT> doFn) {
+              throw new UnsupportedOperationException(
+                  "Not expected to access DoFn.Context from @ProcessElement");
+            }
+
+            @Override
+            public DoFn<InputT, OutputT>.ProcessContext processContext(DoFn<InputT, OutputT> doFn) {
+              return processContext;
+            }
+
+            @Override
+            public DoFn.InputProvider<InputT> inputProvider() {
+              throw new UnsupportedOperationException(
+                  "Not expected to access InputProvider from DoFnTester");
+            }
+
+            @Override
+            public DoFn.OutputReceiver<OutputT> outputReceiver() {
+              throw new UnsupportedOperationException(
+                  "Not expected to access OutputReceiver from DoFnTester");
+            }
+
+            @Override
+            public WindowingInternals<InputT, OutputT> windowingInternals() {
+              throw new UnsupportedOperationException(
+                  "Not expected to access WindowingInternals from a new DoFn");
+            }
+
+            @Override
+            public <RestrictionT> RestrictionTracker<RestrictionT> restrictionTracker() {
+              throw new UnsupportedOperationException(
+                  "Not expected to access RestrictionTracker from a regular DoFn in DoFnTester");
+            }
+
+            @Override
+            public org.apache.beam.sdk.util.state.State state(String stateId) {
+              throw new UnsupportedOperationException("DoFnTester doesn't support state yet");
+            }
+
+            @Override
+            public Timer timer(String timerId) {
+              throw new UnsupportedOperationException("DoFnTester doesn't support timers yet");
+            }
+          });
     } catch (UserCodeException e) {
       unwrapUserCodeException(e);
     }
@@ -318,7 +386,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
         "Must be inside bundle to call finishBundle, but was: %s",
         state);
     try {
-      fnInvoker.invokeFinishBundle(createContext(fn));
+      fnInvoker.invokeFinishBundle(new TestContext());
     } catch (UserCodeException e) {
       unwrapUserCodeException(e);
     }
@@ -543,10 +611,6 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     return mainOutputTag;
   }
 
-  private TestContext createContext(DoFn<InputT, OutputT> fn) {
-    return new TestContext();
-  }
-
   private class TestContext extends DoFn<InputT, OutputT>.Context {
     TestContext() {
       fn.super();
@@ -559,12 +623,27 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
 
     @Override
     public void output(OutputT output) {
-      sideOutput(mainOutputTag, output);
+      throwUnsupportedOutputFromBundleMethods();
     }
 
     @Override
     public void outputWithTimestamp(OutputT output, Instant timestamp) {
-      sideOutputWithTimestamp(mainOutputTag, output, timestamp);
+      throwUnsupportedOutputFromBundleMethods();
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+      throwUnsupportedOutputFromBundleMethods();
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+      throwUnsupportedOutputFromBundleMethods();
+    }
+
+    private void throwUnsupportedOutputFromBundleMethods() {
+      throw new UnsupportedOperationException(
+          "DoFnTester doesn't support output from bundle methods");
     }
 
     @Override
@@ -613,26 +692,6 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
       }
       return aggregator;
     }
-
-    @Override
-    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-
-    }
-
-    @Override
-    public <T> void sideOutput(TupleTag<T> tag, T output) {
-      sideOutputWithTimestamp(tag, output, BoundedWindow.TIMESTAMP_MIN_VALUE);
-    }
-
-    public <T> void noteOutput(TupleTag<T> tag, ValueInSingleWindow<T> output) {
-      getMutableOutput(tag).add(output);
-    }
-  }
-
-  private TestProcessContext createProcessContext(TimestampedValue<InputT> elem) {
-    return new TestProcessContext(
-        ValueInSingleWindow.of(
-            elem.getValue(), elem.getTimestamp(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
   }
 
   private class TestProcessContext extends DoFn<InputT, OutputT>.ProcessContext {
@@ -641,7 +700,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
 
     private TestProcessContext(ValueInSingleWindow<InputT> element) {
       fn.super();
-      this.context = createContext(fn);
+      this.context = new TestContext();
       this.element = element;
     }
 
@@ -699,8 +758,8 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
 
     @Override
     public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-      context.noteOutput(
-          tag, ValueInSingleWindow.of(output, timestamp, element.getWindow(), element.getPane()));
+      getMutableOutput(tag)
+          .add(ValueInSingleWindow.of(output, timestamp, element.getWindow(), element.getPane()));
     }
 
     @Override
@@ -772,6 +831,29 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
 
   private DoFnTester(DoFn<InputT, OutputT> origFn) {
     this.origFn = origFn;
+    DoFnSignature signature = DoFnSignatures.signatureForDoFn(origFn);
+    for (DoFnSignature.Parameter param : signature.processElement().extraParameters()) {
+      param.match(
+          new DoFnSignature.Parameter.Cases.WithDefault<Void>() {
+            @Override
+            public Void dispatch(DoFnSignature.Parameter.ProcessContextParameter p) {
+              // ProcessContext parameter is obviously supported.
+              return null;
+            }
+
+            @Override
+            public Void dispatch(DoFnSignature.Parameter.WindowParameter p) {
+              // We also support the BoundedWindow parameter.
+              return null;
+            }
+
+            @Override
+            protected Void dispatchDefault(DoFnSignature.Parameter p) {
+              throw new UnsupportedOperationException(
+                  "Parameter " + p + " not supported by DoFnTester");
+            }
+          });
+    }
   }
 
   @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/78ac009b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
index ff8a9bc..b47465e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java
@@ -30,13 +30,16 @@ import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.testing.TestPipeline;
+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.util.PCollectionViews;
 import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.hamcrest.Matchers;
+import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Rule;
 import org.junit.Test;
@@ -350,6 +353,37 @@ public class DoFnTesterTest {
     }
   }
 
+  @Test
+  public void testSupportsWindowParameter() throws Exception {
+    Instant now = Instant.now();
+    try (DoFnTester<Integer, KV<Integer, BoundedWindow>> tester =
+        DoFnTester.of(new DoFnWithWindowParameter())) {
+      BoundedWindow firstWindow = new IntervalWindow(now, now.plus(Duration.standardMinutes(1)));
+      tester.processWindowedElement(1, now, firstWindow);
+      tester.processWindowedElement(2, now, firstWindow);
+      BoundedWindow secondWindow = new IntervalWindow(now, now.plus(Duration.standardMinutes(4)));
+      tester.processWindowedElement(3, now, secondWindow);
+      tester.finishBundle();
+
+      assertThat(
+          tester.peekOutputElementsInWindow(firstWindow),
+          containsInAnyOrder(
+              TimestampedValue.of(KV.of(1, firstWindow), now),
+              TimestampedValue.of(KV.of(2, firstWindow), now)));
+      assertThat(
+          tester.peekOutputElementsInWindow(secondWindow),
+          containsInAnyOrder(
+              TimestampedValue.of(KV.of(3, secondWindow), now)));
+    }
+  }
+
+  private static class DoFnWithWindowParameter extends DoFn<Integer, KV<Integer, BoundedWindow>> {
+    @ProcessElement
+    public void processElement(ProcessContext c, BoundedWindow window) {
+      c.output(KV.of(c.element(), window));
+    }
+  }
+
   private static class SideInputDoFn extends DoFn<Integer, Integer> {
     private final PCollectionView<Integer> value;
 


[35/50] [abbrv] incubator-beam git commit: Add TextIO.Write support for runtime-valued output prefix

Posted by ke...@apache.org.
Add TextIO.Write support for runtime-valued output prefix

* Updates to TextIO
* Updates for FileBasedSink to support this change
* Updates to other FileBasedSinks that do not yet support
  runtime values but need to be aware that values are now
  ValueProvider<String> instead of String


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/9a038c4f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/9a038c4f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/9a038c4f

Branch: refs/heads/gearpump-runner
Commit: 9a038c4f3404a3707eca29c5e898014df7fafbf4
Parents: 26eb435
Author: Sam McVeety <sg...@google.com>
Authored: Wed Nov 30 14:06:59 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Dec 2 17:24:12 2016 -0800

----------------------------------------------------------------------
 .../org/apache/beam/sdk/io/FileBasedSink.java   | 22 +++++++++------
 .../java/org/apache/beam/sdk/io/TextIO.java     | 28 ++++++++++++++++----
 .../java/org/apache/beam/sdk/io/XmlSink.java    |  4 +--
 .../org/apache/beam/sdk/io/XmlSinkTest.java     |  6 ++---
 4 files changed, 42 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a038c4f/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java
index 5375b90..1396ab6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java
@@ -41,6 +41,8 @@ import javax.annotation.Nullable;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.SerializableCoder;
 import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.util.IOChannelFactory;
 import org.apache.beam.sdk.util.IOChannelUtils;
@@ -135,7 +137,7 @@ public abstract class FileBasedSink<T> extends Sink<T> {
   /**
    * Base filename for final output files.
    */
-  protected final String baseOutputFilename;
+  protected final ValueProvider<String> baseOutputFilename;
 
   /**
    * The extension to be used for the final output files.
@@ -162,7 +164,8 @@ public abstract class FileBasedSink<T> extends Sink<T> {
    */
   public FileBasedSink(String baseOutputFilename, String extension,
       WritableByteChannelFactory writableByteChannelFactory) {
-    this(baseOutputFilename, extension, ShardNameTemplate.INDEX_OF_MAX, writableByteChannelFactory);
+    this(StaticValueProvider.of(baseOutputFilename), extension,
+        ShardNameTemplate.INDEX_OF_MAX, writableByteChannelFactory);
   }
 
   /**
@@ -173,7 +176,8 @@ public abstract class FileBasedSink<T> extends Sink<T> {
    * <p>See {@link ShardNameTemplate} for a description of file naming templates.
    */
   public FileBasedSink(String baseOutputFilename, String extension, String fileNamingTemplate) {
-    this(baseOutputFilename, extension, fileNamingTemplate, CompressionType.UNCOMPRESSED);
+    this(StaticValueProvider.of(baseOutputFilename), extension, fileNamingTemplate,
+        CompressionType.UNCOMPRESSED);
   }
 
   /**
@@ -182,8 +186,8 @@ public abstract class FileBasedSink<T> extends Sink<T> {
    *
    * <p>See {@link ShardNameTemplate} for a description of file naming templates.
    */
-  public FileBasedSink(String baseOutputFilename, String extension, String fileNamingTemplate,
-      WritableByteChannelFactory writableByteChannelFactory) {
+  public FileBasedSink(ValueProvider<String> baseOutputFilename, String extension,
+      String fileNamingTemplate, WritableByteChannelFactory writableByteChannelFactory) {
     this.writableByteChannelFactory = writableByteChannelFactory;
     this.baseOutputFilename = baseOutputFilename;
     if (!isNullOrEmpty(writableByteChannelFactory.getFilenameSuffix())) {
@@ -198,7 +202,7 @@ public abstract class FileBasedSink<T> extends Sink<T> {
    * Returns the base output filename for this file based sink.
    */
   public String getBaseOutputFilename() {
-    return baseOutputFilename;
+    return baseOutputFilename.get();
   }
 
   @Override
@@ -216,7 +220,9 @@ public abstract class FileBasedSink<T> extends Sink<T> {
     super.populateDisplayData(builder);
 
     String fileNamePattern = String.format("%s%s%s",
-        baseOutputFilename, fileNamingTemplate, getFileExtension(extension));
+        baseOutputFilename.isAccessible()
+        ? baseOutputFilename.get() : baseOutputFilename.toString(),
+        fileNamingTemplate, getFileExtension(extension));
     builder.add(DisplayData.item("fileNamePattern", fileNamePattern)
       .withLabel("File Name Pattern"));
   }
@@ -420,7 +426,7 @@ public abstract class FileBasedSink<T> extends Sink<T> {
     protected final List<String> generateDestinationFilenames(int numFiles) {
       List<String> destFilenames = new ArrayList<>();
       String extension = getSink().extension;
-      String baseOutputFilename = getSink().baseOutputFilename;
+      String baseOutputFilename = getSink().baseOutputFilename.get();
       String fileNamingTemplate = getSink().fileNamingTemplate;
 
       String suffix = getFileExtension(extension);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a038c4f/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
index 84c24ea..e967a27 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
@@ -404,6 +404,13 @@ public class TextIO {
     }
 
     /**
+     * Like {@link #to(String)}, but with a {@link ValueProvider}.
+     */
+    public static Bound<String> to(ValueProvider<String> prefix) {
+      return new Bound<>(DEFAULT_TEXT_CODER).to(prefix);
+    }
+
+    /**
      * Returns a transform for writing to text files that appends the specified suffix
      * to the created files.
      */
@@ -521,7 +528,7 @@ public class TextIO {
       private static final String DEFAULT_SHARD_TEMPLATE = ShardNameTemplate.INDEX_OF_MAX;
 
       /** The prefix of each file written, combined with suffix and shardTemplate. */
-      @Nullable private final String filenamePrefix;
+      private final ValueProvider<String> filenamePrefix;
       /** The suffix of each file written, combined with prefix and shardTemplate. */
       private final String filenameSuffix;
 
@@ -554,7 +561,7 @@ public class TextIO {
             FileBasedSink.CompressionType.UNCOMPRESSED);
       }
 
-      private Bound(String name, String filenamePrefix, String filenameSuffix,
+      private Bound(String name, ValueProvider<String> filenamePrefix, String filenameSuffix,
           @Nullable String header, @Nullable String footer, Coder<T> coder, int numShards,
           String shardTemplate, boolean validate,
           WritableByteChannelFactory writableByteChannelFactory) {
@@ -581,6 +588,15 @@ public class TextIO {
        */
       public Bound<T> to(String filenamePrefix) {
         validateOutputComponent(filenamePrefix);
+        return new Bound<>(name, StaticValueProvider.of(filenamePrefix), filenameSuffix,
+            header, footer, coder, numShards, shardTemplate, validate,
+            writableByteChannelFactory);
+      }
+
+      /**
+       * Like {@link #to(String)}, but with a {@link ValueProvider}.
+       */
+      public Bound<T> to(ValueProvider<String> filenamePrefix) {
         return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards,
             shardTemplate, validate, writableByteChannelFactory);
       }
@@ -745,8 +761,10 @@ public class TextIO {
       public void populateDisplayData(DisplayData.Builder builder) {
         super.populateDisplayData(builder);
 
+        String prefixString = filenamePrefix.isAccessible()
+            ? filenamePrefix.get() : filenamePrefix.toString();
         builder
-            .addIfNotNull(DisplayData.item("filePrefix", filenamePrefix)
+            .addIfNotNull(DisplayData.item("filePrefix", prefixString)
               .withLabel("Output File Prefix"))
             .addIfNotDefault(DisplayData.item("fileSuffix", filenameSuffix)
               .withLabel("Output Fix Suffix"), "")
@@ -779,7 +797,7 @@ public class TextIO {
       }
 
       public String getFilenamePrefix() {
-        return filenamePrefix;
+        return filenamePrefix.get();
       }
 
       public String getShardTemplate() {
@@ -1101,7 +1119,7 @@ public class TextIO {
 
     @VisibleForTesting
     TextSink(
-        String baseOutputFilename, String extension,
+        ValueProvider<String> baseOutputFilename, String extension,
         @Nullable String header, @Nullable String footer,
         String fileNameTemplate, Coder<T> coder,
         WritableByteChannelFactory writableByteChannelFactory) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a038c4f/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java
index 983eed2..0f25aea 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java
@@ -176,7 +176,7 @@ public class XmlSink {
      * <p>The specified class must be able to be used to create a JAXB context.
      */
     public <T> Bound<T> ofRecordClass(Class<T> classToBind) {
-      return new Bound<>(classToBind, rootElementName, baseOutputFilename);
+      return new Bound<>(classToBind, rootElementName, baseOutputFilename.get());
     }
 
     /**
@@ -194,7 +194,7 @@ public class XmlSink {
      * supplied name.
      */
     public Bound<T> withRootElement(String rootElementName) {
-      return new Bound<>(classToBind, rootElementName, baseOutputFilename);
+      return new Bound<>(classToBind, rootElementName, baseOutputFilename.get());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a038c4f/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java
index 400b04a..f9a9655 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSinkTest.java
@@ -93,7 +93,7 @@ public class XmlSinkTest {
             .withRootElement(testRootElement);
     assertEquals(testClass, sink.classToBind);
     assertEquals(testRootElement, sink.rootElementName);
-    assertEquals(testFilePrefix, sink.baseOutputFilename);
+    assertEquals(testFilePrefix, sink.baseOutputFilename.get());
   }
 
   /**
@@ -105,7 +105,7 @@ public class XmlSinkTest {
         XmlSink.writeOf(Bird.class, testRootElement, testFilePrefix);
     assertEquals(testClass, sink.classToBind);
     assertEquals(testRootElement, sink.rootElementName);
-    assertEquals(testFilePrefix, sink.baseOutputFilename);
+    assertEquals(testFilePrefix, sink.baseOutputFilename.get());
   }
 
   /**
@@ -142,7 +142,7 @@ public class XmlSinkTest {
         XmlSink.writeOf(testClass, testRootElement, testFilePrefix);
     XmlWriteOperation<Bird> writeOp = sink.createWriteOperation(options);
     assertEquals(testClass, writeOp.getSink().classToBind);
-    assertEquals(testFilePrefix, writeOp.getSink().baseOutputFilename);
+    assertEquals(testFilePrefix, writeOp.getSink().baseOutputFilename.get());
     assertEquals(testRootElement, writeOp.getSink().rootElementName);
     assertEquals(XmlSink.XML_EXTENSION, writeOp.getSink().extension);
     Path outputPath = new File(testFilePrefix).toPath();


[39/50] [abbrv] incubator-beam git commit: BEAM-1078: Changing the links from GCP to incubator-beam in the project

Posted by ke...@apache.org.
BEAM-1078: Changing the links from GCP to incubator-beam in the project


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/5a997a1a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/5a997a1a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/5a997a1a

Branch: refs/heads/gearpump-runner
Commit: 5a997a1a5d5d977bb84af1737db1128df916de7a
Parents: 8a7919b
Author: Neelesh Srinivas Salian <ns...@cloudera.com>
Authored: Fri Dec 2 17:43:34 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Dec 2 21:44:33 2016 -0800

----------------------------------------------------------------------
 .travis/README.md                                     |  2 +-
 .../java/org/apache/beam/examples/complete/README.md  | 14 +++++++-------
 .../java/org/apache/beam/examples/cookbook/README.md  | 14 +++++++-------
 3 files changed, 15 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5a997a1a/.travis/README.md
----------------------------------------------------------------------
diff --git a/.travis/README.md b/.travis/README.md
index e0c13f2..536692d 100644
--- a/.travis/README.md
+++ b/.travis/README.md
@@ -19,5 +19,5 @@
 
 # Travis Scripts
 
-This directory contains scripts used for [Travis CI](https://travis-ci.org/GoogleCloudPlatform/DataflowJavaSDK)
+This directory contains scripts used for [Travis CI](https://travis-ci.org/apache/incubator-beam/)
 testing.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5a997a1a/examples/java/src/main/java/org/apache/beam/examples/complete/README.md
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/README.md
index b98be7a..b0b6f9d 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/README.md
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/README.md
@@ -22,34 +22,34 @@
 This directory contains end-to-end example pipelines that perform complex data processing tasks. They include:
 
 <ul>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/AutoComplete.java">AutoComplete</a>
+  <li><a href="https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java">AutoComplete</a>
   &mdash; An example that computes the most popular hash tags for every
   prefix, which can be used for auto-completion. Demonstrates how to use the
   same pipeline in both streaming and batch, combiners, and composite
   transforms.</li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/StreamingWordExtract.java">StreamingWordExtract</a>
+  <li><a href="https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java">StreamingWordExtract</a>
   &mdash; A streaming pipeline example that inputs lines of text from a Cloud
   Pub/Sub topic, splits each line into individual words, capitalizes those
   words, and writes the output to a BigQuery table.
   </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/TfIdf.java">TfIdf</a>
+  <li><a href="https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java">TfIdf</a>
   &mdash; An example that computes a basic TF-IDF search table for a directory or
   Cloud Storage prefix. Demonstrates joining data, side inputs, and logging.
   </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/TopWikipediaSessions.java">TopWikipediaSessions</a>
+  <li><a href="https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java">TopWikipediaSessions</a>
   &mdash; An example that reads Wikipedia edit data from Cloud Storage and
   computes the user with the longest string of edits separated by no more than
   an hour within each month. Demonstrates using Cloud Dataflow
   <code>Windowing</code> to perform time-based aggregations of data.
   </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/TrafficMaxLaneFlow.java">TrafficMaxLaneFlow</a>
+  <li><a href="https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java">TrafficMaxLaneFlow</a>
   &mdash; A streaming Beam Example using BigQuery output in the
   <code>traffic sensor</code> domain. Demonstrates the Cloud Dataflow streaming
   runner, sliding windows, Cloud Pub/Sub topic ingestion, the use of the
   <code>AvroCoder</code> to encode a custom class, and custom
   <code>Combine</code> transforms.
   </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/complete/TrafficRoutes.java">TrafficRoutes</a>
+  <li><a href="https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java">TrafficRoutes</a>
   &mdash; A streaming Beam Example using BigQuery output in the
   <code>traffic sensor</code> domain. Demonstrates the Cloud Dataflow streaming
   runner, <code>GroupByKey</code>, keyed state, sliding windows, and Cloud
@@ -57,6 +57,6 @@ This directory contains end-to-end example pipelines that perform complex data p
   </li>
   </ul>
 
-See the [documentation](https://cloud.google.com/dataflow/getting-started) and the [Examples
+See the [documentation](http://beam.incubator.apache.org/get-started/quickstart/) and the [Examples
 README](../../../../../../../../../README.md) for
 information about how to run these examples.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5a997a1a/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md b/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md
index 2edd26b..e709955 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/README.md
@@ -24,12 +24,12 @@ commonly-used data analysis patterns that you would likely incorporate into a
 larger Dataflow pipeline. They include:
 
  <ul>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/BigQueryTornadoes.java">BigQueryTornadoes</a>
+  <li><a href="https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java">BigQueryTornadoes</a>
   &mdash; An example that reads the public samples of weather data from Google
   BigQuery, counts the number of tornadoes that occur in each month, and
   writes the results to BigQuery. Demonstrates reading/writing BigQuery,
   counting a <code>PCollection</code>, and user-defined <code>PTransforms</code>.</li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/CombinePerKeyExamples.java">CombinePerKeyExamples</a>
+  <li><a href="https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java">CombinePerKeyExamples</a>
   &mdash; An example that reads the public &quot;Shakespeare&quot; data, and for
   each word in the dataset that exceeds a given length, generates a string
   containing the list of play names in which that word appears.
@@ -39,13 +39,13 @@ larger Dataflow pipeline. They include:
   </li>
   <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/DatastoreWordCount.java">DatastoreWordCount</a>
   &mdash; An example that shows you how to read from Google Cloud Datastore.</li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/DeDupExample.java">DeDupExample</a>
+  <li><a href="https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/cookbook/DistinctExample.java">DistinctExample</a>
   &mdash; An example that uses Shakespeare's plays as plain text files, and
   removes duplicate lines across all the files. Demonstrates the
   <code>Distinct</code>, <code>TextIO.Read</code>,
   and <code>TextIO.Write</code> transforms, and how to wire transforms together.
   </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/FilterExamples.java">FilterExamples</a>
+  <li><a href="https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java">FilterExamples</a>
   &mdash; An example that shows different approaches to filtering, including
   selection and projection. It also shows how to dynamically set parameters
   by defining and using new pipeline options, and use how to use a value derived
@@ -53,14 +53,14 @@ larger Dataflow pipeline. They include:
   <code>Options</code> configuration, and using pipeline-derived data as a side
   input.
   </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/JoinExamples.java">JoinExamples</a>
+  <li><a href="https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java">JoinExamples</a>
   &mdash; An example that shows how to join two collections. It uses a
   sample of the <a href="http://goo.gl/OB6oin">GDELT &quot;world event&quot;
   data</a>, joining the event <code>action</code> country code against a table
   that maps country codes to country names. Demonstrates the <code>Join</code>
   operation, and using multiple input sources.
   </li>
-  <li><a href="https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/cookbook/MaxPerKeyExamples.java">MaxPerKeyExamples</a>
+  <li><a href="https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java">MaxPerKeyExamples</a>
   &mdash; An example that reads the public samples of weather data from BigQuery,
   and finds the maximum temperature (<code>mean_temp</code>) for each month.
   Demonstrates the <code>Max</code> statistical combination transform, and how to
@@ -68,6 +68,6 @@ larger Dataflow pipeline. They include:
   </li>
   </ul>
 
-See the [documentation](https://cloud.google.com/dataflow/getting-started) and the [Examples
+See the [documentation](http://beam.incubator.apache.org/get-started/quickstart/) and the [Examples
 README](../../../../../../../../../README.md) for
 information about how to run these examples.


[29/50] [abbrv] incubator-beam git commit: BEAM-879: Changing DeDupExample to DistinctExample

Posted by ke...@apache.org.
BEAM-879: Changing DeDupExample to DistinctExample


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/e3dca4ca
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/e3dca4ca
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/e3dca4ca

Branch: refs/heads/gearpump-runner
Commit: e3dca4cab6914166465c70f5f0b4be4f06ddd088
Parents: 1abbb90
Author: Neelesh Srinivas Salian <ns...@cloudera.com>
Authored: Thu Dec 1 20:28:43 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Dec 2 15:20:16 2016 -0800

----------------------------------------------------------------------
 .../beam/examples/cookbook/DeDupExample.java    | 96 --------------------
 .../beam/examples/cookbook/DistinctExample.java | 96 ++++++++++++++++++++
 2 files changed, 96 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e3dca4ca/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
deleted file mode 100644
index 34fb901..0000000
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
+++ /dev/null
@@ -1,96 +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.examples.cookbook;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.DefaultValueFactory;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.Distinct;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-
-/**
- * This example uses as input Shakespeare's plays as plaintext files, and will remove any
- * duplicate lines across all the files. (The output does not preserve any input order).
- *
- * <p>Concepts: the Distinct transform, and how to wire transforms together.
- * Demonstrates {@link org.apache.beam.sdk.io.TextIO.Read}/
- * {@link Distinct}/{@link org.apache.beam.sdk.io.TextIO.Write}.
- *
- * <p>To execute this pipeline locally, specify a local output file or output prefix on GCS:
- *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
- *
- * <p>To change the runner, specify:
- * <pre>{@code
- *   --runner=YOUR_SELECTED_RUNNER
- * }
- * </pre>
- * See examples/java/README.md for instructions about how to configure different runners.
- *
- * <p>The input defaults to {@code gs://apache-beam-samples/shakespeare/*} and can be
- * overridden with {@code --input}.
- */
-public class DeDupExample {
-
-  /**
-   * Options supported by {@link DeDupExample}.
-   *
-   * <p>Inherits standard configuration options.
-   */
-  private interface Options extends PipelineOptions {
-    @Description("Path to the directory or GCS prefix containing files to read from")
-    @Default.String("gs://apache-beam-samples/shakespeare/*")
-    String getInput();
-    void setInput(String value);
-
-    @Description("Path of the file to write to")
-    @Default.InstanceFactory(OutputFactory.class)
-    String getOutput();
-    void setOutput(String value);
-
-    /** Returns gs://${TEMP_LOCATION}/"deduped.txt". */
-    class OutputFactory implements DefaultValueFactory<String> {
-      @Override
-      public String create(PipelineOptions options) {
-        if (options.getTempLocation() != null) {
-          return GcsPath.fromUri(options.getTempLocation())
-              .resolve("deduped.txt").toString();
-        } else {
-          throw new IllegalArgumentException("Must specify --output or --tempLocation");
-        }
-      }
-    }
-  }
-
-
-  public static void main(String[] args)
-      throws Exception {
-
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    Pipeline p = Pipeline.create(options);
-
-    p.apply("ReadLines", TextIO.Read.from(options.getInput()))
-     .apply(Distinct.<String>create())
-     .apply("DedupedShakespeare", TextIO.Write.to(options.getOutput()));
-
-    p.run().waitUntilFinish();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e3dca4ca/examples/java/src/main/java/org/apache/beam/examples/cookbook/DistinctExample.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DistinctExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DistinctExample.java
new file mode 100644
index 0000000..9670b7f
--- /dev/null
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DistinctExample.java
@@ -0,0 +1,96 @@
+/*
+ * 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.examples.cookbook;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Distinct;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+
+/**
+ * This example uses as input Shakespeare's plays as plaintext files, and will remove any
+ * duplicate lines across all the files. (The output does not preserve any input order).
+ *
+ * <p>Concepts: the Distinct transform, and how to wire transforms together.
+ * Demonstrates {@link org.apache.beam.sdk.io.TextIO.Read}/
+ * {@link Distinct}/{@link org.apache.beam.sdk.io.TextIO.Write}.
+ *
+ * <p>To execute this pipeline locally, specify a local output file or output prefix on GCS:
+ *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
+ *
+ * <p>To change the runner, specify:
+ * <pre>{@code
+ *   --runner=YOUR_SELECTED_RUNNER
+ * }
+ * </pre>
+ * See examples/java/README.md for instructions about how to configure different runners.
+ *
+ * <p>The input defaults to {@code gs://apache-beam-samples/shakespeare/*} and can be
+ * overridden with {@code --input}.
+ */
+public class DistinctExample {
+
+  /**
+   * Options supported by {@link DistinctExample}.
+   *
+   * <p>Inherits standard configuration options.
+   */
+  private interface Options extends PipelineOptions {
+    @Description("Path to the directory or GCS prefix containing files to read from")
+    @Default.String("gs://apache-beam-samples/shakespeare/*")
+    String getInput();
+    void setInput(String value);
+
+    @Description("Path of the file to write to")
+    @Default.InstanceFactory(OutputFactory.class)
+    String getOutput();
+    void setOutput(String value);
+
+    /** Returns gs://${TEMP_LOCATION}/"deduped.txt". */
+    class OutputFactory implements DefaultValueFactory<String> {
+      @Override
+      public String create(PipelineOptions options) {
+        if (options.getTempLocation() != null) {
+          return GcsPath.fromUri(options.getTempLocation())
+              .resolve("deduped.txt").toString();
+        } else {
+          throw new IllegalArgumentException("Must specify --output or --tempLocation");
+        }
+      }
+    }
+  }
+
+
+  public static void main(String[] args)
+      throws Exception {
+
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+    Pipeline p = Pipeline.create(options);
+
+    p.apply("ReadLines", TextIO.Read.from(options.getInput()))
+     .apply(Distinct.<String>create())
+     .apply("DedupedShakespeare", TextIO.Write.to(options.getOutput()));
+
+    p.run().waitUntilFinish();
+  }
+}


[43/50] [abbrv] incubator-beam git commit: Autogenerate examples archetype during build

Posted by ke...@apache.org.
Autogenerate examples archetype during build


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/abd96b89
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/abd96b89
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/abd96b89

Branch: refs/heads/gearpump-runner
Commit: abd96b8986ef922057375279312fcd0fa66ab7ba
Parents: 9cb8b5f
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Nov 17 15:19:16 2016 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Sat Dec 3 16:12:00 2016 -0800

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 .../examples/generate-sources.sh                |  68 ++++
 sdks/java/maven-archetypes/examples/pom.xml     |  27 +-
 .../main/resources/archetype-resources/pom.xml  |  19 +-
 .../src/main/java/DebuggingWordCount.java       | 164 ---------
 .../src/main/java/MinimalWordCount.java         | 118 -------
 .../src/main/java/WindowedWordCount.java        | 229 ------------
 .../src/main/java/WordCount.java                | 186 ----------
 .../common/ExampleBigQueryTableOptions.java     |  55 ---
 .../src/main/java/common/ExampleOptions.java    |  37 --
 ...xamplePubsubTopicAndSubscriptionOptions.java |  45 ---
 .../java/common/ExamplePubsubTopicOptions.java  |  45 ---
 .../src/main/java/common/ExampleUtils.java      | 352 -------------------
 .../src/test/java/DebuggingWordCountTest.java   |  52 ---
 .../src/test/java/WordCountTest.java            |  85 -----
 sdks/java/maven-archetypes/pom.xml              |  31 +-
 .../update-examples-archetype.sh                |  59 ----
 17 files changed, 120 insertions(+), 1453 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index fcfeafc..39e2f62 100644
--- a/.gitignore
+++ b/.gitignore
@@ -7,6 +7,7 @@ target/
 bin/
 
 # Ignore generated archetypes
+sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/
 sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/src/
 
 # Ignore IntelliJ files.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/examples/generate-sources.sh
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/generate-sources.sh b/sdks/java/maven-archetypes/examples/generate-sources.sh
new file mode 100755
index 0000000..b1a86a7
--- /dev/null
+++ b/sdks/java/maven-archetypes/examples/generate-sources.sh
@@ -0,0 +1,68 @@
+#!/bin/bash -e
+#
+#    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.
+#
+
+# Updates the examples archetype to match the examples module
+#
+# Usage: Invoke with no arguments from any working directory.
+
+# The directory of this script. Assumes root of the maven-archetypes module.
+HERE="$(dirname $0)"
+
+# The directory of the examples-java module
+EXAMPLES_ROOT="${HERE}/../../../../examples/java"
+
+# The root of the examples archetype
+ARCHETYPE_ROOT="${HERE}/src/main/resources/archetype-resources"
+
+mkdir -p "${ARCHETYPE_ROOT}/src/main/java"
+mkdir -p "${ARCHETYPE_ROOT}/src/test/java"
+
+#
+# Copy the Java subset of the examples project verbatim. 
+#
+rsync -a --exclude cookbook --exclude complete                  \
+    "${EXAMPLES_ROOT}"/src/main/java/org/apache/beam/examples/  \
+    "${ARCHETYPE_ROOT}/src/main/java"
+
+rsync -a --exclude cookbook --exclude complete --exclude '*IT.java'  \
+    "${EXAMPLES_ROOT}"/src/test/java/org/apache/beam/examples/        \
+    "${ARCHETYPE_ROOT}/src/test/java"
+
+#
+# Replace 'package org.apache.beam.examples' with 'package ${package}' in all Java code
+#
+find "${ARCHETYPE_ROOT}/src/main/java" -name '*.java' -print0 \
+    | xargs -0 sed -i.bak 's/^package org\.apache\.beam\.examples/package ${package}/g'
+
+find "${ARCHETYPE_ROOT}/src/test/java" -name '*.java' -print0 \
+    | xargs -0 sed -i.bak 's/^package org\.apache\.beam\.examples/package ${package}/g'
+
+#
+# Replace 'import org.apache.beam.examples.' with 'import ${package}.' in all Java code
+#
+find "${ARCHETYPE_ROOT}/src/main/java" -name '*.java' -print0 \
+    | xargs -0 sed -i.bak 's/^import org\.apache\.beam\.examples/import ${package}/g'
+
+find "${ARCHETYPE_ROOT}/src/test/java" -name '*.java' -print0 \
+    | xargs -0 sed -i.bak 's/^import org\.apache\.beam\.examples/import ${package}/g'
+
+#
+# The use of -i.bak is necessary for the above to work with both GNU and BSD sed. 
+# Delete the files now.
+#
+find "${ARCHETYPE_ROOT}/src" -name '*.bak' -delete

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/examples/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/pom.xml b/sdks/java/maven-archetypes/examples/pom.xml
index b8555bc..637231f 100644
--- a/sdks/java/maven-archetypes/examples/pom.xml
+++ b/sdks/java/maven-archetypes/examples/pom.xml
@@ -15,7 +15,9 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 
   <modelVersion>4.0.0</modelVersion>
 
@@ -29,7 +31,8 @@
   <artifactId>beam-sdks-java-maven-archetypes-examples</artifactId>
   <name>Apache Beam :: SDKs :: Java :: Maven Archetypes :: Examples</name>
   <description>A Maven Archetype to create a project containing all the
-    example pipelines from the Apache Beam Java SDK.</description>
+    example pipelines from the Apache Beam Java SDK.
+  </description>
 
   <packaging>maven-archetype</packaging>
 
@@ -65,7 +68,27 @@
         </plugin>
       </plugins>
     </pluginManagement>
+
+    <plugins>
+      <plugin>
+        <artifactId>exec-maven-plugin</artifactId>
+        <groupId>org.codehaus.mojo</groupId>
+        <executions>
+          <execution>
+            <id>generate-archetype-contents</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <executable>${project.basedir}/generate-sources.sh</executable>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
   </build>
+
   <dependencies>
     <dependency>
       <groupId>org.apache.beam</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
index 48f56fd..df2e9f3 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml
@@ -85,24 +85,7 @@
             <cleanupDaemonThreads>false</cleanupDaemonThreads>
           </configuration>
         </plugin>
- 
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <version>2.10</version>
-        <executions>
-          <execution>
-            <goals><goal>analyze-only</goal></goals>
-            <configuration>
-              <!-- Ignore runtime-only dependencies in analysis -->
-              <ignoreNonCompile>true</ignoreNonCompile>
-              <failOnWarning>true</failOnWarning>
-            </configuration>
-          </execution>
-        </executions>
-       </plugin>
-     </plugins>
-
+      </plugins>
     </pluginManagement>
   </build>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java
deleted file mode 100644
index 99ae796..0000000
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java
+++ /dev/null
@@ -1,164 +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 ${package};
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.regex.Pattern;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * An example that verifies word counts in Shakespeare and includes Beam best practices.
- *
- * <p>This class, {@link DebuggingWordCount}, is the third in a series of four successively more
- * detailed 'word count' examples. You may first want to take a look at {@link MinimalWordCount}
- * and {@link WordCount}. After you've looked at this example, then see the
- * {@link WindowedWordCount} pipeline, for introduction of additional concepts.
- *
- * <p>Basic concepts, also in the MinimalWordCount and WordCount examples:
- * Reading text files; counting a PCollection; executing a Pipeline both locally
- * and using a selected runner; defining DoFns.
- *
- * <p>New Concepts:
- * <pre>
- *   1. Logging using SLF4J, even in a distributed environment
- *   2. Creating a custom aggregator (runners have varying levels of support)
- *   3. Testing your Pipeline via PAssert
- * </pre>
- *
- * <p>To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- *   --project=YOUR_PROJECT_ID
- * }
- * </pre>
- *
- * <p>To change the runner, specify:
- * <pre>{@code
- *   --runner=YOUR_SELECTED_RUNNER
- * }
- * </pre>
- *
- * <p>The input file defaults to a public data set containing the text of of King Lear,
- * by William Shakespeare. You can override it and choose your own input with {@code --inputFile}.
- *
- */
-public class DebuggingWordCount {
-  /** A DoFn that filters for a specific key based upon a regular expression. */
-  public static class FilterTextFn extends DoFn<KV<String, Long>, KV<String, Long>> {
-    /**
-     * Concept #1: The logger below uses the fully qualified class name of FilterTextFn as the
-     * logger. Depending on your SLF4J configuration, log statements will likely be qualified by
-     * this name.
-     *
-     * <p>Note that this is entirely standard SLF4J usage. Some runners may provide a default SLF4J
-     * configuration that is most appropriate for their logging integration.
-     */
-    private static final Logger LOG = LoggerFactory.getLogger(FilterTextFn.class);
-
-    private final Pattern filter;
-    public FilterTextFn(String pattern) {
-      filter = Pattern.compile(pattern);
-    }
-
-    /**
-     * Concept #2: A custom aggregator can track values in your pipeline as it runs. Each
-     * runner provides varying levels of support for aggregators, and may expose them
-     * in a dashboard, etc.
-     */
-    private final Aggregator<Long, Long> matchedWords =
-        createAggregator("matchedWords", new Sum.SumLongFn());
-    private final Aggregator<Long, Long> unmatchedWords =
-        createAggregator("umatchedWords", new Sum.SumLongFn());
-
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      if (filter.matcher(c.element().getKey()).matches()) {
-        // Log at the "DEBUG" level each element that we match. When executing this pipeline
-        // these log lines will appear only if the log level is set to "DEBUG" or lower.
-        LOG.debug("Matched: " + c.element().getKey());
-        matchedWords.addValue(1L);
-        c.output(c.element());
-      } else {
-        // Log at the "TRACE" level each element that is not matched. Different log levels
-        // can be used to control the verbosity of logging providing an effective mechanism
-        // to filter less important information.
-        LOG.trace("Did not match: " + c.element().getKey());
-        unmatchedWords.addValue(1L);
-      }
-    }
-  }
-
-  /**
-   * Options supported by {@link DebuggingWordCount}.
-   *
-   * <p>Inherits standard configuration options and all options defined in
-   * {@link WordCount.WordCountOptions}.
-   */
-  public interface WordCountOptions extends WordCount.WordCountOptions {
-
-    @Description("Regex filter pattern to use in DebuggingWordCount. "
-        + "Only words matching this pattern will be counted.")
-    @Default.String("Flourish|stomach")
-    String getFilterPattern();
-    void setFilterPattern(String value);
-  }
-
-  public static void main(String[] args) {
-    WordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation()
-      .as(WordCountOptions.class);
-    Pipeline p = Pipeline.create(options);
-
-    PCollection<KV<String, Long>> filteredWords =
-        p.apply("ReadLines", TextIO.Read.from(options.getInputFile()))
-         .apply(new WordCount.CountWords())
-         .apply(ParDo.of(new FilterTextFn(options.getFilterPattern())));
-
-    /**
-     * Concept #3: PAssert is a set of convenient PTransforms in the style of
-     * Hamcrest's collection matchers that can be used when writing Pipeline level tests
-     * to validate the contents of PCollections. PAssert is best used in unit tests
-     * with small data sets but is demonstrated here as a teaching tool.
-     *
-     * <p>Below we verify that the set of filtered words matches our expected counts. Note
-     * that PAssert does not provide any output and that successful completion of the
-     * Pipeline implies that the expectations were met. Learn more at
-     * https://cloud.google.com/dataflow/pipelines/testing-your-pipeline on how to test
-     * your Pipeline and see {@link DebuggingWordCountTest} for an example unit test.
-     */
-    List<KV<String, Long>> expectedResults = Arrays.asList(
-        KV.of("Flourish", 3L),
-        KV.of("stomach", 1L));
-    PAssert.that(filteredWords).containsInAnyOrder(expectedResults);
-
-    p.run().waitUntilFinish();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java
deleted file mode 100644
index 97bd824..0000000
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.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 ${package};
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.values.KV;
-
-
-/**
- * An example that counts words in Shakespeare.
- *
- * <p>This class, {@link MinimalWordCount}, is the first in a series of four successively more
- * detailed 'word count' examples. Here, for simplicity, we don't show any error-checking or
- * argument processing, and focus on construction of the pipeline, which chains together the
- * application of core transforms.
- *
- * <p>Next, see the {@link WordCount} pipeline, then the {@link DebuggingWordCount}, and finally the
- * {@link WindowedWordCount} pipeline, for more detailed examples that introduce additional
- * concepts.
- *
- * <p>Concepts:
- *
- * <pre>
- *   1. Reading data from text files
- *   2. Specifying 'inline' transforms
- *   3. Counting items in a PCollection
- *   4. Writing data to text files
- * </pre>
- *
- * <p>No arguments are required to run this pipeline. It will be executed with the DirectRunner. You
- * can see the results in the output files in your current working directory, with names like
- * "wordcounts-00001-of-00005. When running on a distributed service, you would use an appropriate
- * file service.
- */
-public class MinimalWordCount {
-
-  public static void main(String[] args) {
-    // Create a PipelineOptions object. This object lets us set various execution
-    // options for our pipeline, such as the runner you wish to use. This example
-    // will run with the DirectRunner by default, based on the class path configured
-    // in its dependencies.
-    PipelineOptions options = PipelineOptionsFactory.create();
-
-    // Create the Pipeline object with the options we defined above.
-    Pipeline p = Pipeline.create(options);
-
-    // Apply the pipeline's transforms.
-
-    // Concept #1: Apply a root transform to the pipeline; in this case, TextIO.Read to read a set
-    // of input text files. TextIO.Read returns a PCollection where each element is one line from
-    // the input text (a set of Shakespeare's texts).
-
-    // This example reads a public data set consisting of the complete works of Shakespeare.
-    p.apply(TextIO.Read.from("gs://apache-beam-samples/shakespeare/*"))
-
-     // Concept #2: Apply a ParDo transform to our PCollection of text lines. This ParDo invokes a
-     // DoFn (defined in-line) on each element that tokenizes the text line into individual words.
-     // The ParDo returns a PCollection<String>, where each element is an individual word in
-     // Shakespeare's collected texts.
-     .apply("ExtractWords", ParDo.of(new DoFn<String, String>() {
-                       @ProcessElement
-                       public void processElement(ProcessContext c) {
-                         for (String word : c.element().split("[^a-zA-Z']+")) {
-                           if (!word.isEmpty()) {
-                             c.output(word);
-                           }
-                         }
-                       }
-                     }))
-
-     // Concept #3: Apply the Count transform to our PCollection of individual words. The Count
-     // transform returns a new PCollection of key/value pairs, where each key represents a unique
-     // word in the text. The associated value is the occurrence count for that word.
-     .apply(Count.<String>perElement())
-
-     // Apply a MapElements transform that formats our PCollection of word counts into a printable
-     // string, suitable for writing to an output file.
-     .apply("FormatResults", MapElements.via(new SimpleFunction<KV<String, Long>, String>() {
-                       @Override
-                       public String apply(KV<String, Long> input) {
-                         return input.getKey() + ": " + input.getValue();
-                       }
-                     }))
-
-     // Concept #4: Apply a write transform, TextIO.Write, at the end of the pipeline.
-     // TextIO.Write writes the contents of a PCollection (in this case, our PCollection of
-     // formatted strings) to a series of text files.
-     //
-     // By default, it will write to a set of files with names like wordcount-00001-of-00005
-     .apply(TextIO.Write.to("wordcounts"));
-
-    // Run the pipeline.
-    p.run().waitUntilFinish();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
deleted file mode 100644
index 2812531..0000000
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
+++ /dev/null
@@ -1,229 +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 ${package};
-
-import com.google.api.services.bigquery.model.TableFieldSchema;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import ${package}.common.ExampleBigQueryTableOptions;
-import ${package}.common.ExampleOptions;
-import ${package}.common.ExampleUtils;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-
-/**
- * An example that counts words in text, and can run over either unbounded or bounded input
- * collections.
- *
- * <p>This class, {@link WindowedWordCount}, is the last in a series of four successively more
- * detailed 'word count' examples. First take a look at {@link MinimalWordCount},
- * {@link WordCount}, and {@link DebuggingWordCount}.
- *
- * <p>Basic concepts, also in the MinimalWordCount, WordCount, and DebuggingWordCount examples:
- * Reading text files; counting a PCollection; writing to GCS; executing a Pipeline both locally
- * and using a selected runner; defining DoFns; creating a custom aggregator;
- * user-defined PTransforms; defining PipelineOptions.
- *
- * <p>New Concepts:
- * <pre>
- *   1. Unbounded and bounded pipeline input modes
- *   2. Adding timestamps to data
- *   3. Windowing
- *   4. Re-using PTransforms over windowed PCollections
- *   5. Writing to BigQuery
- * </pre>
- *
- * <p>By default, the examples will run with the {@code DirectRunner}.
- * To change the runner, specify:
- * <pre>{@code
- *   --runner=YOUR_SELECTED_RUNNER
- * }
- * </pre>
- * See examples/java/README.md for instructions about how to configure different runners.
- *
- * <p>Optionally specify the input file path via:
- * {@code --inputFile=gs://INPUT_PATH},
- * which defaults to {@code gs://apache-beam-samples/shakespeare/kinglear.txt}.
- *
- * <p>Specify an output BigQuery dataset and optionally, a table for the output. If you don't
- * specify the table, one will be created for you using the job name. If you don't specify the
- * dataset, a dataset called {@code beam_examples} must already exist in your project.
- * {@code --bigQueryDataset=YOUR-DATASET --bigQueryTable=YOUR-NEW-TABLE-NAME}.
- *
- * <p>By default, the pipeline will do fixed windowing, on 1-minute windows.  You can
- * change this interval by setting the {@code --windowSize} parameter, e.g. {@code --windowSize=10}
- * for 10-minute windows.
- *
- * <p>The example will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
- * and then exits.
- */
-public class WindowedWordCount {
-    static final int WINDOW_SIZE = 1;  // Default window duration in minutes
-
-  /**
-   * Concept #2: A DoFn that sets the data element timestamp. This is a silly method, just for
-   * this example, for the bounded data case.
-   *
-   * <p>Imagine that many ghosts of Shakespeare are all typing madly at the same time to recreate
-   * his masterworks. Each line of the corpus will get a random associated timestamp somewhere in a
-   * 2-hour period.
-   */
-  static class AddTimestampFn extends DoFn<String, String> {
-    private static final Duration RAND_RANGE = Duration.standardHours(2);
-    private final Instant minTimestamp;
-
-    AddTimestampFn() {
-      this.minTimestamp = new Instant(System.currentTimeMillis());
-    }
-
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      // Generate a timestamp that falls somewhere in the past two hours.
-      long randMillis = (long) (Math.random() * RAND_RANGE.getMillis());
-      Instant randomTimestamp = minTimestamp.plus(randMillis);
-      /**
-       * Concept #2: Set the data element with that timestamp.
-       */
-      c.outputWithTimestamp(c.element(), new Instant(randomTimestamp));
-    }
-  }
-
-  /** A DoFn that converts a Word and Count into a BigQuery table row. */
-  static class FormatAsTableRowFn extends DoFn<KV<String, Long>, TableRow> {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      TableRow row = new TableRow()
-          .set("word", c.element().getKey())
-          .set("count", c.element().getValue())
-          // include a field for the window timestamp
-         .set("window_timestamp", c.timestamp().toString());
-      c.output(row);
-    }
-  }
-
-  /**
-   * Helper method that defines the BigQuery schema used for the output.
-   */
-  private static TableSchema getSchema() {
-    List<TableFieldSchema> fields = new ArrayList<>();
-    fields.add(new TableFieldSchema().setName("word").setType("STRING"));
-    fields.add(new TableFieldSchema().setName("count").setType("INTEGER"));
-    fields.add(new TableFieldSchema().setName("window_timestamp").setType("TIMESTAMP"));
-    TableSchema schema = new TableSchema().setFields(fields);
-    return schema;
-  }
-
-  /**
-   * Concept #5: We'll stream the results to a BigQuery table. The BigQuery output source is one
-   * that supports both bounded and unbounded data. This is a helper method that creates a
-   * TableReference from input options, to tell the pipeline where to write its BigQuery results.
-   */
-  private static TableReference getTableReference(Options options) {
-    TableReference tableRef = new TableReference();
-    tableRef.setProjectId(options.getProject());
-    tableRef.setDatasetId(options.getBigQueryDataset());
-    tableRef.setTableId(options.getBigQueryTable());
-    return tableRef;
-  }
-
-  /**
-   * Options supported by {@link WindowedWordCount}.
-   *
-   * <p>Inherits standard example configuration options, which allow specification of the BigQuery
-   * table, as well as the {@link WordCount.WordCountOptions} support for
-   * specification of the input file.
-   */
-  public interface Options extends WordCount.WordCountOptions,
-      ExampleOptions, ExampleBigQueryTableOptions {
-    @Description("Fixed window duration, in minutes")
-    @Default.Integer(WINDOW_SIZE)
-    Integer getWindowSize();
-    void setWindowSize(Integer value);
-  }
-
-  public static void main(String[] args) throws IOException {
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-    options.setBigQuerySchema(getSchema());
-    // ExampleUtils creates the necessary input sources to simplify execution of this Pipeline.
-    ExampleUtils exampleUtils = new ExampleUtils(options);
-    exampleUtils.setup();
-
-    Pipeline pipeline = Pipeline.create(options);
-
-    /**
-     * Concept #1: the Beam SDK lets us run the same pipeline with either a bounded or
-     * unbounded input source.
-     */
-    PCollection<String> input = pipeline
-      /** Read from the GCS file. */
-      .apply(TextIO.Read.from(options.getInputFile()))
-      // Concept #2: Add an element timestamp, using an artificial time just to show windowing.
-      // See AddTimestampFn for more detail on this.
-      .apply(ParDo.of(new AddTimestampFn()));
-
-    /**
-     * Concept #3: Window into fixed windows. The fixed window size for this example defaults to 1
-     * minute (you can change this with a command-line option). See the documentation for more
-     * information on how fixed windows work, and for information on the other types of windowing
-     * available (e.g., sliding windows).
-     */
-    PCollection<String> windowedWords = input
-      .apply(Window.<String>into(
-        FixedWindows.of(Duration.standardMinutes(options.getWindowSize()))));
-
-    /**
-     * Concept #4: Re-use our existing CountWords transform that does not have knowledge of
-     * windows over a PCollection containing windowed values.
-     */
-    PCollection<KV<String, Long>> wordCounts = windowedWords.apply(new WordCount.CountWords());
-
-    /**
-     * Concept #5: Format the results for a BigQuery table, then write to BigQuery.
-     * The BigQuery output source supports both bounded and unbounded data.
-     */
-    wordCounts.apply(ParDo.of(new FormatAsTableRowFn()))
-        .apply(BigQueryIO.Write
-          .to(getTableReference(options))
-          .withSchema(getSchema())
-          .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
-          .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND));
-
-    PipelineResult result = pipeline.run();
-
-    // ExampleUtils will try to cancel the pipeline before the program exists.
-    exampleUtils.waitToFinish(result);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
deleted file mode 100644
index 8fe7137..0000000
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
+++ /dev/null
@@ -1,186 +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 ${package};
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.options.Validation.Required;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * An example that counts words in Shakespeare and includes Beam best practices.
- *
- * <p>This class, {@link WordCount}, is the second in a series of four successively more detailed
- * 'word count' examples. You may first want to take a look at {@link MinimalWordCount}.
- * After you've looked at this example, then see the {@link DebuggingWordCount}
- * pipeline, for introduction of additional concepts.
- *
- * <p>For a detailed walkthrough of this example, see
- *   <a href="http://beam.incubator.apache.org/use/walkthroughs/">
- *   http://beam.incubator.apache.org/use/walkthroughs/
- *   </a>
- *
- * <p>Basic concepts, also in the MinimalWordCount example:
- * Reading text files; counting a PCollection; writing to text files
- *
- * <p>New Concepts:
- * <pre>
- *   1. Executing a Pipeline both locally and using the selected runner
- *   2. Using ParDo with static DoFns defined out-of-line
- *   3. Building a composite transform
- *   4. Defining your own pipeline options
- * </pre>
- *
- * <p>Concept #1: you can execute this pipeline either locally or using by selecting another runner.
- * These are now command-line options and not hard-coded as they were in the MinimalWordCount
- * example.
- *
- * <p>To change the runner, specify:
- * <pre>{@code
- *   --runner=YOUR_SELECTED_RUNNER
- * }
- * </pre>
- *
- * <p>To execute this pipeline, specify a local output file (if using the
- * {@code DirectRunner}) or output prefix on a supported distributed file system.
- * <pre>{@code
- *   --output=[YOUR_LOCAL_FILE | YOUR_OUTPUT_PREFIX]
- * }</pre>
- *
- * <p>The input file defaults to a public data set containing the text of of King Lear,
- * by William Shakespeare. You can override it and choose your own input with {@code --inputFile}.
- */
-public class WordCount {
-
-  /**
-   * Concept #2: You can make your pipeline assembly code less verbose by defining your DoFns
-   * statically out-of-line. This DoFn tokenizes lines of text into individual words; we pass it
-   * to a ParDo in the pipeline.
-   */
-  static class ExtractWordsFn extends DoFn<String, String> {
-    private final Aggregator<Long, Long> emptyLines =
-        createAggregator("emptyLines", new Sum.SumLongFn());
-
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      if (c.element().trim().isEmpty()) {
-        emptyLines.addValue(1L);
-      }
-
-      // Split the line into words.
-      String[] words = c.element().split("[^a-zA-Z']+");
-
-      // Output each word encountered into the output PCollection.
-      for (String word : words) {
-        if (!word.isEmpty()) {
-          c.output(word);
-        }
-      }
-    }
-  }
-
-  /** A SimpleFunction that converts a Word and Count into a printable string. */
-  public static class FormatAsTextFn extends SimpleFunction<KV<String, Long>, String> {
-    @Override
-    public String apply(KV<String, Long> input) {
-      return input.getKey() + ": " + input.getValue();
-    }
-  }
-
-  /**
-   * A PTransform that converts a PCollection containing lines of text into a PCollection of
-   * formatted word counts.
-   *
-   * <p>Concept #3: This is a custom composite transform that bundles two transforms (ParDo and
-   * Count) as a reusable PTransform subclass. Using composite transforms allows for easy reuse,
-   * modular testing, and an improved monitoring experience.
-   */
-  public static class CountWords extends PTransform<PCollection<String>,
-      PCollection<KV<String, Long>>> {
-    @Override
-    public PCollection<KV<String, Long>> apply(PCollection<String> lines) {
-
-      // Convert lines of text into individual words.
-      PCollection<String> words = lines.apply(
-          ParDo.of(new ExtractWordsFn()));
-
-      // Count the number of times each word occurs.
-      PCollection<KV<String, Long>> wordCounts =
-          words.apply(Count.<String>perElement());
-
-      return wordCounts;
-    }
-  }
-
-  /**
-   * Options supported by {@link WordCount}.
-   *
-   * <p>Concept #4: Defining your own configuration options. Here, you can add your own arguments
-   * to be processed by the command-line parser, and specify default values for them. You can then
-   * access the options values in your pipeline code.
-   *
-   * <p>Inherits standard configuration options.
-   */
-  public interface WordCountOptions extends PipelineOptions {
-
-    /**
-     * By default, this example reads from a public dataset containing the text of
-     * King Lear. Set this option to choose a different input file or glob.
-     */
-    @Description("Path of the file to read from")
-    @Default.String("gs://apache-beam-samples/shakespeare/kinglear.txt")
-    String getInputFile();
-    void setInputFile(String value);
-
-    /**
-     * Set this required option to specify where to write the output.
-     */
-    @Description("Path of the file to write to")
-    @Required
-    String getOutput();
-    void setOutput(String value);
-  }
-
-  public static void main(String[] args) {
-    WordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation()
-      .as(WordCountOptions.class);
-    Pipeline p = Pipeline.create(options);
-
-    // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the
-    // static FormatAsTextFn() to the ParDo transform.
-    p.apply("ReadLines", TextIO.Read.from(options.getInputFile()))
-     .apply(new CountWords())
-     .apply(MapElements.via(new FormatAsTextFn()))
-     .apply("WriteCounts", TextIO.Write.to(options.getOutput()));
-
-    p.run().waitUntilFinish();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java
deleted file mode 100644
index 6b51074..0000000
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java
+++ /dev/null
@@ -1,55 +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 ${package}.common;
-
-import com.google.api.services.bigquery.model.TableSchema;
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.DefaultValueFactory;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.GcpOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-/**
- * Options that can be used to configure BigQuery tables in Beam examples.
- * The project defaults to the project being used to run the example.
- */
-public interface ExampleBigQueryTableOptions extends GcpOptions {
-  @Description("BigQuery dataset name")
-  @Default.String("beam_examples")
-  String getBigQueryDataset();
-  void setBigQueryDataset(String dataset);
-
-  @Description("BigQuery table name")
-  @Default.InstanceFactory(BigQueryTableFactory.class)
-  String getBigQueryTable();
-  void setBigQueryTable(String table);
-
-  @Description("BigQuery table schema")
-  TableSchema getBigQuerySchema();
-  void setBigQuerySchema(TableSchema schema);
-
-  /**
-   * Returns the job name as the default BigQuery table name.
-   */
-  class BigQueryTableFactory implements DefaultValueFactory<String> {
-    @Override
-    public String create(PipelineOptions options) {
-      return options.getJobName().replace('-', '_');
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleOptions.java
deleted file mode 100644
index 90f935c..0000000
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleOptions.java
+++ /dev/null
@@ -1,37 +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 ${package}.common;
-
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-/**
- * Options that can be used to configure the Beam examples.
- */
-public interface ExampleOptions extends PipelineOptions {
-  @Description("Whether to keep jobs running after local process exit")
-  @Default.Boolean(false)
-  boolean getKeepJobsRunning();
-  void setKeepJobsRunning(boolean keepJobsRunning);
-
-  @Description("Number of workers to use when executing the injector pipeline")
-  @Default.Integer(1)
-  int getInjectorNumWorkers();
-  void setInjectorNumWorkers(int numWorkers);
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicAndSubscriptionOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicAndSubscriptionOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicAndSubscriptionOptions.java
deleted file mode 100644
index daeb398..0000000
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicAndSubscriptionOptions.java
+++ /dev/null
@@ -1,45 +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 ${package}.common;
-
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.DefaultValueFactory;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.GcpOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-/**
- * Options that can be used to configure Pub/Sub topic/subscription in Beam examples.
- */
-public interface ExamplePubsubTopicAndSubscriptionOptions extends ExamplePubsubTopicOptions {
-  @Description("Pub/Sub subscription")
-  @Default.InstanceFactory(PubsubSubscriptionFactory.class)
-  String getPubsubSubscription();
-  void setPubsubSubscription(String subscription);
-
-  /**
-   * Returns a default Pub/Sub subscription based on the project and the job names.
-   */
-  class PubsubSubscriptionFactory implements DefaultValueFactory<String> {
-    @Override
-    public String create(PipelineOptions options) {
-      return "projects/" + options.as(GcpOptions.class).getProject()
-          + "/subscriptions/" + options.getJobName();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java
deleted file mode 100644
index 936bff5..0000000
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java
+++ /dev/null
@@ -1,45 +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 ${package}.common;
-
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.DefaultValueFactory;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.GcpOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-/**
- * Options that can be used to configure Pub/Sub topic in Beam examples.
- */
-public interface ExamplePubsubTopicOptions extends GcpOptions {
-  @Description("Pub/Sub topic")
-  @Default.InstanceFactory(PubsubTopicFactory.class)
-  String getPubsubTopic();
-  void setPubsubTopic(String topic);
-
-  /**
-   * Returns a default Pub/Sub topic based on the project and the job names.
-   */
-  class PubsubTopicFactory implements DefaultValueFactory<String> {
-    @Override
-    public String create(PipelineOptions options) {
-      return "projects/" + options.as(GcpOptions.class).getProject()
-          + "/topics/" + options.getJobName();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleUtils.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleUtils.java
deleted file mode 100644
index 570b382..0000000
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleUtils.java
+++ /dev/null
@@ -1,352 +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 ${package}.common;
-
-import com.google.api.client.googleapis.json.GoogleJsonResponseException;
-import com.google.api.client.googleapis.services.AbstractGoogleClientRequest;
-import com.google.api.client.util.BackOff;
-import com.google.api.client.util.BackOffUtils;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.bigquery.Bigquery;
-import com.google.api.services.bigquery.Bigquery.Datasets;
-import com.google.api.services.bigquery.Bigquery.Tables;
-import com.google.api.services.bigquery.model.Dataset;
-import com.google.api.services.bigquery.model.DatasetReference;
-import com.google.api.services.bigquery.model.Table;
-import com.google.api.services.bigquery.model.TableReference;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.api.services.pubsub.Pubsub;
-import com.google.api.services.pubsub.model.Subscription;
-import com.google.api.services.pubsub.model.Topic;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.Uninterruptibles;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.options.BigQueryOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PubsubOptions;
-import org.apache.beam.sdk.util.FluentBackoff;
-import org.apache.beam.sdk.util.Transport;
-import org.joda.time.Duration;
-
-/**
- * The utility class that sets up and tears down external resources,
- * and cancels the streaming pipelines once the program terminates.
- *
- * <p>It is used to run Beam examples.
- */
-public class ExampleUtils {
-
-  private static final int SC_NOT_FOUND = 404;
-
-  private final PipelineOptions options;
-  private Bigquery bigQueryClient = null;
-  private Pubsub pubsubClient = null;
-  private Set<PipelineResult> pipelinesToCancel = Sets.newHashSet();
-  private List<String> pendingMessages = Lists.newArrayList();
-
-  /**
-   * Do resources and runner options setup.
-   */
-  public ExampleUtils(PipelineOptions options) {
-    this.options = options;
-  }
-
-  /**
-   * Sets up external resources that are required by the example,
-   * such as Pub/Sub topics and BigQuery tables.
-   *
-   * @throws IOException if there is a problem setting up the resources
-   */
-  public void setup() throws IOException {
-    Sleeper sleeper = Sleeper.DEFAULT;
-    BackOff backOff =
-        FluentBackoff.DEFAULT
-            .withMaxRetries(3).withInitialBackoff(Duration.millis(200)).backoff();
-    Throwable lastException = null;
-    try {
-      do {
-        try {
-          setupPubsub();
-          setupBigQueryTable();
-          return;
-        } catch (GoogleJsonResponseException e) {
-          lastException = e;
-        }
-      } while (BackOffUtils.next(sleeper, backOff));
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      // Ignore InterruptedException
-    }
-    throw new RuntimeException(lastException);
-  }
-
-  /**
-   * Sets up the Google Cloud Pub/Sub topic.
-   *
-   * <p>If the topic doesn't exist, a new topic with the given name will be created.
-   *
-   * @throws IOException if there is a problem setting up the Pub/Sub topic
-   */
-  public void setupPubsub() throws IOException {
-    ExamplePubsubTopicAndSubscriptionOptions pubsubOptions =
-        options.as(ExamplePubsubTopicAndSubscriptionOptions.class);
-    if (!pubsubOptions.getPubsubTopic().isEmpty()) {
-      pendingMessages.add("**********************Set Up Pubsub************************");
-      setupPubsubTopic(pubsubOptions.getPubsubTopic());
-      pendingMessages.add("The Pub/Sub topic has been set up for this example: "
-          + pubsubOptions.getPubsubTopic());
-
-      if (!pubsubOptions.getPubsubSubscription().isEmpty()) {
-        setupPubsubSubscription(
-            pubsubOptions.getPubsubTopic(), pubsubOptions.getPubsubSubscription());
-        pendingMessages.add("The Pub/Sub subscription has been set up for this example: "
-            + pubsubOptions.getPubsubSubscription());
-      }
-    }
-  }
-
-  /**
-   * Sets up the BigQuery table with the given schema.
-   *
-   * <p>If the table already exists, the schema has to match the given one. Otherwise, the example
-   * will throw a RuntimeException. If the table doesn't exist, a new table with the given schema
-   * will be created.
-   *
-   * @throws IOException if there is a problem setting up the BigQuery table
-   */
-  public void setupBigQueryTable() throws IOException {
-    ExampleBigQueryTableOptions bigQueryTableOptions =
-        options.as(ExampleBigQueryTableOptions.class);
-    if (bigQueryTableOptions.getBigQueryDataset() != null
-        && bigQueryTableOptions.getBigQueryTable() != null
-        && bigQueryTableOptions.getBigQuerySchema() != null) {
-      pendingMessages.add("******************Set Up Big Query Table*******************");
-      setupBigQueryTable(bigQueryTableOptions.getProject(),
-                         bigQueryTableOptions.getBigQueryDataset(),
-                         bigQueryTableOptions.getBigQueryTable(),
-                         bigQueryTableOptions.getBigQuerySchema());
-      pendingMessages.add("The BigQuery table has been set up for this example: "
-          + bigQueryTableOptions.getProject()
-          + ":" + bigQueryTableOptions.getBigQueryDataset()
-          + "." + bigQueryTableOptions.getBigQueryTable());
-    }
-  }
-
-  /**
-   * Tears down external resources that can be deleted upon the example's completion.
-   */
-  private void tearDown() {
-    pendingMessages.add("*************************Tear Down*************************");
-    ExamplePubsubTopicAndSubscriptionOptions pubsubOptions =
-        options.as(ExamplePubsubTopicAndSubscriptionOptions.class);
-    if (!pubsubOptions.getPubsubTopic().isEmpty()) {
-      try {
-        deletePubsubTopic(pubsubOptions.getPubsubTopic());
-        pendingMessages.add("The Pub/Sub topic has been deleted: "
-            + pubsubOptions.getPubsubTopic());
-      } catch (IOException e) {
-        pendingMessages.add("Failed to delete the Pub/Sub topic : "
-            + pubsubOptions.getPubsubTopic());
-      }
-      if (!pubsubOptions.getPubsubSubscription().isEmpty()) {
-        try {
-          deletePubsubSubscription(pubsubOptions.getPubsubSubscription());
-          pendingMessages.add("The Pub/Sub subscription has been deleted: "
-              + pubsubOptions.getPubsubSubscription());
-        } catch (IOException e) {
-          pendingMessages.add("Failed to delete the Pub/Sub subscription : "
-              + pubsubOptions.getPubsubSubscription());
-        }
-      }
-    }
-
-    ExampleBigQueryTableOptions bigQueryTableOptions =
-        options.as(ExampleBigQueryTableOptions.class);
-    if (bigQueryTableOptions.getBigQueryDataset() != null
-        && bigQueryTableOptions.getBigQueryTable() != null
-        && bigQueryTableOptions.getBigQuerySchema() != null) {
-      pendingMessages.add("The BigQuery table might contain the example's output, "
-          + "and it is not deleted automatically: "
-          + bigQueryTableOptions.getProject()
-          + ":" + bigQueryTableOptions.getBigQueryDataset()
-          + "." + bigQueryTableOptions.getBigQueryTable());
-      pendingMessages.add("Please go to the Developers Console to delete it manually."
-          + " Otherwise, you may be charged for its usage.");
-    }
-  }
-
-  private void setupBigQueryTable(String projectId, String datasetId, String tableId,
-      TableSchema schema) throws IOException {
-    if (bigQueryClient == null) {
-      bigQueryClient = Transport.newBigQueryClient(options.as(BigQueryOptions.class)).build();
-    }
-
-    Datasets datasetService = bigQueryClient.datasets();
-    if (executeNullIfNotFound(datasetService.get(projectId, datasetId)) == null) {
-      Dataset newDataset = new Dataset().setDatasetReference(
-          new DatasetReference().setProjectId(projectId).setDatasetId(datasetId));
-      datasetService.insert(projectId, newDataset).execute();
-    }
-
-    Tables tableService = bigQueryClient.tables();
-    Table table = executeNullIfNotFound(tableService.get(projectId, datasetId, tableId));
-    if (table == null) {
-      Table newTable = new Table().setSchema(schema).setTableReference(
-          new TableReference().setProjectId(projectId).setDatasetId(datasetId).setTableId(tableId));
-      tableService.insert(projectId, datasetId, newTable).execute();
-    } else if (!table.getSchema().equals(schema)) {
-      throw new RuntimeException(
-          "Table exists and schemas do not match, expecting: " + schema.toPrettyString()
-          + ", actual: " + table.getSchema().toPrettyString());
-    }
-  }
-
-  private void setupPubsubTopic(String topic) throws IOException {
-    if (pubsubClient == null) {
-      pubsubClient = Transport.newPubsubClient(options.as(PubsubOptions.class)).build();
-    }
-    if (executeNullIfNotFound(pubsubClient.projects().topics().get(topic)) == null) {
-      pubsubClient.projects().topics().create(topic, new Topic().setName(topic)).execute();
-    }
-  }
-
-  private void setupPubsubSubscription(String topic, String subscription) throws IOException {
-    if (pubsubClient == null) {
-      pubsubClient = Transport.newPubsubClient(options.as(PubsubOptions.class)).build();
-    }
-    if (executeNullIfNotFound(pubsubClient.projects().subscriptions().get(subscription)) == null) {
-      Subscription subInfo = new Subscription()
-        .setAckDeadlineSeconds(60)
-        .setTopic(topic);
-      pubsubClient.projects().subscriptions().create(subscription, subInfo).execute();
-    }
-  }
-
-  /**
-   * Deletes the Google Cloud Pub/Sub topic.
-   *
-   * @throws IOException if there is a problem deleting the Pub/Sub topic
-   */
-  private void deletePubsubTopic(String topic) throws IOException {
-    if (pubsubClient == null) {
-      pubsubClient = Transport.newPubsubClient(options.as(PubsubOptions.class)).build();
-    }
-    if (executeNullIfNotFound(pubsubClient.projects().topics().get(topic)) != null) {
-      pubsubClient.projects().topics().delete(topic).execute();
-    }
-  }
-
-  /**
-   * Deletes the Google Cloud Pub/Sub subscription.
-   *
-   * @throws IOException if there is a problem deleting the Pub/Sub subscription
-   */
-  private void deletePubsubSubscription(String subscription) throws IOException {
-    if (pubsubClient == null) {
-      pubsubClient = Transport.newPubsubClient(options.as(PubsubOptions.class)).build();
-    }
-    if (executeNullIfNotFound(pubsubClient.projects().subscriptions().get(subscription)) != null) {
-      pubsubClient.projects().subscriptions().delete(subscription).execute();
-    }
-  }
-
-  /**
-   * Waits for the pipeline to finish and cancels it before the program exists.
-   */
-  public void waitToFinish(PipelineResult result) {
-    pipelinesToCancel.add(result);
-    if (!options.as(ExampleOptions.class).getKeepJobsRunning()) {
-      addShutdownHook(pipelinesToCancel);
-    }
-    try {
-      result.waitUntilFinish();
-    } catch (UnsupportedOperationException e) {
-      // Do nothing if the given PipelineResult doesn't support waitUntilFinish(),
-      // such as EvaluationResults returned by DirectRunner.
-      tearDown();
-      printPendingMessages();
-    } catch (Exception e) {
-      throw new RuntimeException("Failed to wait the pipeline until finish: " + result);
-    }
-  }
-
-  private void addShutdownHook(final Collection<PipelineResult> pipelineResults) {
-    Runtime.getRuntime().addShutdownHook(new Thread() {
-      @Override
-      public void run() {
-        tearDown();
-        printPendingMessages();
-        for (PipelineResult pipelineResult : pipelineResults) {
-          try {
-            pipelineResult.cancel();
-          } catch (IOException e) {
-            System.out.println("Failed to cancel the job.");
-            System.out.println(e.getMessage());
-          }
-        }
-
-        for (PipelineResult pipelineResult : pipelineResults) {
-          boolean cancellationVerified = false;
-          for (int retryAttempts = 6; retryAttempts > 0; retryAttempts--) {
-            if (pipelineResult.getState().isTerminal()) {
-              cancellationVerified = true;
-              break;
-            } else {
-              System.out.println(
-                  "The example pipeline is still running. Verifying the cancellation.");
-            }
-            Uninterruptibles.sleepUninterruptibly(10, TimeUnit.SECONDS);
-          }
-          if (!cancellationVerified) {
-            System.out.println("Failed to verify the cancellation for job: " + pipelineResult);
-          }
-        }
-      }
-    });
-  }
-
-  private void printPendingMessages() {
-    System.out.println();
-    System.out.println("***********************************************************");
-    System.out.println("***********************************************************");
-    for (String message : pendingMessages) {
-      System.out.println(message);
-    }
-    System.out.println("***********************************************************");
-    System.out.println("***********************************************************");
-  }
-
-  private static <T> T executeNullIfNotFound(
-      AbstractGoogleClientRequest<T> request) throws IOException {
-    try {
-      return request.execute();
-    } catch (GoogleJsonResponseException e) {
-      if (e.getStatusCode() == SC_NOT_FOUND) {
-        return null;
-      } else {
-        throw e;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/DebuggingWordCountTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/DebuggingWordCountTest.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/DebuggingWordCountTest.java
deleted file mode 100644
index 155242d..0000000
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/DebuggingWordCountTest.java
+++ /dev/null
@@ -1,52 +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 ${package};
-
-import com.google.common.io.Files;
-import java.io.File;
-import java.nio.charset.StandardCharsets;
-import ${package}.DebuggingWordCount.WordCountOptions;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link DebuggingWordCount}.
- */
-@RunWith(JUnit4.class)
-public class DebuggingWordCountTest {
-  @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
-
-  @Test
-  public void testDebuggingWordCount() throws Exception {
-    File inputFile = tmpFolder.newFile();
-    File outputFile = tmpFolder.newFile();
-    Files.write(
-        "stomach secret Flourish message Flourish here Flourish",
-        inputFile,
-        StandardCharsets.UTF_8);
-    WordCountOptions options =
-        TestPipeline.testingPipelineOptions().as(WordCountOptions.class);
-    options.setInputFile(inputFile.getAbsolutePath());
-    options.setOutput(outputFile.getAbsolutePath());
-    DebuggingWordCount.main(TestPipeline.convertToArgs(options));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java
deleted file mode 100644
index e86c2aa..0000000
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/test/java/WordCountTest.java
+++ /dev/null
@@ -1,85 +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 ${package};
-
-import java.util.Arrays;
-import java.util.List;
-import ${package}.WordCount.CountWords;
-import ${package}.WordCount.ExtractWordsFn;
-import ${package}.WordCount.FormatAsTextFn;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.RunnableOnService;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.DoFnTester;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.values.PCollection;
-import org.hamcrest.CoreMatchers;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests of WordCount.
- */
-@RunWith(JUnit4.class)
-public class WordCountTest {
-
-  /** Example test that tests a specific {@link DoFn}. */
-  @Test
-  public void testExtractWordsFn() throws Exception {
-    DoFnTester<String, String> extractWordsFn =
-        DoFnTester.of(new ExtractWordsFn());
-
-    Assert.assertThat(extractWordsFn.processBundle(" some  input  words "),
-                      CoreMatchers.hasItems("some", "input", "words"));
-    Assert.assertThat(extractWordsFn.processBundle(" "),
-                      CoreMatchers.<String>hasItems());
-    Assert.assertThat(extractWordsFn.processBundle(" some ", " input", " words"),
-                      CoreMatchers.hasItems("some", "input", "words"));
-  }
-
-  static final String[] WORDS_ARRAY = new String[] {
-    "hi there", "hi", "hi sue bob",
-    "hi sue", "", "bob hi"};
-
-  static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
-
-  static final String[] COUNTS_ARRAY = new String[] {
-      "hi: 5", "there: 1", "sue: 2", "bob: 2"};
-
-  /** Example test that tests a PTransform by using an in-memory input and inspecting the output. */
-  @Test
-  @Category(RunnableOnService.class)
-  public void testCountWords() throws Exception {
-    Pipeline p = TestPipeline.create();
-
-    PCollection<String> input = p.apply(Create.of(WORDS).withCoder(StringUtf8Coder.of()));
-
-    PCollection<String> output = input.apply(new CountWords())
-      .apply(MapElements.via(new FormatAsTextFn()));
-
-    PAssert.that(output).containsInAnyOrder(COUNTS_ARRAY);
-    p.run().waitUntilFinish();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/pom.xml b/sdks/java/maven-archetypes/pom.xml
index 1b1c319..be9ad6a 100644
--- a/sdks/java/maven-archetypes/pom.xml
+++ b/sdks/java/maven-archetypes/pom.xml
@@ -31,7 +31,32 @@
 
   <name>Apache Beam :: SDKs :: Java :: Maven Archetypes</name>
 
+  <modules>
+    <module>starter</module>
+  </modules>
+
   <profiles>
+    <!--
+      The builds for the example archetypes use unix scripting, so are enabled only on
+      platforms that Maven considers "unix".
+
+      The archetypes themselves are perfectly fine on all platforms (namely Windows). This is just
+      the generation and installation of the jar.
+    -->
+    <profile>
+      <id>beam-sdks-java-archetypes-examples</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+        <os>
+          <family>unix</family>
+        </os>
+        <property><name>!beam.sdks.java.archetypes.examples.skip</name></property>
+      </activation>
+      <modules>
+        <module>examples</module>
+      </modules>
+    </profile>
+
     <profile>
       <id>beam-sdks-java-archetypes-examples-java8</id>
       <activation>
@@ -47,10 +72,4 @@
       </modules>
     </profile>
   </profiles>
-
-  <modules>
-    <module>starter</module>
-    <module>examples</module>
-  </modules>
-
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/abd96b89/sdks/java/maven-archetypes/update-examples-archetype.sh
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/update-examples-archetype.sh b/sdks/java/maven-archetypes/update-examples-archetype.sh
deleted file mode 100755
index 3c222be..0000000
--- a/sdks/java/maven-archetypes/update-examples-archetype.sh
+++ /dev/null
@@ -1,59 +0,0 @@
-#!/bin/bash -e
-#
-#    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.
-#
-
-# Updates the examples archetype to match the examples module
-#
-# Usage: Invoke with no arguments from any working directory.
-
-# The directory of this script. Assumes root of the maven-archetypes module.
-HERE="$(dirname $0)"
-
-# The directory of the examples-java module
-EXAMPLES_ROOT="${HERE}/../../../examples/java"
-
-# The root of the examples archetype
-ARCHETYPE_ROOT="${HERE}/examples/src/main/resources/archetype-resources"
-
-#
-# Copy the Java subset of the examples project verbatim. 
-#
-rsync -a --exclude cookbook --exclude complete                  \
-    "${EXAMPLES_ROOT}"/src/main/java/org/apache/beam/examples/  \
-    "${ARCHETYPE_ROOT}/src/main/java"
-
-rsync -a --exclude cookbook --exclude complete --exclude '*IT.java'  \
-    "${EXAMPLES_ROOT}"/src/test/java/org/apache/beam/examples/        \
-    "${ARCHETYPE_ROOT}/src/test/java"
-
-#
-# Replace 'package org.apache.beam.examples' with 'package ${package}' in all Java code
-#
-find "${ARCHETYPE_ROOT}/src/main/java" -name '*.java' -print0 \
-    | xargs -0 sed -i 's/^package org\.apache\.beam\.examples/package ${package}/g'
-
-find "${ARCHETYPE_ROOT}/src/test/java" -name '*.java' -print0 \
-    | xargs -0 sed -i 's/^package org\.apache\.beam\.examples/package ${package}/g'
-
-#
-# Replace 'import org.apache.beam.examples.' with 'import ${package}.' in all Java code
-#
-find "${ARCHETYPE_ROOT}/src/main/java" -name '*.java' -print0 \
-    | xargs -0 sed -i 's/^import org\.apache\.beam\.examples/import ${package}/g'
-
-find "${ARCHETYPE_ROOT}/src/test/java" -name '*.java' -print0 \
-    | xargs -0 sed -i 's/^import org\.apache\.beam\.examples/import ${package}/g'


[03/50] [abbrv] incubator-beam git commit: Add a test of ReleaseInfo

Posted by ke...@apache.org.
Add a test of ReleaseInfo


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/1094fa6a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/1094fa6a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/1094fa6a

Branch: refs/heads/gearpump-runner
Commit: 1094fa6ac32046b4c092294b3cee046c91aea5a1
Parents: 48130f7
Author: Dan Halperin <dh...@google.com>
Authored: Thu Dec 1 09:15:28 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Thu Dec 1 13:10:55 2016 -0800

----------------------------------------------------------------------
 .../apache/beam/sdk/util/ReleaseInfoTest.java   | 45 ++++++++++++++++++++
 1 file changed, 45 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1094fa6a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReleaseInfoTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReleaseInfoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReleaseInfoTest.java
new file mode 100644
index 0000000..fabb7e2
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReleaseInfoTest.java
@@ -0,0 +1,45 @@
+/*
+ * 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.containsString;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+/**
+ * Tests for {@link ReleaseInfo}.
+ */
+public class ReleaseInfoTest {
+
+  @Test
+  public void getReleaseInfo() throws Exception {
+    ReleaseInfo info = ReleaseInfo.getReleaseInfo();
+
+    // Validate name
+    assertThat(info.getName(), containsString("Beam"));
+
+    // Validate semantic version
+    String version = info.getVersion();
+    String pattern = "\\d+\\.\\d+\\.\\d+.*";
+    assertTrue(
+        String.format("%s does not match pattern %s", version, pattern),
+        version.matches(pattern));
+  }
+}


[04/50] [abbrv] incubator-beam git commit: Closes #1480

Posted by ke...@apache.org.
Closes #1480


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/fd4b631f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/fd4b631f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/fd4b631f

Branch: refs/heads/gearpump-runner
Commit: fd4b631f1b4aa1538b779c4de591bd9b18526cd6
Parents: 48130f7 b36048b
Author: Dan Halperin <dh...@google.com>
Authored: Thu Dec 1 13:10:56 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Thu Dec 1 13:10:56 2016 -0800

----------------------------------------------------------------------
 sdks/java/core/pom.xml                          | 29 +++----------
 .../apache/beam/sdk/util/ReleaseInfoTest.java   | 45 ++++++++++++++++++++
 2 files changed, 52 insertions(+), 22 deletions(-)
----------------------------------------------------------------------



[20/50] [abbrv] incubator-beam git commit: Move Towards removing WindowedValue from SDK

Posted by ke...@apache.org.
Move Towards removing WindowedValue from SDK

- Introduces ValueInSingleWindow for purposes of PAssert
- Uses ValueInSingleWindow inside DoFnTester
- Moves WindowMatchers{,Test} to runners-core

After this commit, WindowedValue does not appear in any SDK APIs
used by Pipeline authors.


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/d9891234
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/d9891234
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/d9891234

Branch: refs/heads/gearpump-runner
Commit: d989123424a54699ecb47ba6c0a4e437316cabce
Parents: 0fb5610
Author: Eugene Kirpichov <ki...@google.com>
Authored: Mon Oct 31 15:46:25 2016 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Fri Dec 2 13:16:04 2016 -0800

----------------------------------------------------------------------
 .../beam/runners/core/ReduceFnRunnerTest.java   |   5 +-
 .../beam/runners/core/SplittableParDoTest.java  |  38 ++--
 .../beam/runners/core/WindowMatchers.java       | 204 +++++++++++++++++++
 .../beam/runners/core/WindowMatchersTest.java   |  82 ++++++++
 .../direct/WindowEvaluatorFactoryTest.java      |   4 +-
 .../apache/beam/sdk/testing/GatherAllPanes.java |  88 ++++++++
 .../org/apache/beam/sdk/testing/PAssert.java    |  77 +++----
 .../apache/beam/sdk/testing/PaneExtractors.java |  55 +++--
 .../beam/sdk/testing/ValueInSingleWindow.java   | 134 ++++++++++++
 .../apache/beam/sdk/transforms/DoFnTester.java  |  58 +++---
 .../apache/beam/sdk/util/GatherAllPanes.java    |  86 --------
 .../apache/beam/sdk/util/IdentityWindowFn.java  |   2 +-
 .../org/apache/beam/sdk/WindowMatchers.java     | 204 -------------------
 .../org/apache/beam/sdk/WindowMatchersTest.java |  82 --------
 .../beam/sdk/testing/GatherAllPanesTest.java    | 140 +++++++++++++
 .../beam/sdk/testing/PaneExtractorsTest.java    | 133 ++++++------
 .../testing/ValueInSingleWindowCoderTest.java   |  51 +++++
 .../beam/sdk/util/GatherAllPanesTest.java       | 143 -------------
 18 files changed, 893 insertions(+), 693 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java
index 20eb08b..ba57567 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java
@@ -18,8 +18,8 @@
 package org.apache.beam.runners.core;
 
 import static com.google.common.base.Preconditions.checkArgument;
-import static org.apache.beam.sdk.WindowMatchers.isSingleWindowedValue;
-import static org.apache.beam.sdk.WindowMatchers.isWindowedValue;
+import static org.apache.beam.runners.core.WindowMatchers.isSingleWindowedValue;
+import static org.apache.beam.runners.core.WindowMatchers.isWindowedValue;
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.emptyIterable;
@@ -39,7 +39,6 @@ import com.google.common.collect.Iterables;
 import java.util.Iterator;
 import java.util.List;
 import org.apache.beam.runners.core.triggers.TriggerStateMachine;
-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;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
index 990d892..b13d839 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java
@@ -36,6 +36,7 @@ import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.SerializableCoder;
 import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.ValueInSingleWindow;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFnTester;
@@ -142,14 +143,15 @@ public class SplittableParDoTest {
         PCollection.IsBounded.BOUNDED,
         makeBoundedCollection(pipeline)
             .apply("bounded to bounded", new SplittableParDo<>(makeParDo(boundedFn)))
-            .get(MAIN_OUTPUT_TAG).isBounded());
+            .get(MAIN_OUTPUT_TAG)
+            .isBounded());
     assertEquals(
         "Applying a bounded SDF to an unbounded collection produces an unbounded collection",
         PCollection.IsBounded.UNBOUNDED,
         makeUnboundedCollection(pipeline)
-            .apply(
-                "bounded to unbounded", new SplittableParDo<>(makeParDo(boundedFn)))
-            .get(MAIN_OUTPUT_TAG).isBounded());
+            .apply("bounded to unbounded", new SplittableParDo<>(makeParDo(boundedFn)))
+            .get(MAIN_OUTPUT_TAG)
+            .isBounded());
   }
 
   @Test
@@ -160,18 +162,16 @@ public class SplittableParDoTest {
         "Applying an unbounded SDF to a bounded collection produces a bounded collection",
         PCollection.IsBounded.UNBOUNDED,
         makeBoundedCollection(pipeline)
-            .apply(
-                "unbounded to bounded",
-                new SplittableParDo<>(makeParDo(unboundedFn)))
-            .get(MAIN_OUTPUT_TAG).isBounded());
+            .apply("unbounded to bounded", new SplittableParDo<>(makeParDo(unboundedFn)))
+            .get(MAIN_OUTPUT_TAG)
+            .isBounded());
     assertEquals(
         "Applying an unbounded SDF to an unbounded collection produces an unbounded collection",
         PCollection.IsBounded.UNBOUNDED,
         makeUnboundedCollection(pipeline)
-            .apply(
-                "unbounded to unbounded",
-                new SplittableParDo<>(makeParDo(unboundedFn)))
-            .get(MAIN_OUTPUT_TAG).isBounded());
+            .apply("unbounded to unbounded", new SplittableParDo<>(makeParDo(unboundedFn)))
+            .get(MAIN_OUTPUT_TAG)
+            .isBounded());
   }
 
   // ------------------------------- Tests for ProcessFn ---------------------------------
@@ -224,9 +224,11 @@ public class SplittableParDoTest {
                 Instant timestamp,
                 Collection<? extends BoundedWindow> windows,
                 PaneInfo pane) {
-              tester
-                  .getMutableOutput(tester.getMainOutputTag())
-                  .add(WindowedValue.of(output, timestamp, windows, pane));
+              for (BoundedWindow window : windows) {
+                tester
+                    .getMutableOutput(tester.getMainOutputTag())
+                    .add(ValueInSingleWindow.of(output, timestamp, window, pane));
+              }
             }
 
             @Override
@@ -236,7 +238,11 @@ public class SplittableParDoTest {
                 Instant timestamp,
                 Collection<? extends BoundedWindow> windows,
                 PaneInfo pane) {
-              tester.getMutableOutput(tag).add(WindowedValue.of(output, timestamp, windows, pane));
+              for (BoundedWindow window : windows) {
+                tester
+                    .getMutableOutput(tag)
+                    .add(ValueInSingleWindow.of(output, timestamp, window, pane));
+              }
             }
           });
       // Do not clone since ProcessFn references non-serializable DoFnTester itself

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java
new file mode 100644
index 0000000..6c3a7e2
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java
@@ -0,0 +1,204 @@
+/*
+ * 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.runners.core;
+
+import com.google.common.collect.Lists;
+import java.util.Collection;
+import java.util.Objects;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.hamcrest.Matchers;
+import org.hamcrest.TypeSafeMatcher;
+import org.joda.time.Instant;
+
+/**
+ * Matchers that are useful for working with Windowing, Timestamps, etc.
+ */
+public class WindowMatchers {
+
+  public static <T> Matcher<WindowedValue<? extends T>> isWindowedValue(
+      T value,
+      Instant timestamp,
+      Collection<? extends BoundedWindow> windows,
+      PaneInfo paneInfo) {
+
+    Collection<Matcher<? super BoundedWindow>> windowMatchers =
+        Lists.newArrayListWithCapacity(windows.size());
+    for (BoundedWindow window : windows) {
+      windowMatchers.add(Matchers.equalTo(window));
+    }
+
+    return isWindowedValue(
+        Matchers.equalTo(value),
+        Matchers.equalTo(timestamp),
+        Matchers.containsInAnyOrder(windowMatchers),
+        Matchers.equalTo(paneInfo));
+  }
+
+  public static <T> Matcher<WindowedValue<? extends T>> isWindowedValue(
+      Matcher<? super T> valueMatcher,
+      Matcher<? super Instant> timestampMatcher,
+      Matcher<? super Collection<? extends BoundedWindow>> windowsMatcher,
+      Matcher<? super PaneInfo> paneInfoMatcher) {
+    return new WindowedValueMatcher<>(
+        valueMatcher, timestampMatcher, windowsMatcher, paneInfoMatcher);
+  }
+
+  public static <T> Matcher<WindowedValue<? extends T>> isWindowedValue(
+      Matcher<? super T> valueMatcher,
+      Matcher<? super Instant> timestampMatcher,
+      Matcher<? super Collection<? extends BoundedWindow>> windowsMatcher) {
+    return new WindowedValueMatcher<>(
+        valueMatcher, timestampMatcher, windowsMatcher, Matchers.anything());
+  }
+
+  public static <T> Matcher<WindowedValue<? extends T>> isWindowedValue(
+      Matcher<? super T> valueMatcher, Matcher<? super Instant> timestampMatcher) {
+    return new WindowedValueMatcher<>(
+        valueMatcher, timestampMatcher, Matchers.anything(), Matchers.anything());
+  }
+
+  public static <T> Matcher<WindowedValue<? extends T>> isWindowedValue(
+      Matcher<? super T> valueMatcher) {
+    return new WindowedValueMatcher<>(
+        valueMatcher, Matchers.anything(), Matchers.anything(), Matchers.anything());
+  }
+
+  public static <T> Matcher<WindowedValue<? extends T>> isSingleWindowedValue(
+      T value, long timestamp, long windowStart, long windowEnd) {
+    return WindowMatchers.<T>isSingleWindowedValue(
+        Matchers.equalTo(value), timestamp, windowStart, windowEnd);
+  }
+
+  public static <T> Matcher<WindowedValue<? extends T>> isSingleWindowedValue(
+      T value, Instant timestamp, BoundedWindow window, PaneInfo paneInfo) {
+    return WindowMatchers.<T>isSingleWindowedValue(
+        Matchers.equalTo(value),
+        Matchers.equalTo(timestamp),
+        Matchers.equalTo(window),
+        Matchers.equalTo(paneInfo));
+  }
+
+  public static <T> Matcher<WindowedValue<? extends T>> isSingleWindowedValue(
+      T value, Instant timestamp, BoundedWindow window) {
+    return WindowMatchers.<T>isSingleWindowedValue(
+        Matchers.equalTo(value), Matchers.equalTo(timestamp), Matchers.equalTo(window));
+  }
+
+  public static <T> Matcher<WindowedValue<? extends T>> isSingleWindowedValue(
+      Matcher<T> valueMatcher, long timestamp, long windowStart, long windowEnd) {
+    IntervalWindow intervalWindow =
+        new IntervalWindow(new Instant(windowStart), new Instant(windowEnd));
+    return WindowMatchers.<T>isSingleWindowedValue(
+        valueMatcher,
+        Matchers.describedAs("%0", Matchers.equalTo(new Instant(timestamp)), timestamp),
+        Matchers.<BoundedWindow>equalTo(intervalWindow),
+        Matchers.anything());
+  }
+
+  public static <T> Matcher<WindowedValue<? extends T>> isSingleWindowedValue(
+      Matcher<? super T> valueMatcher,
+      Matcher<? super Instant> timestampMatcher,
+      Matcher<? super BoundedWindow> windowMatcher) {
+    return new WindowedValueMatcher<T>(
+        valueMatcher, timestampMatcher, Matchers.contains(windowMatcher), Matchers.anything());
+  }
+
+  public static <T> Matcher<WindowedValue<? extends T>> isSingleWindowedValue(
+      Matcher<? super T> valueMatcher,
+      Matcher<? super Instant> timestampMatcher,
+      Matcher<? super BoundedWindow> windowMatcher,
+      Matcher<? super PaneInfo> paneInfoMatcher) {
+    return new WindowedValueMatcher<T>(
+        valueMatcher, timestampMatcher, Matchers.contains(windowMatcher), paneInfoMatcher);
+  }
+
+  public static Matcher<IntervalWindow> intervalWindow(long start, long end) {
+    return Matchers.equalTo(new IntervalWindow(new Instant(start), new Instant(end)));
+  }
+
+  public static <T> Matcher<WindowedValue<? extends T>> valueWithPaneInfo(final PaneInfo paneInfo) {
+    return new TypeSafeMatcher<WindowedValue<? extends T>>() {
+      @Override
+      public void describeTo(Description description) {
+        description
+            .appendText("WindowedValue(paneInfo = ").appendValue(paneInfo).appendText(")");
+      }
+
+      @Override
+      protected boolean matchesSafely(WindowedValue<? extends T> item) {
+        return Objects.equals(item.getPane(), paneInfo);
+      }
+
+      @Override
+      protected void describeMismatchSafely(
+          WindowedValue<? extends T> item, Description mismatchDescription) {
+        mismatchDescription.appendValue(item.getPane());
+      }
+    };
+  }
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  @SafeVarargs
+  public static final <W extends BoundedWindow> Matcher<Iterable<W>> ofWindows(
+      Matcher<W>... windows) {
+    return (Matcher) Matchers.<W>containsInAnyOrder(windows);
+  }
+
+  private WindowMatchers() {}
+
+  private static class WindowedValueMatcher<T> extends TypeSafeMatcher<WindowedValue<? extends T>> {
+
+    private Matcher<? super T> valueMatcher;
+    private Matcher<? super Instant> timestampMatcher;
+    private Matcher<? super Collection<? extends BoundedWindow>> windowsMatcher;
+    private Matcher<? super PaneInfo> paneInfoMatcher;
+
+    private WindowedValueMatcher(
+        Matcher<? super T> valueMatcher,
+        Matcher<? super Instant> timestampMatcher,
+        Matcher<? super Collection<? extends BoundedWindow>> windowsMatcher,
+        Matcher<? super PaneInfo> paneInfoMatcher) {
+      this.valueMatcher = valueMatcher;
+      this.timestampMatcher = timestampMatcher;
+      this.windowsMatcher = windowsMatcher;
+      this.paneInfoMatcher = paneInfoMatcher;
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description
+          .appendText("a WindowedValue(").appendValue(valueMatcher)
+          .appendText(", ").appendValue(timestampMatcher)
+          .appendText(", ").appendValue(windowsMatcher)
+          .appendText(", ").appendValue(paneInfoMatcher)
+          .appendText(")");
+    }
+
+    @Override
+    protected boolean matchesSafely(WindowedValue<? extends T> windowedValue) {
+      return valueMatcher.matches(windowedValue.getValue())
+          && timestampMatcher.matches(windowedValue.getTimestamp())
+          && windowsMatcher.matches(windowedValue.getWindows());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java
new file mode 100644
index 0000000..6f4741a
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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.runners.core;
+
+import static org.junit.Assert.assertThat;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link WindowMatchers}.
+ */
+@RunWith(JUnit4.class)
+public class WindowMatchersTest {
+
+  @Test
+  public void testIsWindowedValueExact() {
+    long timestamp = 100;
+    long windowStart = 0;
+    long windowEnd = 200;
+
+    assertThat(
+        WindowedValue.of(
+            "hello",
+            new Instant(timestamp),
+            new IntervalWindow(new Instant(windowStart), new Instant(windowEnd)),
+            PaneInfo.NO_FIRING),
+        WindowMatchers.isWindowedValue(
+            "hello",
+            new Instant(timestamp),
+            ImmutableList.of(new IntervalWindow(new Instant(windowStart), new Instant(windowEnd))),
+            PaneInfo.NO_FIRING));
+  }
+
+  @Test
+  public void testIsWindowedValueReorderedWindows() {
+    long timestamp = 100;
+    long windowStart = 0;
+    long windowEnd = 200;
+    long windowStart2 = 50;
+    long windowEnd2 = 150;
+
+    assertThat(
+        WindowedValue.of(
+            "hello",
+            new Instant(timestamp),
+            ImmutableList.of(
+                new IntervalWindow(new Instant(windowStart), new Instant(windowEnd)),
+                new IntervalWindow(new Instant(windowStart2), new Instant(windowEnd2))),
+            PaneInfo.NO_FIRING),
+        WindowMatchers.isWindowedValue(
+            "hello",
+            new Instant(timestamp),
+            ImmutableList.of(
+                new IntervalWindow(new Instant(windowStart), new Instant(windowEnd)),
+                new IntervalWindow(new Instant(windowStart2), new Instant(windowEnd2))),
+            PaneInfo.NO_FIRING));
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
index e2f987c..66c28ce 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
@@ -17,8 +17,8 @@
  */
 package org.apache.beam.runners.direct;
 
-import static org.apache.beam.sdk.WindowMatchers.isSingleWindowedValue;
-import static org.apache.beam.sdk.WindowMatchers.isWindowedValue;
+import static org.apache.beam.runners.core.WindowMatchers.isSingleWindowedValue;
+import static org.apache.beam.runners.core.WindowMatchers.isWindowedValue;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.when;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/GatherAllPanes.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/GatherAllPanes.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/GatherAllPanes.java
new file mode 100644
index 0000000..2b311b7
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/GatherAllPanes.java
@@ -0,0 +1,88 @@
+/*
+ * 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.testing;
+
+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.Values;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+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.IdentityWindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * Gathers all panes of each window into exactly one output.
+ *
+ * <p>Note that this will delay the output of a window until the garbage collection time (when the
+ * watermark passes the end of the window plus allowed lateness) even if the upstream triggers
+ * closed the window earlier.
+ */
+class GatherAllPanes<T>
+    extends PTransform<PCollection<T>, PCollection<Iterable<ValueInSingleWindow<T>>>> {
+  /**
+   * Gathers all panes of each window into a single output element.
+   *
+   * <p>This will gather all output panes into a single element, which causes them to be colocated
+   * on a single worker. As a result, this is only suitable for {@link PCollection PCollections}
+   * where all of the output elements for each pane fit in memory, such as in tests.
+   */
+  public static <T> GatherAllPanes<T> globally() {
+    return new GatherAllPanes<>();
+  }
+
+  private GatherAllPanes() {}
+
+  @Override
+  public PCollection<Iterable<ValueInSingleWindow<T>>> apply(PCollection<T> input) {
+    WindowFn<?, ?> originalWindowFn = input.getWindowingStrategy().getWindowFn();
+
+    return input
+        .apply(ParDo.of(new ReifyTimestampsAndWindowsFn<T>()))
+        .setCoder(
+            ValueInSingleWindow.Coder.of(
+                input.getCoder(), input.getWindowingStrategy().getWindowFn().windowCoder()))
+        .apply(
+            WithKeys.<Integer, ValueInSingleWindow<T>>of(0)
+                .withKeyType(new TypeDescriptor<Integer>() {}))
+        .apply(
+            Window.into(
+                    new IdentityWindowFn<KV<Integer, ValueInSingleWindow<T>>>(
+                        originalWindowFn.windowCoder()))
+                .triggering(Never.ever())
+                .withAllowedLateness(input.getWindowingStrategy().getAllowedLateness())
+                .discardingFiredPanes())
+        // all values have the same key so they all appear as a single output element
+        .apply(GroupByKey.<Integer, ValueInSingleWindow<T>>create())
+        .apply(Values.<Iterable<ValueInSingleWindow<T>>>create())
+        .setWindowingStrategyInternal(input.getWindowingStrategy());
+  }
+
+  private static class ReifyTimestampsAndWindowsFn<T> extends DoFn<T, ValueInSingleWindow<T>> {
+    @DoFn.ProcessElement
+    public void processElement(ProcessContext c, BoundedWindow window) {
+      c.output(ValueInSingleWindow.of(c.element(), c.timestamp(), window, c.pane()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
index b3a14aa..7dc78d8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
@@ -63,8 +63,6 @@ import org.apache.beam.sdk.transforms.windowing.Trigger;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
 import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.GatherAllPanes;
-import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PBegin;
@@ -349,7 +347,7 @@ public class PAssert {
   private static class PCollectionContentsAssert<T> implements IterableAssert<T> {
     private final PCollection<T> actual;
     private final AssertionWindows rewindowingStrategy;
-    private final SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> paneExtractor;
+    private final SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> paneExtractor;
 
     public PCollectionContentsAssert(PCollection<T> actual) {
       this(actual, IntoGlobalWindow.<T>of(), PaneExtractors.<T>allPanes());
@@ -358,7 +356,7 @@ public class PAssert {
     public PCollectionContentsAssert(
         PCollection<T> actual,
         AssertionWindows rewindowingStrategy,
-        SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> paneExtractor) {
+        SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> paneExtractor) {
       this.actual = actual;
       this.rewindowingStrategy = rewindowingStrategy;
       this.paneExtractor = paneExtractor;
@@ -391,7 +389,7 @@ public class PAssert {
 
     private PCollectionContentsAssert<T> withPane(
         BoundedWindow window,
-        SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> paneExtractor) {
+        SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> paneExtractor) {
       @SuppressWarnings({"unchecked", "rawtypes"})
       Coder<BoundedWindow> windowCoder =
           (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder();
@@ -523,7 +521,7 @@ public class PAssert {
     private final PCollection<Iterable<T>> actual;
     private final Coder<T> elementCoder;
     private final AssertionWindows rewindowingStrategy;
-    private final SimpleFunction<Iterable<WindowedValue<Iterable<T>>>, Iterable<Iterable<T>>>
+    private final SimpleFunction<Iterable<ValueInSingleWindow<Iterable<T>>>, Iterable<Iterable<T>>>
         paneExtractor;
 
     public PCollectionSingletonIterableAssert(PCollection<Iterable<T>> actual) {
@@ -533,7 +531,8 @@ public class PAssert {
     public PCollectionSingletonIterableAssert(
         PCollection<Iterable<T>> actual,
         AssertionWindows rewindowingStrategy,
-        SimpleFunction<Iterable<WindowedValue<Iterable<T>>>, Iterable<Iterable<T>>> paneExtractor) {
+        SimpleFunction<Iterable<ValueInSingleWindow<Iterable<T>>>, Iterable<Iterable<T>>>
+            paneExtractor) {
       this.actual = actual;
 
       @SuppressWarnings("unchecked")
@@ -571,7 +570,8 @@ public class PAssert {
 
     private PCollectionSingletonIterableAssert<T> withPanes(
         BoundedWindow window,
-        SimpleFunction<Iterable<WindowedValue<Iterable<T>>>, Iterable<Iterable<T>>> paneExtractor) {
+        SimpleFunction<Iterable<ValueInSingleWindow<Iterable<T>>>, Iterable<Iterable<T>>>
+            paneExtractor) {
       @SuppressWarnings({"unchecked", "rawtypes"})
       Coder<BoundedWindow> windowCoder =
           (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder();
@@ -620,7 +620,8 @@ public class PAssert {
     private final PCollection<ElemT> actual;
     private final PTransform<PCollection<ElemT>, PCollectionView<ViewT>> view;
     private final AssertionWindows rewindowActuals;
-    private final SimpleFunction<Iterable<WindowedValue<ElemT>>, Iterable<ElemT>> paneExtractor;
+    private final SimpleFunction<Iterable<ValueInSingleWindow<ElemT>>, Iterable<ElemT>>
+        paneExtractor;
     private final Coder<ViewT> coder;
 
     protected PCollectionViewAssert(
@@ -634,7 +635,7 @@ public class PAssert {
         PCollection<ElemT> actual,
         PTransform<PCollection<ElemT>, PCollectionView<ViewT>> view,
         AssertionWindows rewindowActuals,
-        SimpleFunction<Iterable<WindowedValue<ElemT>>, Iterable<ElemT>> paneExtractor,
+        SimpleFunction<Iterable<ValueInSingleWindow<ElemT>>, Iterable<ElemT>> paneExtractor,
         Coder<ViewT> coder) {
       this.actual = actual;
       this.view = view;
@@ -660,7 +661,7 @@ public class PAssert {
 
     private PCollectionViewAssert<ElemT, ViewT> inPane(
         BoundedWindow window,
-        SimpleFunction<Iterable<WindowedValue<ElemT>>, Iterable<ElemT>> paneExtractor) {
+        SimpleFunction<Iterable<ValueInSingleWindow<ElemT>>, Iterable<ElemT>> paneExtractor) {
       return new PCollectionViewAssert<>(
           actual,
           view,
@@ -738,13 +739,14 @@ public class PAssert {
 
     private final transient PCollection<T> actual;
     private final transient AssertionWindows rewindowActuals;
-    private final transient SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> extractPane;
+    private final transient SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>>
+        extractPane;
     private final transient PTransform<PCollection<T>, PCollectionView<ActualT>> actualView;
 
     public static <T, ActualT> CreateActual<T, ActualT> from(
         PCollection<T> actual,
         AssertionWindows rewindowActuals,
-        SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> extractPane,
+        SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> extractPane,
         PTransform<PCollection<T>, PCollectionView<ActualT>> actualView) {
       return new CreateActual<>(actual, rewindowActuals, extractPane, actualView);
     }
@@ -752,7 +754,7 @@ public class PAssert {
     private CreateActual(
         PCollection<T> actual,
         AssertionWindows rewindowActuals,
-        SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> extractPane,
+        SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> extractPane,
         PTransform<PCollection<T>, PCollectionView<ActualT>> actualView) {
       this.actual = actual;
       this.rewindowActuals = rewindowActuals;
@@ -822,7 +824,7 @@ public class PAssert {
    * a single empty iterable, even though in practice most runners will not produce any element.
    */
   private static class GroupGlobally<T>
-      extends PTransform<PCollection<T>, PCollection<Iterable<WindowedValue<T>>>>
+      extends PTransform<PCollection<T>, PCollection<Iterable<ValueInSingleWindow<T>>>>
       implements Serializable {
     private final AssertionWindows rewindowingStrategy;
 
@@ -831,20 +833,20 @@ public class PAssert {
     }
 
     @Override
-    public PCollection<Iterable<WindowedValue<T>>> apply(PCollection<T> input) {
+    public PCollection<Iterable<ValueInSingleWindow<T>>> apply(PCollection<T> input) {
       final int combinedKey = 42;
 
       // Remove the triggering on both
       PTransform<
-              PCollection<KV<Integer, Iterable<WindowedValue<T>>>>,
-              PCollection<KV<Integer, Iterable<WindowedValue<T>>>>>
+              PCollection<KV<Integer, Iterable<ValueInSingleWindow<T>>>>,
+              PCollection<KV<Integer, Iterable<ValueInSingleWindow<T>>>>>
           removeTriggering =
-              Window.<KV<Integer, Iterable<WindowedValue<T>>>>triggering(Never.ever())
+              Window.<KV<Integer, Iterable<ValueInSingleWindow<T>>>>triggering(Never.ever())
                   .discardingFiredPanes()
                   .withAllowedLateness(input.getWindowingStrategy().getAllowedLateness());
       // Group the contents by key. If it is empty, this PCollection will be empty, too.
       // Then key it again with a dummy key.
-      PCollection<KV<Integer, Iterable<WindowedValue<T>>>> groupedContents =
+      PCollection<KV<Integer, Iterable<ValueInSingleWindow<T>>>> groupedContents =
           // TODO: Split the filtering from the rewindowing, and apply filtering before the Gather
           // if the grouping of extra records
           input
@@ -852,45 +854,47 @@ public class PAssert {
               .apply("GatherAllOutputs", GatherAllPanes.<T>globally())
               .apply(
                   "RewindowActuals",
-                  rewindowingStrategy.<Iterable<WindowedValue<T>>>windowActuals())
-              .apply("KeyForDummy", WithKeys.<Integer, Iterable<WindowedValue<T>>>of(combinedKey))
+                  rewindowingStrategy.<Iterable<ValueInSingleWindow<T>>>windowActuals())
+              .apply(
+                  "KeyForDummy",
+                  WithKeys.<Integer, Iterable<ValueInSingleWindow<T>>>of(combinedKey))
               .apply("RemoveActualsTriggering", removeTriggering);
 
       // Create another non-empty PCollection that is keyed with a distinct dummy key
-      PCollection<KV<Integer, Iterable<WindowedValue<T>>>> keyedDummy =
+      PCollection<KV<Integer, Iterable<ValueInSingleWindow<T>>>> keyedDummy =
           input
               .getPipeline()
               .apply(
                   Create.of(
                           KV.of(
                               combinedKey,
-                              (Iterable<WindowedValue<T>>)
-                                  Collections.<WindowedValue<T>>emptyList()))
+                              (Iterable<ValueInSingleWindow<T>>)
+                                  Collections.<ValueInSingleWindow<T>>emptyList()))
                       .withCoder(groupedContents.getCoder()))
               .apply(
                   "WindowIntoDummy",
-                  rewindowingStrategy.<KV<Integer, Iterable<WindowedValue<T>>>>windowDummy())
+                  rewindowingStrategy.<KV<Integer, Iterable<ValueInSingleWindow<T>>>>windowDummy())
               .apply("RemoveDummyTriggering", removeTriggering);
 
       // Flatten them together and group by the combined key to get a single element
-      PCollection<KV<Integer, Iterable<Iterable<WindowedValue<T>>>>> dummyAndContents =
+      PCollection<KV<Integer, Iterable<Iterable<ValueInSingleWindow<T>>>>> dummyAndContents =
           PCollectionList.of(groupedContents)
               .and(keyedDummy)
               .apply(
                   "FlattenDummyAndContents",
-                  Flatten.<KV<Integer, Iterable<WindowedValue<T>>>>pCollections())
+                  Flatten.<KV<Integer, Iterable<ValueInSingleWindow<T>>>>pCollections())
               .apply(
                   "NeverTrigger",
-                  Window.<KV<Integer, Iterable<WindowedValue<T>>>>triggering(Never.ever())
+                  Window.<KV<Integer, Iterable<ValueInSingleWindow<T>>>>triggering(Never.ever())
                       .withAllowedLateness(input.getWindowingStrategy().getAllowedLateness())
                       .discardingFiredPanes())
               .apply(
                   "GroupDummyAndContents",
-                  GroupByKey.<Integer, Iterable<WindowedValue<T>>>create());
+                  GroupByKey.<Integer, Iterable<ValueInSingleWindow<T>>>create());
 
       return dummyAndContents
-          .apply(Values.<Iterable<Iterable<WindowedValue<T>>>>create())
-          .apply(ParDo.of(new ConcatFn<WindowedValue<T>>()));
+          .apply(Values.<Iterable<Iterable<ValueInSingleWindow<T>>>>create())
+          .apply(ParDo.of(new ConcatFn<ValueInSingleWindow<T>>()));
     }
   }
 
@@ -909,12 +913,12 @@ public class PAssert {
       implements Serializable {
     private final SerializableFunction<Iterable<T>, Void> checkerFn;
     private final AssertionWindows rewindowingStrategy;
-    private final SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> paneExtractor;
+    private final SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> paneExtractor;
 
     private GroupThenAssert(
         SerializableFunction<Iterable<T>, Void> checkerFn,
         AssertionWindows rewindowingStrategy,
-        SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> paneExtractor) {
+        SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> paneExtractor) {
       this.checkerFn = checkerFn;
       this.rewindowingStrategy = rewindowingStrategy;
       this.paneExtractor = paneExtractor;
@@ -940,13 +944,14 @@ public class PAssert {
       extends PTransform<PCollection<Iterable<T>>, PDone> implements Serializable {
     private final SerializableFunction<Iterable<T>, Void> checkerFn;
     private final AssertionWindows rewindowingStrategy;
-    private final SimpleFunction<Iterable<WindowedValue<Iterable<T>>>, Iterable<Iterable<T>>>
+    private final SimpleFunction<Iterable<ValueInSingleWindow<Iterable<T>>>, Iterable<Iterable<T>>>
         paneExtractor;
 
     private GroupThenAssertForSingleton(
         SerializableFunction<Iterable<T>, Void> checkerFn,
         AssertionWindows rewindowingStrategy,
-        SimpleFunction<Iterable<WindowedValue<Iterable<T>>>, Iterable<Iterable<T>>> paneExtractor) {
+        SimpleFunction<Iterable<ValueInSingleWindow<Iterable<T>>>, Iterable<Iterable<T>>>
+            paneExtractor) {
       this.checkerFn = checkerFn;
       this.rewindowingStrategy = rewindowingStrategy;
       this.paneExtractor = paneExtractor;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java
index db72a0c..dd1fac9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java
@@ -25,14 +25,13 @@ import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.SimpleFunction;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
-import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TypeDescriptor;
 
 /**
- * {@link PTransform PTransforms} which take an {@link Iterable} of {@link WindowedValue
- * WindowedValues} and outputs an {@link Iterable} of all values in the specified pane, dropping the
- * {@link WindowedValue} metadata.
+ * {@link PTransform PTransforms} which take an {@link Iterable} of {@link ValueInSingleWindow
+ * ValueInSingleWindows} and outputs an {@link Iterable} of all values in the specified pane,
+ * dropping the {@link ValueInSingleWindow} metadata.
  *
  * <p>Although all of the method signatures return SimpleFunction, users should ensure to set the
  * coder of any output {@link PCollection}, as appropriate {@link TypeDescriptor TypeDescriptors}
@@ -42,36 +41,36 @@ final class PaneExtractors {
   private PaneExtractors() {
   }
 
-  static <T> SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> onlyPane() {
+  static <T> SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> onlyPane() {
     return new ExtractOnlyPane<>();
   }
 
-  static <T> SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> onTimePane() {
+  static <T> SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> onTimePane() {
     return new ExtractOnTimePane<>();
   }
 
-  static <T> SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> finalPane() {
+  static <T> SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> finalPane() {
     return new ExtractFinalPane<>();
   }
 
-  static <T> SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> nonLatePanes() {
+  static <T> SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> nonLatePanes() {
     return new ExtractNonLatePanes<>();
   }
 
-  static <T> SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> earlyPanes() {
+  static <T> SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> earlyPanes() {
     return new ExtractEarlyPanes<>();
   }
 
-  static <T> SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> allPanes() {
+  static <T> SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> allPanes() {
     return new ExtractAllPanes<>();
   }
 
   private static class ExtractOnlyPane<T>
-      extends SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> {
+      extends SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> {
     @Override
-    public Iterable<T> apply(Iterable<WindowedValue<T>> input) {
+    public Iterable<T> apply(Iterable<ValueInSingleWindow<T>> input) {
       List<T> outputs = new ArrayList<>();
-      for (WindowedValue<T> value : input) {
+      for (ValueInSingleWindow<T> value : input) {
         checkState(value.getPane().isFirst() && value.getPane().isLast(),
             "Expected elements to be produced by a trigger that fires at most once, but got"
                 + "a value in a pane that is %s. Actual Pane Info: %s",
@@ -85,11 +84,11 @@ final class PaneExtractors {
 
 
   private static class ExtractOnTimePane<T>
-      extends SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> {
+      extends SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> {
     @Override
-    public Iterable<T> apply(Iterable<WindowedValue<T>> input) {
+    public Iterable<T> apply(Iterable<ValueInSingleWindow<T>> input) {
       List<T> outputs = new ArrayList<>();
-      for (WindowedValue<T> value : input) {
+      for (ValueInSingleWindow<T> value : input) {
         if (value.getPane().getTiming().equals(Timing.ON_TIME)) {
           outputs.add(value.getValue());
         }
@@ -100,11 +99,11 @@ final class PaneExtractors {
 
 
   private static class ExtractFinalPane<T>
-      extends SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> {
+      extends SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> {
     @Override
-    public Iterable<T> apply(Iterable<WindowedValue<T>> input) {
+    public Iterable<T> apply(Iterable<ValueInSingleWindow<T>> input) {
       List<T> outputs = new ArrayList<>();
-      for (WindowedValue<T> value : input) {
+      for (ValueInSingleWindow<T> value : input) {
         if (value.getPane().isLast()) {
           outputs.add(value.getValue());
         }
@@ -115,11 +114,11 @@ final class PaneExtractors {
 
 
   private static class ExtractAllPanes<T>
-      extends SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> {
+      extends SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> {
     @Override
-    public Iterable<T> apply(Iterable<WindowedValue<T>> input) {
+    public Iterable<T> apply(Iterable<ValueInSingleWindow<T>> input) {
       List<T> outputs = new ArrayList<>();
-      for (WindowedValue<T> value : input) {
+      for (ValueInSingleWindow<T> value : input) {
         outputs.add(value.getValue());
       }
       return outputs;
@@ -128,11 +127,11 @@ final class PaneExtractors {
 
 
   private static class ExtractNonLatePanes<T>
-      extends SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> {
+      extends SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> {
     @Override
-    public Iterable<T> apply(Iterable<WindowedValue<T>> input) {
+    public Iterable<T> apply(Iterable<ValueInSingleWindow<T>> input) {
       List<T> outputs = new ArrayList<>();
-      for (WindowedValue<T> value : input) {
+      for (ValueInSingleWindow<T> value : input) {
         if (value.getPane().getTiming() != PaneInfo.Timing.LATE) {
           outputs.add(value.getValue());
         }
@@ -142,11 +141,11 @@ final class PaneExtractors {
   }
 
   private static class ExtractEarlyPanes<T>
-      extends SimpleFunction<Iterable<WindowedValue<T>>, Iterable<T>> {
+      extends SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> {
     @Override
-    public Iterable<T> apply(Iterable<WindowedValue<T>> input) {
+    public Iterable<T> apply(Iterable<ValueInSingleWindow<T>> input) {
       List<T> outputs = new ArrayList<>();
-      for (WindowedValue<T> value : input) {
+      for (ValueInSingleWindow<T> value : input) {
         if (value.getPane().getTiming() == PaneInfo.Timing.EARLY) {
           outputs.add(value.getValue());
         }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/ValueInSingleWindow.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/ValueInSingleWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/ValueInSingleWindow.java
new file mode 100644
index 0000000..9ec030f
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/ValueInSingleWindow.java
@@ -0,0 +1,134 @@
+/*
+ * 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.testing;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.auto.value.AutoValue;
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.InstantCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.joda.time.Instant;
+
+/**
+ * An immutable tuple of value, timestamp, window, and pane.
+ *
+ * @param <T> the type of the value
+ */
+@AutoValue
+public abstract class ValueInSingleWindow<T> {
+  /** Returns the value of this {@code ValueInSingleWindow}. */
+  @Nullable
+  public abstract T getValue();
+
+  /** Returns the timestamp of this {@code ValueInSingleWindow}. */
+  public abstract Instant getTimestamp();
+
+  /** Returns the window of this {@code ValueInSingleWindow}. */
+  public abstract BoundedWindow getWindow();
+
+  /** Returns the pane of this {@code ValueInSingleWindow} in its window. */
+  public abstract PaneInfo getPane();
+
+  public static <T> ValueInSingleWindow<T> of(
+      T value, Instant timestamp, BoundedWindow window, PaneInfo paneInfo) {
+    return new AutoValue_ValueInSingleWindow<>(value, timestamp, window, paneInfo);
+  }
+
+  /** A coder for {@link ValueInSingleWindow}. */
+  public static class Coder<T> extends StandardCoder<ValueInSingleWindow<T>> {
+    private final org.apache.beam.sdk.coders.Coder<T> valueCoder;
+    private final org.apache.beam.sdk.coders.Coder<BoundedWindow> windowCoder;
+
+    public static <T> Coder<T> of(
+        org.apache.beam.sdk.coders.Coder<T> valueCoder,
+        org.apache.beam.sdk.coders.Coder<? extends BoundedWindow> windowCoder) {
+      return new Coder<>(valueCoder, windowCoder);
+    }
+
+    @JsonCreator
+    public static <T> Coder<T> of(
+        @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
+            List<org.apache.beam.sdk.coders.Coder<?>> components) {
+      checkArgument(components.size() == 2, "Expecting 2 components, got %s", components.size());
+      @SuppressWarnings("unchecked")
+      org.apache.beam.sdk.coders.Coder<T> valueCoder =
+          (org.apache.beam.sdk.coders.Coder<T>) components.get(0);
+      @SuppressWarnings("unchecked")
+      org.apache.beam.sdk.coders.Coder<BoundedWindow> windowCoder =
+          (org.apache.beam.sdk.coders.Coder<BoundedWindow>) components.get(1);
+      return new Coder<>(valueCoder, windowCoder);
+    }
+
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    Coder(
+        org.apache.beam.sdk.coders.Coder<T> valueCoder,
+        org.apache.beam.sdk.coders.Coder<? extends BoundedWindow> windowCoder) {
+      this.valueCoder = valueCoder;
+      this.windowCoder = (org.apache.beam.sdk.coders.Coder) windowCoder;
+    }
+
+    @Override
+    public void encode(ValueInSingleWindow<T> windowedElem, OutputStream outStream, Context context)
+        throws IOException {
+      Context nestedContext = context.nested();
+      valueCoder.encode(windowedElem.getValue(), outStream, nestedContext);
+      InstantCoder.of().encode(windowedElem.getTimestamp(), outStream, nestedContext);
+      windowCoder.encode(windowedElem.getWindow(), outStream, nestedContext);
+      PaneInfo.PaneInfoCoder.INSTANCE.encode(windowedElem.getPane(), outStream, context);
+    }
+
+    @Override
+    public ValueInSingleWindow<T> decode(InputStream inStream, Context context) throws IOException {
+      Context nestedContext = context.nested();
+      T value = valueCoder.decode(inStream, nestedContext);
+      Instant timestamp = InstantCoder.of().decode(inStream, nestedContext);
+      BoundedWindow window = windowCoder.decode(inStream, nestedContext);
+      PaneInfo pane = PaneInfo.PaneInfoCoder.INSTANCE.decode(inStream, nestedContext);
+      return new AutoValue_ValueInSingleWindow<>(value, timestamp, window, pane);
+    }
+
+    @Override
+    public List<? extends org.apache.beam.sdk.coders.Coder<?>> getCoderArguments() {
+      // Coder arguments are coders for the type parameters of the coder - i.e. only T.
+      return ImmutableList.of(valueCoder);
+    }
+
+    @Override
+    public List<? extends org.apache.beam.sdk.coders.Coder<?>> getComponents() {
+      // Coder components are all inner coders that it uses - i.e. both T and BoundedWindow.
+      return ImmutableList.of(valueCoder, windowCoder);
+    }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {
+      valueCoder.verifyDeterministic();
+      windowCoder.verifyDeterministic();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
index 0c6043f..17fa612 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
@@ -23,7 +23,6 @@ import static com.google.common.base.Preconditions.checkState;
 import com.google.common.base.Function;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -35,8 +34,10 @@ import java.util.Map;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.ValueInSingleWindow;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.util.TimerInternals;
@@ -353,10 +354,10 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
   public List<TimestampedValue<OutputT>> peekOutputElementsWithTimestamp() {
     // TODO: Should we return an unmodifiable list?
     return Lists.transform(getImmutableOutput(mainOutputTag),
-        new Function<WindowedValue<OutputT>, TimestampedValue<OutputT>>() {
+        new Function<ValueInSingleWindow<OutputT>, TimestampedValue<OutputT>>() {
           @Override
           @SuppressWarnings("unchecked")
-          public TimestampedValue<OutputT> apply(WindowedValue<OutputT> input) {
+          public TimestampedValue<OutputT> apply(ValueInSingleWindow<OutputT> input) {
             return TimestampedValue.of(input.getValue(), input.getTimestamp());
           }
         });
@@ -378,8 +379,8 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
       TupleTag<OutputT> tag,
       BoundedWindow window) {
     ImmutableList.Builder<TimestampedValue<OutputT>> valuesBuilder = ImmutableList.builder();
-    for (WindowedValue<OutputT> value : getImmutableOutput(tag)) {
-      if (value.getWindows().contains(window)) {
+    for (ValueInSingleWindow<OutputT> value : getImmutableOutput(tag)) {
+      if (value.getWindow().equals(window)) {
         valuesBuilder.add(TimestampedValue.of(value.getValue(), value.getTimestamp()));
       }
     }
@@ -434,10 +435,10 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
   public <T> List<T> peekSideOutputElements(TupleTag<T> tag) {
     // TODO: Should we return an unmodifiable list?
     return Lists.transform(getImmutableOutput(tag),
-        new Function<WindowedValue<T>, T>() {
+        new Function<ValueInSingleWindow<T>, T>() {
           @SuppressWarnings("unchecked")
           @Override
-          public T apply(WindowedValue<T> input) {
+          public T apply(ValueInSingleWindow<T> input) {
             return input.getValue();
           }});
   }
@@ -510,16 +511,16 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
     return combiner.extractOutput(accumulator);
   }
 
-  private <T> List<WindowedValue<T>> getImmutableOutput(TupleTag<T> tag) {
+  private <T> List<ValueInSingleWindow<T>> getImmutableOutput(TupleTag<T> tag) {
     @SuppressWarnings({"unchecked", "rawtypes"})
-    List<WindowedValue<T>> elems = (List) outputs.get(tag);
+    List<ValueInSingleWindow<T>> elems = (List) outputs.get(tag);
     return ImmutableList.copyOf(
-        MoreObjects.firstNonNull(elems, Collections.<WindowedValue<T>>emptyList()));
+        MoreObjects.firstNonNull(elems, Collections.<ValueInSingleWindow<T>>emptyList()));
   }
 
   @SuppressWarnings({"unchecked", "rawtypes"})
-  public <T> List<WindowedValue<T>> getMutableOutput(TupleTag<T> tag) {
-    List<WindowedValue<T>> outputList = (List) outputs.get(tag);
+  public <T> List<ValueInSingleWindow<T>> getMutableOutput(TupleTag<T> tag) {
+    List<ValueInSingleWindow<T>> outputList = (List) outputs.get(tag);
     if (outputList == null) {
       outputList = new ArrayList<>();
       outputs.put(tag, (List) outputList);
@@ -612,23 +613,22 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
       sideOutputWithTimestamp(tag, output, BoundedWindow.TIMESTAMP_MIN_VALUE);
     }
 
-    public <T> void noteOutput(TupleTag<T> tag, WindowedValue<T> output) {
+    public <T> void noteOutput(TupleTag<T> tag, ValueInSingleWindow<T> output) {
       getMutableOutput(tag).add(output);
     }
   }
 
   private TestProcessContext createProcessContext(TimestampedValue<InputT> elem) {
-    WindowedValue<InputT> windowedValue = WindowedValue.timestampedValueInGlobalWindow(
-        elem.getValue(), elem.getTimestamp());
-
-    return new TestProcessContext(windowedValue);
+    return new TestProcessContext(
+        ValueInSingleWindow.of(
+            elem.getValue(), elem.getTimestamp(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
   }
 
   private class TestProcessContext extends OldDoFn<InputT, OutputT>.ProcessContext {
     private final TestContext context;
-    private final WindowedValue<InputT> element;
+    private final ValueInSingleWindow<InputT> element;
 
-    private TestProcessContext(WindowedValue<InputT> element) {
+    private TestProcessContext(ValueInSingleWindow<InputT> element) {
       fn.super();
       this.context = createContext(fn);
       this.element = element;
@@ -661,7 +661,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
 
     @Override
     public BoundedWindow window() {
-      return Iterables.getOnlyElement(element.getWindows());
+      return element.getWindow();
     }
 
     @Override
@@ -683,7 +683,10 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
             Instant timestamp,
             Collection<? extends BoundedWindow> windows,
             PaneInfo pane) {
-          context.noteOutput(mainOutputTag, WindowedValue.of(output, timestamp, windows, pane));
+          for (BoundedWindow window : windows) {
+            context.noteOutput(
+                mainOutputTag, ValueInSingleWindow.of(output, timestamp, window, pane));
+          }
         }
 
         @Override
@@ -693,7 +696,10 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
             Instant timestamp,
             Collection<? extends BoundedWindow> windows,
             PaneInfo pane) {
-          context.noteOutput(tag, WindowedValue.of(output, timestamp, windows, pane));
+          for (BoundedWindow window : windows) {
+            context.noteOutput(
+                tag, ValueInSingleWindow.of(output, timestamp, window, pane));
+          }
         }
 
         @Override
@@ -703,7 +709,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
 
         @Override
         public Collection<? extends BoundedWindow> windows() {
-          return element.getWindows();
+          return Collections.singleton(element.getWindow());
         }
 
         @Override
@@ -742,8 +748,8 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
 
     @Override
     public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-      context.noteOutput(tag,
-          WindowedValue.of(output, timestamp, element.getWindows(), element.getPane()));
+      context.noteOutput(
+          tag, ValueInSingleWindow.of(output, timestamp, element.getWindow(), element.getPane()));
     }
 
     @Override
@@ -803,7 +809,7 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
   OldDoFn<InputT, OutputT> fn;
 
   /** The outputs from the {@link DoFn} under test. */
-  private Map<TupleTag<?>, List<WindowedValue<?>>> outputs;
+  private Map<TupleTag<?>, List<ValueInSingleWindow<?>>> outputs;
 
   private InMemoryStateInternals<?> stateInternals;
   private InMemoryTimerInternals timerInternals;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/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
deleted file mode 100644
index 52a2ba8..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GatherAllPanes.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 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.Values;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-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.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.TypeDescriptor;
-
-/**
- * Gathers all panes of each window into exactly one output.
- *
- * <p>Note that this will delay the output of a window until the garbage collection time (when the
- * watermark passes the end of the window plus allowed lateness) even if the upstream triggers
- * closed the window earlier.
- */
-public class GatherAllPanes<T>
-    extends PTransform<PCollection<T>, PCollection<Iterable<WindowedValue<T>>>> {
-  /**
-   * Gathers all panes of each window into a single output element.
-   *
-   * <p>This will gather all output panes into a single element, which causes them to be colocated
-   * on a single worker. As a result, this is only suitable for {@link PCollection PCollections}
-   * where all of the output elements for each pane fit in memory, such as in tests.
-   */
-  public static <T> GatherAllPanes<T> globally() {
-    return new GatherAllPanes<>();
-  }
-
-  private GatherAllPanes() {}
-
-  @Override
-  public PCollection<Iterable<WindowedValue<T>>> apply(PCollection<T> input) {
-    WindowFn<?, ?> originalWindowFn = input.getWindowingStrategy().getWindowFn();
-
-    return input
-        .apply(ParDo.of(new ReifyTimestampsAndWindowsFn<T>()))
-        .setCoder(
-            WindowedValue.FullWindowedValueCoder.of(
-                input.getCoder(), input.getWindowingStrategy().getWindowFn().windowCoder()))
-        .apply(
-            WithKeys.<Integer, WindowedValue<T>>of(0).withKeyType(new TypeDescriptor<Integer>() {}))
-        .apply(
-            Window.into(
-                    new IdentityWindowFn<KV<Integer, WindowedValue<T>>>(
-                        originalWindowFn.windowCoder()))
-                .triggering(Never.ever())
-                .withAllowedLateness(input.getWindowingStrategy().getAllowedLateness())
-                .discardingFiredPanes())
-        // all values have the same key so they all appear as a single output element
-        .apply(GroupByKey.<Integer, WindowedValue<T>>create())
-        .apply(Values.<Iterable<WindowedValue<T>>>create())
-        .setWindowingStrategyInternal(input.getWindowingStrategy());
-  }
-
-  private static class ReifyTimestampsAndWindowsFn<T> extends DoFn<T, WindowedValue<T>> {
-    @DoFn.ProcessElement
-    public void processElement(ProcessContext c, BoundedWindow window) {
-      c.output(WindowedValue.of(c.element(), c.timestamp(), window, c.pane()));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java
index 8ca1bfd..c02e1f4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java
@@ -45,7 +45,7 @@ import org.joda.time.Instant;
  * <p>This {@link WindowFn} is an internal implementation detail of sdk-provided utilities, and
  * should not be used by {@link Pipeline} writers.
  */
-class IdentityWindowFn<T> extends NonMergingWindowFn<T, BoundedWindow> {
+public class IdentityWindowFn<T> extends NonMergingWindowFn<T, BoundedWindow> {
 
   /**
    * The coder of the type of windows of the input {@link PCollection}. This is not an arbitrary

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java
deleted file mode 100644
index 3531a86..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java
+++ /dev/null
@@ -1,204 +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;
-
-import com.google.common.collect.Lists;
-import java.util.Collection;
-import java.util.Objects;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.hamcrest.Description;
-import org.hamcrest.Matcher;
-import org.hamcrest.Matchers;
-import org.hamcrest.TypeSafeMatcher;
-import org.joda.time.Instant;
-
-/**
- * Matchers that are useful for working with Windowing, Timestamps, etc.
- */
-public class WindowMatchers {
-
-  public static <T> Matcher<WindowedValue<? extends T>> isWindowedValue(
-      T value,
-      Instant timestamp,
-      Collection<? extends BoundedWindow> windows,
-      PaneInfo paneInfo) {
-
-    Collection<Matcher<? super BoundedWindow>> windowMatchers =
-        Lists.newArrayListWithCapacity(windows.size());
-    for (BoundedWindow window : windows) {
-      windowMatchers.add(Matchers.equalTo(window));
-    }
-
-    return isWindowedValue(
-        Matchers.equalTo(value),
-        Matchers.equalTo(timestamp),
-        Matchers.containsInAnyOrder(windowMatchers),
-        Matchers.equalTo(paneInfo));
-  }
-
-  public static <T> Matcher<WindowedValue<? extends T>> isWindowedValue(
-      Matcher<? super T> valueMatcher,
-      Matcher<? super Instant> timestampMatcher,
-      Matcher<? super Collection<? extends BoundedWindow>> windowsMatcher,
-      Matcher<? super PaneInfo> paneInfoMatcher) {
-    return new WindowedValueMatcher<>(
-        valueMatcher, timestampMatcher, windowsMatcher, paneInfoMatcher);
-  }
-
-  public static <T> Matcher<WindowedValue<? extends T>> isWindowedValue(
-      Matcher<? super T> valueMatcher,
-      Matcher<? super Instant> timestampMatcher,
-      Matcher<? super Collection<? extends BoundedWindow>> windowsMatcher) {
-    return new WindowedValueMatcher<>(
-        valueMatcher, timestampMatcher, windowsMatcher, Matchers.anything());
-  }
-
-  public static <T> Matcher<WindowedValue<? extends T>> isWindowedValue(
-      Matcher<? super T> valueMatcher, Matcher<? super Instant> timestampMatcher) {
-    return new WindowedValueMatcher<>(
-        valueMatcher, timestampMatcher, Matchers.anything(), Matchers.anything());
-  }
-
-  public static <T> Matcher<WindowedValue<? extends T>> isWindowedValue(
-      Matcher<? super T> valueMatcher) {
-    return new WindowedValueMatcher<>(
-        valueMatcher, Matchers.anything(), Matchers.anything(), Matchers.anything());
-  }
-
-  public static <T> Matcher<WindowedValue<? extends T>> isSingleWindowedValue(
-      T value, long timestamp, long windowStart, long windowEnd) {
-    return WindowMatchers.<T>isSingleWindowedValue(
-        Matchers.equalTo(value), timestamp, windowStart, windowEnd);
-  }
-
-  public static <T> Matcher<WindowedValue<? extends T>> isSingleWindowedValue(
-      T value, Instant timestamp, BoundedWindow window, PaneInfo paneInfo) {
-    return WindowMatchers.<T>isSingleWindowedValue(
-        Matchers.equalTo(value),
-        Matchers.equalTo(timestamp),
-        Matchers.equalTo(window),
-        Matchers.equalTo(paneInfo));
-  }
-
-  public static <T> Matcher<WindowedValue<? extends T>> isSingleWindowedValue(
-      T value, Instant timestamp, BoundedWindow window) {
-    return WindowMatchers.<T>isSingleWindowedValue(
-        Matchers.equalTo(value), Matchers.equalTo(timestamp), Matchers.equalTo(window));
-  }
-
-  public static <T> Matcher<WindowedValue<? extends T>> isSingleWindowedValue(
-      Matcher<T> valueMatcher, long timestamp, long windowStart, long windowEnd) {
-    IntervalWindow intervalWindow =
-        new IntervalWindow(new Instant(windowStart), new Instant(windowEnd));
-    return WindowMatchers.<T>isSingleWindowedValue(
-        valueMatcher,
-        Matchers.describedAs("%0", Matchers.equalTo(new Instant(timestamp)), timestamp),
-        Matchers.<BoundedWindow>equalTo(intervalWindow),
-        Matchers.anything());
-  }
-
-  public static <T> Matcher<WindowedValue<? extends T>> isSingleWindowedValue(
-      Matcher<? super T> valueMatcher,
-      Matcher<? super Instant> timestampMatcher,
-      Matcher<? super BoundedWindow> windowMatcher) {
-    return new WindowedValueMatcher<T>(
-        valueMatcher, timestampMatcher, Matchers.contains(windowMatcher), Matchers.anything());
-  }
-
-  public static <T> Matcher<WindowedValue<? extends T>> isSingleWindowedValue(
-      Matcher<? super T> valueMatcher,
-      Matcher<? super Instant> timestampMatcher,
-      Matcher<? super BoundedWindow> windowMatcher,
-      Matcher<? super PaneInfo> paneInfoMatcher) {
-    return new WindowedValueMatcher<T>(
-        valueMatcher, timestampMatcher, Matchers.contains(windowMatcher), paneInfoMatcher);
-  }
-
-  public static Matcher<IntervalWindow> intervalWindow(long start, long end) {
-    return Matchers.equalTo(new IntervalWindow(new Instant(start), new Instant(end)));
-  }
-
-  public static <T> Matcher<WindowedValue<? extends T>> valueWithPaneInfo(final PaneInfo paneInfo) {
-    return new TypeSafeMatcher<WindowedValue<? extends T>>() {
-      @Override
-      public void describeTo(Description description) {
-        description
-            .appendText("WindowedValue(paneInfo = ").appendValue(paneInfo).appendText(")");
-      }
-
-      @Override
-      protected boolean matchesSafely(WindowedValue<? extends T> item) {
-        return Objects.equals(item.getPane(), paneInfo);
-      }
-
-      @Override
-      protected void describeMismatchSafely(
-          WindowedValue<? extends T> item, Description mismatchDescription) {
-        mismatchDescription.appendValue(item.getPane());
-      }
-    };
-  }
-
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  @SafeVarargs
-  public static final <W extends BoundedWindow> Matcher<Iterable<W>> ofWindows(
-      Matcher<W>... windows) {
-    return (Matcher) Matchers.<W>containsInAnyOrder(windows);
-  }
-
-  private WindowMatchers() {}
-
-  private static class WindowedValueMatcher<T> extends TypeSafeMatcher<WindowedValue<? extends T>> {
-
-    private Matcher<? super T> valueMatcher;
-    private Matcher<? super Instant> timestampMatcher;
-    private Matcher<? super Collection<? extends BoundedWindow>> windowsMatcher;
-    private Matcher<? super PaneInfo> paneInfoMatcher;
-
-    private WindowedValueMatcher(
-        Matcher<? super T> valueMatcher,
-        Matcher<? super Instant> timestampMatcher,
-        Matcher<? super Collection<? extends BoundedWindow>> windowsMatcher,
-        Matcher<? super PaneInfo> paneInfoMatcher) {
-      this.valueMatcher = valueMatcher;
-      this.timestampMatcher = timestampMatcher;
-      this.windowsMatcher = windowsMatcher;
-      this.paneInfoMatcher = paneInfoMatcher;
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      description
-          .appendText("a WindowedValue(").appendValue(valueMatcher)
-          .appendText(", ").appendValue(timestampMatcher)
-          .appendText(", ").appendValue(windowsMatcher)
-          .appendText(", ").appendValue(paneInfoMatcher)
-          .appendText(")");
-    }
-
-    @Override
-    protected boolean matchesSafely(WindowedValue<? extends T> windowedValue) {
-      return valueMatcher.matches(windowedValue.getValue())
-          && timestampMatcher.matches(windowedValue.getTimestamp())
-          && windowsMatcher.matches(windowedValue.getWindows());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java
deleted file mode 100644
index 89637e2..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java
+++ /dev/null
@@ -1,82 +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;
-
-import static org.junit.Assert.assertThat;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link WindowMatchers}.
- */
-@RunWith(JUnit4.class)
-public class WindowMatchersTest {
-
-  @Test
-  public void testIsWindowedValueExact() {
-    long timestamp = 100;
-    long windowStart = 0;
-    long windowEnd = 200;
-
-    assertThat(
-        WindowedValue.of(
-            "hello",
-            new Instant(timestamp),
-            new IntervalWindow(new Instant(windowStart), new Instant(windowEnd)),
-            PaneInfo.NO_FIRING),
-        WindowMatchers.isWindowedValue(
-            "hello",
-            new Instant(timestamp),
-            ImmutableList.of(new IntervalWindow(new Instant(windowStart), new Instant(windowEnd))),
-            PaneInfo.NO_FIRING));
-  }
-
-  @Test
-  public void testIsWindowedValueReorderedWindows() {
-    long timestamp = 100;
-    long windowStart = 0;
-    long windowEnd = 200;
-    long windowStart2 = 50;
-    long windowEnd2 = 150;
-
-    assertThat(
-        WindowedValue.of(
-            "hello",
-            new Instant(timestamp),
-            ImmutableList.of(
-                new IntervalWindow(new Instant(windowStart), new Instant(windowEnd)),
-                new IntervalWindow(new Instant(windowStart2), new Instant(windowEnd2))),
-            PaneInfo.NO_FIRING),
-        WindowMatchers.isWindowedValue(
-            "hello",
-            new Instant(timestamp),
-            ImmutableList.of(
-                new IntervalWindow(new Instant(windowStart), new Instant(windowEnd)),
-                new IntervalWindow(new Instant(windowStart2), new Instant(windowEnd2))),
-            PaneInfo.NO_FIRING));
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d9891234/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java
new file mode 100644
index 0000000..417147f
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.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.testing;
+
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.Iterables;
+import java.io.Serializable;
+import org.apache.beam.sdk.io.CountingInput;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.WithTimestamps;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link GatherAllPanes}. */
+@RunWith(JUnit4.class)
+public class GatherAllPanesTest implements Serializable {
+  @Test
+  @Category(NeedsRunner.class)
+  public void singlePaneSingleReifiedPane() {
+    TestPipeline p = TestPipeline.create();
+    PCollection<Iterable<ValueInSingleWindow<Iterable<Long>>>> accumulatedPanes =
+        p.apply(CountingInput.upTo(20000))
+            .apply(
+                WithTimestamps.of(
+                    new SerializableFunction<Long, Instant>() {
+                      @Override
+                      public Instant apply(Long input) {
+                        return new Instant(input * 10);
+                      }
+                    }))
+            .apply(
+                Window.<Long>into(FixedWindows.of(Duration.standardMinutes(1)))
+                    .triggering(AfterWatermark.pastEndOfWindow())
+                    .withAllowedLateness(Duration.ZERO)
+                    .discardingFiredPanes())
+            .apply(WithKeys.<Void, Long>of((Void) null).withKeyType(new TypeDescriptor<Void>() {}))
+            .apply(GroupByKey.<Void, Long>create())
+            .apply(Values.<Iterable<Long>>create())
+            .apply(GatherAllPanes.<Iterable<Long>>globally());
+
+    PAssert.that(accumulatedPanes)
+        .satisfies(
+            new SerializableFunction<
+                Iterable<Iterable<ValueInSingleWindow<Iterable<Long>>>>, Void>() {
+              @Override
+              public Void apply(Iterable<Iterable<ValueInSingleWindow<Iterable<Long>>>> input) {
+                for (Iterable<ValueInSingleWindow<Iterable<Long>>> windowedInput : input) {
+                  if (Iterables.size(windowedInput) > 1) {
+                    fail("Expected all windows to have exactly one pane, got " + windowedInput);
+                    return null;
+                  }
+                }
+                return null;
+              }
+            });
+
+    p.run();
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void multiplePanesMultipleReifiedPane() {
+    TestPipeline p = TestPipeline.create();
+
+    PCollection<Long> someElems = p.apply("someLongs", CountingInput.upTo(20000));
+    PCollection<Long> otherElems = p.apply("otherLongs", CountingInput.upTo(20000));
+    PCollection<Iterable<ValueInSingleWindow<Iterable<Long>>>> accumulatedPanes =
+        PCollectionList.of(someElems)
+            .and(otherElems)
+            .apply(Flatten.<Long>pCollections())
+            .apply(
+                WithTimestamps.of(
+                    new SerializableFunction<Long, Instant>() {
+                      @Override
+                      public Instant apply(Long input) {
+                        return new Instant(input * 10);
+                      }
+                    }))
+            .apply(
+                Window.<Long>into(FixedWindows.of(Duration.standardMinutes(1)))
+                    .triggering(
+                        AfterWatermark.pastEndOfWindow()
+                            .withEarlyFirings(AfterPane.elementCountAtLeast(1)))
+                    .withAllowedLateness(Duration.ZERO)
+                    .discardingFiredPanes())
+            .apply(WithKeys.<Void, Long>of((Void) null).withKeyType(new TypeDescriptor<Void>() {}))
+            .apply(GroupByKey.<Void, Long>create())
+            .apply(Values.<Iterable<Long>>create())
+            .apply(GatherAllPanes.<Iterable<Long>>globally());
+
+    PAssert.that(accumulatedPanes)
+        .satisfies(
+            new SerializableFunction<
+                Iterable<Iterable<ValueInSingleWindow<Iterable<Long>>>>, Void>() {
+              @Override
+              public Void apply(Iterable<Iterable<ValueInSingleWindow<Iterable<Long>>>> input) {
+                for (Iterable<ValueInSingleWindow<Iterable<Long>>> windowedInput : input) {
+                  if (Iterables.size(windowedInput) > 1) {
+                    return null;
+                  }
+                }
+                fail("Expected at least one window to have multiple panes");
+                return null;
+              }
+            });
+
+    p.run();
+  }
+}


[24/50] [abbrv] incubator-beam git commit: Explicitly Throw in TransformExecutorTest

Posted by ke...@apache.org.
Explicitly Throw in TransformExecutorTest


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/b4ee8b73
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/b4ee8b73
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/b4ee8b73

Branch: refs/heads/gearpump-runner
Commit: b4ee8b730bffb31ee1178303f1dbd5058eb22a11
Parents: 37e891f
Author: Thomas Groh <tg...@google.com>
Authored: Fri Dec 2 10:56:15 2016 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Fri Dec 2 13:58:38 2016 -0800

----------------------------------------------------------------------
 .../runners/direct/TransformExecutorTest.java   | 184 ++++++++++---------
 1 file changed, 97 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b4ee8b73/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java
index 85eff65..08b1e18 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java
@@ -37,13 +37,10 @@ import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.beam.runners.direct.CommittedResult.OutputType;
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
-import org.apache.beam.sdk.coders.ByteArrayCoder;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.util.IllegalMutationException;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
@@ -63,7 +60,9 @@ import org.mockito.MockitoAnnotations;
 public class TransformExecutorTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
   private PCollection<String> created;
-  private PCollection<KV<Integer, String>> downstream;
+
+  private AppliedPTransform<?, ?, ?> createdProducer;
+  private AppliedPTransform<?, ?, ?> downstreamProducer;
 
   private CountDownLatch evaluatorCompleted;
 
@@ -88,15 +87,17 @@ public class TransformExecutorTest {
 
     TestPipeline p = TestPipeline.create();
     created = p.apply(Create.of("foo", "spam", "third"));
-    downstream = created.apply(WithKeys.<Integer, String>of(3));
+    PCollection<KV<Integer, String>> downstream = created.apply(WithKeys.<Integer, String>of(3));
+
+    createdProducer = created.getProducingTransformInternal();
+    downstreamProducer = downstream.getProducingTransformInternal();
 
     when(evaluationContext.getMetrics()).thenReturn(metrics);
   }
 
   @Test
   public void callWithNullInputBundleFinishesBundleAndCompletes() throws Exception {
-    final TransformResult<Object> result =
-        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
+    final TransformResult<Object> result = StepTransformResult.withoutHold(createdProducer).build();
     final AtomicBoolean finishCalled = new AtomicBoolean(false);
     TransformEvaluator<Object> evaluator =
         new TransformEvaluator<Object>() {
@@ -112,8 +113,7 @@ public class TransformExecutorTest {
           }
         };
 
-    when(registry.forApplication(created.getProducingTransformInternal(), null))
-        .thenReturn(evaluator);
+    when(registry.forApplication(createdProducer, null)).thenReturn(evaluator);
 
     TransformExecutor<Object> executor =
         TransformExecutor.create(
@@ -121,7 +121,7 @@ public class TransformExecutorTest {
             registry,
             Collections.<ModelEnforcementFactory>emptyList(),
             null,
-            created.getProducingTransformInternal(),
+            createdProducer,
             completionCallback,
             transformEvaluationState);
     executor.run();
@@ -133,7 +133,7 @@ public class TransformExecutorTest {
 
   @Test
   public void nullTransformEvaluatorTerminates() throws Exception {
-    when(registry.forApplication(created.getProducingTransformInternal(), null)).thenReturn(null);
+    when(registry.forApplication(createdProducer, null)).thenReturn(null);
 
     TransformExecutor<Object> executor =
         TransformExecutor.create(
@@ -141,7 +141,7 @@ public class TransformExecutorTest {
             registry,
             Collections.<ModelEnforcementFactory>emptyList(),
             null,
-            created.getProducingTransformInternal(),
+            createdProducer,
             completionCallback,
             transformEvaluationState);
     executor.run();
@@ -154,7 +154,7 @@ public class TransformExecutorTest {
   @Test
   public void inputBundleProcessesEachElementFinishesAndCompletes() throws Exception {
     final TransformResult<String> result =
-        StepTransformResult.<String>withoutHold(downstream.getProducingTransformInternal()).build();
+        StepTransformResult.<String>withoutHold(downstreamProducer).build();
     final Collection<WindowedValue<String>> elementsProcessed = new ArrayList<>();
     TransformEvaluator<String> evaluator =
         new TransformEvaluator<String>() {
@@ -175,8 +175,7 @@ public class TransformExecutorTest {
     WindowedValue<String> third = WindowedValue.valueInGlobalWindow("third");
     CommittedBundle<String> inputBundle =
         bundleFactory.createBundle(created).add(foo).add(spam).add(third).commit(Instant.now());
-    when(registry.<String>forApplication(downstream.getProducingTransformInternal(), inputBundle))
-        .thenReturn(evaluator);
+    when(registry.<String>forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator);
 
     TransformExecutor<String> executor =
         TransformExecutor.create(
@@ -184,7 +183,7 @@ public class TransformExecutorTest {
             registry,
             Collections.<ModelEnforcementFactory>emptyList(),
             inputBundle,
-            downstream.getProducingTransformInternal(),
+            downstreamProducer,
             completionCallback,
             transformEvaluationState);
 
@@ -200,7 +199,7 @@ public class TransformExecutorTest {
   @Test
   public void processElementThrowsExceptionCallsback() throws Exception {
     final TransformResult<String> result =
-        StepTransformResult.<String>withoutHold(downstream.getProducingTransformInternal()).build();
+        StepTransformResult.<String>withoutHold(downstreamProducer).build();
     final Exception exception = new Exception();
     TransformEvaluator<String> evaluator =
         new TransformEvaluator<String>() {
@@ -218,8 +217,7 @@ public class TransformExecutorTest {
     WindowedValue<String> foo = WindowedValue.valueInGlobalWindow("foo");
     CommittedBundle<String> inputBundle =
         bundleFactory.createBundle(created).add(foo).commit(Instant.now());
-    when(registry.<String>forApplication(downstream.getProducingTransformInternal(), inputBundle))
-        .thenReturn(evaluator);
+    when(registry.<String>forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator);
 
     TransformExecutor<String> executor =
         TransformExecutor.create(
@@ -227,7 +225,7 @@ public class TransformExecutorTest {
             registry,
             Collections.<ModelEnforcementFactory>emptyList(),
             inputBundle,
-            downstream.getProducingTransformInternal(),
+            downstreamProducer,
             completionCallback,
             transformEvaluationState);
     Executors.newSingleThreadExecutor().submit(executor);
@@ -252,10 +250,8 @@ public class TransformExecutorTest {
           }
         };
 
-    CommittedBundle<String> inputBundle =
-        bundleFactory.createBundle(created).commit(Instant.now());
-    when(registry.<String>forApplication(downstream.getProducingTransformInternal(), inputBundle))
-        .thenReturn(evaluator);
+    CommittedBundle<String> inputBundle = bundleFactory.createBundle(created).commit(Instant.now());
+    when(registry.<String>forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator);
 
     TransformExecutor<String> executor =
         TransformExecutor.create(
@@ -263,7 +259,7 @@ public class TransformExecutorTest {
             registry,
             Collections.<ModelEnforcementFactory>emptyList(),
             inputBundle,
-            downstream.getProducingTransformInternal(),
+            downstreamProducer,
             completionCallback,
             transformEvaluationState);
     Executors.newSingleThreadExecutor().submit(executor);
@@ -277,7 +273,7 @@ public class TransformExecutorTest {
   @Test
   public void callWithEnforcementAppliesEnforcement() throws Exception {
     final TransformResult<Object> result =
-        StepTransformResult.withoutHold(downstream.getProducingTransformInternal()).build();
+        StepTransformResult.withoutHold(downstreamProducer).build();
 
     TransformEvaluator<Object> evaluator =
         new TransformEvaluator<Object>() {
@@ -294,8 +290,7 @@ public class TransformExecutorTest {
     WindowedValue<String> barElem = WindowedValue.valueInGlobalWindow("bar");
     CommittedBundle<String> inputBundle =
         bundleFactory.createBundle(created).add(fooElem).add(barElem).commit(Instant.now());
-    when(registry.forApplication(downstream.getProducingTransformInternal(), inputBundle))
-        .thenReturn(evaluator);
+    when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator);
 
     TestEnforcementFactory enforcement = new TestEnforcementFactory();
     TransformExecutor<String> executor =
@@ -304,7 +299,7 @@ public class TransformExecutorTest {
             registry,
             Collections.<ModelEnforcementFactory>singleton(enforcement),
             inputBundle,
-            downstream.getProducingTransformInternal(),
+            downstreamProducer,
             completionCallback,
             transformEvaluationState);
 
@@ -321,21 +316,8 @@ public class TransformExecutorTest {
 
   @Test
   public void callWithEnforcementThrowsOnFinishPropagates() throws Exception {
-    PCollection<byte[]> pcBytes =
-        created.apply(
-            new PTransform<PCollection<String>, PCollection<byte[]>>() {
-              @Override
-              public PCollection<byte[]> apply(PCollection<String> input) {
-                return PCollection.<byte[]>createPrimitiveOutputInternal(
-                        input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
-                    .setCoder(ByteArrayCoder.of());
-              }
-            });
-
     final TransformResult<Object> result =
-        StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build();
-    final CountDownLatch testLatch = new CountDownLatch(1);
-    final CountDownLatch evaluatorLatch = new CountDownLatch(1);
+        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
 
     TransformEvaluator<Object> evaluator =
         new TransformEvaluator<Object>() {
@@ -344,62 +326,42 @@ public class TransformExecutorTest {
 
           @Override
           public TransformResult<Object> finishBundle() throws Exception {
-            testLatch.countDown();
-            evaluatorLatch.await();
             return result;
           }
         };
 
-    WindowedValue<byte[]> fooBytes = WindowedValue.valueInGlobalWindow("foo".getBytes());
-    CommittedBundle<byte[]> inputBundle =
-        bundleFactory.createBundle(pcBytes).add(fooBytes).commit(Instant.now());
-    when(registry.forApplication(pcBytes.getProducingTransformInternal(), inputBundle))
-        .thenReturn(evaluator);
+    WindowedValue<String> fooBytes = WindowedValue.valueInGlobalWindow("foo");
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createBundle(created).add(fooBytes).commit(Instant.now());
+    when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator);
 
-    TransformExecutor<byte[]> executor =
+    TransformExecutor<String> executor =
         TransformExecutor.create(
             evaluationContext,
             registry,
-            Collections.<ModelEnforcementFactory>singleton(ImmutabilityEnforcementFactory.create()),
+            Collections.<ModelEnforcementFactory>singleton(
+                new ThrowingEnforcementFactory(ThrowingEnforcementFactory.When.AFTER_BUNDLE)),
             inputBundle,
-            pcBytes.getProducingTransformInternal(),
+            downstreamProducer,
             completionCallback,
             transformEvaluationState);
 
     Future<?> task = Executors.newSingleThreadExecutor().submit(executor);
-    testLatch.await();
-    fooBytes.getValue()[0] = 'b';
-    evaluatorLatch.countDown();
 
-    thrown.expectCause(isA(IllegalMutationException.class));
+    thrown.expectCause(isA(RuntimeException.class));
+    thrown.expectMessage("afterFinish");
     task.get();
   }
 
   @Test
   public void callWithEnforcementThrowsOnElementPropagates() throws Exception {
-    PCollection<byte[]> pcBytes =
-        created.apply(
-            new PTransform<PCollection<String>, PCollection<byte[]>>() {
-              @Override
-              public PCollection<byte[]> apply(PCollection<String> input) {
-                return PCollection.<byte[]>createPrimitiveOutputInternal(
-                        input.getPipeline(), input.getWindowingStrategy(), input.isBounded())
-                    .setCoder(ByteArrayCoder.of());
-              }
-            });
-
     final TransformResult<Object> result =
-        StepTransformResult.withoutHold(pcBytes.getProducingTransformInternal()).build();
-    final CountDownLatch testLatch = new CountDownLatch(1);
-    final CountDownLatch evaluatorLatch = new CountDownLatch(1);
+        StepTransformResult.withoutHold(created.getProducingTransformInternal()).build();
 
     TransformEvaluator<Object> evaluator =
         new TransformEvaluator<Object>() {
           @Override
-          public void processElement(WindowedValue<Object> element) throws Exception {
-            testLatch.countDown();
-            evaluatorLatch.await();
-          }
+          public void processElement(WindowedValue<Object> element) throws Exception {}
 
           @Override
           public TransformResult<Object> finishBundle() throws Exception {
@@ -407,28 +369,26 @@ public class TransformExecutorTest {
           }
         };
 
-    WindowedValue<byte[]> fooBytes = WindowedValue.valueInGlobalWindow("foo".getBytes());
-    CommittedBundle<byte[]> inputBundle =
-        bundleFactory.createBundle(pcBytes).add(fooBytes).commit(Instant.now());
-    when(registry.forApplication(pcBytes.getProducingTransformInternal(), inputBundle))
-        .thenReturn(evaluator);
+    WindowedValue<String> fooBytes = WindowedValue.valueInGlobalWindow("foo");
+    CommittedBundle<String> inputBundle =
+        bundleFactory.createBundle(created).add(fooBytes).commit(Instant.now());
+    when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator);
 
-    TransformExecutor<byte[]> executor =
+    TransformExecutor<String> executor =
         TransformExecutor.create(
             evaluationContext,
             registry,
-            Collections.<ModelEnforcementFactory>singleton(ImmutabilityEnforcementFactory.create()),
+            Collections.<ModelEnforcementFactory>singleton(
+                new ThrowingEnforcementFactory(ThrowingEnforcementFactory.When.AFTER_ELEMENT)),
             inputBundle,
-            pcBytes.getProducingTransformInternal(),
+            downstreamProducer,
             completionCallback,
             transformEvaluationState);
 
     Future<?> task = Executors.newSingleThreadExecutor().submit(executor);
-    testLatch.await();
-    fooBytes.getValue()[0] = 'b';
-    evaluatorLatch.countDown();
 
-    thrown.expectCause(isA(IllegalMutationException.class));
+    thrown.expectCause(isA(RuntimeException.class));
+    thrown.expectMessage("afterElement");
     task.get();
   }
 
@@ -509,4 +469,54 @@ public class TransformExecutorTest {
       finishedBundles.add(result);
     }
   }
+
+  private static class ThrowingEnforcementFactory implements ModelEnforcementFactory {
+    private final When when;
+
+    private ThrowingEnforcementFactory(When when) {
+      this.when = when;
+    }
+
+    enum When {
+      BEFORE_BUNDLE,
+      BEFORE_ELEMENT,
+      AFTER_ELEMENT,
+      AFTER_BUNDLE
+    }
+
+    @Override
+    public <T> ModelEnforcement<T> forBundle(
+        CommittedBundle<T> input, AppliedPTransform<?, ?, ?> consumer) {
+      if (when == When.BEFORE_BUNDLE) {
+        throw new RuntimeException("forBundle");
+      }
+      return new ThrowingEnforcement<>();
+    }
+
+    private class ThrowingEnforcement<T> implements ModelEnforcement<T> {
+      @Override
+      public void beforeElement(WindowedValue<T> element) {
+        if (when == When.BEFORE_ELEMENT) {
+          throw new RuntimeException("beforeElement");
+        }
+      }
+
+      @Override
+      public void afterElement(WindowedValue<T> element) {
+        if (when == When.AFTER_ELEMENT) {
+          throw new RuntimeException("afterElement");
+        }
+      }
+
+      @Override
+      public void afterFinish(
+          CommittedBundle<T> input,
+          TransformResult<T> result,
+          Iterable<? extends CommittedBundle<?>> outputs) {
+        if (when == When.AFTER_BUNDLE) {
+          throw new RuntimeException("afterFinish");
+        }
+      }
+    }
+  }
 }


[36/50] [abbrv] incubator-beam git commit: Closes #1475

Posted by ke...@apache.org.
Closes #1475


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/c8404557
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/c8404557
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/c8404557

Branch: refs/heads/gearpump-runner
Commit: c84045573948a7cba72e37e5e562c7f63375e9ea
Parents: 26eb435 9a038c4
Author: Dan Halperin <dh...@google.com>
Authored: Fri Dec 2 17:25:36 2016 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Dec 2 17:25:36 2016 -0800

----------------------------------------------------------------------
 .../org/apache/beam/sdk/io/FileBasedSink.java   | 22 +++++++++------
 .../java/org/apache/beam/sdk/io/TextIO.java     | 28 ++++++++++++++++----
 .../java/org/apache/beam/sdk/io/XmlSink.java    |  4 +--
 .../org/apache/beam/sdk/io/XmlSinkTest.java     |  6 ++---
 4 files changed, 42 insertions(+), 18 deletions(-)
----------------------------------------------------------------------



[10/50] [abbrv] incubator-beam git commit: Migrate TransformTreeNode to an Inner Class

Posted by ke...@apache.org.
Migrate TransformTreeNode to an Inner Class

TransformTreeNode requires access to the hierarchy it is contained
within, and generally cannot be separated from TransformHierarchy. It is
primarily an implementation detail of TransformHierarchy, so can be
relocated to within it.


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/569e8d70
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/569e8d70
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/569e8d70

Branch: refs/heads/gearpump-runner
Commit: 569e8d7085cf4e6effd379f23716202c6c5daf52
Parents: 24fab9f
Author: Thomas Groh <tg...@google.com>
Authored: Thu Dec 1 13:19:14 2016 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Thu Dec 1 14:34:21 2016 -0800

----------------------------------------------------------------------
 .../translation/ApexPipelineTranslator.java     |  12 +-
 .../apex/translation/TranslationContext.java    |   6 +-
 .../direct/ConsumerTrackingPipelineVisitor.java |  12 +-
 .../runners/direct/DisplayDataValidator.java    |   6 +-
 .../direct/KeyedPValueTrackingVisitor.java      |  10 +-
 .../apache/beam/runners/flink/FlinkRunner.java  |  12 +-
 .../FlinkBatchPipelineTranslator.java           |  14 +-
 .../FlinkStreamingPipelineTranslator.java       |  16 +-
 .../PipelineTranslationOptimizer.java           |  10 +-
 .../dataflow/DataflowPipelineTranslator.java    |   8 +-
 .../beam/runners/dataflow/DataflowRunner.java   |  10 +-
 .../runners/dataflow/DataflowRunnerTest.java    |   4 +-
 .../dataflow/RecordingPipelineVisitor.java      |   6 +-
 .../apache/beam/runners/spark/SparkRunner.java  |  21 +-
 .../beam/sdk/AggregatorPipelineExtractor.java   |   6 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java |  17 +-
 .../beam/sdk/runners/TransformHierarchy.java    | 243 +++++++++++++++-
 .../beam/sdk/runners/TransformTreeNode.java     | 282 -------------------
 .../sdk/AggregatorPipelineExtractorTest.java    |  20 +-
 .../sdk/runners/TransformHierarchyTest.java     |  26 +-
 .../beam/sdk/runners/TransformTreeTest.java     |   8 +-
 .../display/DisplayDataEvaluator.java           |   8 +-
 22 files changed, 343 insertions(+), 414 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java
index d38faf7..8d6db84 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java
@@ -19,17 +19,15 @@
 package org.apache.beam.runners.apex.translation;
 
 import com.datatorrent.api.DAG;
-
 import java.util.HashMap;
 import java.util.Map;
-
 import org.apache.beam.runners.apex.ApexPipelineOptions;
 import org.apache.beam.runners.apex.ApexRunner.CreateApexPCollectionView;
 import org.apache.beam.runners.apex.translation.operators.ApexReadUnboundedInputOperator;
 import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.Flatten;
 import org.apache.beam.sdk.transforms.GroupByKey;
@@ -84,18 +82,18 @@ public class ApexPipelineTranslator implements Pipeline.PipelineVisitor {
   }
 
   @Override
-  public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
     LOG.debug("entering composite transform {}", node.getTransform());
     return CompositeBehavior.ENTER_TRANSFORM;
   }
 
   @Override
-  public void leaveCompositeTransform(TransformTreeNode node) {
+  public void leaveCompositeTransform(TransformHierarchy.Node node) {
     LOG.debug("leaving composite transform {}", node.getTransform());
   }
 
   @Override
-  public void visitPrimitiveTransform(TransformTreeNode node) {
+  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
     LOG.debug("visiting transform {}", node.getTransform());
     PTransform transform = node.getTransform();
     TransformTranslator translator = getTransformTranslator(transform.getClass());
@@ -108,7 +106,7 @@ public class ApexPipelineTranslator implements Pipeline.PipelineVisitor {
   }
 
   @Override
-  public void visitValue(PValue value, TransformTreeNode producer) {
+  public void visitValue(PValue value, TransformHierarchy.Node producer) {
     LOG.debug("visiting value {}", value);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java
index e016730..259afbd 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java
@@ -24,18 +24,16 @@ import com.datatorrent.api.DAG;
 import com.datatorrent.api.Operator;
 import com.datatorrent.api.Operator.InputPort;
 import com.datatorrent.api.Operator.OutputPort;
-
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-
 import org.apache.beam.runners.apex.ApexPipelineOptions;
 import org.apache.beam.runners.apex.translation.utils.ApexStateInternals;
 import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple;
 import org.apache.beam.runners.apex.translation.utils.CoderAdapterStreamCodec;
 import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
@@ -73,7 +71,7 @@ class TranslationContext {
     this.pipelineOptions = pipelineOptions;
   }
 
-  public void setCurrentTransform(TransformTreeNode treeNode) {
+  public void setCurrentTransform(TransformHierarchy.Node treeNode) {
     this.currentTransform = AppliedPTransform.of(treeNode.getFullName(),
         treeNode.getInput(), treeNode.getOutput(), (PTransform) treeNode.getTransform());
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
index 4fdfea0..acfad16 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ConsumerTrackingPipelineVisitor.java
@@ -28,7 +28,7 @@ import java.util.Set;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor;
 import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PCollectionView;
@@ -50,7 +50,7 @@ public class ConsumerTrackingPipelineVisitor extends PipelineVisitor.Defaults {
   private boolean finalized = false;
 
   @Override
-  public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
     checkState(
         !finalized,
         "Attempting to traverse a pipeline (node %s) with a %s "
@@ -61,7 +61,7 @@ public class ConsumerTrackingPipelineVisitor extends PipelineVisitor.Defaults {
   }
 
   @Override
-  public void leaveCompositeTransform(TransformTreeNode node) {
+  public void leaveCompositeTransform(TransformHierarchy.Node node) {
     checkState(
         !finalized,
         "Attempting to traverse a pipeline (node %s) with a %s which is already finalized",
@@ -73,7 +73,7 @@ public class ConsumerTrackingPipelineVisitor extends PipelineVisitor.Defaults {
   }
 
   @Override
-  public void visitPrimitiveTransform(TransformTreeNode node) {
+  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
     toFinalize.removeAll(node.getInput().expand());
     AppliedPTransform<?, ?, ?> appliedTransform = getAppliedTransform(node);
     stepNames.put(appliedTransform, genStepName());
@@ -86,7 +86,7 @@ public class ConsumerTrackingPipelineVisitor extends PipelineVisitor.Defaults {
     }
   }
 
-  private AppliedPTransform<?, ?, ?> getAppliedTransform(TransformTreeNode node) {
+  private AppliedPTransform<?, ?, ?> getAppliedTransform(TransformHierarchy.Node node) {
     @SuppressWarnings({"rawtypes", "unchecked"})
     AppliedPTransform<?, ?, ?> application = AppliedPTransform.of(
         node.getFullName(), node.getInput(), node.getOutput(), (PTransform) node.getTransform());
@@ -94,7 +94,7 @@ public class ConsumerTrackingPipelineVisitor extends PipelineVisitor.Defaults {
   }
 
   @Override
-  public void visitValue(PValue value, TransformTreeNode producer) {
+  public void visitValue(PValue value, TransformHierarchy.Node producer) {
     toFinalize.add(value);
     for (PValue expandedValue : value.expand()) {
       valueToConsumers.put(expandedValue, new ArrayList<AppliedPTransform<?, ?, ?>>());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java
index e09fe62..c77cb48 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java
@@ -18,7 +18,7 @@
 package org.apache.beam.runners.direct;
 
 import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.HasDisplayData;
 
@@ -51,7 +51,7 @@ class DisplayDataValidator {
     private static final Visitor INSTANCE = new Visitor();
 
     @Override
-    public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+    public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
       if (!node.isRootNode()) {
         evaluateDisplayData(node.getTransform());
       }
@@ -60,7 +60,7 @@ class DisplayDataValidator {
     }
 
     @Override
-    public void visitPrimitiveTransform(TransformTreeNode node) {
+    public void visitPrimitiveTransform(TransformHierarchy.Node node) {
       evaluateDisplayData(node.getTransform());
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
index 47b0857..5dc24c2 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
@@ -22,7 +22,7 @@ import static com.google.common.base.Preconditions.checkState;
 import java.util.HashSet;
 import java.util.Set;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PValue;
@@ -55,7 +55,7 @@ class KeyedPValueTrackingVisitor implements PipelineVisitor {
   }
 
   @Override
-  public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
     checkState(
         !finalized,
         "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)",
@@ -65,7 +65,7 @@ class KeyedPValueTrackingVisitor implements PipelineVisitor {
   }
 
   @Override
-  public void leaveCompositeTransform(TransformTreeNode node) {
+  public void leaveCompositeTransform(TransformHierarchy.Node node) {
     checkState(
         !finalized,
         "Attempted to use a %s that has already been finalized on a pipeline (visiting node %s)",
@@ -79,10 +79,10 @@ class KeyedPValueTrackingVisitor implements PipelineVisitor {
   }
 
   @Override
-  public void visitPrimitiveTransform(TransformTreeNode node) {}
+  public void visitPrimitiveTransform(TransformHierarchy.Node node) {}
 
   @Override
-  public void visitValue(PValue value, TransformTreeNode producer) {
+  public void visitValue(PValue value, TransformHierarchy.Node producer) {
     if (producesKeyedOutputs.contains(producer.getTransform().getClass())) {
       keyedValues.addAll(value.expand());
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
index 488c170..0b92734 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
@@ -31,7 +31,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
-
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.coders.Coder;
@@ -41,7 +40,7 @@ import org.apache.beam.sdk.coders.ListCoder;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsValidator;
 import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.OldDoFn;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -55,7 +54,6 @@ import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
-
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.client.program.DetachedEnvironment;
 import org.slf4j.Logger;
@@ -259,18 +257,18 @@ public class FlinkRunner extends PipelineRunner<PipelineResult> {
       final SortedSet<String> ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>();
       pipeline.traverseTopologically(new Pipeline.PipelineVisitor() {
         @Override
-        public void visitValue(PValue value, TransformTreeNode producer) {
+        public void visitValue(PValue value, TransformHierarchy.Node producer) {
         }
 
         @Override
-        public void visitPrimitiveTransform(TransformTreeNode node) {
+        public void visitPrimitiveTransform(TransformHierarchy.Node node) {
           if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
             ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
           }
         }
 
         @Override
-        public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+        public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
           if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
             ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
           }
@@ -278,7 +276,7 @@ public class FlinkRunner extends PipelineRunner<PipelineResult> {
         }
 
         @Override
-        public void leaveCompositeTransform(TransformTreeNode node) {
+        public void leaveCompositeTransform(TransformHierarchy.Node node) {
         }
       });
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
index f36be6b..805c41c 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
@@ -19,7 +19,7 @@ package org.apache.beam.runners.flink.translation;
 
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.flink.api.java.DataSet;
@@ -63,7 +63,7 @@ public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
   // --------------------------------------------------------------------------------------------
 
   @Override
-  public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
     LOG.info(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node));
     this.depth++;
 
@@ -79,13 +79,13 @@ public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
   }
 
   @Override
-  public void leaveCompositeTransform(TransformTreeNode node) {
+  public void leaveCompositeTransform(TransformHierarchy.Node node) {
     this.depth--;
     LOG.info(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node));
   }
 
   @Override
-  public void visitPrimitiveTransform(TransformTreeNode node) {
+  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
     LOG.info(genSpaces(this.depth) + "visitPrimitiveTransform- " + formatNodeName(node));
 
     // get the transformation corresponding to the node we are
@@ -103,7 +103,7 @@ public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
 
   private <T extends PTransform<?, ?>> void applyBatchTransform(
       PTransform<?, ?> transform,
-      TransformTreeNode node,
+      TransformHierarchy.Node node,
       BatchTransformTranslator<?> translator) {
 
     @SuppressWarnings("unchecked")
@@ -128,7 +128,7 @@ public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
   /**
    * Returns a translator for the given node, if it is possible, otherwise null.
    */
-  private static BatchTransformTranslator<?> getTranslator(TransformTreeNode node) {
+  private static BatchTransformTranslator<?> getTranslator(TransformHierarchy.Node node) {
     PTransform<?, ?> transform = node.getTransform();
 
     // Root of the graph is null
@@ -139,7 +139,7 @@ public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
     return FlinkBatchTransformTranslators.getTranslator(transform);
   }
 
-  private static String formatNodeName(TransformTreeNode node) {
+  private static String formatNodeName(TransformHierarchy.Node node) {
     return node.toString().split("@")[1] + node.getTransform();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
index e5c0d76..a07dc3d 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
@@ -18,7 +18,7 @@
 package org.apache.beam.runners.flink.translation;
 
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PValue;
@@ -50,7 +50,7 @@ public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
   // --------------------------------------------------------------------------------------------
 
   @Override
-  public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
     LOG.info(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node));
     this.depth++;
 
@@ -69,13 +69,13 @@ public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
   }
 
   @Override
-  public void leaveCompositeTransform(TransformTreeNode node) {
+  public void leaveCompositeTransform(TransformHierarchy.Node node) {
     this.depth--;
     LOG.info(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node));
   }
 
   @Override
-  public void visitPrimitiveTransform(TransformTreeNode node) {
+  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
     LOG.info(genSpaces(this.depth) + "visitPrimitiveTransform- " + formatNodeName(node));
     // get the transformation corresponding to hte node we are
     // currently visiting and translate it into its Flink alternative.
@@ -93,13 +93,13 @@ public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
   }
 
   @Override
-  public void visitValue(PValue value, TransformTreeNode producer) {
+  public void visitValue(PValue value, TransformHierarchy.Node producer) {
     // do nothing here
   }
 
   private <T extends PTransform<?, ?>> void applyStreamingTransform(
       PTransform<?, ?> transform,
-      TransformTreeNode node,
+      TransformHierarchy.Node node,
       StreamTransformTranslator<?> translator) {
 
     @SuppressWarnings("unchecked")
@@ -116,7 +116,7 @@ public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
 
   private <T extends PTransform<?, ?>> boolean applyCanTranslate(
       PTransform<?, ?> transform,
-      TransformTreeNode node,
+      TransformHierarchy.Node node,
       StreamTransformTranslator<?> translator) {
 
     @SuppressWarnings("unchecked")
@@ -151,7 +151,7 @@ public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
     }
   }
 
-  private static String formatNodeName(TransformTreeNode node) {
+  private static String formatNodeName(TransformHierarchy.Node node) {
     return node.toString().split("@")[1] + node.getTransform();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/PipelineTranslationOptimizer.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/PipelineTranslationOptimizer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/PipelineTranslationOptimizer.java
index 97d123c..99f7ceb 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/PipelineTranslationOptimizer.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/PipelineTranslationOptimizer.java
@@ -19,7 +19,7 @@ package org.apache.beam.runners.flink.translation;
 
 import org.apache.beam.runners.flink.FlinkPipelineOptions;
 import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PValue;
 import org.slf4j.Logger;
@@ -52,15 +52,15 @@ public class PipelineTranslationOptimizer extends FlinkPipelineTranslator {
   }
 
   @Override
-  public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+  public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
     return CompositeBehavior.ENTER_TRANSFORM;
   }
 
   @Override
-  public void leaveCompositeTransform(TransformTreeNode node) {}
+  public void leaveCompositeTransform(TransformHierarchy.Node node) {}
 
   @Override
-  public void visitPrimitiveTransform(TransformTreeNode node) {
+  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
     Class<? extends PTransform> transformClass = node.getTransform().getClass();
     if (transformClass == Read.Unbounded.class) {
       LOG.info("Found {}. Switching to streaming execution.", transformClass);
@@ -69,5 +69,5 @@ public class PipelineTranslationOptimizer extends FlinkPipelineTranslator {
   }
 
   @Override
-  public void visitValue(PValue value, TransformTreeNode producer) {}
+  public void visitValue(PValue value, TransformHierarchy.Node producer) {}
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
index 2af2cae..1cff42a 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
@@ -66,7 +66,7 @@ import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.IterableCoder;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.options.StreamingOptions;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.DoFn;
@@ -517,11 +517,11 @@ public class DataflowPipelineTranslator {
 
 
     @Override
-    public void leaveCompositeTransform(TransformTreeNode node) {
+    public void leaveCompositeTransform(TransformHierarchy.Node node) {
     }
 
     @Override
-    public void visitPrimitiveTransform(TransformTreeNode node) {
+    public void visitPrimitiveTransform(TransformHierarchy.Node node) {
       PTransform<?, ?> transform = node.getTransform();
       TransformTranslator translator =
           getTransformTranslator(transform.getClass());
@@ -537,7 +537,7 @@ public class DataflowPipelineTranslator {
     }
 
     @Override
-    public void visitValue(PValue value, TransformTreeNode producer) {
+    public void visitValue(PValue value, TransformHierarchy.Node producer) {
       LOG.debug("Checking translation of {}", value);
       if (value.getProducingTransformInternal() == null) {
         throw new RuntimeException(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index 0099856..6ed386a 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -119,7 +119,7 @@ import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsValidator;
 import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
@@ -684,18 +684,18 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
       final SortedSet<String> ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>();
       pipeline.traverseTopologically(new PipelineVisitor() {
         @Override
-        public void visitValue(PValue value, TransformTreeNode producer) {
+        public void visitValue(PValue value, TransformHierarchy.Node producer) {
         }
 
         @Override
-        public void visitPrimitiveTransform(TransformTreeNode node) {
+        public void visitPrimitiveTransform(TransformHierarchy.Node node) {
           if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
             ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
           }
         }
 
         @Override
-        public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+        public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
           if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
             ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
           }
@@ -703,7 +703,7 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
         }
 
         @Override
-        public void leaveCompositeTransform(TransformTreeNode node) {
+        public void leaveCompositeTransform(TransformHierarchy.Node node) {
         }
       });
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
index 3925ed4..5375c95 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
@@ -81,7 +81,7 @@ import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.runners.dataflow.TestCountingSource;
 import org.apache.beam.sdk.testing.ExpectedLogs;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -912,7 +912,7 @@ public class DataflowRunnerTest {
     private List<PTransform<?, ?>> transforms = new ArrayList<>();
 
     @Override
-    public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+    public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
       if (node.getTransform() != null) {
         transforms.add(node.getTransform());
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java
index 2090877..1d5a7f5 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java
@@ -20,7 +20,7 @@ package org.apache.beam.runners.dataflow;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PValue;
 
@@ -34,12 +34,12 @@ class RecordingPipelineVisitor extends Pipeline.PipelineVisitor.Defaults {
   public final List<PValue> values = new ArrayList<>();
 
   @Override
-  public void visitPrimitiveTransform(TransformTreeNode node) {
+  public void visitPrimitiveTransform(TransformHierarchy.Node node) {
     transforms.add(node.getTransform());
   }
 
   @Override
-  public void visitValue(PValue value, TransformTreeNode producer) {
+  public void visitValue(PValue value, TransformHierarchy.Node producer) {
     values.add(value);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
index 49e0113..63f77c0 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
@@ -32,7 +32,7 @@ import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.options.PipelineOptionsValidator;
 import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -213,7 +213,7 @@ public final class SparkRunner extends PipelineRunner<EvaluationResult> {
     }
 
     @Override
-    public void visitPrimitiveTransform(TransformTreeNode node) {
+    public void visitPrimitiveTransform(TransformHierarchy.Node node) {
       if (translationMode.equals(TranslationMode.BATCH)) {
         Class<? extends PTransform> transformClass = node.getTransform().getClass();
         if (transformClass == Read.Unbounded.class) {
@@ -239,7 +239,7 @@ public final class SparkRunner extends PipelineRunner<EvaluationResult> {
     }
 
     @Override
-    public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+    public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
       if (node.getTransform() != null) {
         @SuppressWarnings("unchecked")
         Class<PTransform<?, ?>> transformClass =
@@ -254,7 +254,7 @@ public final class SparkRunner extends PipelineRunner<EvaluationResult> {
       return CompositeBehavior.ENTER_TRANSFORM;
     }
 
-    private boolean shouldDefer(TransformTreeNode node) {
+    private boolean shouldDefer(TransformHierarchy.Node node) {
       PInput input = node.getInput();
       // if the input is not a PCollection, or it is but with non merging windows, don't defer.
       if (!(input instanceof PCollection)
@@ -283,12 +283,12 @@ public final class SparkRunner extends PipelineRunner<EvaluationResult> {
     }
 
     @Override
-    public void visitPrimitiveTransform(TransformTreeNode node) {
+    public void visitPrimitiveTransform(TransformHierarchy.Node node) {
       doVisitTransform(node);
     }
 
     <TransformT extends PTransform<? super PInput, POutput>> void
-        doVisitTransform(TransformTreeNode node) {
+        doVisitTransform(TransformHierarchy.Node node) {
       @SuppressWarnings("unchecked")
       TransformT transform = (TransformT) node.getTransform();
       @SuppressWarnings("unchecked")
@@ -304,11 +304,12 @@ public final class SparkRunner extends PipelineRunner<EvaluationResult> {
     }
 
     /**
-     *  Determine if this Node belongs to a Bounded branch of the pipeline, or Unbounded, and
-     *  translate with the proper translator.
+     * Determine if this Node belongs to a Bounded branch of the pipeline, or Unbounded, and
+     * translate with the proper translator.
      */
-    private <TransformT extends PTransform<? super PInput, POutput>> TransformEvaluator<TransformT>
-        translate(TransformTreeNode node, TransformT transform, Class<TransformT> transformClass) {
+    private <TransformT extends PTransform<? super PInput, POutput>>
+        TransformEvaluator<TransformT> translate(
+            TransformHierarchy.Node node, TransformT transform, Class<TransformT> transformClass) {
       //--- determine if node is bounded/unbounded.
       // usually, the input determines if the PCollection to apply the next transformation to
       // is BOUNDED or UNBOUNDED, meaning RDD/DStream.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java
index 0e79abe..d2130d0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorPipelineExtractor.java
@@ -23,7 +23,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.AggregatorRetriever;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -62,7 +62,7 @@ class AggregatorPipelineExtractor {
     }
 
     @Override
-    public void visitPrimitiveTransform(TransformTreeNode node) {
+    public void visitPrimitiveTransform(TransformHierarchy.Node node) {
       PTransform<?, ?> transform = node.getTransform();
       addStepToAggregators(transform, getAggregators(transform));
     }
@@ -86,6 +86,6 @@ class AggregatorPipelineExtractor {
     }
 
     @Override
-    public void visitValue(PValue value, TransformTreeNode producer) {}
+    public void visitValue(PValue value, TransformHierarchy.Node producer) {}
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
index c8a4439..7a16f9d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
@@ -30,7 +30,6 @@ import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.TransformHierarchy;
-import org.apache.beam.sdk.runners.TransformTreeNode;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -218,25 +217,25 @@ public class Pipeline {
      *
      * <p>The return value controls whether or not child transforms are visited.
      */
-    CompositeBehavior enterCompositeTransform(TransformTreeNode node);
+    CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node);
 
     /**
      * Called for each composite transform after all of its component transforms and their outputs
      * have been visited.
      */
-    void leaveCompositeTransform(TransformTreeNode node);
+    void leaveCompositeTransform(TransformHierarchy.Node node);
 
     /**
      * Called for each primitive transform after all of its topological predecessors
      * and inputs have been visited.
      */
-    void visitPrimitiveTransform(TransformTreeNode node);
+    void visitPrimitiveTransform(TransformHierarchy.Node node);
 
     /**
      * Called for each value after the transform that produced the value has been
      * visited.
      */
-    void visitValue(PValue value, TransformTreeNode producer);
+    void visitValue(PValue value, TransformHierarchy.Node producer);
 
     /**
      * Control enum for indicating whether or not a traversal should process the contents of
@@ -253,18 +252,18 @@ public class Pipeline {
      */
     class Defaults implements PipelineVisitor {
       @Override
-      public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+      public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
         return CompositeBehavior.ENTER_TRANSFORM;
       }
 
       @Override
-      public void leaveCompositeTransform(TransformTreeNode node) { }
+      public void leaveCompositeTransform(TransformHierarchy.Node node) { }
 
       @Override
-      public void visitPrimitiveTransform(TransformTreeNode node) { }
+      public void visitPrimitiveTransform(TransformHierarchy.Node node) { }
 
       @Override
-      public void visitValue(PValue value, TransformTreeNode producer) { }
+      public void visitValue(PValue value, TransformHierarchy.Node producer) { }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
index d3fd497..662acc1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
@@ -20,13 +20,18 @@ package org.apache.beam.sdk.runners;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
 
+import com.google.common.annotations.VisibleForTesting;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import javax.annotation.Nullable;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
@@ -37,13 +42,13 @@ import org.apache.beam.sdk.values.PValue;
  * associated {@link PValue}s.
  */
 public class TransformHierarchy {
-  private final TransformTreeNode root;
-  private final Map<POutput, TransformTreeNode> producers;
+  private final Node root;
+  private final Map<POutput, Node> producers;
   // Maintain a stack based on the enclosing nodes
-  private TransformTreeNode current;
+  private Node current;
 
   public TransformHierarchy() {
-    root = TransformTreeNode.root(this);
+    root = new Node(null, null, "", null);
     current = root;
     producers = new HashMap<>();
   }
@@ -58,20 +63,22 @@ public class TransformHierarchy {
    *
    * @return the added node
    */
-  public TransformTreeNode pushNode(String name, PInput input, PTransform<?, ?> transform) {
+  public Node pushNode(String name, PInput input, PTransform<?, ?> transform) {
     checkNotNull(
         transform, "A %s must be provided for all Nodes", PTransform.class.getSimpleName());
     checkNotNull(
         name, "A name must be provided for all %s Nodes", PTransform.class.getSimpleName());
     checkNotNull(
         input, "An input must be provided for all %s Nodes", PTransform.class.getSimpleName());
-    current = TransformTreeNode.subtransform(current, transform, name, input);
+    Node node = new Node(current, transform, name, input);
+    current.addComposite(node);
+    current = node;
     return current;
   }
 
   /**
    * Finish specifying all of the input {@link PValue PValues} of the current {@link
-   * TransformTreeNode}. Ensures that all of the inputs to the current node have been fully
+   * Node}. Ensures that all of the inputs to the current node have been fully
    * specified, and have been produced by a node in this graph.
    */
   public void finishSpecifyingInput() {
@@ -84,7 +91,7 @@ public class TransformHierarchy {
   }
 
   /**
-   * Set the output of the current {@link TransformTreeNode}. If the output is new (setOutput has
+   * Set the output of the current {@link Node}. If the output is new (setOutput has
    * not previously been called with it as the parameter), the current node is set as the producer
    * of that {@link POutput}.
    *
@@ -114,7 +121,7 @@ public class TransformHierarchy {
     checkState(current != null, "Can't pop the root node of a TransformHierarchy");
   }
 
-  TransformTreeNode getProducer(PValue produced) {
+  Node getProducer(PValue produced) {
     return producers.get(produced);
   }
 
@@ -122,10 +129,10 @@ public class TransformHierarchy {
    * Returns all producing transforms for the {@link PValue PValues} contained
    * in {@code output}.
    */
-  List<TransformTreeNode> getProducingTransforms(POutput output) {
-    List<TransformTreeNode> producingTransforms = new ArrayList<>();
+  List<Node> getProducingTransforms(POutput output) {
+    List<Node> producingTransforms = new ArrayList<>();
     for (PValue value : output.expand()) {
-      TransformTreeNode producer = getProducer(value);
+      Node producer = getProducer(value);
       if (producer != null) {
         producingTransforms.add(producer);
       }
@@ -139,7 +146,217 @@ public class TransformHierarchy {
     return visitedValues;
   }
 
-  public TransformTreeNode getCurrent() {
+  public Node getCurrent() {
     return current;
   }
+
+  /**
+   * Provides internal tracking of transform relationships with helper methods
+   * for initialization and ordered visitation.
+   */
+  public class Node {
+    private final Node enclosingNode;
+    // The PTransform for this node, which may be a composite PTransform.
+    // The root of a TransformHierarchy is represented as a Node
+    // with a null transform field.
+    private final PTransform<?, ?> transform;
+
+    private final String fullName;
+
+    // Nodes for sub-transforms of a composite transform.
+    private final Collection<Node> parts = new ArrayList<>();
+
+    // Input to the transform, in unexpanded form.
+    private final PInput input;
+
+    // TODO: track which outputs need to be exported to parent.
+    // Output of the transform, in unexpanded form.
+    private POutput output;
+
+    @VisibleForTesting
+    boolean finishedSpecifying = false;
+
+    /**
+     * Creates a new Node with the given parent and transform.
+     *
+     * <p>EnclosingNode and transform may both be null for a root-level node, which holds all other
+     * nodes.
+     *
+     * @param enclosingNode the composite node containing this node
+     * @param transform the PTransform tracked by this node
+     * @param fullName the fully qualified name of the transform
+     * @param input the unexpanded input to the transform
+     */
+    private Node(
+        @Nullable Node enclosingNode,
+        @Nullable PTransform<?, ?> transform,
+        String fullName,
+        @Nullable PInput input) {
+      this.enclosingNode = enclosingNode;
+      this.transform = transform;
+      this.fullName = fullName;
+      this.input = input;
+    }
+
+    /**
+     * Returns the transform associated with this transform node.
+     */
+    public PTransform<?, ?> getTransform() {
+      return transform;
+    }
+
+    /**
+     * Returns the enclosing composite transform node, or null if there is none.
+     */
+    public Node getEnclosingNode() {
+      return enclosingNode;
+    }
+
+    /**
+     * Adds a composite operation to the transform node.
+     *
+     * <p>As soon as a node is added, the transform node is considered a
+     * composite operation instead of a primitive transform.
+     */
+    public void addComposite(Node node) {
+      parts.add(node);
+    }
+
+    /**
+     * Returns true if this node represents a composite transform that does not perform processing
+     * of its own, but merely encapsulates a sub-pipeline (which may be empty).
+     *
+     * <p>Note that a node may be composite with no sub-transforms if it returns its input directly
+     * extracts a component of a tuple, or other operations that occur at pipeline assembly time.
+     */
+    public boolean isCompositeNode() {
+      return !parts.isEmpty() || isRootNode() || returnsOthersOutput();
+    }
+
+    private boolean returnsOthersOutput() {
+      PTransform<?, ?> transform = getTransform();
+      if (output != null) {
+        for (PValue outputValue : output.expand()) {
+          if (!getProducer(outputValue).getTransform().equals(transform)) {
+            return true;
+          }
+        }
+      }
+      return false;
+    }
+
+    public boolean isRootNode() {
+      return transform == null;
+    }
+
+    public String getFullName() {
+      return fullName;
+    }
+
+    /**
+     * Returns the transform input, in unexpanded form.
+     */
+    public PInput getInput() {
+      return input;
+    }
+
+    /**
+     * Adds an output to the transform node.
+     */
+    public void setOutput(POutput output) {
+      checkState(!finishedSpecifying);
+      checkState(
+          this.output == null, "Tried to specify more than one output for %s", getFullName());
+      checkNotNull(output, "Tried to set the output of %s to null", getFullName());
+      this.output = output;
+
+      // Validate that a primitive transform produces only primitive output, and a composite
+      // transform does not produce primitive output.
+      Set<Node> outputProducers = new HashSet<>();
+      for (PValue outputValue : output.expand()) {
+        outputProducers.add(getProducer(outputValue));
+      }
+      if (outputProducers.contains(this) && outputProducers.size() != 1) {
+        Set<String> otherProducerNames = new HashSet<>();
+        for (Node outputProducer : outputProducers) {
+          if (outputProducer != this) {
+            otherProducerNames.add(outputProducer.getFullName());
+          }
+        }
+        throw new IllegalArgumentException(
+            String.format(
+                "Output of transform [%s] contains a %s produced by it as well as other "
+                    + "Transforms. A primitive transform must produce all of its outputs, and "
+                    + "outputs of a composite transform must be produced by a component transform "
+                    + "or be part of the input."
+                    + "%n    Other Outputs: %s"
+                    + "%n    Other Producers: %s",
+                getFullName(), POutput.class.getSimpleName(), output.expand(), otherProducerNames));
+      }
+    }
+
+    /** Returns the transform output, in unexpanded form. */
+    public POutput getOutput() {
+      return output;
+    }
+
+    AppliedPTransform<?, ?, ?> toAppliedPTransform() {
+      return AppliedPTransform.of(
+          getFullName(), getInput(), getOutput(), (PTransform) getTransform());
+    }
+    /**
+     * Visit the transform node.
+     *
+     * <p>Provides an ordered visit of the input values, the primitive transform (or child nodes for
+     * composite transforms), then the output values.
+     */
+    public void visit(PipelineVisitor visitor, Set<PValue> visitedValues) {
+      if (!finishedSpecifying) {
+        finishSpecifying();
+      }
+
+      if (!isRootNode()) {
+        // Visit inputs.
+        for (PValue inputValue : input.expand()) {
+          if (visitedValues.add(inputValue)) {
+            visitor.visitValue(inputValue, getProducer(inputValue));
+          }
+        }
+      }
+
+      if (isCompositeNode()) {
+        PipelineVisitor.CompositeBehavior recurse = visitor.enterCompositeTransform(this);
+
+        if (recurse.equals(CompositeBehavior.ENTER_TRANSFORM)) {
+          for (Node child : parts) {
+            child.visit(visitor, visitedValues);
+          }
+        }
+        visitor.leaveCompositeTransform(this);
+      } else {
+        visitor.visitPrimitiveTransform(this);
+      }
+
+      if (!isRootNode()) {
+        // Visit outputs.
+        for (PValue pValue : output.expand()) {
+          if (visitedValues.add(pValue)) {
+            visitor.visitValue(pValue, this);
+          }
+        }
+      }
+    }
+
+    /**
+     * Finish specifying a transform.
+     *
+     * <p>All inputs are finished first, then the transform, then all outputs.
+     */
+    public void finishSpecifying() {
+      if (finishedSpecifying) {
+        return;
+      }
+      finishedSpecifying = true;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java
deleted file mode 100644
index ea94bd9..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java
+++ /dev/null
@@ -1,282 +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.runners;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.common.annotations.VisibleForTesting;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.Pipeline.PipelineVisitor;
-import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-import org.apache.beam.sdk.values.PValue;
-
-/**
- * Provides internal tracking of transform relationships with helper methods
- * for initialization and ordered visitation.
- */
-public class TransformTreeNode {
-  private final TransformHierarchy hierarchy;
-  private final TransformTreeNode enclosingNode;
-
-  // The PTransform for this node, which may be a composite PTransform.
-  // The root of a TransformHierarchy is represented as a TransformTreeNode
-  // with a null transform field.
-  private final PTransform<?, ?> transform;
-
-  private final String fullName;
-
-  // Nodes for sub-transforms of a composite transform.
-  private final Collection<TransformTreeNode> parts = new ArrayList<>();
-
-  // Input to the transform, in unexpanded form.
-  private final PInput input;
-
-  // TODO: track which outputs need to be exported to parent.
-  // Output of the transform, in unexpanded form.
-  private POutput output;
-
-  @VisibleForTesting
-  boolean finishedSpecifying = false;
-
-  /**
-   * Create a root {@link TransformTreeNode}. This transform is the root of the provided {@link
-   * TransformHierarchy} - it has no enclosing node, no {@link PTransform}, no {@link PInput input},
-   * no {@link POutput output}, and an empty name. It contains all {@link PTransform transforms}
-   * within a {@link Pipeline} as component transforms.
-   */
-  public static TransformTreeNode root(TransformHierarchy hierarchy) {
-    return new TransformTreeNode(hierarchy, null, null, "", null);
-  }
-
-  /**
-   * Create a subtransform of the provided {@link TransformTreeNode node}. The enclosing node is a
-   * composite that contains this transform.
-   *
-   * <p>The returned node is a component node of the enclosing node.
-   */
-  public static TransformTreeNode subtransform(
-      TransformTreeNode enclosing, PTransform<?, ?> transform, String fullName, PInput input) {
-    checkNotNull(enclosing);
-    checkNotNull(transform);
-    checkNotNull(fullName);
-    checkNotNull(input);
-    TransformTreeNode node =
-        new TransformTreeNode(enclosing.hierarchy, enclosing, transform, fullName, input);
-    enclosing.addComposite(node);
-    return node;
-  }
-
-  /**
-   * Creates a new TransformTreeNode with the given parent and transform.
-   *
-   * <p>EnclosingNode and transform may both be null for a root-level node, which holds all other
-   * nodes.
-   *
-   * @param enclosingNode the composite node containing this node
-   * @param transform the PTransform tracked by this node
-   * @param fullName the fully qualified name of the transform
-   * @param input the unexpanded input to the transform
-   */
-  private TransformTreeNode(
-      TransformHierarchy hierarchy,
-      @Nullable TransformTreeNode enclosingNode,
-      @Nullable PTransform<?, ?> transform,
-      String fullName,
-      @Nullable PInput input) {
-    this.hierarchy = hierarchy;
-    this.enclosingNode = enclosingNode;
-    this.transform = transform;
-    this.fullName = fullName;
-    this.input = input;
-  }
-
-  /**
-   * Returns the transform associated with this transform node.
-   */
-  public PTransform<?, ?> getTransform() {
-    return transform;
-  }
-
-  /**
-   * Returns the enclosing composite transform node, or null if there is none.
-   */
-  public TransformTreeNode getEnclosingNode() {
-    return enclosingNode;
-  }
-
-  /**
-   * Adds a composite operation to the transform node.
-   *
-   * <p>As soon as a node is added, the transform node is considered a
-   * composite operation instead of a primitive transform.
-   */
-  public void addComposite(TransformTreeNode node) {
-    parts.add(node);
-  }
-
-  /**
-   * Returns true if this node represents a composite transform that does not perform processing of
-   * its own, but merely encapsulates a sub-pipeline (which may be empty).
-   *
-   * <p>Note that a node may be composite with no sub-transforms if it returns its input directly
-   * extracts a component of a tuple, or other operations that occur at pipeline assembly time.
-   */
-  public boolean isCompositeNode() {
-    return !parts.isEmpty() || isRootNode() || returnsOthersOutput();
-  }
-
-  private boolean returnsOthersOutput() {
-    PTransform<?, ?> transform = getTransform();
-    if (output != null) {
-      for (PValue outputValue : output.expand()) {
-        if (!hierarchy.getProducer(outputValue).getTransform().equals(transform)) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  public boolean isRootNode() {
-    return transform == null;
-  }
-
-  public String getFullName() {
-    return fullName;
-  }
-
-  /**
-   * Returns the transform input, in unexpanded form.
-   */
-  public PInput getInput() {
-    return input;
-  }
-
-  /**
-   * Adds an output to the transform node.
-   */
-  public void setOutput(POutput output) {
-    checkState(!finishedSpecifying);
-    checkState(this.output == null, "Tried to specify more than one output for %s", getFullName());
-    checkNotNull(output, "Tried to set the output of %s to null", getFullName());
-    this.output = output;
-
-    // Validate that a primitive transform produces only primitive output, and a composite transform
-    // does not produce primitive output.
-    Set<TransformTreeNode> outputProducers = new HashSet<>();
-    for (PValue outputValue : output.expand()) {
-      outputProducers.add(hierarchy.getProducer(outputValue));
-    }
-    if (outputProducers.contains(this) && outputProducers.size() != 1) {
-      Set<String> otherProducerNames = new HashSet<>();
-      for (TransformTreeNode outputProducer : outputProducers) {
-        if (outputProducer != this) {
-          otherProducerNames.add(outputProducer.getFullName());
-        }
-      }
-      throw new IllegalArgumentException(
-          String.format(
-              "Output of transform [%s] contains a %s produced by it as well as other Transforms. "
-                  + "A primitive transform must produce all of its outputs, and outputs of a "
-                  + "composite transform must be produced by a component transform or be part of"
-                  + "the input."
-                  + "%n    Other Outputs: %s"
-                  + "%n    Other Producers: %s",
-              getFullName(), POutput.class.getSimpleName(), output.expand(), otherProducerNames));
-    }
-  }
-
-  /**
-   * Returns the transform output, in unexpanded form.
-   */
-  public POutput getOutput() {
-    return output;
-  }
-
-  AppliedPTransform<?, ?, ?> toAppliedPTransform() {
-    return AppliedPTransform.of(
-        getFullName(), getInput(), getOutput(), (PTransform) getTransform());
-  }
-  /**
-   * Visit the transform node.
-   *
-   * <p>Provides an ordered visit of the input values, the primitive
-   * transform (or child nodes for composite transforms), then the
-   * output values.
-   */
-  public void visit(PipelineVisitor visitor,
-                    Set<PValue> visitedValues) {
-    if (!finishedSpecifying) {
-      finishSpecifying();
-    }
-
-    if (!isRootNode()) {
-      // Visit inputs.
-      for (PValue inputValue : input.expand()) {
-        if (visitedValues.add(inputValue)) {
-          visitor.visitValue(inputValue, hierarchy.getProducer(inputValue));
-        }
-      }
-    }
-
-    if (isCompositeNode()) {
-      PipelineVisitor.CompositeBehavior recurse = visitor.enterCompositeTransform(this);
-
-      if (recurse.equals(CompositeBehavior.ENTER_TRANSFORM)) {
-        for (TransformTreeNode child : parts) {
-          child.visit(visitor, visitedValues);
-        }
-      }
-      visitor.leaveCompositeTransform(this);
-    } else {
-      visitor.visitPrimitiveTransform(this);
-    }
-
-    if (!isRootNode()) {
-      // Visit outputs.
-      for (PValue pValue : output.expand()) {
-        if (visitedValues.add(pValue)) {
-          visitor.visitValue(pValue, this);
-        }
-      }
-    }
-  }
-
-  /**
-   * Finish specifying a transform.
-   *
-   * <p>All inputs are finished first, then the transform, then
-   * all outputs.
-   */
-  public void finishSpecifying() {
-    if (finishedSpecifying) {
-      return;
-    }
-    finishedSpecifying = true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java
index 335d81f..b4de768 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/AggregatorPipelineExtractorTest.java
@@ -30,7 +30,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.Max;
@@ -73,7 +73,7 @@ public class AggregatorPipelineExtractorTest {
     Aggregator<Long, Long> aggregatorOne = fn.addAggregator(new Sum.SumLongFn());
     Aggregator<Integer, Integer> aggregatorTwo = fn.addAggregator(new Min.MinIntegerFn());
 
-    TransformTreeNode transformNode = mock(TransformTreeNode.class);
+    TransformHierarchy.Node transformNode = mock(TransformHierarchy.Node.class);
     when(transformNode.getTransform()).thenReturn(bound);
 
     doAnswer(new VisitNodesAnswer(ImmutableList.of(transformNode)))
@@ -101,7 +101,7 @@ public class AggregatorPipelineExtractorTest {
     Aggregator<Long, Long> aggregatorOne = fn.addAggregator(new Max.MaxLongFn());
     Aggregator<Double, Double> aggregatorTwo = fn.addAggregator(new Min.MinDoubleFn());
 
-    TransformTreeNode transformNode = mock(TransformTreeNode.class);
+    TransformHierarchy.Node transformNode = mock(TransformHierarchy.Node.class);
     when(transformNode.getTransform()).thenReturn(bound);
 
     doAnswer(new VisitNodesAnswer(ImmutableList.of(transformNode)))
@@ -132,9 +132,9 @@ public class AggregatorPipelineExtractorTest {
     Aggregator<Long, Long> aggregatorOne = fn.addAggregator(new Sum.SumLongFn());
     Aggregator<Double, Double> aggregatorTwo = fn.addAggregator(new Min.MinDoubleFn());
 
-    TransformTreeNode transformNode = mock(TransformTreeNode.class);
+    TransformHierarchy.Node transformNode = mock(TransformHierarchy.Node.class);
     when(transformNode.getTransform()).thenReturn(bound);
-    TransformTreeNode otherTransformNode = mock(TransformTreeNode.class);
+    TransformHierarchy.Node otherTransformNode = mock(TransformHierarchy.Node.class);
     when(otherTransformNode.getTransform()).thenReturn(otherBound);
 
     doAnswer(new VisitNodesAnswer(ImmutableList.of(transformNode, otherTransformNode)))
@@ -172,9 +172,9 @@ public class AggregatorPipelineExtractorTest {
 
     when(otherBound.getFn()).thenReturn(otherFn);
 
-    TransformTreeNode transformNode = mock(TransformTreeNode.class);
+    TransformHierarchy.Node transformNode = mock(TransformHierarchy.Node.class);
     when(transformNode.getTransform()).thenReturn(bound);
-    TransformTreeNode otherTransformNode = mock(TransformTreeNode.class);
+    TransformHierarchy.Node otherTransformNode = mock(TransformHierarchy.Node.class);
     when(otherTransformNode.getTransform()).thenReturn(otherBound);
 
     doAnswer(new VisitNodesAnswer(ImmutableList.of(transformNode, otherTransformNode)))
@@ -192,16 +192,16 @@ public class AggregatorPipelineExtractorTest {
   }
 
   private static class VisitNodesAnswer implements Answer<Object> {
-    private final List<TransformTreeNode> nodes;
+    private final List<TransformHierarchy.Node> nodes;
 
-    public VisitNodesAnswer(List<TransformTreeNode> nodes) {
+    public VisitNodesAnswer(List<TransformHierarchy.Node> nodes) {
       this.nodes = nodes;
     }
 
     @Override
     public Object answer(InvocationOnMock invocation) throws Throwable {
       PipelineVisitor visitor = (PipelineVisitor) invocation.getArguments()[0];
-      for (TransformTreeNode node : nodes) {
+      for (TransformHierarchy.Node node : nodes) {
         visitor.visitPrimitiveTransform(node);
       }
       return null;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
index 3bf6d64..f4488f4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
@@ -77,8 +77,8 @@ public class TransformHierarchyTest {
 
   @Test
   public void pushThenPopSucceeds() {
-    TransformTreeNode root = hierarchy.getCurrent();
-    TransformTreeNode node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1));
+    TransformHierarchy.Node root = hierarchy.getCurrent();
+    TransformHierarchy.Node node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1));
     assertThat(hierarchy.getCurrent(), equalTo(node));
     hierarchy.popNode();
     assertThat(node.finishedSpecifying, is(true));
@@ -90,12 +90,12 @@ public class TransformHierarchyTest {
     PCollection<Long> created =
         PCollection.createPrimitiveOutputInternal(
             pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED);
-    TransformTreeNode node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1));
+    TransformHierarchy.Node node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1));
     hierarchy.setOutput(created);
     hierarchy.popNode();
     PCollectionList<Long> pcList = PCollectionList.of(created);
 
-    TransformTreeNode emptyTransform =
+    TransformHierarchy.Node emptyTransform =
         hierarchy.pushNode(
             "Extract",
             pcList,
@@ -149,7 +149,7 @@ public class TransformHierarchyTest {
 
   @Test
   public void visitVisitsAllPushed() {
-    TransformTreeNode root = hierarchy.getCurrent();
+    TransformHierarchy.Node root = hierarchy.getCurrent();
     PBegin begin = PBegin.in(pipeline);
 
     Create.Values<Long> create = Create.of(1L);
@@ -170,7 +170,7 @@ public class TransformHierarchyTest {
         PCollection.createPrimitiveOutputInternal(
             pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED);
 
-    TransformTreeNode compositeNode = hierarchy.pushNode("Create", begin, create);
+    TransformHierarchy.Node compositeNode = hierarchy.pushNode("Create", begin, create);
     assertThat(hierarchy.getCurrent(), equalTo(compositeNode));
     assertThat(compositeNode.getInput(), Matchers.<PInput>equalTo(begin));
     assertThat(compositeNode.getTransform(), Matchers.<PTransform<?, ?>>equalTo(create));
@@ -178,7 +178,7 @@ public class TransformHierarchyTest {
     assertThat(compositeNode.getOutput(), nullValue());
     assertThat(compositeNode.getEnclosingNode().isRootNode(), is(true));
 
-    TransformTreeNode primitiveNode = hierarchy.pushNode("Create/Read", begin, read);
+    TransformHierarchy.Node primitiveNode = hierarchy.pushNode("Create/Read", begin, read);
     assertThat(hierarchy.getCurrent(), equalTo(primitiveNode));
     hierarchy.setOutput(created);
     hierarchy.popNode();
@@ -194,30 +194,30 @@ public class TransformHierarchyTest {
     assertThat(hierarchy.getProducer(created), equalTo(primitiveNode));
     hierarchy.popNode();
 
-    TransformTreeNode otherPrimitive = hierarchy.pushNode("ParDo", created, map);
+    TransformHierarchy.Node otherPrimitive = hierarchy.pushNode("ParDo", created, map);
     hierarchy.setOutput(mapped);
     hierarchy.popNode();
 
-    final Set<TransformTreeNode> visitedCompositeNodes = new HashSet<>();
-    final Set<TransformTreeNode> visitedPrimitiveNodes = new HashSet<>();
+    final Set<TransformHierarchy.Node> visitedCompositeNodes = new HashSet<>();
+    final Set<TransformHierarchy.Node> visitedPrimitiveNodes = new HashSet<>();
     final Set<PValue> visitedValuesInVisitor = new HashSet<>();
 
     Set<PValue> visitedValues =
         hierarchy.visit(
             new PipelineVisitor.Defaults() {
               @Override
-              public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+              public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
                 visitedCompositeNodes.add(node);
                 return CompositeBehavior.ENTER_TRANSFORM;
               }
 
               @Override
-              public void visitPrimitiveTransform(TransformTreeNode node) {
+              public void visitPrimitiveTransform(TransformHierarchy.Node node) {
                 visitedPrimitiveNodes.add(node);
               }
 
               @Override
-              public void visitValue(PValue value, TransformTreeNode producer) {
+              public void visitValue(PValue value, TransformHierarchy.Node producer) {
                 visitedValuesInVisitor.add(value);
               }
             });

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
index b95fa70..a81fb1a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
@@ -51,7 +51,7 @@ import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
 /**
- * Tests for {@link TransformTreeNode} and {@link TransformHierarchy}.
+ * Tests for {@link TransformHierarchy.Node} and {@link TransformHierarchy}.
  */
 @RunWith(JUnit4.class)
 public class TransformTreeTest {
@@ -128,7 +128,7 @@ public class TransformTreeTest {
 
     p.traverseTopologically(new Pipeline.PipelineVisitor.Defaults() {
       @Override
-      public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+      public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
         PTransform<?, ?> transform = node.getTransform();
         if (transform instanceof Sample.SampleAny) {
           assertTrue(visited.add(TransformsSeen.SAMPLE_ANY));
@@ -144,7 +144,7 @@ public class TransformTreeTest {
       }
 
       @Override
-      public void leaveCompositeTransform(TransformTreeNode node) {
+      public void leaveCompositeTransform(TransformHierarchy.Node node) {
         PTransform<?, ?> transform = node.getTransform();
         if (transform instanceof Sample.SampleAny) {
           assertTrue(left.add(TransformsSeen.SAMPLE_ANY));
@@ -152,7 +152,7 @@ public class TransformTreeTest {
       }
 
       @Override
-      public void visitPrimitiveTransform(TransformTreeNode node) {
+      public void visitPrimitiveTransform(TransformHierarchy.Node node) {
         PTransform<?, ?> transform = node.getTransform();
         // Pick is a composite, should not be visited here.
         assertThat(transform, not(instanceOf(Sample.SampleAny.class)));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/569e8d70/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
index b758ed6..31ac913 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
@@ -24,7 +24,7 @@ import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -143,7 +143,7 @@ public class DisplayDataEvaluator {
     }
 
     @Override
-    public CompositeBehavior enterCompositeTransform(TransformTreeNode node) {
+    public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
       if (Objects.equals(root, node.getTransform())) {
         inCompositeRoot = true;
       }
@@ -152,14 +152,14 @@ public class DisplayDataEvaluator {
     }
 
     @Override
-    public void leaveCompositeTransform(TransformTreeNode node) {
+    public void leaveCompositeTransform(TransformHierarchy.Node node) {
       if (Objects.equals(root, node.getTransform())) {
         inCompositeRoot = false;
       }
     }
 
     @Override
-    public void visitPrimitiveTransform(TransformTreeNode node) {
+    public void visitPrimitiveTransform(TransformHierarchy.Node node) {
       if (inCompositeRoot || Objects.equals(root, node.getTransform())) {
         displayData.add(DisplayData.from(node.getTransform()));
       }


[45/50] [abbrv] incubator-beam git commit: This closes #1344

Posted by ke...@apache.org.
This closes #1344


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/c22b97dd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/c22b97dd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/c22b97dd

Branch: refs/heads/gearpump-runner
Commit: c22b97dd103886b5268645985d1c2298951b3a07
Parents: 8f712fd abd96b8
Author: Davor Bonaci <da...@google.com>
Authored: Sat Dec 3 16:15:45 2016 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Sat Dec 3 16:15:45 2016 -0800

----------------------------------------------------------------------
 .gitignore                                      |   4 +
 pom.xml                                         |   8 -
 .../examples-java8/generate-sources.sh          |  82 +++++
 .../maven-archetypes/examples-java8/pom.xml     | 177 ++++++++++
 .../META-INF/maven/archetype-metadata.xml       |  39 ++
 .../main/resources/archetype-resources/pom.xml  | 209 +++++++++++
 .../projects/basic/archetype.properties         |  21 ++
 .../src/test/resources/projects/basic/goal.txt  |   1 +
 .../examples/generate-sources.sh                |  68 ++++
 sdks/java/maven-archetypes/examples/pom.xml     |  27 +-
 .../src/main/java/DebuggingWordCount.java       | 164 ---------
 .../src/main/java/MinimalWordCount.java         | 118 -------
 .../src/main/java/WindowedWordCount.java        | 229 ------------
 .../src/main/java/WordCount.java                | 186 ----------
 .../common/ExampleBigQueryTableOptions.java     |  55 ---
 .../src/main/java/common/ExampleOptions.java    |  37 --
 ...xamplePubsubTopicAndSubscriptionOptions.java |  45 ---
 .../java/common/ExamplePubsubTopicOptions.java  |  45 ---
 .../src/main/java/common/ExampleUtils.java      | 352 -------------------
 .../src/test/java/DebuggingWordCountTest.java   |  52 ---
 .../src/test/java/WordCountTest.java            |  85 -----
 sdks/java/maven-archetypes/pom.xml              |  38 +-
 .../update-examples-archetype.sh                |  59 ----
 23 files changed, 663 insertions(+), 1438 deletions(-)
----------------------------------------------------------------------



[07/50] [abbrv] incubator-beam git commit: This closes #1261

Posted by ke...@apache.org.
This closes #1261


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/24fab9f5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/24fab9f5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/24fab9f5

Branch: refs/heads/gearpump-runner
Commit: 24fab9f53a8b3a7ef5fb35195dbe9417bbcc4101
Parents: fd4b631 87ff5ac
Author: Thomas Groh <tg...@google.com>
Authored: Thu Dec 1 14:16:58 2016 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Thu Dec 1 14:16:58 2016 -0800

----------------------------------------------------------------------
 .../core/ElementAndRestrictionCoder.java        |   8 +
 .../runners/core/GBKIntoKeyedWorkItems.java     |  55 ---
 .../beam/runners/core/SplittableParDo.java      | 378 +++++++++++++++----
 .../beam/runners/core/SplittableParDoTest.java  | 134 +++++--
 ...ectGBKIntoKeyedWorkItemsOverrideFactory.java |  41 +-
 .../beam/runners/direct/DirectGroupByKey.java   |   2 +-
 .../beam/runners/direct/DirectRunner.java       |   8 +-
 .../runners/direct/DoFnLifecycleManager.java    |   4 +-
 .../beam/runners/direct/ParDoEvaluator.java     |  26 +-
 .../runners/direct/ParDoEvaluatorFactory.java   |  63 +++-
 .../direct/ParDoMultiOverrideFactory.java       |   2 +-
 ...littableProcessElementsEvaluatorFactory.java | 144 +++++++
 .../direct/TransformEvaluatorRegistry.java      |   5 +
 .../beam/runners/direct/SplittableDoFnTest.java | 194 +++++++++-
 .../org/apache/beam/sdk/transforms/DoFn.java    |  12 +
 .../apache/beam/sdk/transforms/DoFnTester.java  |  51 ++-
 .../sdk/util/state/TimerInternalsFactory.java   |  36 ++
 17 files changed, 905 insertions(+), 258 deletions(-)
----------------------------------------------------------------------



[31/50] [abbrv] incubator-beam git commit: Removes DoFnTester.of(OldDoFn)

Posted by ke...@apache.org.
Removes DoFnTester.of(OldDoFn)


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/86173a83
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/86173a83
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/86173a83

Branch: refs/heads/gearpump-runner
Commit: 86173a839f57cf7ed45566b380e557cf1defcba9
Parents: 78ac009
Author: Eugene Kirpichov <ki...@google.com>
Authored: Fri Dec 2 11:44:02 2016 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Dec 2 15:42:33 2016 -0800

----------------------------------------------------------------------
 .../org/apache/beam/sdk/transforms/DoFnTester.java   | 15 ---------------
 1 file changed, 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/86173a83/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
index 7c1abef..9f32aec 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
@@ -23,7 +23,6 @@ import static com.google.common.base.Preconditions.checkState;
 import com.google.common.base.Function;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -100,20 +99,6 @@ public class DoFnTester<InputT, OutputT> implements AutoCloseable {
   }
 
   /**
-   * Returns a {@code DoFnTester} supporting unit-testing of the given
-   * {@link OldDoFn}.
-   *
-   * @see #of(DoFn)
-   */
-  @SuppressWarnings("unchecked")
-  @Deprecated
-  public static <InputT, OutputT> DoFnTester<InputT, OutputT>
-      of(OldDoFn<InputT, OutputT> fn) {
-    checkNotNull(fn, "fn can't be null");
-    return new DoFnTester<>(fn.toDoFn());
-  }
-
-  /**
    * Registers the tuple of values of the side input {@link PCollectionView}s to
    * pass to the {@link DoFn} under test.
    *