[ 
https://issues.apache.org/jira/browse/KAFKA-4277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16366008#comment-16366008
 ] 

Nico Meyer commented on KAFKA-4277:
-----------------------------------

We also just encountered the same problem. Kafka 0.11.0.1 and ZooKeeper 3.4.6.

It seems that Zookeeper does not guarantee that the ephemeral nodes are gone 
after a session expired. At least not if one connects to a different Zookeeper 
node, which is exactly what happened in our case:
{code:java}
2018-02-15 13:43:03,464 INFO  zookeeper.ClientCnxn 
(ClientCnxn.java:logStartConnect(1032)) - Opening socket connection to server 
172.20.0.206/172.20.0.206:2181. Will not attempt to authenticate using SASL 
(unknown error)
2018-02-15 13:43:04,462 INFO  zookeeper.ClientCnxn 
(ClientCnxn.java:primeConnection(876)) - Socket connection established to 
172.20.0.206/172.20.0.206:2181, initiating session
2018-02-15 13:43:04,549 WARN  zookeeper.ClientCnxn 
(ClientCnxn.java:onConnected(1285)) - Unable to reconnect to ZooKeeper service, 
session 0x1614ad433b00b17 has expired
2018-02-15 13:43:04,549 INFO  zookeeper.ClientCnxn (ClientCnxn.java:run(1154)) 
- Unable to reconnect to ZooKeeper service, session 0x1614ad433b00b17 has 
expired, closing socket connection
2018-02-15 13:43:04,549 INFO  zkclient.ZkClient 
(ZkClient.java:processStateChanged(713)) - zookeeper state changed (Expired)
2018-02-15 13:43:04,550 INFO  zookeeper.ZooKeeper (ZooKeeper.java:<init>(438)) 
- Initiating client connection, 
connectString=172.20.0.215:2181,172.20.0.244:2181,172.20.0.204:2181,172.20.0.206:2181,172.20.0.208:2181/3rdparty/kafka08
 sessionTimeout=6000 watcher=org.I0Itec.zkclient.ZkClient@199bd995
2018-02-15 13:43:04,550 INFO  zookeeper.ClientCnxn (ClientCnxn.java:run(519)) - 
EventThread shut down for session: 0x1614ad433b00b17
2018-02-15 13:43:04,552 INFO  zookeeper.ClientCnxn 
(ClientCnxn.java:logStartConnect(1032)) - Opening socket connection to server 
172.20.0.204/172.20.0.204:2181. Will not attempt to authenticate using SASL 
(unknown error)
2018-02-15 13:43:04,553 INFO  zookeeper.ClientCnxn 
(ClientCnxn.java:primeConnection(876)) - Socket connection established to 
172.20.0.204/172.20.0.204:2181, initiating session
2018-02-15 13:43:04,553 INFO  zookeeper.ClientCnxn (ClientCnxn.java:run(1158)) 
- Unable to read additional data from server sessionid 0x0, likely server has 
closed socket, closing socket connection and attempting reconnect
2018-02-15 13:43:04,699 INFO  zkclient.ZkClient 
(ZkClient.java:waitForKeeperState(936)) - Waiting for keeper state SyncConnected
2018-02-15 13:43:04,699 INFO  zkclient.ZkClient 
(ZkClient.java:waitForKeeperState(936)) - Waiting for keeper state SyncConnected
2018-02-15 13:43:05,373 INFO  zookeeper.ClientCnxn 
(ClientCnxn.java:logStartConnect(1032)) - Opening socket connection to server 
172.20.0.206/172.20.0.206:2181. Will not attempt to authenticate using SASL 
(unknown error)
2018-02-15 13:43:05,374 INFO  zookeeper.ClientCnxn 
(ClientCnxn.java:primeConnection(876)) - Socket connection established to 
172.20.0.206/172.20.0.206:2181, initiating session
2018-02-15 13:43:06,575 WARN  zookeeper.ClientCnxn (ClientCnxn.java:run(1108)) 
- Client session timed out, have not heard from server in 1201ms for sessionid 
0x0
2018-02-15 13:43:06,575 INFO  zookeeper.ClientCnxn (ClientCnxn.java:run(1156)) 
- Client session timed out, have not heard from server in 1201ms for sessionid 
0x0, closing socket connection and attempting reconnect
2018-02-15 13:43:07,180 INFO  zookeeper.ClientCnxn 
(ClientCnxn.java:logStartConnect(1032)) - Opening socket connection to server 
172.20.0.215/172.20.0.215:2181. Will not attempt to authenticate using SASL 
(unknown error)
2018-02-15 13:43:07,180 INFO  zookeeper.ClientCnxn 
(ClientCnxn.java:primeConnection(876)) - Socket connection established to 
172.20.0.215/172.20.0.215:2181, initiating session
2018-02-15 13:43:07,182 INFO  zookeeper.ClientCnxn 
(ClientCnxn.java:onConnected(1299)) - Session establishment complete on server 
172.20.0.215/172.20.0.215:2181, sessionid = 0x161997d55350158, negotiated 
timeout = 6000
2018-02-15 13:43:07,182 INFO  zkclient.ZkClient 
(ZkClient.java:processStateChanged(713)) - zookeeper state changed 
(SyncConnected)
2018-02-15 13:43:07,184 INFO  server.KafkaHealthcheck$SessionExpireListener 
(Logging.scala:info(70)) - re-registering broker info in ZK for broker 11
2018-02-15 13:43:07,203 INFO  utils.ZKCheckedEphemeral (Logging.scala:info(70)) 
- Creating /brokers/ids/11 (is it secure? false)
2018-02-15 13:43:07,246 INFO  utils.ZKCheckedEphemeral (Logging.scala:info(70)) 
- Result of znode creation is: NODEEXISTS
2018-02-15 13:43:07,304 ERROR zkclient.ZkEventThread 
(ZkEventThread.java:run(78)) - Error handling event ZkEvent[New session event 
sent to kafka.server.KafkaHealthcheck$SessionExpireListener@bb373b3]
java.lang.RuntimeException: A broker is already registered on the path 
/brokers/ids/11. 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:417)
        at kafka.utils.ZkUtils.registerBrokerInZk(ZkUtils.scala:403)
        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:736)
        at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:72){code}
 

Afterward a lot of these are logged until restart:
{code:java}
2018-02-15 13:45:56,322 INFO  cluster.Partition (Logging.scala:info(70)) - 
Partition [access_log,40] on broker 11: Shrinking ISR from 6,5,11 to 6,11
2018-02-15 13:45:56,328 INFO  cluster.Partition (Logging.scala:info(70)) - 
Partition [access_log,40] on broker 11: Cached zkVersion [1411] not equal to 
that in zookeeper, skip updating ISR

{code}
 

 

> 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
>            Priority: Major
>
> 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
(v7.6.3#76005)

Reply via email to