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

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

                Author: ASF GitHub Bot
            Created on: 17/Dec/18 14:44
            Start Date: 17/Dec/18 14:44
    Worklog Time Spent: 10m 
      Work Description: tweise closed pull request #7291: [BEAM-6227] Do not 
compare recovered state against structural null value
URL: https://github.com/apache/beam/pull/7291
 
 
   

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 b2f5aede9dfd..6db4306c14ea 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
@@ -328,8 +328,7 @@ public void add(T input) {
         Iterable<T> result = partitionedState.get();
         if (storesVoidValues) {
           return () -> {
-            Iterator underlying = result.iterator();
-            Object structuralNullValue = VoidCoder.of().structuralValue(null);
+            final Iterator underlying = result.iterator();
             return new Iterator<T>() {
               @Override
               public boolean hasNext() {
@@ -338,11 +337,10 @@ public boolean hasNext() {
 
               @Override
               public T next() {
-                Object next = underlying.next();
-                Preconditions.checkState(
-                    structuralNullValue.equals(next),
-                    "Expected to receive structural null value but was: %s",
-                    next);
+                // Simply move the iterator forward but ignore the value.
+                // The value can be the structural null value or NULL itself,
+                // if this has been restored from serialized state.
+                underlying.next();
                 return null;
               }
             };


 

----------------------------------------------------------------
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:
[email protected]


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

    Worklog Id:     (was: 176050)
    Time Spent: 1h 20m  (was: 1h 10m)

> 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: 1h 20m
>  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