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

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

                Author: ASF GitHub Bot
            Created on: 18/Nov/20 22:47
            Start Date: 18/Nov/20 22:47
    Worklog Time Spent: 10m 
      Work Description: je-ik commented on a change in pull request #13353:
URL: https://github.com/apache/beam/pull/13353#discussion_r526473302



##########
File path: 
runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslatorTest.java
##########
@@ -120,4 +138,95 @@ public void testAutoBalanceShardKeyCacheMaxSize() throws 
Exception {
     assertThat(
         fn.getCache().size(), 
equalTo(FlinkAutoBalancedShardKeyShardingFunction.CACHE_MAX_SIZE));
   }
+
+  @Test
+  public void testStatefulParDoAfterCombineChaining() {
+    final JobGraph stablePartitioning = 
getStatefulParDoAfterCombineChainingJobGraph(true);
+    final JobGraph unstablePartitioning = 
getStatefulParDoAfterCombineChainingJobGraph(false);
+    // We expect an extra shuffle stage for unstable partitioning.
+    Assert.assertEquals(
+        1,
+        Iterables.size(unstablePartitioning.getVertices())
+            - Iterables.size(stablePartitioning.getVertices()));
+  }
+
+  private JobGraph getStatefulParDoAfterCombineChainingJobGraph(boolean 
stablePartitioning) {
+    final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+    final FlinkStreamingPipelineTranslator translator =
+        new FlinkStreamingPipelineTranslator(env, 
PipelineOptionsFactory.create());
+    final PipelineOptions pipelineOptions = PipelineOptionsFactory.create();
+    pipelineOptions.setRunner(FlinkRunner.class);

Review comment:
       Do we need to set runner if we enforce the translator?

##########
File path: 
runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslatorTest.java
##########
@@ -120,4 +138,95 @@ public void testAutoBalanceShardKeyCacheMaxSize() throws 
Exception {
     assertThat(
         fn.getCache().size(), 
equalTo(FlinkAutoBalancedShardKeyShardingFunction.CACHE_MAX_SIZE));
   }
+
+  @Test
+  public void testStatefulParDoAfterCombineChaining() {
+    final JobGraph stablePartitioning = 
getStatefulParDoAfterCombineChainingJobGraph(true);
+    final JobGraph unstablePartitioning = 
getStatefulParDoAfterCombineChainingJobGraph(false);
+    // We expect an extra shuffle stage for unstable partitioning.
+    Assert.assertEquals(
+        1,
+        Iterables.size(unstablePartitioning.getVertices())
+            - Iterables.size(stablePartitioning.getVertices()));
+  }
+
+  private JobGraph getStatefulParDoAfterCombineChainingJobGraph(boolean 
stablePartitioning) {

Review comment:
       I would call this parameter `forcesShuffle` or something similar.

##########
File path: 
runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslatorTest.java
##########
@@ -120,4 +138,95 @@ public void testAutoBalanceShardKeyCacheMaxSize() throws 
Exception {
     assertThat(
         fn.getCache().size(), 
equalTo(FlinkAutoBalancedShardKeyShardingFunction.CACHE_MAX_SIZE));
   }
+
+  @Test
+  public void testStatefulParDoAfterCombineChaining() {
+    final JobGraph stablePartitioning = 
getStatefulParDoAfterCombineChainingJobGraph(true);
+    final JobGraph unstablePartitioning = 
getStatefulParDoAfterCombineChainingJobGraph(false);
+    // We expect an extra shuffle stage for unstable partitioning.
+    Assert.assertEquals(
+        1,
+        Iterables.size(unstablePartitioning.getVertices())
+            - Iterables.size(stablePartitioning.getVertices()));
+  }
+
+  private JobGraph getStatefulParDoAfterCombineChainingJobGraph(boolean 
stablePartitioning) {
+    final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+    final FlinkStreamingPipelineTranslator translator =
+        new FlinkStreamingPipelineTranslator(env, 
PipelineOptionsFactory.create());
+    final PipelineOptions pipelineOptions = PipelineOptionsFactory.create();
+    pipelineOptions.setRunner(FlinkRunner.class);
+    final Pipeline pipeline = Pipeline.create(pipelineOptions);
+    PCollection<KV<String, Long>> aggregate =
+        pipeline
+            .apply(Create.of("foo", "bar").withCoder(StringUtf8Coder.of()))
+            .apply(Count.perElement());
+    if (!stablePartitioning) {
+      // When we insert any element-wise "map" operation between aggregation 
and stateful ParDo, we
+      // can no longer assume that partitioning did not change, therefore we 
need an extra shuffle
+      aggregate = aggregate.apply(ParDo.of(new StatelessIdentityDoFn<>()));

Review comment:
       Instead of the `StatelessIdentityDoFn` we could use 
`MapElements.into(...).via(e -> KV.of("", e.getValue())`, which would enforce 
shuffle semantically. That might improve readability a bit.

##########
File path: 
runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslatorTest.java
##########
@@ -120,4 +138,95 @@ public void testAutoBalanceShardKeyCacheMaxSize() throws 
Exception {
     assertThat(
         fn.getCache().size(), 
equalTo(FlinkAutoBalancedShardKeyShardingFunction.CACHE_MAX_SIZE));
   }
+
+  @Test
+  public void testStatefulParDoAfterCombineChaining() {
+    final JobGraph stablePartitioning = 
getStatefulParDoAfterCombineChainingJobGraph(true);
+    final JobGraph unstablePartitioning = 
getStatefulParDoAfterCombineChainingJobGraph(false);
+    // We expect an extra shuffle stage for unstable partitioning.
+    Assert.assertEquals(
+        1,
+        Iterables.size(unstablePartitioning.getVertices())
+            - Iterables.size(stablePartitioning.getVertices()));
+  }
+
+  private JobGraph getStatefulParDoAfterCombineChainingJobGraph(boolean 
stablePartitioning) {
+    final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+    final FlinkStreamingPipelineTranslator translator =
+        new FlinkStreamingPipelineTranslator(env, 
PipelineOptionsFactory.create());
+    final PipelineOptions pipelineOptions = PipelineOptionsFactory.create();
+    pipelineOptions.setRunner(FlinkRunner.class);
+    final Pipeline pipeline = Pipeline.create(pipelineOptions);
+    PCollection<KV<String, Long>> aggregate =
+        pipeline
+            .apply(Create.of("foo", "bar").withCoder(StringUtf8Coder.of()))
+            .apply(Count.perElement());
+    if (!stablePartitioning) {
+      // When we insert any element-wise "map" operation between aggregation 
and stateful ParDo, we
+      // can no longer assume that partitioning did not change, therefore we 
need an extra shuffle
+      aggregate = aggregate.apply(ParDo.of(new StatelessIdentityDoFn<>()));
+    }
+    aggregate.apply(ParDo.of(new StatefulNoopDoFn<>()));
+    translator.translate(pipeline);
+    return env.getStreamGraph().getJobGraph();
+  }
+
+  @Test
+  public void testStatefulParDoAfterGroupByKeyChaining() {
+    final JobGraph stablePartitioning = 
getStatefulParDoAfterGroupByKeyChainingJobGraph(true);
+    final JobGraph unstablePartitioning = 
getStatefulParDoAfterGroupByKeyChainingJobGraph(false);
+    // We expect an extra shuffle stage for unstable partitioning.
+    Assert.assertEquals(
+        1,
+        Iterables.size(unstablePartitioning.getVertices())
+            - Iterables.size(stablePartitioning.getVertices()));
+  }
+
+  private JobGraph getStatefulParDoAfterGroupByKeyChainingJobGraph(boolean 
stablePartitioning) {
+    final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+    final FlinkStreamingPipelineTranslator translator =
+        new FlinkStreamingPipelineTranslator(env, 
PipelineOptionsFactory.create());
+    final PipelineOptions pipelineOptions = PipelineOptionsFactory.create();
+    pipelineOptions.setRunner(FlinkRunner.class);
+    final Pipeline pipeline = Pipeline.create(pipelineOptions);
+    PCollection<KV<String, Iterable<Long>>> aggregate =
+        pipeline
+            .apply(
+                Create.of(KV.of("foo", 1L), KV.of("bar", 1L))
+                    .withCoder(KvCoder.of(StringUtf8Coder.of(), 
VarLongCoder.of())))
+            .apply(GroupByKey.create());

Review comment:
       Can we merge the two methods, that seem to differ only by this 
PTransform applied 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: 513831)
    Time Spent: 2h 20m  (was: 2h 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: 2h 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