Re: feedback zkclient

2009-10-05 Thread Patrick Hunt

You might want to add a link to zkclient on this page:

http://wiki.apache.org/hadoop/ZooKeeper/UsefulTools

Patrick

Patrick Hunt wrote:

Ted Dunning wrote:
Judging by history and that fact that only 40/127 issues are resolved, 
3.3

is probably 3-6 months away.  Is that a fair assessment?


Yes, that's fair.

Patrick


On Thu, Oct 1, 2009 at 11:13 AM, Patrick Hunt ph...@apache.org wrote:


One nice thing about ephemeral is that the Stat contains the owner
sessionid. As you say, it's highly implementation dependent. It's also
something we recognize is a problem for users, we've slated it for 3.3.0
http://issues.apache.org/jira/browse/ZOOKEEPER-22







Re: feedback zkclient

2009-10-05 Thread Stefan Groschupf

Hi Patrick,
thanks - we will do, also thanks for all the feedback.
Stefan

On Oct 5, 2009, at 11:22 AM, Patrick Hunt wrote:


You might want to add a link to zkclient on this page:

http://wiki.apache.org/hadoop/ZooKeeper/UsefulTools

Patrick

Patrick Hunt wrote:

Ted Dunning wrote:
Judging by history and that fact that only 40/127 issues are  
resolved, 3.3

is probably 3-6 months away.  Is that a fair assessment?

Yes, that's fair.
Patrick
On Thu, Oct 1, 2009 at 11:13 AM, Patrick Hunt ph...@apache.org  
wrote:



One nice thing about ephemeral is that the Stat contains the owner
sessionid. As you say, it's highly implementation dependent. It's  
also
something we recognize is a problem for users, we've slated it  
for 3.3.0

http://issues.apache.org/jira/browse/ZOOKEEPER-22









~~~
Hadoop training and consulting
http://www.scaleunlimited.com
http://www.101tec.com





Zookeeper Test Project

2009-10-05 Thread Steve Cheng
Hi,

The test project on the ProjectSuggestions page, to create tests for the
examples, sounded like a nice way to get started with ZooKeeper, I've
created one queue test and one lock test so far.

I'd like to make a good range of tests, a number of basic sanity ones like
the queue test, and some more involved ones like the lock test.  The lock
test does polling right now, another test would also test watching.  I think
it's useful to test both.

Any comments on this would be great, I noticed the bug about the test case
runtime so I'm not sure what that means for more tests, thanks,

Steven
Index: src/java/test/org/apache/zookeeper/test/ExampleTest.java
===
--- src/java/test/org/apache/zookeeper/test/ExampleTest.java	(revision 0)
+++ src/java/test/org/apache/zookeeper/test/ExampleTest.java	(revision 0)
@@ -0,0 +1,149 @@
+package org.apache.zookeeper.test;
+
+import java.util.Calendar;
+import java.util.List;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.junit.Test;
+
+public class ExampleTest extends ClientBase {
+	protected static final Logger LOG = Logger.getLogger(ExampleTest.class);
+
+	@Override
+	protected void tearDown() throws Exception {
+		super.tearDown();
+		LOG.info(FINISHED  + getName());
+	}
+
+	static class CountdownTimer {
+		long startTime;
+		long waitTime;
+
+		CountdownTimer(long millis) {
+			waitTime = millis;
+			startTime = Calendar.getInstance().getTimeInMillis();
+		}
+
+		boolean waiting() {
+			return (Calendar.getInstance().getTimeInMillis() - startTime)  waitTime;
+		}
+	}
+
+	@Test
+	public void testQueue() throws Exception {
+		ZooKeeper zkProducer = null;
+		ZooKeeper zkConsumer = null;
+		String queue_handle = /queue;
+		try {
+
+			zkProducer = createClient();
+			zkConsumer = createClient();
+
+			zkProducer.create(queue_handle, new byte[0], Ids.OPEN_ACL_UNSAFE,
+	CreateMode.PERSISTENT);
+			zkProducer.create(queue_handle + /element, 0.getBytes(),
+	Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL);
+			zkProducer.create(queue_handle + /element, 1.getBytes(),
+	Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL);
+			ListString children = null;
+			CountdownTimer waitForQueueElements = new CountdownTimer(1000);
+			while (waitForQueueElements.waiting()
+	 ((children = zkConsumer.getChildren(queue_handle, true))
+			.size() != 2)) {
+			}
+			assertEquals(children.size(), 2);
+			String child1 = children.get(0);
+			String child2 = children.get(1);
+			int compareResult = child1.compareTo(child2);
+			assertNotSame(compareResult, 0);
+			if (compareResult  0) {
+			} else {
+String temp = child1;
+child1 = child2;
+child2 = temp;
+			}
+			String child1data = new String(zkConsumer.getData(queue_handle
+	+ / + child1, false, null));
+			String child2data = new String(zkConsumer.getData(queue_handle
+	+ / + child2, false, null));
+			assertEquals(child1data, 0);
+			assertEquals(child2data, 1);
+		} finally {
+			if (zkProducer != null) {
+zkProducer.close();
+			}
+			if (zkConsumer != null) {
+zkConsumer.close();
+			}
+		}
+
+	}
+
+	@Test
+	public void testLock() throws Exception {
+		final int num_contenders = 10;
+		final ZooKeeper zookeepers[] = new ZooKeeper[num_contenders];
+		final String contenderLockNodes[] = new String[num_contenders];
+		final String lock_handle = /lock;
+		final String lock_element = /node-;
+		final int lock_prefix_length = lock_handle.length() + lock_element.length();
+		try{
+			for(int i=0; i  zookeepers.length; i++){
+zookeepers[i] = createClient();
+			}
+
+			zookeepers[0].create(lock_handle, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+			//Get lock nodes
+			for(int i=0; i zookeepers.length; i++){
+contenderLockNodes[i] = zookeepers[i].create(lock_handle+lock_element, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
+			}
+
+			boolean contenderSucceeded[] = new boolean[num_contenders];
+			//Poll the locks in a round robin
+			CountdownTimer lockAcquire = new CountdownTimer(1000*num_contenders);
+			
+			boolean waiting = true;
+			while(lockAcquire.waiting()  waiting){
+waiting = false;
+for(int contenderId=0; contenderId  num_contenders; contenderId++){
+	if(contenderSucceeded[contenderId]) break;
+	ZooKeeper zooKeeper = zookeepers[contenderId];
+	String lockNode = contenderLockNodes[contenderId];
+	long lockId = Long.parseLong(lockNode.substring(lock_prefix_length));
+
+	ListString children = zooKeeper.getChildren(lock_handle, false);
+	
+	long coveredId = -1;
+	for(String child : children){
+		long childId = Long.parseLong(child.substring(lock_prefix_length));
+		if((coveredId  childId)  (childId  lockId)){
+			coveredId = childId;
+		}
+	}		

Re: Zookeeper Test Project

2009-10-05 Thread Mahadev Konar

Hi Steve,
Can you please create a jira and upload the code to that jira for
feedback? I am guessing that you want to contribute these tests back to
zookeeper right ? 
  Also, do take a look at src/recipes/lock for an implementation of locks on
top of zookeeper. They have some tests as well.

Also, if you are willing to work a little more on your queue implementation,
we could have that as a part of src/recipes and then people could use that.
Tests can be added to this recipe to verify them...

Thanks
mahadev

On 10/5/09 1:45 PM, Steve Cheng steve...@gmail.com wrote:

 Hi,
 
 The test project on the ProjectSuggestions page, to create tests for the
 examples, sounded like a nice way to get started with ZooKeeper, I've created
 one queue test and one lock test so far.
 
 I'd like to make a good range of tests, a number of basic sanity ones like the
 queue test, and some more involved ones like the lock test.  The lock test
 does polling right now, another test would also test watching.  I think it's
 useful to test both.
 
 Any comments on this would be great, I noticed the bug about the test case
 runtime so I'm not sure what that means for more tests, thanks,
 
 Steven
 
 



Re: Zookeeper Test Project

2009-10-05 Thread Mahadev Konar
 Sounds good, I have created an ASF JIRA account, but it doesn't look like I
 can attach a file in Step 2 of the Create Issue screen.  Do I have to
 create the jira first then attach the file?
Yes. Please take a look at

http://wiki.apache.org/hadoop/ZooKeeper/HowToContribute

On some guidelines on contributing.

Thanks
mahadev

 
 
 
 Also, if you are willing to work a little more on your queue
 implementation,
 we could have that as a part of src/recipes and then people could use that.
 Tests can be added to this recipe to verify them...
 
 
 Just took a look at the lock recipe, I'll start working on a queue recipe
 and tests.
 
 Steven
 
 
 
 
 Thanks
 mahadev
 
 On 10/5/09 1:45 PM, Steve Cheng steve...@gmail.com wrote:
 
 Hi,
 
 The test project on the ProjectSuggestions page, to create tests for the
 examples, sounded like a nice way to get started with ZooKeeper, I've
 created
 one queue test and one lock test so far.
 
 I'd like to make a good range of tests, a number of basic sanity ones
 like the
 queue test, and some more involved ones like the lock test.  The lock
 test
 does polling right now, another test would also test watching.  I think
 it's
 useful to test both.
 
 Any comments on this would be great, I noticed the bug about the test
 case
 runtime so I'm not sure what that means for more tests, thanks,
 
 Steven
 
 
 
 



[jira] Updated: (ZOOKEEPER-510) zkpython lumps all exceptions as IOError, needs specialized exceptions for KeeperException types

2009-10-05 Thread Henry Robinson (JIRA)

 [ 
https://issues.apache.org/jira/browse/ZOOKEEPER-510?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Henry Robinson updated ZOOKEEPER-510:
-

Attachment: ZOOKEEPER-510.patch

This patch adds exceptions, and expands the test suite to cover some commonly 
thrown exception cases. 

At the same time, it also includes a patch for the segfault issue when reusing 
a closed handle (since getting the exceptions right here requires fixing that 
issue, and fixing that issue requires proper exception testing). 

There were a couple of other bugs discovered during this process which this 
patch hopefully also squashes. 

 zkpython lumps all exceptions as IOError, needs specialized exceptions for 
 KeeperException types
 

 Key: ZOOKEEPER-510
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-510
 Project: Zookeeper
  Issue Type: Bug
  Components: contrib-bindings
Affects Versions: 3.2.0
Reporter: Patrick Hunt
Assignee: Henry Robinson
 Fix For: 3.3.0

 Attachments: ZOOKEEPER-510.patch, ZOOKEEPER-510.patch, 
 ZOOKEEPER-510.patch, ZOOKEEPER-510.patch


 The current zkpython bindings always throw IOError(text) exceptions, even 
 for ZK specific exceptions such as NODEEXISTS. This makes it difficult (error 
 prone) to handle exceptions in python code. You can't easily pickup a 
 connection loss vs a node exists for example. Of course you could match the 
 error string, but this seems like a bad idea imo.
 We need to add specific exception types to the python binding that map 
 directly to KeeperException/java types. It would also be useful to include 
 the information provided by the KeeperException (like path in some cases), 
 etc... as part of the error thrown to the python code. Would probably be a 
 good idea to stay as close to java api as possible wrt mapping the errors.

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



[jira] Created: (ZOOKEEPER-542) c-client can spin when server unresponsive

2009-10-05 Thread Christian Wiedmann (JIRA)
c-client can spin when server unresponsive
--

 Key: ZOOKEEPER-542
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-542
 Project: Zookeeper
  Issue Type: Bug
  Components: c client
Affects Versions: 3.2.0
Reporter: Christian Wiedmann


Due to a mismatch between zookeeper_interest() and zookeeper_process(), when 
the zookeeper server is unresponsive the client can spin when reconnecting to 
the server.

In particular, zookeeper_interest() adds ZOOKEEPER_WRITE whenever there is data 
to be sent, but flush_send_queue() only writes the data if the state is 
ZOO_CONNECTED_STATE.  When in ZOO_ASSOCIATING_STATE, this results in spinning.

This probably doesn't affect production, but I had a runaway process in a 
development deployment that caused performance issues on the node.  This is 
easy to reproduce in a single node environment by doing a kill -STOP on the 
server and waiting for the session timeout.

Patch to be added.

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



[jira] Updated: (ZOOKEEPER-542) c-client can spin when server unresponsive

2009-10-05 Thread Christian Wiedmann (JIRA)

 [ 
https://issues.apache.org/jira/browse/ZOOKEEPER-542?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Christian Wiedmann updated ZOOKEEPER-542:
-

Attachment: ZOOKEEPER-542.patch

 c-client can spin when server unresponsive
 --

 Key: ZOOKEEPER-542
 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-542
 Project: Zookeeper
  Issue Type: Bug
  Components: c client
Affects Versions: 3.2.0
Reporter: Christian Wiedmann
 Attachments: ZOOKEEPER-542.patch


 Due to a mismatch between zookeeper_interest() and zookeeper_process(), when 
 the zookeeper server is unresponsive the client can spin when reconnecting to 
 the server.
 In particular, zookeeper_interest() adds ZOOKEEPER_WRITE whenever there is 
 data to be sent, but flush_send_queue() only writes the data if the state is 
 ZOO_CONNECTED_STATE.  When in ZOO_ASSOCIATING_STATE, this results in spinning.
 This probably doesn't affect production, but I had a runaway process in a 
 development deployment that caused performance issues on the node.  This is 
 easy to reproduce in a single node environment by doing a kill -STOP on the 
 server and waiting for the session timeout.
 Patch to be added.

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