zookeeper-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Fournier, Camille F." <Camille.Fourn...@gs.com>
Subject RE: help on Zookeeper code walk through?
Date Mon, 18 Jul 2011 16:30:38 GMT
The client connection sets two values. One is the negotiatedSessionTimeout, which is the time
that the server will go before timing out a session it has not received a message from. The
second is the readTimeout, which is set after session is established to: readTimeout = negotiatedSessionTimeout
* 2 / 3

What should happen is the following: Imagine the client has a network partition from the ZK
cluster. It will try to ping the cluster in the readTimeout, and detect that it is disconnected.
Your code will then see a disconnected event. Now, you still have 1/3 of the negotiated session
timeout before the session is completely timed out. So while you could have some bit of time
between when the disconnected event happens and the client sees and acts on it, it should
be less than the session timeout, and it is fair for the master to continue to act as the
master during this time because no other server should have taken over as the master yet,
since the master's lock node still exists on the ZK server. Now, if you are in a debugger
let's say, and have paused the event thread while the main thread continues running, then
yes, you can continue to incorrectly act as the master. But as long as your timeouts are reasonable,
the interval between when the disconnected event happens and the master detects the disconnected
event and ceases to act as the master should be long enough that you are never improperly
acting as the master because your node will still exist on the ZK. Note that this does mean
that the actions inside of your master while loop down there need to be able to complete well
within 1/3 of the negotiated session timeout, or else you will act improperly as the master.

The only exception to this I can think of in "normal use" is the following: I receive a disconnected
event, and before processing it I have a full GC that takes so long that the session was fully
expired before it completed. Then, there may possibly be a period after resumption of the
VM where I am acting as the master when in fact I haven't processed a disconnected/session
expired event. 


-----Original Message-----
From: Yang [mailto:teddyyyy123@gmail.com] 
Sent: Monday, July 18, 2011 12:02 PM
To: user@zookeeper.apache.org
Subject: Re: help on Zookeeper code walk through?

Thanks Camille.

now I see that it's the Watcher.Event.KeeperState.Disconnected  event
being generated,
 by the ClientCnxn.SenderThread.run().   .... queueEvent  ,  and then
processed by EventThread.run() .... watcher.process()
it seems that the same scenario I gave above could still happen, i.e.
the ClientCnxn.SenderThread or the EventThread could be stopped and
the main application thread keeps going happily along .  though this
is a very slight possibility, theoretically it is still possible.
or am I missing something?

Thanks
Yang


On Mon, Jul 18, 2011 at 6:51 AM, Fournier, Camille F.
<Camille.Fournier@gs.com> wrote:
> If the zk cluster doesn't get pings from your existing master, the zk client on that
master should see a disconnected state event, not a node deletion event. Upon seeing that
event, it should stop acting as master until such time as it can determine whether it has
reconnected and is still master, or it reconnects and sees that its original session has failed
or the master node is deleted.
>
> C
>
>
>
> ----- Original Message -----
> From: Yang <teddyyyy123@gmail.com>
> To: user@zookeeper.apache.org <user@zookeeper.apache.org>
> Sent: Mon Jul 18 04:00:04 2011
> Subject: Re: help on Zookeeper code walk through?
>
> Thanks  Camille and Ben.
>
> I get the basic picture now.
>
> I have another question: in a leader election scenario (for example
> HBase Master election), I want to make sure that at any time ,  there
> is only at most one node running as master, and there is indeed one
> running as master all the time except for very short failover time
> period.
>
> then if only the connection between current master and ZK  is down,
> ZK senses the lack of pings, and kills the session and ephemeral child
> node owned by the leader, and the next client node kicks in as leader.
> at this time, if the current leader machine is still working fine, its
> traffic going out to the its application servers as normal, would it
> be blissfully still acting as a leader, and violate our "single
> master" goal?   for example if the Watcher.process()  catches the
> nodeDelete event, and tries to set some var to stop the application
> server, but if this thread is stopped before the var is set, and is
> never invoked again, then the application server could just keep
> happily going along...?
>
> for example, the following dummy code
>
> class MyApplication {
>    volatile boolean should_stop = false;
>    class MyZKWatcher implements Zookeeper.Watcher {
>             public void process(Event e) {
>                  if ( e is nodeDelete of my owner node ) {
>                           should_stop = true ;  //*************
>                  }
>    }
>
>    public void runApp() {
>          zk = new ZooKeeper(hostPort, 3000, this);
>          while ( ! should_shop ) {
>               send_out_some_messages to my application servers
> assuming I'm leader
>          }
>   }
>
>   public static void main(String args[]) {
>      new MyApplication().runApp();
>  }
> }
>
>
> basically if the nodeDelete event is caught but the Watcher stops
> right at "//*****" line , then the
> application main loop could still be going on?? otherwise I have to
> put a node exists() check before I send out every application message?
>
>
> Thanks  a lot
> Yang
> 7 PM, Benjamin Reed <breed@apache.org> wrote:
>> if you are running with multiple servers, it is the leader that
>> declares sessions dead, so the leader will call killSession(). the
>> followers track the liveness of the clients with pings and will
>> periodically send liveness summaries to the leader.
>>
>> see camille's email the specific classes to look at.
>>
>> ben
>>
>> On Sat, Jul 16, 2011 at 1:44 AM, Yang <teddyyyy123@gmail.com> wrote:
>>> I'm wondering if a client loses session to its ephemeral znode, under
>>> the hood, how
>>> is the watcher triggered?
>>>
>>> went through the code , and found something that looks related:
>>> ZKDataBase.killSession()-->DataTree.killSession()--->DataTree.deleteNode()--->WatchManager.triggerWatch()--->Watcher.process()
>>>
>>> but how is ZKDataBase.killSession() called?  from the info given in
>>> http://zookeeper.apache.org/doc/r3.3.3/zookeeperProgrammers.html#ch_zkSessions
>>>  I can see the ZooKeeper client code does periodically ping the server
>>> to maintain liveness. but how the server checks for this liveness and
>>> trigger killSession(), here I'm having difficulty connecting the dots.
>>>
>>> could you please give me some help walking through this piece of code?
>>>
>>> Thanks
>>> Yang
>>>
>>
>

Mime
View raw message