You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by "Michael Han (JIRA)" <ji...@apache.org> on 2018/06/01 22:00:00 UTC

[jira] [Commented] (ZOOKEEPER-2919) expired ephemeral node reappears after ZK leader change

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

Michael Han commented on ZOOKEEPER-2919:
----------------------------------------

[~junrao] I think this is a bug in ZK. We promised (as [~fpj] also pointed in the [email thread|http://mail-archives.apache.org/mod_mbox/zookeeper-user/201701.mbox/%3CB512F6DE-C0BF-45CE-8102-6F242988268E%40apache.org%3E]) that {quote} Receiving a session expiration means
that a closeSession has been committed {quote}, but this is not true in every case, and sometimes even when a client receives session expire event it's possible the session close is not committed across the quorum. This is because we don't differentiate "session closing" with "session closed" state internally at ZK. Between a session starts expiring and the session expired, a client will receive session expire for its requests to server but if the quorum is down at the same time, it's possible the close session is not committed.

Here is a canonical example similar to this Kafka case: 

Client A connects to Server 1. Server 1, 2 and 3 forms an ensemble.
1. For some reasons like network interruption, Server 1 thinks A is dead, so it starts expiring the session. It starts by setting the "is Closing" property of session object associated with A to true.
2. A fires a request to Server 1 (and network heals) and gets back session expire event. A thinks its session is expired but actually the session expiration was not finished yet. A gave up and starts a new session.
3. Meanwhile, Server 1 proposes to close the old session of A and at the same time, the quorum is down (1 was leader but got partitioned from 2 and 3). The proposal never landed at Server 2 and Server 3. 
4. 2 and 3 can talk, and they elects 2 as leader, who still has old session of A.
5. Network heals, 1 reconnects to ensemble, and found there is already a leader 2, sync with 2, truncate its log when needed.
6. 2 now happily accepts a new session request from A.
7. Eventually 2 expired old session object because there was not active client associates with that old session. Now between the new session established and the old session expiration, the old ephemeral nodes will be visible to clients.

There are two approaches to solve this issue. 

1. ZK users (e.g. Kafka ZK util client) could check the presence of the old ephemeral (either through polling or register a watcher) before trying recreate another ephemeral with same old path.
2. ZK fixes the semantic of session expire so when a client sees session expired, the session is really expired (instead of expiring). 

I'll propose a fix for 2, starting with a test case that can deterministically reproduce this issue.

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