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

Kafka source fails during checkpoint notifications with NPE

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 0.10.0
    • 0.10.2, 1.0.0
    • Connectors / Kafka
    • None

    Description

      While running a job with many kafka sources I experienced the following error during the checkpoint notifications:

      java.lang.RuntimeException: Error while confirming checkpoint
      at org.apache.flink.runtime.taskmanager.Task$2.run(Task.java:935)
      at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
      at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
      at java.lang.Thread.run(Thread.java:745)
      Caused by: java.lang.NullPointerException
      at org.I0Itec.zkclient.ZkConnection.writeDataReturnStat(ZkConnection.java:115)
      at org.I0Itec.zkclient.ZkClient$10.call(ZkClient.java:817)
      at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
      at org.I0Itec.zkclient.ZkClient.writeDataReturnStat(ZkClient.java:813)
      at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:808)
      at org.I0Itec.zkclient.ZkClient.writeData(ZkClient.java:777)
      at kafka.utils.ZkUtils$.updatePersistentPath(ZkUtils.scala:332)
      at kafka.utils.ZkUtils.updatePersistentPath(ZkUtils.scala)
      at org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.setOffsetInZooKeeper(ZookeeperOffsetHandler.java:112)
      at org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler.commit(ZookeeperOffsetHandler.java:80)
      at org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer.notifyCheckpointComplete(FlinkKafkaConsumer.java:563)
      at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.notifyOfCompletedCheckpoint(AbstractUdfStreamOperator.java:176)
      at org.apache.flink.streaming.runtime.tasks.StreamTask.notifyCheckpointComplete(StreamTask.java:478)
      at org.apache.flink.runtime.taskmanager.Task$2.run(Task.java:931)
      ... 5 more
      06:23:28,373 INFO org.apache.flink.runtime.executiongraph.ExecutionGraph - Source: Kafka[event.rakdos.log] (1/1) (d79e6b7a25b1ac307d2e0c8

      This resulted in the job crashing and getting stuck during cancelling which subsequently lead to having to restart the cluster.

      This might be a zookeeper issue but we should be able to handle it (catch the exception maybe).

      Attachments

        Issue Links

          Activity

            People

              rmetzger Robert Metzger
              gyfora Gyula Fora
              Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: