Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-5885

NPE in ZKClient

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 0.10.2.1
    • None
    • zkclient
    • None

    Description

      A null znode for a topic (reason how this happen isn't totally clear, but not the focus of this issue) can currently cause controller leader election to fail. When looking at the broker logging, you can see there is a NullPointerException emanating from the ZKClient:

      [2017-09-11 00:00:21,441] ERROR Error while electing or becoming leader on broker 1010674 (kafka.server.ZookeeperLeaderElector)
      kafka.common.KafkaException: Can't parse json string: null
              at kafka.utils.Json$.liftedTree1$1(Json.scala:40)
              at kafka.utils.Json$.parseFull(Json.scala:36)
              at kafka.utils.ZkUtils$$anonfun$getReplicaAssignmentForTopics$1.apply(ZkUtils.scala:704)
              at kafka.utils.ZkUtils$$anonfun$getReplicaAssignmentForTopics$1.apply(ZkUtils.scala:700)
              at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
              at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
              at kafka.utils.ZkUtils.getReplicaAssignmentForTopics(ZkUtils.scala:700)
              at kafka.controller.KafkaController.initializeControllerContext(KafkaController.scala:742)
              at kafka.controller.KafkaController.onControllerFailover(KafkaController.scala:333)
              at kafka.controller.KafkaController$$anonfun$1.apply$mcV$sp(KafkaController.scala:160)
              at kafka.server.ZookeeperLeaderElector.elect(ZookeeperLeaderElector.scala:85)
              at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply$mcZ$sp(ZookeeperLeaderElector.scala:154)
              at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:154)
              at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:154)
              at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:213)
              at kafka.server.ZookeeperLeaderElector$LeaderChangeListener.handleDataDeleted(ZookeeperLeaderElector.scala:153)
              at org.I0Itec.zkclient.ZkClient$9.run(ZkClient.java:825)
              at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:72)
      Caused by: java.lang.NullPointerException
      

      Regardless of how a null topic znode ended up in ZooKeeper, we can probably handle this better, at least by printing the path up to the problematic znode in the log. The way this particular problem was resolved was by using the ``kafka-topics`` command and seeing it persistently fail trying to read a particular topic with this same message. Then deleting the null znode allowed the leader election to complete.

      Attachments

        Activity

          People

            Unassigned Unassigned
            cotedm Dustin Cote
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated: