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

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

                Author: ASF GitHub Bot
            Created on: 21/Mar/18 21:41
            Start Date: 21/Mar/18 21:41
    Worklog Time Spent: 10m 
      Work Description: lukecwik commented on a change in pull request #4777: 
[BEAM-3565] Add FusedPipeline#toPipeline
URL: https://github.com/apache/beam/pull/4777#discussion_r176246992
 
 

 ##########
 File path: 
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 fusedProto = 
fused.toPipeline(protoPipeline.getComponents());
+
+    assertThat(
+        "Root Transforms should all be present in the Pipeline Components",
+        fusedProto.getComponents().getTransformsMap().keySet(),
+        hasItems(fusedProto.getRootTransformIdsList().toArray(new String[0])));
+    assertThat(
+        "Should contain Impulse, GroupByKey, and two Environment Stages",
+        fusedProto.getRootTransformIdsCount(),
+        equalTo(4));
+    assertThat(fusedProto.getRootTransformIdsList(), hasItems("impulse", 
"gbk"));
+    assertRootsInTopologicalOrder(fusedProto);
+    // 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",
+        fusedProto.getComponents().getTransformsMap(),
+        allOf(hasKey(startsWith("map")), hasKey(startsWith("key")), 
hasKey(startsWith("values"))));
+    assertThat(
+        "Fused transforms shouldn't be present in the root IDs",
+        fusedProto.getRootTransformIdsList(),
+        not(hasItems(startsWith("map"), startsWith("key"), 
startsWith("values"))));
+
+    // The other components should be those of the original pipeline.
+    assertThat(
+        fusedProto.getComponents().getCodersMap(),
+        equalTo(protoPipeline.getComponents().getCodersMap()));
+    assertThat(
+        fusedProto.getComponents().getWindowingStrategiesMap(),
+        equalTo(protoPipeline.getComponents().getWindowingStrategiesMap()));
+    assertThat(
+        fusedProto.getComponents().getEnvironmentsMap(),
+        equalTo(protoPipeline.getComponents().getEnvironmentsMap()));
+    assertThat(
+        fusedProto.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) {
 
 Review comment:
   Don't all roots have zero consumed PCollections so this is just checking 
that no two roots produce the same PCollection.

----------------------------------------------------------------
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: 82933)
    Time Spent: 16h  (was: 15h 50m)

> 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: 16h
>  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