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:41:18 UTC

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

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