Repository: incubator-beam
Updated Branches:
  refs/heads/master 077d9118d -> ca6ab6c68


Remove getProducingTransformInternal from DirectGraphVisitorTest


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

Branch: refs/heads/master
Commit: ec1eff387a711039801289c8f59c4240b1f1d007
Parents: d6c6ad3
Author: Thomas Groh <tg...@google.com>
Authored: Fri Dec 2 14:26:04 2016 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Tue Dec 6 10:46:39 2016 -0800

----------------------------------------------------------------------
 .../runners/direct/DirectGraphVisitorTest.java  | 70 +++++++++++++-------
 1 file changed, 47 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ec1eff38/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
index fb84de8..5ad278b 100644
--- 
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
@@ -19,25 +19,34 @@ package org.apache.beam.runners.direct;
 
 import static org.hamcrest.Matchers.emptyIterable;
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
 import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 
+import com.google.common.collect.Iterables;
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.List;
 import org.apache.beam.sdk.io.CountingInput;
+import org.apache.beam.sdk.io.CountingSource;
+import org.apache.beam.sdk.io.Read;
 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.Flatten.FlattenPCollectionList;
 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.PBegin;
 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.apache.beam.sdk.values.POutput;
 import org.hamcrest.Matchers;
 import org.junit.Rule;
 import org.junit.Test;
@@ -48,7 +57,6 @@ import org.junit.runners.JUnit4;
 /**
  * Tests for {@link DirectGraphVisitor}.
  */
-// TODO: Replace uses of getProducing
 @RunWith(JUnit4.class)
 public class DirectGraphVisitorTest implements Serializable {
   @Rule public transient ExpectedException thrown = ExpectedException.none();
@@ -81,26 +89,36 @@ public class DirectGraphVisitorTest implements Serializable 
{
   @Test
   public void getRootTransformsContainsPBegins() {
     PCollection<String> created = p.apply(Create.of("foo", "bar"));
-    PCollection<Long> counted = p.apply(CountingInput.upTo(1234L));
+    PCollection<Long> counted = p.apply(Read.from(CountingSource.upTo(1234L)));
     PCollection<Long> unCounted = p.apply(CountingInput.unbounded());
     p.traverseTopologically(visitor);
+    DirectGraph graph = visitor.getGraph();
+    assertThat(graph.getRootTransforms(), hasSize(3));
+    List<PTransform<?, ?>> unapplied = new ArrayList<>();
     assertThat(
-        visitor.getGraph().getRootTransforms(),
+        graph.getRootTransforms(),
         Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
-            created.getProducingTransformInternal(),
-            counted.getProducingTransformInternal(),
-            unCounted.getProducingTransformInternal()));
+            graph.getProducer(created), graph.getProducer(counted), 
graph.getProducer(unCounted)));
+    for (AppliedPTransform<?, ?, ?> root : graph.getRootTransforms())  {
+      assertTrue(root.getInput() instanceof PBegin);
+      assertThat(root.getOutput(), Matchers.<POutput>isOneOf(created, counted, 
unCounted));
+    }
   }
 
   @Test
   public void getRootTransformsContainsEmptyFlatten() {
-    PCollection<String> empty =
-        PCollectionList.<String>empty(p).apply(Flatten.<String>pCollections());
+    FlattenPCollectionList<String> flatten = Flatten.pCollections();
+    PCollectionList<String> emptyList = PCollectionList.empty(p);
+    PCollection<String> empty = emptyList.apply(flatten);
     p.traverseTopologically(visitor);
+    DirectGraph graph = visitor.getGraph();
     assertThat(
-        visitor.getGraph().getRootTransforms(),
-        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
-            empty.getProducingTransformInternal()));
+        graph.getRootTransforms(),
+        Matchers.<AppliedPTransform<?, ?, 
?>>containsInAnyOrder(graph.getProducer(empty)));
+    AppliedPTransform<?, ?, ?> onlyRoot = 
Iterables.getOnlyElement(graph.getRootTransforms());
+    assertThat(onlyRoot.getTransform(), Matchers.<PTransform<?, 
?>>equalTo(flatten));
+    assertThat(onlyRoot.getInput(), Matchers.<PInput>equalTo(emptyList));
+    assertThat(onlyRoot.getOutput(), Matchers.<POutput>equalTo(empty));
   }
 
   @Test
@@ -122,16 +140,20 @@ public class DirectGraphVisitorTest implements 
Serializable {
 
     p.traverseTopologically(visitor);
 
+    DirectGraph graph = visitor.getGraph();
+    AppliedPTransform<?, ?, ?> transformedProducer =
+        graph.getProducer(transformed);
+    AppliedPTransform<?, ?, ?> flattenedProducer =
+        graph.getProducer(flattened);
+
     assertThat(
-        visitor.getGraph().getPrimitiveConsumers(created),
+        graph.getPrimitiveConsumers(created),
         Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
-            transformed.getProducingTransformInternal(),
-            flattened.getProducingTransformInternal()));
+            transformedProducer, flattenedProducer));
     assertThat(
-        visitor.getGraph().getPrimitiveConsumers(transformed),
-        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
-            flattened.getProducingTransformInternal()));
-    assertThat(visitor.getGraph().getPrimitiveConsumers(flattened), 
emptyIterable());
+        graph.getPrimitiveConsumers(transformed),
+        Matchers.<AppliedPTransform<?, ?, 
?>>containsInAnyOrder(flattenedProducer));
+    assertThat(graph.getPrimitiveConsumers(flattened), emptyIterable());
   }
 
   @Test
@@ -143,12 +165,14 @@ public class DirectGraphVisitorTest implements 
Serializable {
 
     p.traverseTopologically(visitor);
 
+    DirectGraph graph = visitor.getGraph();
+    AppliedPTransform<?, ?, ?> flattenedProducer = 
graph.getProducer(flattened);
+
     assertThat(
-        visitor.getGraph().getPrimitiveConsumers(created),
-        Matchers.<AppliedPTransform<?, ?, ?>>containsInAnyOrder(
-            flattened.getProducingTransformInternal(),
-            flattened.getProducingTransformInternal()));
-    assertThat(visitor.getGraph().getPrimitiveConsumers(flattened), 
emptyIterable());
+        graph.getPrimitiveConsumers(created),
+        Matchers.<AppliedPTransform<?, ?, 
?>>containsInAnyOrder(flattenedProducer,
+            flattenedProducer));
+    assertThat(graph.getPrimitiveConsumers(flattened), emptyIterable());
   }
 
   @Test

Reply via email to