[
https://issues.apache.org/jira/browse/BEAM-4297?focusedWorklogId=105256&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-105256
]
ASF GitHub Bot logged work on BEAM-4297:
----------------------------------------
Author: ASF GitHub Bot
Created on: 23/May/18 18:43
Start Date: 23/May/18 18:43
Worklog Time Spent: 10m
Work Description: bsidhom commented on a change in pull request #5407:
[BEAM-4297] Streaming executable stage translation and operator for portable
Flink runner.
URL: https://github.com/apache/beam/pull/5407#discussion_r190358677
##########
File path:
runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java
##########
@@ -423,8 +432,133 @@ private void translateImpulse(
String id,
RunnerApi.Pipeline pipeline,
StreamingTranslationContext context) {
+ // TODO: Fail on stateful DoFns for now.
+ // TODO: Support stateful DoFns by inserting group-by-keys where necessary.
+ // TODO: Fail on splittable DoFns.
+ // TODO: Special-case single outputs to avoid multiplexing PCollections.
+ RunnerApi.Components components = pipeline.getComponents();
+ RunnerApi.PTransform transform = components.getTransformsOrThrow(id);
+ Map<String, String> outputs = transform.getOutputsMap();
+ RehydratedComponents rehydratedComponents =
+ RehydratedComponents.forComponents(components);
+
+ BiMap<String, Integer> outputMap =
+ FlinkPipelineTranslatorUtils.createOutputMap(outputs.keySet());
+ Map<String, Coder<WindowedValue<?>>> outputCoders = Maps.newHashMap();
+ for (String localOutputName : new TreeMap<>(outputMap.inverse()).values())
{
+ String collectionId = outputs.get(localOutputName);
+ Coder<WindowedValue<?>> windowCoder = (Coder)
instantiateCoder(collectionId, components);
+ outputCoders.put(localOutputName, windowCoder);
+ }
+
+ final RunnerApi.ExecutableStagePayload stagePayload;
+ try {
+ stagePayload =
RunnerApi.ExecutableStagePayload.parseFrom(transform.getSpec().getPayload());
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+
+ String inputPCollectionId =
+ Iterables.getOnlyElement(transform.getInputsMap().values());
Review comment:
Unfortunately, Flink needs to have an associated serializer
(TypeInformation, aka Coder) with each distributed collection. This
TypeInformation needs to be known at pipeline construction time. It need not
match the exact coder being used to materialize elements over gRPC, but it does
need to match the in-memory element type.
We could get around this partially by representing everything as bytes. The
downside is that each runner-native operation that requires structure (e.g.,
GBK) will require an additional operation to break elements into their
constituent parts. This step itself also requires knowledge of the coded type,
so we ultimately run into the same issue.
----------------------------------------------------------------
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: 105256)
Time Spent: 2h 40m (was: 2.5h)
> Flink portable runner executable stage operator for streaming
> -------------------------------------------------------------
>
> Key: BEAM-4297
> URL: https://issues.apache.org/jira/browse/BEAM-4297
> Project: Beam
> Issue Type: Task
> Components: runner-flink
> Reporter: Thomas Weise
> Assignee: Thomas Weise
> Priority: Major
> Labels: portability
> Time Spent: 2h 40m
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)