To amplify slightly, there are a range of possible strategies you can use in
the disconnect scenario.

One, is to have a special "master in waiting" state that is entered as soon
as a disconnect is seen.  This is the most common strategy.

A second option is to continue operating as master for a some part of the
session expiration period.  This is appropriate when not having a master is
a really bad thing and you want to minimize the time that this can happen.
 Of course, if you can't reach ZK, the chances that you can function as a
master are somewhat limited.

A variant on the second option would be to serve as a read-only master
during the time period between the disconnect event and the estimated
session expiration.

The second and third options imply that you trust your clock which may be a
bad assumption in a virtual environment like EC2.

On Mon, Jul 18, 2011 at 6:51 AM, Fournier, Camille F. <
[email protected]> 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 <[email protected]>
> To: [email protected] <[email protected]>
> 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 <[email protected]> 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 <[email protected]> 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
> >>
> >
>

Reply via email to