You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by GitBox <gi...@apache.org> on 2020/03/23 14:35:38 UTC

[GitHub] [beam] iemejia commented on a change in pull request #11055: [BEAM-9436] Improve GBK in spark structured streaming runner

iemejia commented on a change in pull request #11055: [BEAM-9436] Improve GBK in spark structured streaming runner
URL: https://github.com/apache/beam/pull/11055#discussion_r396494207
 
 

 ##########
 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:
   I am not familiar with this function but [the documentation explicitly says](https://spark.apache.org/docs/2.4.5/api/java/org/apache/spark/sql/KeyValueGroupedDataset.html#flatMapGroups-org.apache.spark.api.java.function.FlatMapGroupsFunction-org.apache.spark.sql.Encoder-) `...as a result requires shuffling all the data in the Dataset. If an application intends to perform an aggregation over each key, it is best to use the reduce function or an org.apache.spark.sql.expressions#Aggregator`. It is probably a good idea that we test/ensure somehow that GbK + flatMapGroups does not end up producing a double shuffle otherwise the improvement would become a regression.
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services