[
https://issues.apache.org/jira/browse/BEAM-12153?focusedWorklogId=647506&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-647506
]
ASF GitHub Bot logged work on BEAM-12153:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 07/Sep/21 18:05
Start Date: 07/Sep/21 18:05
Worklog Time Spent: 10m
Work Description: ibzib commented on a change in pull request #15267:
URL: https://github.com/apache/beam/pull/15267#discussion_r703719653
##########
File path:
sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
##########
@@ -83,6 +84,7 @@ public void setup() throws IOException {
@Rule public TestPipeline pipeline = TestPipeline.create();
+ @Ignore("Temporary ignore this test waiting for a fix because it is flaky")
Review comment:
I agree we should ignore these tests, but since they're unrelated to the
Spark changes we should do it in a separate PR.
Also make sure to include the JIRA number in the message.
##########
File path:
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java
##########
@@ -17,74 +17,95 @@
*/
package org.apache.beam.runners.spark.structuredstreaming.translation.batch;
-import java.io.Serializable;
-import org.apache.beam.runners.core.InMemoryStateInternals;
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.StateInternalsFactory;
-import org.apache.beam.runners.core.SystemReduceFn;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.runners.core.Concatenate;
import
org.apache.beam.runners.spark.structuredstreaming.translation.AbstractTranslationContext;
import
org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator;
-import
org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions.GroupAlsoByWindowViaOutputBufferFn;
import
org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers;
import
org.apache.beam.runners.spark.structuredstreaming.translation.helpers.KVHelpers;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.IterableCoder;
import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.Combine;
import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.WindowingStrategy;
+import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.KeyValueGroupedDataset;
+import scala.Tuple2;
class GroupByKeyTranslatorBatch<K, V>
- implements TransformTranslator<
- PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>>> {
+ implements TransformTranslator<
Review comment:
I am not sure why the indentation changed here. Do we not use Spotless
on the Spark structured streaming project?
##########
File path:
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java
##########
@@ -17,74 +17,95 @@
*/
package org.apache.beam.runners.spark.structuredstreaming.translation.batch;
-import java.io.Serializable;
-import org.apache.beam.runners.core.InMemoryStateInternals;
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.StateInternalsFactory;
-import org.apache.beam.runners.core.SystemReduceFn;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.runners.core.Concatenate;
import
org.apache.beam.runners.spark.structuredstreaming.translation.AbstractTranslationContext;
import
org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator;
-import
org.apache.beam.runners.spark.structuredstreaming.translation.batch.functions.GroupAlsoByWindowViaOutputBufferFn;
import
org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers;
import
org.apache.beam.runners.spark.structuredstreaming.translation.helpers.KVHelpers;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.IterableCoder;
import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.Combine;
import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.WindowingStrategy;
+import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.KeyValueGroupedDataset;
+import scala.Tuple2;
class GroupByKeyTranslatorBatch<K, V>
- implements TransformTranslator<
- PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>>> {
+ implements TransformTranslator<
+ PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>>> {
@Override
public void translateTransform(
PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>>
transform,
AbstractTranslationContext context) {
@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();
+ final PCollection<KV<K, V>> input = (PCollection<KV<K, V>>)
context.getInput();
+ @SuppressWarnings("unchecked")
+ final PCollection<KV<K, List<V>>> output = (PCollection<KV<K, List<V>>>)
context.getOutput();
+ @SuppressWarnings("unchecked")
Review comment:
I don't think the suppression is necessary here. We're constructing a
fresh object so it shouldn't count as an unchecked cast.
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 647506)
Time Spent: 8h (was: 7h 50m)
> OOM on GBK in SparkRunner and SpartStructuredStreamingRunner
> ------------------------------------------------------------
>
> Key: BEAM-12153
> URL: https://issues.apache.org/jira/browse/BEAM-12153
> Project: Beam
> Issue Type: Bug
> Components: runner-spark
> Reporter: Etienne Chauchot
> Assignee: Etienne Chauchot
> Priority: P3
> Time Spent: 8h
> Remaining Estimate: 0h
>
> We have being experiencing OOM errors on GroupByKey in batch mode in both
> Spark runners even if behind the woods spark spills data to disk in such
> cases: taking a look at the translation in the two runners, it might be due
> to using ReduceFnRunner for merging windows in GBK translation.
> ReduceFnRunner.processElements expects to have all elements to merge the
> windows between each other.:
> RDD spark runner:
> https://github.com/apache/beam/blob/752798e3b0e6911fa84f8d138dacccdb6fcc85ef/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java#L99
> structured streaming spark: runner:
> [https://github.com/apache/beam/blob/752798e3b0e6911fa84f8d138dacccdb6fcc85ef/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java#L74|https://github.com/apache/beam/blob/752798e3b0e6911fa84f8d138dacccdb6fcc85ef/runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java#L106]
> Even replacing the Iterable with an Iterator in ReduceFnRunner to avoid
> materialization does not work because deep in
> ReduceFnRunner.processElements(), the collection is iterated twice.
> It could be better to do what flink runner does and translate GBK as
> CombinePerKey with a Concatenate combine fn and thus avoid elements
> materialization.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)