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

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

                Author: ASF GitHub Bot
            Created on: 23/Mar/18 17:35
            Start Date: 23/Mar/18 17:35
    Worklog Time Spent: 10m 
      Work Description: tgroh closed pull request #4777: [BEAM-3565] Add 
FusedPipeline#toPipeline
URL: https://github.com/apache/beam/pull/4777
 
 
   

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/FusedPipeline.java
 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FusedPipeline.java
index c371920d47e..68da5c3961b 100644
--- 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FusedPipeline.java
+++ 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FusedPipeline.java
@@ -19,54 +19,95 @@
 package org.apache.beam.runners.core.construction.graph;
 
 import com.google.auto.value.AutoValue;
+import com.google.common.collect.Sets;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components.Builder;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
 import 
org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
 
-/**
- * A {@link Pipeline} which has been separated into collections of executable 
components.
- */
+/** A {@link Pipeline} which has been separated into collections of executable 
components. */
 @AutoValue
 public abstract class FusedPipeline {
   static FusedPipeline of(
-      Set<ExecutableStage> environmentalStages, Set<PTransformNode> 
runnerStages) {
-    return new AutoValue_FusedPipeline(environmentalStages, runnerStages);
+      Components components,
+      Set<ExecutableStage> environmentalStages,
+      Set<PTransformNode> runnerStages) {
+    return new AutoValue_FusedPipeline(components, environmentalStages, 
runnerStages);
   }
 
-  /**
-   * The {@link ExecutableStage executable stages} that are executed by SDK 
harnesses.
-   */
+  abstract Components getComponents();
+
+  /** The {@link ExecutableStage executable stages} that are executed by SDK 
harnesses. */
   public abstract Set<ExecutableStage> getFusedStages();
 
+  /** The {@link PTransform PTransforms} that a runner is responsible for 
executing. */
+  public abstract Set<PTransformNode> getRunnerExecutedTransforms();
+
   /**
-   * The {@link PTransform PTransforms} that a runner is responsible for 
executing.
+   * Returns the {@link RunnerApi.Pipeline} representation of this {@link 
FusedPipeline}.
+   *
+   * <p>The {@link Components} of the returned pipeline will contain all of 
the {@link PTransform
+   * PTransforms} present in the original Pipeline that this {@link 
FusedPipeline} was created from,
+   * plus all of the {@link ExecutableStage ExecutableStages} contained within 
this {@link
+   * FusedPipeline}. The {@link Pipeline#getRootTransformIdsList()} will 
contain all of the runner
+   * executed transforms and all of the {@link ExecutableStage execuable 
stages} contained within
+   * the Pipeline.
    */
-  public abstract Set<PTransformNode> getRunnerExecutedTransforms();
+  public RunnerApi.Pipeline toPipeline() {
+    Map<String, PTransform> executableStageTransforms = 
getEnvironmentExecutedTransforms();
+    Set<String> executableTransformIds =
+        Sets.union(
+            executableStageTransforms.keySet(),
+            getRunnerExecutedTransforms()
+                .stream()
+                .map(PTransformNode::getId)
+                .collect(Collectors.toSet()));
+
+    // Augment the initial transforms with all of the executable transforms.
+    Components fusedComponents =
+        
getComponents().toBuilder().putAllTransforms(executableStageTransforms).build();
+    List<String> rootTransformIds =
+        StreamSupport.stream(
+                QueryablePipeline.forTransforms(executableTransformIds, 
fusedComponents)
+                    .getTopologicallyOrderedTransforms()
+                    .spliterator(),
+                false)
+            .map(PTransformNode::getId)
+            .collect(Collectors.toList());
+    return Pipeline.newBuilder()
+        .setComponents(fusedComponents)
+        .addAllRootTransformIds(rootTransformIds)
+        .build();
+  }
 
   /**
-   * Return a {@link Components} like the {@code base} components, but with 
the only transforms
-   * equal to this fused pipeline.
+   * Return a map of IDs to {@link PTransform} which are executed by an SDK 
Harness.
    *
-   * <p>The only composites will be the stages returned by {@link 
#getFusedStages()}.
+   * <p>The transforms that are present in the returned map are the {@link 
RunnerApi.PTransform}
+   * versions of the {@link ExecutableStage ExecutableStages} returned in 
{@link #getFusedStages()}.
+   * The IDs of the returned transforms will not collide with any transform ID 
present in {@link
+   * #getComponents()}.
    */
-  public Components asComponents(Components base) {
-    Builder newComponents = base.toBuilder().clearTransforms();
-    for (PTransformNode runnerExecuted : getRunnerExecutedTransforms()) {
-      newComponents.putTransforms(runnerExecuted.getId(), 
runnerExecuted.getTransform());
-    }
+  private Map<String, PTransform> getEnvironmentExecutedTransforms() {
+    Map<String, PTransform> topLevelTransforms = new HashMap<>();
     for (ExecutableStage stage : getFusedStages()) {
-      for (PTransformNode fusedTransform : stage.getTransforms()) {
-        newComponents.putTransforms(fusedTransform.getId(), 
fusedTransform.getTransform());
-      }
-      newComponents.putTransforms(stageId(stage, newComponents), 
stage.toPTransform());
+      topLevelTransforms.put(
+          generateStageId(
+              stage,
+              Sets.union(getComponents().getTransformsMap().keySet(), 
topLevelTransforms.keySet())),
+          stage.toPTransform());
     }
-    return newComponents.build();
+    return topLevelTransforms;
   }
 
-  private String stageId(ExecutableStage stage, Components.Builder cbuilder) {
+  private String generateStageId(ExecutableStage stage, Set<String> 
existingIds) {
     int i = 0;
     String name;
     do {
@@ -78,7 +119,7 @@ private String stageId(ExecutableStage stage, 
Components.Builder cbuilder) {
               stage.getEnvironment().getUrl(),
               i);
       i++;
-    } while (cbuilder.containsTransforms(name));
+    } while (existingIds.contains(name));
     return name;
   }
 }
diff --git 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java
 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java
index fa09bacfa52..35e079151bd 100644
--- 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java
+++ 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java
@@ -80,7 +80,7 @@ private GreedyPipelineFuser(Pipeline p) {
    */
   public static FusedPipeline fuse(Pipeline p) {
     GreedyPipelineFuser fuser = new GreedyPipelineFuser(p);
-    return FusedPipeline.of(fuser.stages, fuser.unfusedTransforms);
+    return FusedPipeline.of(p.getComponents(), fuser.stages, 
fuser.unfusedTransforms);
   }
 
   /**
diff --git 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/PipelineNode.java
 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/PipelineNode.java
index ee956421001..3829142058d 100644
--- 
a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/PipelineNode.java
+++ 
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/PipelineNode.java
@@ -34,6 +34,8 @@ static PCollectionNode pCollection(String id, PCollection 
collection) {
     return new AutoValue_PipelineNode_PCollectionNode(id, collection);
   }
 
+  String getId();
+
   /**
    * A {@link PipelineNode} which contains a {@link PCollection}.
    */
@@ -43,7 +45,6 @@ static PCollectionNode pCollection(String id, PCollection 
collection) {
     public abstract PCollection getPCollection();
   }
 
-
   /**
    * A {@link PipelineNode} which contains a {@link PTransform}.
    */
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 c13e5eb1b78..27c46fdabe3 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
@@ -28,12 +28,14 @@
 import com.google.protobuf.InvalidProtocolBufferException;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
@@ -170,6 +172,16 @@ private static boolean isPrimitiveTransform(PTransform 
transform) {
     return network;
   }
 
+  public Iterable<PTransformNode> getTopologicallyOrderedTransforms() {
+    return StreamSupport.stream(
+            Networks.topologicalOrder(pipelineNetwork, 
Comparator.comparing(PipelineNode::getId))
+                .spliterator(),
+            false)
+        .filter(PTransformNode.class::isInstance)
+        .map(PTransformNode.class::cast)
+        .collect(Collectors.toList());
+  }
+
   /**
    * Get the transforms that are roots of this {@link QueryablePipeline}. 
These are all nodes which
    * have no input {@link PCollection}.
diff --git 
a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/FusedPipelineTest.java
 
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/FusedPipelineTest.java
new file mode 100644
index 00000000000..2a350054486
--- /dev/null
+++ 
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/FusedPipelineTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.construction.graph;
+
+import static com.google.common.base.Preconditions.checkState;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.hasItems;
+import static org.hamcrest.Matchers.hasKey;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.startsWith;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.collect.ImmutableList;
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
+import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
+import org.apache.beam.runners.core.construction.PipelineTranslation;
+import org.apache.beam.sdk.Pipeline;
+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.Values;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link FusedPipeline}. */
+@RunWith(JUnit4.class)
+public class FusedPipelineTest implements Serializable {
+  @Test
+  public void testToProto() {
+    Pipeline p = Pipeline.create();
+    p.apply("impulse", Impulse.create())
+        .apply("map", MapElements.into(TypeDescriptors.integers()).via(bytes 
-> bytes.length))
+        .apply("key", WithKeys.of("foo"))
+        .apply("gbk", GroupByKey.create())
+        .apply("values", Values.create());
+
+    RunnerApi.Pipeline protoPipeline = PipelineTranslation.toProto(p);
+    checkState(
+        protoPipeline
+            .getRootTransformIdsList()
+            .containsAll(ImmutableList.of("impulse", "map", "key", "gbk", 
"values")),
+        "Unexpected Root Transform IDs %s",
+        protoPipeline.getRootTransformIdsList());
+
+    FusedPipeline fused = GreedyPipelineFuser.fuse(protoPipeline);
+    checkState(
+        fused.getRunnerExecutedTransforms().size() == 2,
+        "Unexpected number of runner transforms %s",
+        fused.getRunnerExecutedTransforms());
+    checkState(
+        fused.getFusedStages().size() == 2,
+        "Unexpected number of fused stages %s",
+        fused.getFusedStages());
+    RunnerApi.Pipeline fusedPipelineProto = fused.toPipeline();
+
+    assertThat(
+        "Root Transforms should all be present in the Pipeline Components",
+        fusedPipelineProto.getComponents().getTransformsMap().keySet(),
+        hasItems(fusedPipelineProto.getRootTransformIdsList().toArray(new 
String[0])));
+    assertThat(
+        "Should contain Impulse, GroupByKey, and two Environment Stages",
+        fusedPipelineProto.getRootTransformIdsCount(),
+        equalTo(4));
+    assertThat(fusedPipelineProto.getRootTransformIdsList(), 
hasItems("impulse", "gbk"));
+    assertRootsInTopologicalOrder(fusedPipelineProto);
+    // Since MapElements, WithKeys, and Values are all composites of a ParDo, 
we do prefix matching
+    // instead of looking at the inside of their expansions
+    assertThat(
+        "Fused transforms should be present in the components",
+        fusedPipelineProto.getComponents().getTransformsMap(),
+        allOf(hasKey(startsWith("map")), hasKey(startsWith("key")), 
hasKey(startsWith("values"))));
+    assertThat(
+        "Fused transforms shouldn't be present in the root IDs",
+        fusedPipelineProto.getRootTransformIdsList(),
+        not(hasItems(startsWith("map"), startsWith("key"), 
startsWith("values"))));
+
+    // The other components should be those of the original pipeline.
+    assertThat(
+        fusedPipelineProto.getComponents().getCodersMap(),
+        equalTo(protoPipeline.getComponents().getCodersMap()));
+    assertThat(
+        fusedPipelineProto.getComponents().getWindowingStrategiesMap(),
+        equalTo(protoPipeline.getComponents().getWindowingStrategiesMap()));
+    assertThat(
+        fusedPipelineProto.getComponents().getEnvironmentsMap(),
+        equalTo(protoPipeline.getComponents().getEnvironmentsMap()));
+    assertThat(
+        fusedPipelineProto.getComponents().getPcollectionsMap(),
+        equalTo(protoPipeline.getComponents().getPcollectionsMap()));
+  }
+
+  // For each transform in the root transforms, asserts that all consumed 
PCollections have been
+  // produced, and no produced PCollection has been consumed
+  private void assertRootsInTopologicalOrder(RunnerApi.Pipeline fusedProto) {
+    Set<String> consumedPCollections = new HashSet<>();
+    Set<String> producedPCollections = new HashSet<>();
+    for (int i = 0; i < fusedProto.getRootTransformIdsCount(); i++) {
+      PTransform rootTransform =
+          
fusedProto.getComponents().getTransformsOrThrow(fusedProto.getRootTransformIds(i));
+      assertThat(
+          String.format(
+              "All %s consumed by %s must be produced before it",
+              PCollection.class.getSimpleName(), 
fusedProto.getRootTransformIds(i)),
+          producedPCollections,
+          hasItems(rootTransform.getInputsMap().values().toArray(new 
String[0])));
+      for (String consumed : consumedPCollections) {
+        assertThat(
+            String.format(
+                "%s %s was consumed before all of its producers produced it",
+                PCollection.class.getSimpleName(), consumed),
+            rootTransform.getOutputsMap().values(),
+            not(hasItem(consumed)));
+      }
+      consumedPCollections.addAll(rootTransform.getInputsMap().values());
+      producedPCollections.addAll(rootTransform.getOutputsMap().values());
+    }
+  }
+}


 

----------------------------------------------------------------
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:
[email protected]


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

    Worklog Id:     (was: 83709)
    Time Spent: 18h 10m  (was: 18h)

> Add utilities for producing a collection of PTransforms that can execute in a 
> single SDK Harness
> ------------------------------------------------------------------------------------------------
>
>                 Key: BEAM-3565
>                 URL: https://issues.apache.org/jira/browse/BEAM-3565
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-core
>            Reporter: Thomas Groh
>            Assignee: Thomas Groh
>            Priority: Major
>              Labels: portability
>             Fix For: 2.4.0
>
>          Time Spent: 18h 10m
>  Remaining Estimate: 0h
>
> An SDK Harness executes some ("fused") collection of PTransforms. The java 
> runner libraries should provide some way to take a Pipeline that executes in 
> both a runner and an environment and construct a collection of transforms 
> which can execute within a single environment.



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

Reply via email to