Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Fixed
-
0.10.0
-
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
- is related to
-
KAFKA-824 java.lang.NullPointerException in commitOffsets
- Resolved