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

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

                Author: ASF GitHub Bot
            Created on: 14/Dec/18 16:41
            Start Date: 14/Dec/18 16:41
    Worklog Time Spent: 10m 
      Work Description: tweise closed pull request #7282: [BEAM-6227] Fix 
GroupByKey with null values in Flink Runner
URL: https://github.com/apache/beam/pull/7282
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
index 02a2ebee74d0..b2f5aede9dfd 100644
--- 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
+++ 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
@@ -17,13 +17,16 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.Map;
+import javax.annotation.Nonnull;
 import org.apache.beam.runners.core.StateInternals;
 import org.apache.beam.runners.core.StateNamespace;
 import org.apache.beam.runners.core.StateTag;
@@ -31,6 +34,7 @@
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.coders.InstantCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.state.BagState;
 import org.apache.beam.sdk.state.CombiningState;
 import org.apache.beam.sdk.state.MapState;
@@ -49,6 +53,7 @@
 import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
 import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.util.CombineContextFactory;
+import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.state.MapStateDescriptor;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
@@ -274,6 +279,7 @@ public int hashCode() {
     private final String stateId;
     private final ListStateDescriptor<T> flinkStateDescriptor;
     private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
+    private final boolean storesVoidValues;
 
     FlinkBagState(
         KeyedStateBackend<ByteBuffer> flinkStateBackend,
@@ -284,17 +290,24 @@ public int hashCode() {
       this.namespace = namespace;
       this.stateId = stateId;
       this.flinkStateBackend = flinkStateBackend;
-
-      flinkStateDescriptor = new ListStateDescriptor<>(stateId, new 
CoderTypeSerializer<>(coder));
+      this.storesVoidValues = coder instanceof VoidCoder;
+      this.flinkStateDescriptor =
+          new ListStateDescriptor<>(stateId, new CoderTypeSerializer<>(coder));
     }
 
     @Override
     public void add(T input) {
       try {
-        flinkStateBackend
-            .getPartitionedState(
-                namespace.stringKey(), StringSerializer.INSTANCE, 
flinkStateDescriptor)
-            .add(input);
+        ListState<T> partitionedState =
+            flinkStateBackend.getPartitionedState(
+                namespace.stringKey(), StringSerializer.INSTANCE, 
flinkStateDescriptor);
+        if (storesVoidValues) {
+          Preconditions.checkState(input == null, "Expected to a null value 
but was: %s", input);
+          // Flink does not allow storing null values
+          // If we have null values, we use the structural null value
+          input = (T) VoidCoder.of().structuralValue((Void) input);
+        }
+        partitionedState.add(input);
       } catch (Exception e) {
         throw new RuntimeException("Error adding to bag state.", e);
       }
@@ -306,14 +319,35 @@ public void add(T input) {
     }
 
     @Override
+    @Nonnull
     public Iterable<T> read() {
       try {
-        Iterable<T> result =
-            flinkStateBackend
-                .getPartitionedState(
-                    namespace.stringKey(), StringSerializer.INSTANCE, 
flinkStateDescriptor)
-                .get();
-
+        ListState<T> partitionedState =
+            flinkStateBackend.getPartitionedState(
+                namespace.stringKey(), StringSerializer.INSTANCE, 
flinkStateDescriptor);
+        Iterable<T> result = partitionedState.get();
+        if (storesVoidValues) {
+          return () -> {
+            Iterator underlying = result.iterator();
+            Object structuralNullValue = VoidCoder.of().structuralValue(null);
+            return new Iterator<T>() {
+              @Override
+              public boolean hasNext() {
+                return underlying.hasNext();
+              }
+
+              @Override
+              public T next() {
+                Object next = underlying.next();
+                Preconditions.checkState(
+                    structuralNullValue.equals(next),
+                    "Expected to receive structural null value but was: %s",
+                    next);
+                return null;
+              }
+            };
+          };
+        }
         return result != null ? ImmutableList.copyOf(result) : 
Collections.emptyList();
       } catch (Exception e) {
         throw new RuntimeException("Error reading state.", e);
diff --git 
a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByWithNullValuesTest.java
 
b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByWithNullValuesTest.java
new file mode 100644
index 000000000000..7411c94e06bd
--- /dev/null
+++ 
b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByWithNullValuesTest.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.streaming;
+
+import static org.junit.Assert.assertNull;
+
+import java.io.Serializable;
+import org.apache.beam.runners.flink.FlinkPipelineOptions;
+import org.apache.beam.runners.flink.FlinkRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.GenerateSequence;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+
+/** Tests grouping with null values. */
+public class GroupByWithNullValuesTest implements Serializable {
+
+  @Test
+  public void testGroupByWithNullValues() {
+    FlinkPipelineOptions options = 
PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+
+    options.setRunner(FlinkRunner.class);
+    options.setStreaming(true);
+
+    Pipeline pipeline = Pipeline.create(options);
+    PCollection<Integer> result =
+        pipeline
+            .apply(
+                GenerateSequence.from(0)
+                    .to(100)
+                    .withTimestampFn(
+                        new SerializableFunction<Long, Instant>() {
+                          @Override
+                          public Instant apply(Long input) {
+                            return new Instant(input);
+                          }
+                        }))
+            .apply(Window.into(FixedWindows.of(Duration.millis(10))))
+            .apply(
+                ParDo.of(
+                    new DoFn<Long, KV<String, Void>>() {
+                      @ProcessElement
+                      public void processElement(ProcessContext pc) {
+                        pc.output(KV.of("hello", null));
+                      }
+                    }))
+            .apply(GroupByKey.create())
+            .apply(
+                ParDo.of(
+                    new DoFn<KV<String, Iterable<Void>>, Integer>() {
+                      @ProcessElement
+                      public void processElement(ProcessContext pc) {
+                        int count = 0;
+                        for (Void aVoid : pc.element().getValue()) {
+                          assertNull("Element should be null", aVoid);
+                          count++;
+                        }
+                        pc.output(count);
+                      }
+                    }));
+
+    PAssert.that(result).containsInAnyOrder(10, 10, 10, 10, 10, 10, 10, 10, 
10, 10);
+
+    pipeline.run();
+  }
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 175447)
    Time Spent: 0.5h  (was: 20m)

> FlinkRunner errors if GroupByKey contains null values (streaming mode only)
> ---------------------------------------------------------------------------
>
>                 Key: BEAM-6227
>                 URL: https://issues.apache.org/jira/browse/BEAM-6227
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-flink
>    Affects Versions: 2.9.0
>            Reporter: Maximilian Michels
>            Assignee: Maximilian Michels
>            Priority: Major
>             Fix For: 2.10.0
>
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> Apparently this passed ValidatesRunner in streaming mode although this is a 
> quite common operation:
> {noformat}
>     FlinkPipelineOptions options = 
> PipelineOptionsFactory.as(FlinkPipelineOptions.class);
>     options.setRunner(FlinkRunner.class);
>     // force streaming mode
>     options.setStreaming(true);
>     Pipeline pipeline = Pipeline.create(options);
>     pipeline
>         .apply(GenerateSequence.from(0).to(100))
>         .apply(Window.into(FixedWindows.of(Duration.millis(10))))
>         .apply(ParDo.of(
>             new DoFn<Long, KV<String, Void>>() {
>               @ProcessElement
>               public void processElement(ProcessContext pc) {
>                 pc.output(KV.of("hello", null));
>               }
>             }
>         ))
>         .apply(GroupByKey.create());
>     pipeline.run();
> {noformat}
> Throws:
> {noformat}
> Caused by: java.lang.RuntimeException: Error adding to bag state.
>       at 
> org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals$FlinkBagState.add(FlinkStateInternals.java:299)
>       at 
> org.apache.beam.runners.core.SystemReduceFn.processValue(SystemReduceFn.java:115)
>       at 
> org.apache.beam.runners.core.ReduceFnRunner.processElement(ReduceFnRunner.java:608)
>       at 
> org.apache.beam.runners.core.ReduceFnRunner.processElements(ReduceFnRunner.java:349)
>       at 
> org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetNewDoFn.processElement(GroupAlsoByWindowViaWindowSetNewDoFn.java:136)
> Caused by: java.lang.NullPointerException: You cannot add null to a ListState.
>       at 
> org.apache.flink.util.Preconditions.checkNotNull(Preconditions.java:75)
>       at 
> org.apache.flink.runtime.state.heap.HeapListState.add(HeapListState.java:89)
>       at 
> org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals$FlinkBagState.add(FlinkStateInternals.java:297)
>       at 
> org.apache.beam.runners.core.SystemReduceFn.processValue(SystemReduceFn.java:115)
>       at 
> org.apache.beam.runners.core.ReduceFnRunner.processElement(ReduceFnRunner.java:608)
>       at 
> org.apache.beam.runners.core.ReduceFnRunner.processElements(ReduceFnRunner.java:349)
>       at 
> org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetNewDoFn.processElement(GroupAlsoByWindowViaWindowSetNewDoFn.java:136)
>       at 
> org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetNewDoFn$DoFnInvoker.invokeProcessElement(Unknown
>  Source)
>       at 
> org.apache.beam.runners.core.SimpleDoFnRunner.invokeProcessElement(SimpleDoFnRunner.java:275)
>       at 
> org.apache.beam.runners.core.SimpleDoFnRunner.processElement(SimpleDoFnRunner.java:240)
>       at 
> org.apache.beam.runners.core.LateDataDroppingDoFnRunner.processElement(LateDataDroppingDoFnRunner.java:80)
>       at 
> org.apache.beam.runners.flink.metrics.DoFnRunnerWithMetricsUpdate.processElement(DoFnRunnerWithMetricsUpdate.java:63)
>       at 
> org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator.processElement(DoFnOperator.java:460)
>       at 
> org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:202)
>       at 
> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:104)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:306)
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:712)
>       at java.lang.Thread.run(Thread.java:745)
> {noformat}
> Will do a follow-up for running ValidatesRunner in streaming mode.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to