[
https://issues.apache.org/jira/browse/BEAM-6294?focusedWorklogId=179477&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-179477
]
ASF GitHub Bot logged work on BEAM-6294:
----------------------------------------
Author: ASF GitHub Bot
Created on: 28/Dec/18 14:48
Start Date: 28/Dec/18 14:48
Worklog Time Spent: 10m
Work Description: mxm commented on pull request #7360: [BEAM-6294] Use
Flink rebalance for shuffle.
URL: https://github.com/apache/beam/pull/7360#discussion_r244335373
##########
File path:
runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobInvocation.java
##########
@@ -98,41 +99,71 @@ private FlinkJobInvocation(
private PipelineResult runPipeline() throws Exception {
MetricsEnvironment.setMetricsSupported(false);
+ FlinkPortablePipelineTranslator<?> translator;
+ if (!pipelineOptions.isStreaming() && !hasUnboundedPCollections(pipeline))
{
+ // TODO: Do we need to inspect for unbounded sources before fusing?
+ translator = FlinkBatchPortablePipelineTranslator.createTranslator();
+ } else {
+ translator = new FlinkStreamingPortablePipelineTranslator();
+ }
+ return runPipelineWithTranslator(translator);
+ }
+
+ private <T extends FlinkPortablePipelineTranslator.TranslationContext>
+ PipelineResult
runPipelineWithTranslator(FlinkPortablePipelineTranslator<T> translator)
+ throws Exception {
LOG.info("Translating pipeline to Flink program.");
+
+ // Don't let the fuser fuse any subcomponents.
+ RunnerApi.Pipeline trimmedPipeline = makeKnownUrnsPrimitives(pipeline,
translator.knownUrns());
+ System.out.println(trimmedPipeline.getComponents().getTransformsMap());
+
// Fused pipeline proto.
- RunnerApi.Pipeline fusedPipeline =
GreedyPipelineFuser.fuse(pipeline).toPipeline();
+ RunnerApi.Pipeline fusedPipeline =
GreedyPipelineFuser.fuse(trimmedPipeline).toPipeline();
JobInfo jobInfo =
JobInfo.create(
id,
pipelineOptions.getJobName(),
retrievalToken,
PipelineOptionsTranslation.toProto(pipelineOptions));
- final JobExecutionResult result;
- if (!pipelineOptions.isStreaming() &&
!hasUnboundedPCollections(fusedPipeline)) {
- // TODO: Do we need to inspect for unbounded sources before fusing?
- // batch translation
- FlinkBatchPortablePipelineTranslator translator =
- FlinkBatchPortablePipelineTranslator.createTranslator();
- FlinkBatchPortablePipelineTranslator.BatchTranslationContext context =
- FlinkBatchPortablePipelineTranslator.createTranslationContext(
- jobInfo, pipelineOptions, confDir, filesToStage);
- translator.translate(context, fusedPipeline);
- result =
context.getExecutionEnvironment().execute(pipelineOptions.getJobName());
- } else {
- // streaming translation
- FlinkStreamingPortablePipelineTranslator translator =
- new FlinkStreamingPortablePipelineTranslator();
- FlinkStreamingPortablePipelineTranslator.StreamingTranslationContext
context =
- FlinkStreamingPortablePipelineTranslator.createTranslationContext(
- jobInfo, pipelineOptions, confDir, filesToStage);
- translator.translate(context, fusedPipeline);
- result =
context.getExecutionEnvironment().execute(pipelineOptions.getJobName());
- }
+ FlinkPortablePipelineTranslator.Executor executor =
+ translator.translate(
+ translator.createTranslationContext(jobInfo, pipelineOptions,
confDir, filesToStage),
+ fusedPipeline);
+ final JobExecutionResult result =
executor.execute(pipelineOptions.getJobName());
return FlinkRunner.createPipelineResult(result, pipelineOptions);
}
+ private RunnerApi.Pipeline makeKnownUrnsPrimitives(
+ RunnerApi.Pipeline pipeline, Set<String> knownUrns) {
+ RunnerApi.Pipeline.Builder trimmedPipeline = pipeline.toBuilder();
+ for (String ptransformId :
pipeline.getComponents().getTransformsMap().keySet()) {
+ if (knownUrns.contains(
+
pipeline.getComponents().getTransformsOrThrow(ptransformId).getSpec().getUrn()))
{
+ System.out.println("FOUND " + ptransformId);
Review comment:
We can change this to a debug log output.
----------------------------------------------------------------
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: 179477)
Time Spent: 20m (was: 10m)
> Use Flink's redistribute for reshuffle.
> ---------------------------------------
>
> Key: BEAM-6294
> URL: https://issues.apache.org/jira/browse/BEAM-6294
> Project: Beam
> Issue Type: New Feature
> Components: runner-flink, sdk-py-core
> Reporter: Robert Bradshaw
> Assignee: Robert Bradshaw
> Priority: Major
> Time Spent: 20m
> Remaining Estimate: 0h
>
> Python needs to publish the URN over the FnAPI which is pretty easy, but
> Flink also needs to ensure that the composite structure does not get fused.
> Unlike with GBK, we can't assume all runners implement this as a primitive.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)