zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Patrick Hunt (JIRA)" <j...@apache.org>
Subject [jira] Created: (ZOOKEEPER-79) Document jacob's leader election on the wiki recipes page
Date Thu, 17 Jul 2008 16:45:31 GMT
Document jacob's leader election on the wiki recipes page
---------------------------------------------------------

                 Key: ZOOKEEPER-79
                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-79
             Project: Zookeeper
          Issue Type: New Feature
          Components: documentation
            Reporter: Patrick Hunt
            Assignee: Patrick Hunt


The following discussion occurred on the zookeeper-user list. We need to formalize this recipe
and document on the wiki recipes page:

---------------------from jacob ----------------
Avinash

 

The following protocol will help you fix the observed misbehavior. As Flavio points out, you
cannot rely on the order of nodes in getChildren, you must use an intrinsic property of each
node to determine who is the leader. The protocol devised by Runping Qi and described here
will do that.

 

First of all, when you create child nodes of the node that holds the leadership bids, you
must create them with the EPHEMERAL and SEQUENCE flag. ZooKeeper guarantees to give you an
ephemeral node named uniquely and with a sequence number larger by at least one than any previously
created node in the sequence. You provide a prefix, like "L_" or your own choice, and ZooKeeper
creates nodes named "L_23", "L_24", etc. The sequence number starts at 0 and increases monotonously.

 

Once you've placed your leadership bid, you search backwards from the sequence number of *your*
node to see if there are any preceding (in terms of the sequence number) nodes. When you find
one, you place a watch on it and wait for it to disappear. When you get the watch notification,
you search again, until you do not find a preceding node, then you know you're the leader.
This protocol guarantees that there is at any time only one node that thinks it is the leader.
But it does not disseminate information about who is the leader. If you want everyone to know
who is the leader, you can have an additional Znode whose value is the name of the current
leader (or some identifying information on how to contact the leader, etc.). Note that this
cannot be done atomically, so by the time other nodes find out who the leader is, the leadership
may already have passed on to a different node.

 

Flavio

 

Might it make sense to provide a standardized implementation of leader election in the library
code in Java?

 

--Jacob

 

From: zookeeper-user-bounces@lists.sourceforge.net [mailto:zookeeper-user-bounces@lists.sourceforge.net]
On Behalf Of Flavio Junqueira
Sent: Friday, July 11, 2008 1:02 AM
To: zookeeper-user@lists.sourceforge.net
Cc: zookeeper-user@hadoop.apache.org
Subject: Re: [Zookeeper-user] Leader election

 

Hi Avinash, getChildren returns a list in lexicographic order, so if you are updating the
children of the election node concurrently, then you may get a different first node with different
clients. If you are using the sequence flag to create nodes, then you may consider stripping
the prefix of the node name and using the sufix value to determine order.

Hope it helps.

-Flavio

 

----- Original Message ----
From: Avinash Lakshman <avinash.lakshman@gmail.com>
To: zookeeper-user@lists.sourceforge.net
Sent: Friday, July 11, 2008 7:20:06 AM
Subject: [Zookeeper-user] Leader election

Hi

I am trying to elect leader among 50 nodes. There is always one odd guy who seems to think
that someone else distinct from what some other nodes see as leader. Could someone please
tell me what is wrong with the following code for leader election:

public void electLeader()
        {           
            ZooKeeper zk = StorageService.instance().getZooKeeperHandle();
            String path = "/Leader";
            try
            {
                String createPath = path + "/L-";                               
                LeaderElector.createLock_.lock();
                while( true )
                {
                    /* Get all znodes under the Leader znode */
                    List<String> values = zk.getChildren(path, false);
                    /*
                     * Get the first znode and if it is the
                     * pathCreated created above then the data
                     * in that znode is the leader's identity.
                    */
                    if ( leader_ == null )
                    {
                        leader_ = new AtomicReference<EndPoint>( EndPoint.fromBytes(
zk.getData(path + "/" + values.get(0), false, null) ) );
                    }
                    else
                    {
                        leader_.set( EndPoint.fromBytes( zk.getData(path + "/" + values .get(0),
false, null) ) );
                        /* Disseminate the state as to who the leader is. */
                        onLeaderElection();
                    }
                    logger_.debug("Elected leader is " + leader_ + " @ znode " + ( path +
"/" + values.get(0) ) );                  
                    Collections.sort(values);
                    /* We need only the last portion of this znode */
                    String[] peices = pathCreated_.split("/");
                    int index = Collections.binarySearch(values, peices[peices.length - 1]);
                 
                    if ( index > 0 )
                    {
                        String pathToCheck = path + "/" + values.get(index - 1);
                        Stat stat = zk.exists(pathToCheck, true);
                        if ( stat != null )
                        {
                            logger_.debug("Awaiting my turn ...");
                            condition_.await();
                            logger_.debug("Checking to see if leader is around ...");
                        }
                    }
                    else
                    {
                        break;
                    }
                }
            }
            catch ( InterruptedException ex )
            {
                logger_.warn(LogUtil.throwableToString(ex));
            }
            catch ( KeeperException ex )
            {
                logger_.warn(LogUtil.throwableToString(ex));
            }
            finally
            {
                LeaderElector.createLock_.unlock();
            }
        }
    }

Thanks
Avinash

 



-------------------------------------------------------------------------
Sponsored by: SourceForge.net Community Choice Awards: VOTE NOW!
Studies have shown that voting for your favorite open source project,
along with a healthy diet, reduces your potential for chronic lameness
and boredom. Vote Now at http://www.sourceforge.net/community/cca08



_______________________________________________
Zookeeper-user mailing list
Zookeeper-user@lists.sourceforge.net
https://lists.sourceforge.net/lists/listinfo/zookeeper-user


-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


Mime
View raw message