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

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

                Author: ASF GitHub Bot
            Created on: 21/Mar/18 23:18
            Start Date: 21/Mar/18 23:18
    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_r176267337
 
 

 ##########
 File path: 
runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FusedPipeline.java
 ##########
 @@ -36,25 +36,45 @@
 @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);
   }
 
+  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();
 
-  public RunnerApi.Pipeline toPipeline(Components initialComponents) {
-    Map<String, PTransform> executableTransforms = 
getExecutableTransforms(initialComponents);
-    Components fusedComponents = initialComponents.toBuilder()
-        .putAllTransforms(executableTransforms)
-        .putAllTransforms(getFusedTransforms())
-        .build();
+  /**
+   * 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 Root Transform IDs will contain all of the runner 
executed transforms and
 
 Review comment:
   The upper casing on Root Transform IDs is strange, would you rather link the 
Pipeline root transform ids method?

----------------------------------------------------------------
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: 82986)
    Time Spent: 17h 40m  (was: 17.5h)

> 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: 17h 40m
>  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