[ 
https://issues.apache.org/jira/browse/BEAM-4073?focusedWorklogId=98060&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-98060
 ]

ASF GitHub Bot logged work on BEAM-4073:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 03/May/18 22:20
            Start Date: 03/May/18 22:20
    Worklog Time Spent: 10m 
      Work Description: tgroh closed pull request #5228: [BEAM-4073] Expand 
ExecutableGraph, and add a Portable Representation
URL: https://github.com/apache/beam/pull/5228
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
index 11006014018..1e24a8ec540 100644
--- 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
+++ 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
@@ -165,6 +165,15 @@ private static boolean isPrimitiveTransform(PTransform 
transform) {
     return network;
   }
 
+  public Collection<PTransformNode> getTransforms() {
+    return pipelineNetwork
+        .nodes()
+        .stream()
+        .filter(PTransformNode.class::isInstance)
+        .map(PTransformNode.class::cast)
+        .collect(Collectors.toList());
+  }
+
   public Iterable<PTransformNode> getTopologicallyOrderedTransforms() {
     return StreamSupport.stream(
             Networks.topologicalOrder(pipelineNetwork, 
Comparator.comparing(PipelineNode::getId))
@@ -217,6 +226,19 @@ public PTransformNode getProducer(PCollectionNode 
pcollection) {
         .collect(Collectors.toSet());
   }
 
+  /**
+   * Gets each {@link PCollectionNode} that the provided {@link 
PTransformNode} consumes on a
+   * per-element basis.
+   */
+  public Set<PCollectionNode> getPerElementInputPCollections(PTransformNode 
ptransform) {
+    return pipelineNetwork
+        .inEdges(ptransform)
+        .stream()
+        .filter(PipelineEdge::isPerElement)
+        .map(edge -> (PCollectionNode) 
pipelineNetwork.incidentNodes(edge).source())
+        .collect(Collectors.toSet());
+  }
+
   public Set<PCollectionNode> getOutputPCollections(PTransformNode ptransform) 
{
     return pipelineNetwork
         .successors(ptransform)
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
index f52c667abe0..e5a2fa8e641 100644
--- 
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
@@ -81,10 +81,10 @@ private DirectGraph(
   }
 
   @Override
-  public Collection<PValue> getProduced(AppliedPTransform<?, ?, ?> toRefresh) {
+  public Collection<PValue> getProduced(AppliedPTransform<?, ?, ?> producer) {
     // TODO: This must only be called on primitive transforms; composites 
should return empty
     // values.
-    return toRefresh.getOutputs().values();
+    return producer.getOutputs().values();
   }
 
   @Override
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutableGraph.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutableGraph.java
index bfdf692c85a..48f119b7463 100644
--- 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutableGraph.java
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutableGraph.java
@@ -35,7 +35,7 @@
 
   ExecutableT getProducer(CollectionT collection);
 
-  Collection<CollectionT> getProduced(ExecutableT toRefresh);
+  Collection<CollectionT> getProduced(ExecutableT producer);
 
   Collection<CollectionT> getPerElementInputs(ExecutableT transform);
 
diff --git 
a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PortableGraph.java
 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PortableGraph.java
new file mode 100644
index 00000000000..0e349a2203b
--- /dev/null
+++ 
b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PortableGraph.java
@@ -0,0 +1,69 @@
+/*
+ * 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.model.pipeline.v1.RunnerApi;
+import 
org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
+import 
org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
+import org.apache.beam.runners.core.construction.graph.QueryablePipeline;
+
+/** A {@link ExecutableGraph} for a Portable {@link RunnerApi.Pipeline}. */
+class PortableGraph implements ExecutableGraph<PTransformNode, 
PCollectionNode> {
+  private final QueryablePipeline queryablePipeline;
+
+  public static PortableGraph forPipeline(RunnerApi.Pipeline p) {
+    return new PortableGraph(p);
+  }
+
+  private PortableGraph(RunnerApi.Pipeline p) {
+    this.queryablePipeline =
+        QueryablePipeline.forTransforms(p.getRootTransformIdsList(), 
p.getComponents());
+  }
+
+  @Override
+  public Collection<PTransformNode> getRootTransforms() {
+    return queryablePipeline.getRootTransforms();
+  }
+
+  @Override
+  public Collection<PTransformNode> getExecutables() {
+    return queryablePipeline.getTransforms();
+  }
+
+  @Override
+  public PTransformNode getProducer(PCollectionNode collection) {
+    return queryablePipeline.getProducer(collection);
+  }
+
+  @Override
+  public Collection<PCollectionNode> getProduced(PTransformNode producer) {
+    return queryablePipeline.getOutputPCollections(producer);
+  }
+
+  @Override
+  public Collection<PCollectionNode> getPerElementInputs(PTransformNode 
transform) {
+    return queryablePipeline.getPerElementInputPCollections(transform);
+  }
+
+  @Override
+  public Collection<PTransformNode> getPerElementConsumers(PCollectionNode 
pCollection) {
+    return queryablePipeline.getPerElementConsumers(pCollection);
+  }
+}
diff --git 
a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/PortableGraphTest.java
 
b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/PortableGraphTest.java
new file mode 100644
index 00000000000..010aaac3b7c
--- /dev/null
+++ 
b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/PortableGraphTest.java
@@ -0,0 +1,178 @@
+/*
+ * 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.collect.Iterables.getOnlyElement;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.hasValue;
+import static org.junit.Assert.assertThat;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
+import org.apache.beam.runners.core.construction.PTransformTranslation;
+import org.apache.beam.runners.core.construction.PipelineTranslation;
+import org.apache.beam.runners.core.construction.graph.ExecutableStage;
+import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser;
+import org.apache.beam.runners.core.construction.graph.PipelineNode;
+import 
org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
+import 
org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link PortableGraph}. */
+@RunWith(JUnit4.class)
+public class PortableGraphTest implements Serializable {
+  @Test
+  public void getRootTransformsSucceeds() {
+    Pipeline pipeline = Pipeline.create();
+    pipeline.apply("impulse", Impulse.create());
+    pipeline.apply("otherImpulse", Impulse.create());
+
+    PortableGraph graph = 
PortableGraph.forPipeline(PipelineTranslation.toProto(pipeline));
+    assertThat(graph.getRootTransforms(), hasSize(2));
+
+    assertThat(
+        
graph.getRootTransforms().stream().map(PTransformNode::getId).collect(Collectors.toSet()),
+        containsInAnyOrder("impulse", "otherImpulse"));
+  }
+
+  @Test
+  public void getExecutablesReturnsTransforms() {
+    Pipeline pipeline = Pipeline.create();
+    pipeline
+        .apply("Impulse", Impulse.create())
+        .apply(
+            "ParDo",
+            ParDo.of(
+                new DoFn<byte[], KV<String, String>>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext ctxt) {
+                    ctxt.output(KV.of("foo", "bar"));
+                  }
+                }))
+        .apply(GroupByKey.create())
+        .apply(Values.create());
+
+    PortableGraph graph = 
PortableGraph.forPipeline(PipelineTranslation.toProto(pipeline));
+    assertThat(graph.getExecutables(), hasSize(4));
+  }
+
+  @Test
+  public void getExecutablesWithStages() {
+    Pipeline pipeline = Pipeline.create();
+    pipeline
+        .apply("Impulse", Impulse.create())
+        .apply(
+            "ParDo",
+            ParDo.of(
+                new DoFn<byte[], KV<String, String>>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext ctxt) {
+                    ctxt.output(KV.of("foo", "bar"));
+                  }
+                }))
+        .apply(
+            MapElements.into(new TypeDescriptor<KV<String, Integer>>() {})
+                .via(input -> KV.of(input.getKey(), 
input.getValue().hashCode())))
+        .apply(GroupByKey.create())
+        .apply(Values.create());
+
+    RunnerApi.Pipeline proto = PipelineTranslation.toProto(pipeline);
+    RunnerApi.Pipeline fused = GreedyPipelineFuser.fuse(proto).toPipeline();
+    PortableGraph graph = PortableGraph.forPipeline(fused);
+    assertThat(graph.getExecutables(), hasSize(4));
+
+    Stream<FunctionSpec> specStream =
+        
graph.getExecutables().stream().map(PTransformNode::getTransform).map(PTransform::getSpec);
+
+    List<String> urns = 
specStream.map(FunctionSpec::getUrn).collect(Collectors.toList());
+    assertThat(
+        urns,
+        containsInAnyOrder(
+            PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN,
+            PTransformTranslation.IMPULSE_TRANSFORM_URN,
+            ExecutableStage.URN,
+            ExecutableStage.URN));
+  }
+
+  @Test
+  public void getProducedAndGetProducerSucceed() {
+    Pipeline pipeline = Pipeline.create();
+    TupleTag<KV<String, String>> mainTag = new TupleTag<>();
+    TupleTag<Long> otherTag = new TupleTag<Long>() {};
+    pipeline
+        .apply("Impulse", Impulse.create())
+        .apply(
+            "ParDo",
+            ParDo.of(
+                    new DoFn<byte[], KV<String, String>>() {
+                      @ProcessElement
+                      public void processElement(ProcessContext ctxt) {
+                        ctxt.output(KV.of("foo", "bar"));
+                      }
+                    })
+                .withOutputTags(mainTag, TupleTagList.of(otherTag)))
+        .get(mainTag)
+        .apply(
+            MapElements.into(new TypeDescriptor<KV<String, Integer>>() {})
+                .via(input -> KV.of(input.getKey(), 
Objects.hash(input.getValue()))))
+        .apply("gbk", GroupByKey.create())
+        .apply("vals", Values.create());
+
+    RunnerApi.Pipeline proto = PipelineTranslation.toProto(pipeline);
+    PortableGraph graph = PortableGraph.forPipeline(proto);
+
+    PTransformNode gbkNode =
+        PipelineNode.pTransform("gbk", 
proto.getComponents().getTransformsOrThrow("gbk"));
+    Collection<PCollectionNode> gbkOutput =
+        graph.getProduced(gbkNode);
+    assertThat(gbkOutput, hasSize(1));
+    assertThat(graph.getProducer(getOnlyElement(gbkOutput)), equalTo(gbkNode));
+
+    PTransformNode parDoNode =
+        PipelineNode.pTransform("ParDo", 
proto.getComponents().getTransformsOrThrow("ParDo"));
+    Collection<PCollectionNode> parDoOutput = graph.getProduced(parDoNode);
+    assertThat(parDoOutput, hasSize(2));
+    for (PCollectionNode parDoOutputNode : parDoOutput) {
+      assertThat(graph.getProducer(parDoOutputNode), equalTo(parDoNode));
+      assertThat(parDoNode.getTransform().getOutputsMap(), 
hasValue(parDoOutputNode.getId()));
+    }
+  }
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 98060)
    Time Spent: 1h 10m  (was: 1h)

> The DirectRunner should interact with a Pipeline via an abstraction of the 
> Graph rather than SDK types
> ------------------------------------------------------------------------------------------------------
>
>                 Key: BEAM-4073
>                 URL: https://issues.apache.org/jira/browse/BEAM-4073
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-direct
>            Reporter: Thomas Groh
>            Assignee: Thomas Groh
>            Priority: Major
>              Labels: portability
>          Time Spent: 1h 10m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to