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

Error while creating ephemeral at /brokers/ids/BROKER_ID

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.1.0
    • 2.2.0
    • core, zkclient
    • None

    Description

      Kafka version: 1.1.0

      Zookeeper version: 3.4.12

      4 Kafka Brokers

      4 Zookeeper servers

       

      In one of the 4 brokers of the cluster, we detect the following error:

      [2018-07-14 04:38:23,784] INFO Unable to read additional data from server sessionid 0x3000c2420cb458d, likely server has closed socket, closing socket connection and attempting reconnect (org.apache.zookeeper.ClientCnxn)
      [2018-07-14 04:38:24,509] INFO Opening socket connection to server ZOOKEEPER_SERVER_1:PORT. Will not attempt to authenticate using SASL (unknown error) (org.apache.zookeeper.ClientCnxn)
      [2018-07-14 04:38:24,510] INFO Socket connection established to ZOOKEEPER_SERVER_1:PORT, initiating session (org.apache.zookeeper.ClientCnxn)
      [2018-07-14 04:38:24,513] INFO Unable to read additional data from server sessionid 0x3000c2420cb458d, likely server has closed socket, closing socket connection and attempting reconnect (org.apache.zookeeper.ClientCnxn)
      [2018-07-14 04:38:25,287] INFO Opening socket connection to server ZOOKEEPER_SERVER_2:PORT. Will not attempt to authenticate using SASL (unknown error) (org.apache.zookeeper.ClientCnxn)
      [2018-07-14 04:38:25,287] INFO Socket connection established to ZOOKEEPER_SERVER_2:PORT, initiating session (org.apache.zookeeper.ClientCnxn)
      [2018-07-14 04:38:25,954] INFO Partition TOPIC_NAME-PARTITION-# broker=1 Shrinking ISR from 1,3,4,2 to 1,4,2 (kafka.cluster.Partition)
      [2018-07-14 04:38:26,444] WARN Unable to reconnect to ZooKeeper service, session 0x3000c2420cb458d has expired (org.apache.zookeeper.ClientCnxn)
      [2018-07-14 04:38:26,444] INFO Unable to reconnect to ZooKeeper service, session 0x3000c2420cb458d has expired, closing socket connection (org.apache.zookeeper.ClientCnxn)
      [2018-07-14 04:38:26,445] INFO EventThread shut down for session: 0x3000c2420cb458d (org.apache.zookeeper.ClientCnxn)
      [2018-07-14 04:38:26,446] INFO [ZooKeeperClient] Session expired. (kafka.zookeeper.ZooKeeperClient)
      [2018-07-14 04:38:26,459] INFO [ZooKeeperClient] Initializing a new session to ZOOKEEPER_SERVER_1:PORT,ZOOKEEPER_SERVER_2:PORT,ZOOKEEPER_SERVER_3:PORT,ZOOKEEPER_SERVER_4:PORT. (kafka.zookeeper.ZooKeeperClient)
      [2018-07-14 04:38:26,459] INFO Initiating client connection, connectString=ZOOKEEPER_SERVER_1:PORT,ZOOKEEPER_SERVER_2:PORT,ZOOKEEPER_SERVER_3:PORT,ZOOKEEPER_SERVER_4:PORT sessionTimeout=6000 watcher=kafka.zookeeper.ZooKeeperClient$ZooKeeperClientWatcher$@44821a96 (org.apache.zookeeper.ZooKeeper)
      [2018-07-14 04:38:26,465] INFO Opening socket connection to server ZOOKEEPER_SERVER_1:PORT. Will not attempt to authenticate using SASL (unknown error) (org.apache.zookeeper.ClientCnxn)
      [2018-07-14 04:38:26,477] INFO Socket connection established to ZOOKEEPER_SERVER_1:PORT, initiating session (org.apache.zookeeper.ClientCnxn)
      [2018-07-14 04:38:26,484] INFO Session establishment complete on server ZOOKEEPER_SERVER_1:PORT, sessionid = 0x4005b59eb6a0000, negotiated timeout = 6000 (org.apache.zookeeper.ClientCnxn)
      [2018-07-14 04:38:26,496] INFO Creating /brokers/ids/1 (is it secure? false) (kafka.zk.KafkaZkClient)
      [2018-07-14 04:38:26,500] INFO Processing notification(s) to /config/changes (kafka.common.ZkNodeChangeNotificationListener)
      [2018-07-14 04:38:26,547] ERROR Error while creating ephemeral at /brokers/ids/1, node already exists and owner '216186131422332301' does not match current session '288330817911521280' (kafka.zk.KafkaZkClient$CheckedEphemeral)
      [2018-07-14 04:38:26,547] INFO Result of znode creation at /brokers/ids/1 is: NODEEXISTS (kafka.zk.KafkaZkClient)
      [2018-07-14 04:38:26,559] ERROR Uncaught exception in scheduled task 'isr-expiration' (kafka.utils.KafkaScheduler)

      org.apache.zookeeper.KeeperException$SessionExpiredException: KeeperErrorCode = Session expired for /brokers/topics/TOPIC_NAME/partitions/PARTITION-#/state
      at org.apache.zookeeper.KeeperException.create(KeeperException.java:127)
      at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
      at kafka.zookeeper.AsyncResponse.resultException(ZooKeeperClient.scala:465)
      at kafka.zk.KafkaZkClient.conditionalUpdatePath(KafkaZkClient.scala:621)
      at kafka.utils.ReplicationUtils$.updateLeaderAndIsr(ReplicationUtils.scala:33)
      at kafka.cluster.Partition.kafka$cluster$Partition$$updateIsr(Partition.scala:669)
      at kafka.cluster.Partition$$anonfun$4.apply$mcZ$sp(Partition.scala:513)
      at kafka.cluster.Partition$$anonfun$4.apply(Partition.scala:504)
      at kafka.cluster.Partition$$anonfun$4.apply(Partition.scala:504)
      at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:250)
      at kafka.utils.CoreUtils$.inWriteLock(CoreUtils.scala:258)
      at kafka.cluster.Partition.maybeShrinkIsr(Partition.scala:503)
      at kafka.server.ReplicaManager$$anonfun$kafka$server$ReplicaManager$$maybeShrinkIsr$2.apply(ReplicaManager.scala:1335)
      at kafka.server.ReplicaManager$$anonfun$kafka$server$ReplicaManager$$maybeShrinkIsr$2.apply(ReplicaManager.scala:1335)
      at scala.collection.Iterator$class.foreach(Iterator.scala:891)
      at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
      at kafka.server.ReplicaManager.kafka$server$ReplicaManager$$maybeShrinkIsr(ReplicaManager.scala:1335)
      at kafka.server.ReplicaManager$$anonfun$2.apply$mcV$sp(ReplicaManager.scala:322)
      at kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:110)
      at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:62)
      at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
      at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
      at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
      at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      at java.lang.Thread.run(Thread.java:748)

      [2018-07-14 04:38:45,938] INFO Partition TOPIC_NAME-PARTITION-# broker=1 Shrinking ISR from 1,3,4,2 to 1 (kafka.cluster.Partition)
      [2018-07-14 04:38:45,952] INFO Partition TOPIC_NAME-PARTITION-# broker=1 Cached zkVersion [0] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
      [2018-07-14 04:38:45,952] INFO Partition __consumer_offsets-# broker=1 Shrinking ISR from 1,2,3,4 to 1 (kafka.cluster.Partition)
      [2018-07-14 04:38:45,992] INFO Partition __consumer_offsets-# broker=1 Cached zkVersion [139] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)

      ...

      ...

       

      The previous exception continues showing constantly without stopping. Since the cluster was still up, I have left the error showing up for about 6 hours to check if the broker recovers itself.

      The only solution, to put back the broker into the cluster, was restarting.

      After the restart, the replicas sync each other and the broker was able to serve again.

      I imagine this is not a normal behavior.

      Attachments

        Issue Links

          Activity

            People

              pachilo Jonathan Santilli
              pachilo Jonathan Santilli
              Votes:
              1 Vote for this issue
              Watchers:
              12 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: