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

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

                Author: ASF GitHub Bot
            Created on: 24/Mar/20 15:36
            Start Date: 24/Mar/20 15:36
    Worklog Time Spent: 10m 
      Work Description: echauchot commented on pull request #11055: [BEAM-9436] 
Improve GBK in spark structured streaming runner
URL: https://github.com/apache/beam/pull/11055#discussion_r397230829
 
 

 ##########
 File path: 
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java
 ##########
 @@ -53,50 +49,23 @@ public void translateTransform(
 
     @SuppressWarnings("unchecked")
     final PCollection<KV<K, V>> inputPCollection = (PCollection<KV<K, V>>) 
context.getInput();
-
     Dataset<WindowedValue<KV<K, V>>> input = 
context.getDataset(inputPCollection);
-
     WindowingStrategy<?, ?> windowingStrategy = 
inputPCollection.getWindowingStrategy();
     KvCoder<K, V> kvCoder = (KvCoder<K, V>) inputPCollection.getCoder();
+    Coder<V> valueCoder = kvCoder.getValueCoder();
 
     // group by key only
     Coder<K> keyCoder = kvCoder.getKeyCoder();
     KeyValueGroupedDataset<K, WindowedValue<KV<K, V>>> groupByKeyOnly =
         input.groupByKey(KVHelpers.extractKey(), 
EncoderHelpers.fromBeamCoder(keyCoder));
 
-    // Materialize groupByKeyOnly values, potential OOM because of creation of 
new iterable
-    Coder<V> valueCoder = kvCoder.getValueCoder();
-    WindowedValue.WindowedValueCoder<V> wvCoder =
-        WindowedValue.FullWindowedValueCoder.of(
-            valueCoder, 
inputPCollection.getWindowingStrategy().getWindowFn().windowCoder());
-    IterableCoder<WindowedValue<V>> iterableCoder = IterableCoder.of(wvCoder);
-    Dataset<KV<K, Iterable<WindowedValue<V>>>> materialized =
-        groupByKeyOnly.mapGroups(
-            (MapGroupsFunction<K, WindowedValue<KV<K, V>>, KV<K, 
Iterable<WindowedValue<V>>>>)
-                (key, iterator) -> {
-                  List<WindowedValue<V>> values = new ArrayList<>();
-                  while (iterator.hasNext()) {
-                    WindowedValue<KV<K, V>> next = iterator.next();
-                    values.add(
-                        WindowedValue.of(
-                            next.getValue().getValue(),
-                            next.getTimestamp(),
-                            next.getWindows(),
-                            next.getPane()));
-                  }
-                  KV<K, Iterable<WindowedValue<V>>> kv =
-                      KV.of(key, Iterables.unmodifiableIterable(values));
-                  return kv;
-                },
-            EncoderHelpers.fromBeamCoder(KvCoder.of(keyCoder, iterableCoder)));
-
     // group also by windows
     WindowedValue.FullWindowedValueCoder<KV<K, Iterable<V>>> outputCoder =
         WindowedValue.FullWindowedValueCoder.of(
             KvCoder.of(keyCoder, IterableCoder.of(valueCoder)),
             windowingStrategy.getWindowFn().windowCoder());
     Dataset<WindowedValue<KV<K, Iterable<V>>>> output =
-        materialized.flatMap(
+        groupByKeyOnly.flatMapGroups(
 
 Review comment:
   GBK will always trigger a shuffle. dataset.gbk is a logical operation (see 
catalyst). One needs to take at the physical plans. In the previous version the 
physical exec plan showed that the shuffle occurred before the mapgroups. Now 
it occurres before the flatmapGroups. The 30% gain (see numbers) of this change 
resides in the fact that we previously had a mapgroups + flatmap and now we 
have only one flatmapgroups + we no more have to instanciate KV pairs. Take a 
look at the phusical plans if you need to be convinced
 
----------------------------------------------------------------
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: 408851)
    Time Spent: 10.5h  (was: 10h 20m)

> Try to avoid elements list materialization in GBK
> -------------------------------------------------
>
>                 Key: BEAM-9436
>                 URL: https://issues.apache.org/jira/browse/BEAM-9436
>             Project: Beam
>          Issue Type: Improvement
>          Components: runner-spark
>            Reporter: Etienne Chauchot
>            Assignee: Etienne Chauchot
>            Priority: Major
>              Labels: structured-streaming
>          Time Spent: 10.5h
>  Remaining Estimate: 0h
>




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

Reply via email to