I am a bit confused by the code

On Jan 25, 2017, at 1:33 PM, Michael Han 
<h...@cloudera.com<mailto:h...@cloudera.com>> wrote:

Does ZK guarantee that ephemeral nodes from a client are removed on the
sever by the time the client receives a session expiration event?

"the server" is a vague definition, as ZooKeeper ensemble is composed of
multiple servers :).

Therefore, it seems to be possible for a client to connect to another
server to see the node there.

This seems the only case I can think of that lead to the inconsistent view
from client side. I'll elaborate as follows, first the guarantees of
ZooKeeper that's relevant to this case:


* ZooKeeper quorum should have already committed the transaction of closing
the session when a client receives the session expire event.


Here are the code that throws KeeperException.SessionExpiredException

public synchronized void checkSession(long sessionId, Object owner)

        throws KeeperException.SessionExpiredException,
        KeeperException.SessionMovedException,
        KeeperException.UnknownSessionException {
    ……….
    if (session.isClosing()) {
        throw new KeeperException.SessionExpiredException();
    }
}

Here is the code to set it to be closing directly

synchronized public void setSessionClosing(long sessionId) {
    if (LOG.isTraceEnabled()) {
        LOG.trace("Session closing: 0x" + Long.toHexString(sessionId));
    }
    SessionImpl s = sessionsById.get(sessionId);
    if (s == null) {
        return;
    }
    s.isClosing = true;
}

and here is the code that call the above
public void runImpl() throws InterruptedException {

    while (running) {
       ……...

        for (SessionImpl s : sessionExpiryQueue.poll()) {
            setSessionClosing(s.sessionId);
            expirer.expire(s);
        }
    }
}

the expire function looks like this

public void expire(Session session) {
    long sessionId = session.getSessionId();

    close(sessionId);
}

and close function here

private void close(long sessionId) {
    Request si = new Request(null, sessionId, 0, OpCode.closeSession, null, 
null);
    setLocalSessionFlag(si);
    submitRequest(si);
}

so it looks to me that the session is marked as closing first and then the 
closeSession is send. This will happen on
the lead only though as the checkSession is only called on the lead but it is 
called even on read operation.

May I missing something?


* Clean up of ephemeral nodes associated with the session is part of the
closing session transaction, so for the quorum of servers who have already
committed the transaction, the ephemeral nodes should have gone already, on
those servers.

* ZooKeeper quorum would not have processed the new session establishment
request for the same client, until after the closing session request has
been processed because transactions are ordered across quorum.

Given these guarantees, if a client reestablishes a new session via
connecting to a server which was the quorum of servers that committed the
closing session transaction, then the client should not see the old
ephemeral node upon new session established.

ZooKeeper does not guarantee a write transaction occur synchronously across
all of the servers, since a write request only requires a quorum of servers
to acknowledge. As a result, it is valid that some servers might lag behind
the state of the quorum. I suspect this case is possible:

* Client receives session expire event, and client close its connection to
server A.

* Client reconnects to server B, which lags behind quorum, that does not
contain the changes to the data tree regarding ephemeral nodes.

* Client sees the ephemeral node so it does nothing. Later the node is
cleaned up when server B sync with quorum.

Client can ensure it always see the state of truth of the quorum by issuing
a sync() request before issuing a read request. A sync request will force
the server it's connecting to sync with the quorum. If Kafka does this,
will the bug go away? Of course, retry creating ephemeral nodes can also
solve the problem (there are possible other solutions as well, by having
client to do some book keeping work to differentiate versions between
ephemeral nodes).


On Wed, Jan 25, 2017 at 11:32 AM, Ryan Zhang 
<yangzhangr...@hotmail.com<mailto:yangzhangr...@hotmail.com>>
wrote:

Good question, AFAIK, it’s not the case.

The server will throw an SessionExpiredException during checkSession call
as soon as the session is marked as isClosing. However, session expiration
actually requires a transaction (of type OpCode.closeSession) which will be
send to the leader to go through the quorum.  The session and ephemeral
node will only be removed after the transaction is  committed and processed
in the final processor on other nodes. Therefore, it seems to be possible
for a client to connect to another server to see the node there. I am not
entirely sure if it can use the same session id though, it seems possible
as the session close is only based on the session expire time and there can
be delays in session pings.

On Jan 25, 2017, at 8:53 AM, Jun Rao 
<jun...@gmail.com<mailto:jun...@gmail.com><mailto:junra
o...@gmail.com<mailto:o...@gmail.com>>> wrote:

Hi,

Does ZK guarantee that ephemeral nodes from a client are removed on the
sever by the time the client receives a session expiration event? I am
getting conflicting info on this (
https://issues.apache.org/jira/browse/KAFKA-4277). Could someone clarify?

Thanks,

Jun




--
Cheers
Michael.

Reply via email to