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

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

                Author: ASF GitHub Bot
            Created on: 15/Dec/18 18:08
            Start Date: 15/Dec/18 18:08
    Worklog Time Spent: 10m 
      Work Description: mxm opened a new pull request #7291: [BEAM-6227] Do not 
compare recovered state against structural null value
URL: https://github.com/apache/beam/pull/7291
 
 
   When we store the structural null value in a Flink bag state, we must take 
into
   account that it is serialized by the VoidCoder and will be deserialized as
   `null` value. We cannot compare it against the structural null value.
   
   Note that we cannot store null in the first place because Flink does not 
allow
   storing null values in its bag state.
   
   Post-Commit Tests Status (on master branch)
   
------------------------------------------------------------------------------------------------
   
   Lang | SDK | Apex | Dataflow | Flink | Gearpump | Samza | Spark
   --- | --- | --- | --- | --- | --- | --- | ---
   Go | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/)
 | --- | --- | --- | --- | --- | ---
   Java | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/)
 [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/)
   Python | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Python_Verify/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python_Verify/lastCompletedBuild/)
 | --- | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/)
 </br> [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/)
 | --- | --- | ---
   
   
   
   
   

----------------------------------------------------------------
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: 175724)
    Time Spent: 40m  (was: 0.5h)

> 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: 40m
>  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