[ https://issues.apache.org/jira/browse/BEAM-3565?focusedWorklogId=82380&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-82380 ]
ASF GitHub Bot logged work on BEAM-3565: ---------------------------------------- Author: ASF GitHub Bot Created on: 20/Mar/18 18:01 Start Date: 20/Mar/18 18:01 Worklog Time Spent: 10m Work Description: tgroh commented on a change in pull request #4777: [BEAM-3565] Add FusedPipeline#toPipeline URL: https://github.com/apache/beam/pull/4777#discussion_r175868128 ########## File path: runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FusedPipeline.java ########## @@ -19,54 +19,84 @@ 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); } - /** - * The {@link ExecutableStage executable stages} that are executed by SDK harnesses. - */ + /** 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. - */ + /** The {@link PTransform PTransforms} that a runner is responsible for executing. */ public abstract Set<PTransformNode> getRunnerExecutedTransforms(); + public RunnerApi.Pipeline toPipeline(Components initialComponents) { + Components executableComponents = + initialComponents + .toBuilder() + .clearTransforms() + .putAllTransforms(getTopLevelTransforms(initialComponents)) + .build(); + List<String> rootTransformIds = + StreamSupport.stream( + QueryablePipeline.forComponents(executableComponents) + .getTopologicallyOrderedTransforms() + .spliterator(), + false) + .map(PTransformNode::getId) + .collect(Collectors.toList()); + return Pipeline.newBuilder() + .setComponents(executableComponents.toBuilder().putAllTransforms(getFusedTransforms())) + .addAllRootTransformIds(rootTransformIds) + .build(); + } + /** * Return a {@link Components} like the {@code base} components, but with the only transforms * equal to this fused pipeline. * - * <p>The only composites will be the stages returned by {@link #getFusedStages()}. + * <p>The only composites will be the stages returned by {@link #getFusedStages()}, and the only + * primitives will be the {@link PTransformNode transforms} returned by {@link + * #getRunnerExecutedTransforms()}. */ - public Components asComponents(Components base) { - Builder newComponents = base.toBuilder().clearTransforms(); + private Map<String, PTransform> getTopLevelTransforms(Components base) { Review comment: `getExecutableTransforms` is what I am calling this now. All of the transforms are primitives, but there are non-executable transforms (within a stage) that get merged into the components as well. ---------------------------------------------------------------- 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: 82380) Time Spent: 14h 20m (was: 14h 10m) > 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: 14h 20m > 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)