[ https://issues.apache.org/jira/browse/ZOOKEEPER-80?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12614466#action_12614466 ]
Doug Cutting commented on ZOOKEEPER-80: --------------------------------------- Whoa! Big issue description! Perhaps you could have gone with a link to the mail archive? Descriptions are included in every message about the issue... In any case, I think perhaps each recipe deserves its own code-tree, and should hence be a separate contrib module. Perhaps, instead of 'contrib/' these should just be under 'recipes/', with a separate src/, lib/, doc/, build.xml, README.txt, etc. for each? Multiple language implementations would go in different src/ subdirectories. Does that work? Also, I am -1 for making these subversion-only. Only released software is fully covered by the Apache license. Subversion is for internal exchange by Apache of works-in-progress, not for end users. > Document process for client recipe contributions > ------------------------------------------------ > > Key: ZOOKEEPER-80 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-80 > Project: Zookeeper > Issue Type: Task > Components: documentation > Reporter: Patrick Hunt > Assignee: Patrick Hunt > > How do we accept zk client recipe contributions? Initiated by the following > discussion on the mailing list: > ------ ben reed wrote -------- > Excellent proposal. The only thing I would add is that there should be > an english description of the recipe in subversion. That way if someone > wanted to do a compatible binding they can do it. If the recipe is on > the wiki it would be hard to keep it in sync, so it is important that it > is in subversion. My preference would be that the doc would be in the > same contrib subdirectory as the source for ease of maintenance. > ben > Patrick Hunt wrote: > > > James, thanks for the contribution! Tests and everything. :-) > > > > > > Jacob sent some mail to the list recently (attached) that details a > > > protocol that he's used successfully (and picked up by some zk users). > > > I have a todo item to document this protocol on the recipes wiki page, > > > haven't gotten to it yet. Not sure how/if this matches what you've > > > done but we should sync up (also see below). > > > https://issues.apache.org/jira/browse/ZOOKEEPER-79 > > > > > > There has been some discussion on client side helper code in the past > > > however this is the first contribution. We need to make some decisions > > > and outline what/how we will accept. > > > > > > 1) I think we should have a > > > "contrib/recipes/{java/{main,test}/org/apache/zookeeper/... ,c/,...}" > > > hierarchy for contributions that implement recipes, including any > > > helper code > > > > > > 2) We should first document recipes on the wiki, then implement them > > > in the code > > > http://wiki.apache.org/hadoop/ZooKeeper/ZooKeeperRecipes > > > The code should fully document the api/implementation, and refer to > > > wiki page for protocol specifics. > > > > > > 3) What should we do relative to ZK releases. Are recipes included in > > > a release? Will bugs in recipes hold up a release? > > > > > > My initial thought is that contrib is available through svn, but not > > > included in the release. If users want to access/use this code they > > > will be required to checkout/build themselves. (at least initially) > > > > > > 4) We will not require "parody" btw the various client languages. > > > Currently we support Java/C clients, we will be adding various > > > scripting languages soon. Contributions will be submitted for various > > > clients (James' submission is for java), that will be placed into > > > contrib, if someone else contributes C bindings (etc...) we will add > > > those to contrib/recipes as well. > > > > > > 5) Implementations should strive to implement similar recipe protocols > > > (see 2 above, a good reason to document before implement). There may > > > be multiple, different, protocols each with their own implementation, > > > but for a particular protocol the implementations should be the same. > > > > > > We may want to stress 5 even more - if multiple clients > > > implementations (c/java/...) are participating in a single instance of > > > leader election it will be CRITICAL for them to be inter-operable. > > > > > > > > > Comments, questions, suggestion? > > > > > > Patrick > > > > > > James Strachan wrote: > >> >> So having recently discovered ZooKeeper, I'm really liking it - good > >> >> job folks! > >> >> > >> >> I've seen discussions of building high level features from the core ZK > >> >> library and had not seen any available on the interweb so figured I'd > >> >> have a try creating a simple one. Feel free to ignore it if a ZK ninja > >> >> can think of a neater way of doing it - I've basically followed the > >> >> protocol defined in the recent ZK presentation... > >> >> http://developer.yahoo.com/blogs/hadoop/2008/03/intro-to-zookeeper-video.html > >> >> > >> >> > >> >> I've submitted the code as a patch here... > >> >> https://issues.apache.org/jira/browse/ZOOKEEPER-78 > >> >> > >> >> I figured the Java Client might as well come with some helper code to > >> >> make doing things like exclusive locks or leader elections easier; we > >> >> could always spin them out into a separate library if and when > >> >> required etc. Right now its one fairly simple class :) > >> >> > >> >> Currently its a simple class where you can register a Runnable to be > >> >> invoked when you have the lock; or you can just keep asking if you > >> >> have the lock now and again as you see fit etc. > >> >> > >> >> WriteLock locker = new WriteLock(zookeeper, "/foo/bar"); > >> >> locker.setWhenOwner(new Runnable() {...}); // fire this code when > >> >> owner... > >> >> > >> >> // lets try own it > >> >> locker.acquire(); > >> >> > >> >> // I may or may not have the lock now > >> >> if (locker.isOwner()) {....} > >> >> > >> >> // time passes > >> >> locker.close(); > >> >> > >> >> > >> >> Thoughts? > >> >> > > > > > > ------------------------------------------------------------------------ > > > > > > Subject: > > > Re: [Zookeeper-user] Leader election > > > From: > > > "Jacob Levy" <[EMAIL PROTECTED]> > > > Date: > > > Fri, 11 Jul 2008 10:42:33 -0700 > > > To: > > > "Flavio Junqueira" <[EMAIL PROTECTED]>, > > > <[EMAIL PROTECTED]>, <[EMAIL PROTECTED]> > > > > > > To: > > > "Flavio Junqueira" <[EMAIL PROTECTED]>, > > > <[EMAIL PROTECTED]>, <[EMAIL PROTECTED]> > > > CC: > > > [EMAIL PROTECTED] > > > > > > > > > *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:* [EMAIL PROTECTED] > > > [mailto:[EMAIL PROTECTED] *On Behalf Of > > > *Flavio Junqueira > > > *Sent:* Friday, July 11, 2008 1:02 AM > > > *To:* [EMAIL PROTECTED] > > > *Cc:* [EMAIL PROTECTED] > > > *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 <[EMAIL PROTECTED]> > > > To: [EMAIL PROTECTED] > > > 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 > > > [EMAIL PROTECTED] > > > 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.