I am sorry that I still didn’t get it.  Let me post my scenarios again

1.  Client A connected to Leader with session S
2.  Session S expired on Leader, which according to the code below it will set 
the session as closing
3.  Leader then send out a close session request to its own first processor 
which will go through the usual pipeline as you mentioned
4. Client A send a request which will go through the checkSession code I pasted 
below and it should get a SessionExpiredException.
5. Client now knows its session expired but for some reason it tries to connect 
to another server B and issues a read.

Now this is a race between the following two chain of operations

1. The closesession request needs to go through quorum and get the majority and 
server B get the commit (or inform) request and actually kill the session in 
the final request processor

2. The read operation that goes through learnerZK process chain in memory (as 
far as I see, there is no session check on read at all so it won’t know until 
it reads the tree)

The first chain can get stuck on a various places like some quorum nodes are 
stuck on some other proposals so it looks to me that the second chain is very 
much likely to win as it does not need to go through any quorum operation. The 
key issue is the client needs to reconnect fast (or in parallel as the original 
post seems to indicate). I think I can definitely simulate this with a test but 
it will be tricky to make it pass/fail deterministically so I didn’t try.


you seems to agree that my step 5 will happen if the client connect with a 
valid session ID so there won’t be any quorum operation before establishing the 
session. So the race after that can happen, why not?  Also, there is no session 
check for read operation on the learners as far as I can see (please point me 
to the code if you find it). Furthermore,  the mark the session closing code I 
posted only run on the lead as far as I can see (again, please point me to the 
code)

Just to repeat, the race is between the learner gets the quorum closeSession 
and the client issue a read. No?


On Feb 8, 2017, at 9:03 PM, Michael Han 
<h...@cloudera.com<mailto:h...@cloudera.com>> wrote:

Hi Ryan,

>> From the code I saw,  there is no quorum operation when a ZK reconnect with 
>> a session ID

The "createSession(cnxn, passwd, sessionTimeout)" will submit a session create 
request that will go through processor pipeline, and that is where the quorum 
operation is involved. Also note that while we are creating a new session for 
client, we disabled server socket connection appertains to this client, so 
server would not process any requests from the client, until the session is 
finalized (we then enable the same socket connection).

There is another case here - covered by the "else" clause, which is when the 
client request embeds a valid session ID. If at this point in time, ZooKeeper 
has already started the session clean up for this particular session ID, then 
the session will be marked as closing. As you noticed previously, ZK validates 
session for every operation (even the read ones), and a session in the state of 
"closing" will not pass validation. On client side it will receive 
SessionExpire event (as a result of failed session validation) for the read 
requests it previously issued on ephemeral nodes, so our invariant still holds.

Hope these help.


On Wed, Feb 8, 2017 at 11:55 AM, Ryan Zhang 
<yangzhangr...@hotmail.com<mailto:yangzhangr...@hotmail.com>> wrote:
Hi, Michael, I am sorry that I spoke too soon.  From the code I saw,  there is 
no quorum operation when a ZK reconnect with a session ID (pasted the code 
below). Did I miss something again?


long sessionId = connReq.getSessionId();
if (sessionId == 0) {
    LOG.info("Client attempting to establish new session at "
            + cnxn.getRemoteSocketAddress());
    createSession(cnxn, passwd, sessionTimeout);
} else {
    long clientSessionId = connReq.getSessionId();
    LOG.info("Client attempting to renew session 0x"
            + Long.toHexString(clientSessionId)
            + " at " + cnxn.getRemoteSocketAddress());
    if (serverCnxnFactory != null) {
        serverCnxnFactory.closeSession(sessionId);
    }
    if (secureServerCnxnFactory != null) {
        secureServerCnxnFactory.closeSession(sessionId);
    }
    cnxn.setSessionId(sessionId);
    reopenSession(cnxn, sessionId, passwd, sessionTimeout);
}




On Feb 7, 2017, at 3:46 PM, Ryan Zhang 
<yangzhangr...@hotmail.com<mailto:yangzhangr...@hotmail.com>> wrote:

Thank you very much Michael, I am now clear.

On Feb 7, 2017, at 3:21 PM, Michael Han 
<h...@cloudera.com<mailto:h...@cloudera.com><mailto:h...@cloudera.com>> wrote:

Hi Ryan,

Thanks for the elaborations. I see why you are confused. Your case is valid, 
except in step 5:

Client now knows its session expired but for some reason it tries to connect to 
another server B and issues a read.

At this point of time, there are two possible cases:
* Client A's session has been cleaned up. As part of the session clean up, the 
ephemeral nodes appertain with A will be gone. After A reconnect to the quorum, 
it will not see the old ephemeral nodes.
* Client A's session has not finished cleaned up. In this case, if client A is 
trying to issue a read, it has to first establish a new session with quorum. 
Because the session clean up transaction of A is already in flight, and because 
of the sequential consistency guarantee provided by ZooKeeper, client A will 
not be able to reconnect to the quorum / establish the session, until the 
previous session up transaction has finished. So we will be back to first case 
here, which also guarantee the old ephemeral nodes are gone.


On Tue, Feb 7, 2017 at 2:02 PM, Ryan Zhang 
<yangzhangr...@hotmail.com<mailto:yangzhangr...@hotmail.com><mailto:yangzhangr...@hotmail.com>>
 wrote:
Hi, Michael,
Thank you for your clarification. What you said seems to be consistent with 
what I saw in the code. However, I am still confused by your conclusion. My 
conclusion from the code is that it is totally possible for a client to get 
session expired and reconnect to another sever to still see the ephemeral node. 
The exact reason is caused by the code I pasted below.  I am not sure if we are 
on the same page as you seem to suggest that it is not possible. Let me 
elaborate a bit on how this can happen.

1.  Client A connected to Leader with session S
2.  Session S expired on Leader, which according to the code below it will set 
the session as closing
3.  Leader then send out a close session request to its own first processor 
which will go through the usual pipeline as you mentioned
4. Client A send a request which will go through the checkSession code I pasted 
below and it should get a SessionExpiredException.
5. Client now knows its session expired but for some reason it tries to connect 
to another server B and issues a read.

Now this is a race between the following two chain of operations

1. The closesession request needs to go through quorum and get the majority and 
server B get the commit (or inform) request and actually kill the session in 
the final request processor

2. The read operation that goes through learnerZK process chain in memory (as 
far as I see, there is no session check on read at all so it won’t know until 
it reads the tree)

The first chain can get stuck on a various places like some quorum nodes are 
stuck on some other proposals so it looks to me that the second chain is very 
much likely to win as it does not need to go through any quorum operation. The 
key issue is the client needs to reconnect fast (or in parallel as the original 
post seems to indicate). I think I can definitely simulate this with a test but 
it will be tricky to make it pass/fail deterministically so I didn’t try.

Am I missing something?

-Ryan



On Feb 7, 2017, at 1:24 PM, Michael Han 
<h...@cloudera.com<mailto:h...@cloudera.com><mailto:h...@cloudera.com>> wrote:

Hi Ryan,

I am not sure what you were confused about regarding session cleaning up code. 
Here is my understanding, hope it helps.

* Session clean up is started from marking the state of a session as closed, as 
you noticed. This is because each session clean up will take a while so we need 
make sure that during session clean up, server will not continue processing the 
requests from the client that appertain with this session.

* Once session is marked as closing, we will send a request so the closing of 
the session not only applies to the leader but also on the quorum servers. This 
is just like all other requests that will go through the normal requests 
process pipeline.

* A valid session is a prerequisite for any of the client operations (including 
read operations), so the liveness of the session is validated before processing 
a read operation.


On Wed, Jan 25, 2017 at 2:25 PM, Ryan Zhang 
<yangzhangr...@hotmail.com<mailto:yangzhangr...@hotmail.com><mailto:yangzhangr...@hotmail.com>>
 wrote:
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><mailto:h...@cloudera.com><mailto: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><mailto:yangzhangr...@hotmail.com><mailto: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:jun...@gmail.com><mailto:jun...@gmail.com<mailto:jun...@gmail.com>><mailto:junra<mailto:junra><mailto:junra<mailto:junra>>
o...@gmail.com<mailto:o...@gmail.com><mailto:o...@gmail.com><mailto: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.




--
Cheers
Michael.




--
Cheers
Michael.





--
Cheers
Michael.

Reply via email to