Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-3339

Checkpointing NPE when using filterWithState

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 1.0.0
    • 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
      

      Attachments

        Activity

          People

            aljoscha Aljoscha Krettek
            shikhar Shikhar Bhushan
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: