Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Fixed
-
None
-
None
-
None
Description
(1.0-SNAPSHOT)
I am using the Scala API keyedStream.filterWithState(..), where the state is an Option[Long]
I am seeing the following error which goes away if I remove the filter.
02/04/2016 14:10:19 Job execution switched to status FAILING. java.lang.RuntimeException: Error triggering a checkpoint as the result of receiving checkpoint barrier at org.apache.flink.streaming.runtime.tasks.StreamTask$2.onEvent(StreamTask.java:651) at org.apache.flink.streaming.runtime.tasks.StreamTask$2.onEvent(StreamTask.java:644) at org.apache.flink.streaming.runtime.io.BarrierBuffer.processBarrier(BarrierBuffer.java:201) at org.apache.flink.streaming.runtime.io.BarrierBuffer.getNextNonBlocked(BarrierBuffer.java:127) at org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:173) at org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63) at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:218) at org.apache.flink.runtime.taskmanager.Task.run(Task.java:562) at java.lang.Thread.run(Thread.java:745) Caused by: java.lang.NullPointerException at org.apache.flink.api.common.typeutils.base.LongSerializer.serialize(LongSerializer.java:63) at org.apache.flink.api.common.typeutils.base.LongSerializer.serialize(LongSerializer.java:27) at org.apache.flink.runtime.state.memory.AbstractMemState.snapshot(AbstractMemState.java:74) at org.apache.flink.runtime.state.AbstractStateBackend.snapshotPartitionedState(AbstractStateBackend.java:245) at org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotOperatorState(AbstractStreamOperator.java:174) at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.snapshotOperatorState(AbstractUdfStreamOperator.java:119) at org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpoint(StreamTask.java:470) at org.apache.flink.streaming.runtime.tasks.StreamTask$2.onEvent(StreamTask.java:648) ... 8 more