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

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

                Author: ASF GitHub Bot
            Created on: 23/May/18 18:25
            Start Date: 23/May/18 18:25
    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_r190351647
 
 

 ##########
 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);
 
 Review comment:
   Unfortunately, we cannot simply pass the ExecutableStage onto operators 
here. Flink requires that operators be serializable. Operator constructors run 
on the client JVM, but operators are initialized via lifecycle methods on 
TaskManagers. For this reason, we use the executable stage _payload_ in the 
batch translator. The same applies here.
   
   We could decide to use a different serialized representation here for 
operator tasks, but it seems convenient here to reuse what we already have.

----------------------------------------------------------------
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: 105234)
    Time Spent: 1h 50m  (was: 1h 40m)

> 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: 1h 50m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to