[ https://issues.apache.org/jira/browse/ZOOKEEPER-2919?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Michael Han reassigned ZOOKEEPER-2919: -------------------------------------- Assignee: Michael Han > expired ephemeral node reappears after ZK leader change > ------------------------------------------------------- > > Key: ZOOKEEPER-2919 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2919 > Project: ZooKeeper > Issue Type: Bug > Affects Versions: 3.4.9 > Reporter: Jun Rao > Assignee: Michael Han > Attachments: zk-myid-4.log, zk-myid-5.log > > > We found the following issue when using ZK. A client (a Kafka broker) > registered an ephemeral node in ZK. The client then received a session > expiration event and created the new session. The client tried to create the > same ephemeral node in ZK in the new session but received a > NodeExistException. The following are the details. > From Kafka broker 1: > Broker 1 received the expiration of session 55bcff0f02d0002 at 13:33:26. > {code:java} > [2017-07-29 13:33:26,706] INFO Unable to reconnect to ZooKeeper service, > session 0x55bcff0f02d0002 has expired, closing socket connection > (org.apache.zookeeper.ClientCnxn) > {code} > It then established a new session 55d8f690ca20038 at 13:33:33. > {code:java} > [2017-07-29 13:33:33,405] INFO Session establishment complete on server > rdalnydbbdqs10/10.122.104.12:2181, sessionid = 0x55d8f690ca20038, negotiated > timeout = 6000 (org.apache.zookeeper.ClientCnxn) > {code} > However, the re-registration of the broker id fails. > {code:java} > [2017-07-29 13:33:33,408] INFO Result of znode creation is: NODEEXISTS > (kafka.utils.ZKCheckedEphemeral) > [2017-07-29 13:33:33,408] ERROR Error handling event ZkEvent[New session > event sent to kafka.server.KafkaHealthcheck$SessionExpireListener@74ad6d14] > (org.I0Itec.zkclient.ZkEvent > Thread) > java.lang.RuntimeException: A broker is already registered on the path > /brokers/ids/1. 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:375) > at kafka.utils.ZkUtils.registerBrokerInZk(ZkUtils.scala:361) > at kafka.server.KafkaHealthcheck.register(KafkaHealthcheck.scala:71) > at > kafka.server.KafkaHealthcheck$SessionExpireListener.handleNewSession(KafkaHealthcheck.scala:105) > at org.I0Itec.zkclient.ZkClient$6.run(ZkClient.java:736) > at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:72) > {code} > From ZK server (my id 4) : > It expired the old session 55bcff0f02d0002 correctly before broker received > the session expiration. It then went to ZK leader election soon after. > {code:java} > [2017-07-29 13:33:26,000] INFO Expiring session 0x55bcff0f02d0002, timeout of > 6000ms exceeded (org.apache.zookeeper.server.ZooKeeperServer) > [2017-07-29 13:33:26,019] INFO Processed session termination for sessionid: > 0x55bcff0f02d0002 (org.apache.zookeeper.server.PrepRequestProcessor) > [2017-07-29 13:33:33,582] INFO Shutting down > (org.apache.zookeeper.server.quorum.CommitProcessor) > [2017-07-29 13:33:34,344] INFO New election. My id = 4, proposed > zxid=0x5830d1163b (org.apache.zookeeper.server.quorum.FastLeaderElection) > [2017-07-29 13:34:22,499] INFO FOLLOWING - LEADER ELECTION TOOK - 48915 > (org.apache.zookeeper.server.quorum.Learner) > {code} > From ZK server (my id 5): > It lost the connection to the old session 55bcff0f02d0002 before the session > got expired. It then went into ZK leader election and became the leader. > However, it didn't think the old session 55bcff0f02d0002 was expired after > becoming the leader. Therefore, the new session 55d8f690ca20038 failed to > create /brokers/ids/1. Only after that, it eventually expired the old session > 55bcff0f02d0002. > {code:java} > [2017-07-29 13:33:24,216] WARN caught end of stream exception > (org.apache.zookeeper.server.NIOServerCnxn) > EndOfStreamException: Unable to read additional data from client sessionid > 0x55bcff0f02d0002, likely client has closed socket > at > org.apache.zookeeper.server.NIOServerCnxn.doIO(NIOServerCnxn.java:228) > at > org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:203) > at java.lang.Thread.run(Thread.java:745) > 2017-07-29 13:33:24,216] INFO Closed socket connection for client > /10.122.73.147:59615 which had sessionid 0x55bcff0f02d0002 > (org.apache.zookeeper.server.NIOServerCnxn) > [2017-07-29 13:33:30,921] INFO New election. My id = 5, proposed > zxid=0x5830d1113f (org.apache.zookeeper.server.quorum.FastLeaderElection) > [2017-07-29 13:33:31,126] INFO LEADING - LEADER ELECTION TOOK - 1122 > (org.apache.zookeeper.server.quorum.Leader) > [2017-07-29 13:33:33,405] INFO Established session 0x55d8f690ca20038 with > negotiated timeout 6000 for client /10.122.73.147:47106 > (org.apache.zookeeper.server.ZooKeeperServer) > [2017-07-29 13:33:33,407] INFO Got user-level KeeperException when processing > sessionid:0x55d8f690ca20038 type:create cxid:0x5 zxid:0x5900000352 txntype:-1 > reqpath:n/a Error Path:/brokers/ids/1 Error:KeeperErrorCode = NodeExists for > /brokers/ids/1 (org.apache.zookeeper.server.PrepRequestProcessor) > [2017-07-29 13:33:40,002] INFO Expiring session 0x55bcff0f02d0002, timeout of > 6000ms exceeded (org.apache.zookeeper.server.ZooKeeperServer) > [2017-07-29 13:33:40,074] INFO Processed session termination for sessionid: > 0x55bcff0f02d0002 (org.apache.zookeeper.server.PrepRequestProcessor) > {code} > According to > http://mail-archives.apache.org/mod_mbox/zookeeper-user/201701.mbox/%3CB512F6DE-C0BF-45CE-8102-6F242988268E%40apache.org%3E > from [~fpj], a ZK client in a new session shouldn't see the ephemeral node > created in its previous session. So, could this be a potential bug in ZK > during ZK leader transition? -- This message was sent by Atlassian JIRA (v6.4.14#64029)