[
https://issues.apache.org/jira/browse/BEAM-4297?focusedWorklogId=105302&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-105302
]
ASF GitHub Bot logged work on BEAM-4297:
----------------------------------------
Author: ASF GitHub Bot
Created on: 23/May/18 20:00
Start Date: 23/May/18 20:00
Worklog Time Spent: 10m
Work Description: lukecwik 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_r190380409
##########
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:
I'm just arguing for pushing most of the manipulation done within
ExecutableProcessBundleDescriptor into the ExecutableStage payload (minus the
ApiServiceDescriptor binding) so it doesn't need modification. This would allow
the ExecutableStage to concretely answer what are the input coders, output
coders, side input coders, state coders, ... in addition to any other
information.
Longer term it seems if we had a way for the runner to say whether we need a
keyed input context or grouped keyed output context makes sense as the runner
could then say. These are the cases I know of:
* KV<Key, Value> for SplittableDoFn input, StatefulDoFn input, GBK input,
Multimap side input materialization input, window mapping input
* KV<Key, Iterable<Value>> for GBK output
Do you know of any others?
----------------------------------------------------------------
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: 105302)
Time Spent: 3h (was: 2h 50m)
> 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: 3h
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)