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

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

                Author: ASF GitHub Bot
            Created on: 16/Nov/20 21:54
            Start Date: 16/Nov/20 21:54
    Worklog Time Spent: 10m 
      Work Description: mxm commented on a change in pull request #13353:
URL: https://github.com/apache/beam/pull/13353#discussion_r524634584



##########
File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
##########
@@ -971,7 +987,9 @@ public void translateNode(
               .transform(fullName, outputTypeInfo, (OneInputStreamOperator) 
doFnOperator)
               .uid(fullName);
 
-      context.setOutputDataStream(context.getOutput(transform), outDataStream);
+      final PCollection<KV<K, Iterable<InputT>>> output = 
context.getOutput(transform);
+      context.setOutputDataStream(output, outDataStream);
+      context.setProducer(output, transform);

Review comment:
       It would be nice if these explicit calls wouldn't be required. I believe 
`context.setOutputDataStream` internally has the current transform available. 
So we could perform it internally in the context.

##########
File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
##########
@@ -1127,7 +1148,9 @@ public void translateNode(
 
         
keyedWorkItemStream.getExecutionEnvironment().addOperator(rawFlinkTransform);
 
-        context.setOutputDataStream(context.getOutput(transform), 
outDataStream);
+        final PCollection<KV<K, OutputT>> output = 
context.getOutput(transform);
+        context.setOutputDataStream(output, outDataStream);
+        context.setProducer(output, transform);

Review comment:
       Same here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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: 512612)
    Time Spent: 1h 20m  (was: 1h 10m)

> Remove unnecessary reshuffle for stateful ParDo after keyed operation
> ---------------------------------------------------------------------
>
>                 Key: BEAM-11267
>                 URL: https://issues.apache.org/jira/browse/BEAM-11267
>             Project: Beam
>          Issue Type: Improvement
>          Components: runner-flink
>            Reporter: David Morávek
>            Assignee: David Morávek
>            Priority: P3
>          Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> When we have stateful pardo after GBK / Combine, we can safely assume that 
> partitioning remains consistent and we can safe an extra shuffle.
> The use case for this are user defined timers / datadriven triggers. This 
> code path is stressed for example by 
> org.apache.beam.sdk.transforms.ParDoTest.TimerTests#testGbkFollowedByUserTimers.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to