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]


Reply via email to