[ https://issues.apache.org/jira/browse/KAFKA-4277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15866415#comment-15866415 ]
Wrikken commented on KAFKA-4277: -------------------------------- Reproduced on our side with Scala 2.11 kafka_2.11-0.10.1.1, zookeeper v. 3.4.5 (3.4.5+dfsg-2+deb8u1 from Debian Jessie). [2017-02-13 06:05:11,793] INFO re-registering broker info in ZK for broker 1 (kafka.server.KafkaHealthcheck$SessionExpireListener) [2017-02-13 06:05:11,795] INFO Creating /brokers/ids/1 (is it secure? false) (kafka.utils.ZKCheckedEphemeral) [2017-02-13 06:05:11,797] INFO Partition [com.takeaway.events.order.create,5] on broker 1: Shrinking ISR for partition [com.takeaway.events.order.create,5] from 3,1 to 1 (kafka.cluster.Partition) [2017-02-13 06:05:11,801] INFO Partition [__consumer_offsets,31] on broker 1: Shrinking ISR for partition [__consumer_offsets,31] from 3,2,1 to 2,1 (kafka.cluster.Partition) [2017-02-13 06:05:11,801] INFO Result of znode creation is: NODEEXISTS (kafka.utils.ZKCheckedEphemeral) The more annoying issue is I can deal with Kafka fully failing at that point, which would prompt a restart. However, currently it 'silently' fails, but keeps running, forcing us to set up extra monitoring to force a restart when this happens. > creating ephemeral node already exist > ------------------------------------- > > Key: KAFKA-4277 > URL: https://issues.apache.org/jira/browse/KAFKA-4277 > Project: Kafka > Issue Type: Bug > Affects Versions: 0.10.0.0 > Reporter: Feixiang Yan > > I use zookeeper 3.4.6. > Zookeeper session time out, zkClient try reconnect failed. Then re-establish > the session and re-registering broker info in ZK, throws NODEEXISTS Exception. > I think it is because the ephemeral node which created by old session has > not removed. > I read the > [ZkUtils.scala|https://github.com/apache/kafka/blob/0.8.1/core/src/main/scala/kafka/utils/ZkUtils.scala] > of 0.8.1, createEphemeralPathExpectConflictHandleZKBug try create node in a > while loop until create success. This can solve the issue. But in > [ZkUtils.scala|https://github.com/apache/kafka/blob/0.10.0.1/core/src/main/scala/kafka/utils/ZkUtils.scala] > 0.10.1 the function removed. > {noformat} > [2016-10-07 19:00:32,562] INFO Socket connection established to > 10.191.155.238/10.191.155.238:21819, initiating session > (org.apache.zookeeper.ClientCnxn) > [2016-10-07 19:00:32,563] INFO zookeeper state changed (Expired) > (org.I0Itec.zkclient.ZkClient) > [2016-10-07 19:00:32,564] INFO Unable to reconnect to ZooKeeper service, > session 0x1576b11f9b201bd has expired, closing socket connection > (org.apache.zookeeper.ClientCnxn) > [2016-10-07 19:00:32,564] INFO Initiating client connection, > connectString=10.191.155.237:21819,10.191.155.238:21819,10.191.155.239:21819/cluster2 > sessionTimeout=6000 watcher=org.I0Itec.zkclient.ZkClient@ae71be2 > (org.apache.zookeeper.ZooKeeper) > [2016-10-07 19:00:32,566] INFO Opening socket connection to server > 10.191.155.237/10.191.155.237:21819. Will not attempt to authenticate using > SASL (unknown error) (org.apache.zookeeper.ClientCnxn) > [2016-10-07 19:00:32,566] INFO Socket connection established to > 10.191.155.237/10.191.155.237:21819, initiating session > (org.apache.zookeeper.ClientCnxn) > [2016-10-07 19:00:32,566] INFO EventThread shut down > (org.apache.zookeeper.ClientCnxn) > [2016-10-07 19:00:32,567] INFO Session establishment complete on server > 10.191.155.237/10.191.155.237:21819, sessionid = 0x1579ecd39c20006, > negotiated timeout = 6000 (org.apache.zookeeper.ClientCnxn) > [2016-10-07 19:00:32,567] INFO zookeeper state changed (SyncConnected) > (org.I0Itec.zkclient.ZkClient) > [2016-10-07 19:00:32,608] INFO re-registering broker info in ZK for broker 3 > (kafka.server.KafkaHealthcheck$SessionExpireListener) > [2016-10-07 19:00:32,610] INFO Creating /brokers/ids/3 (is it secure? false) > (kafka.utils.ZKCheckedEphemeral) > [2016-10-07 19:00:32,611] INFO Result of znode creation is: NODEEXISTS > (kafka.utils.ZKCheckedEphemeral) > [2016-10-07 19:00:32,614] ERROR Error handling event ZkEvent[New session > event sent to kafka.server.KafkaHealthcheck$SessionExpireListener@324f1bc] > (org.I0Itec.zkclient.ZkEventThread) > java.lang.RuntimeException: A broker is already registered on the path > /brokers/ids/3. This probably indicates that you either have configured a > brokerid that is already in use, or else you have shutdown this broker and > restarted it faster than the zookeeper timeout so it appears to be > re-registering. > at kafka.utils.ZkUtils.registerBrokerInZk(ZkUtils.scala:305) > at kafka.utils.ZkUtils.registerBrokerInZk(ZkUtils.scala:291) > at kafka.server.KafkaHealthcheck.register(KafkaHealthcheck.scala:70) > at > kafka.server.KafkaHealthcheck$SessionExpireListener.handleNewSession(KafkaHealthcheck.scala:104) > at org.I0Itec.zkclient.ZkClient$6.run(ZkClient.java:735) > at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) > {noformat} -- This message was sent by Atlassian JIRA (v6.3.15#6346)