svn commit: r689668 - in /hadoop/zookeeper/trunk: ./ src/java/main/org/apache/zookeeper/ src/java/test/org/apache/zookeeper/test/

2008-08-27 Thread breed
Author: breed
Date: Wed Aug 27 16:36:40 2008
New Revision: 689668

URL: http://svn.apache.org/viewvc?rev=689668view=rev
Log:
 ZOOKEEPER-63. Race condition in client close() operation. (phunt via breed)


Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncTest.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/DataTreeTest.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/QuorumTest.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SessionTest.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SyncCallTest.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=689668r1=689667r2=689668view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Wed Aug 27 16:36:40 2008
@@ -47,3 +47,5 @@
 
  ZOOKEEPER-125. Remove unwanted class declaration in FastLeaderElection. 
  (Flavio Paiva Junqueira via mahadev)
+
+ ZOOKEEPER-63. Race condition in client close() operation. (phunt via breed)

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java?rev=689668r1=689667r2=689668view=diff
==
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java 
(original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java 
Wed Aug 27 16:36:40 2008
@@ -34,11 +34,10 @@
 import java.util.Set;
 import java.util.concurrent.LinkedBlockingQueue;
 
-import org.apache.log4j.Logger;
-
 import org.apache.jute.BinaryInputArchive;
 import org.apache.jute.BinaryOutputArchive;
 import org.apache.jute.Record;
+import org.apache.log4j.Logger;
 import org.apache.zookeeper.AsyncCallback.ACLCallback;
 import org.apache.zookeeper.AsyncCallback.ChildrenCallback;
 import org.apache.zookeeper.AsyncCallback.DataCallback;
@@ -63,7 +62,6 @@
 import org.apache.zookeeper.proto.SetDataResponse;
 import org.apache.zookeeper.proto.WatcherEvent;
 import org.apache.zookeeper.server.ByteBufferInputStream;
-import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooTrace;
 
 /**
@@ -75,7 +73,8 @@
 class ClientCnxn {
 private static final Logger LOG = Logger.getLogger(ClientCnxn.class);
 
-private ArrayListInetSocketAddress serverAddrs = new 
ArrayListInetSocketAddress();
+private ArrayListInetSocketAddress serverAddrs =
+new ArrayListInetSocketAddress();
 
 static class AuthData {
 AuthData(String scheme, byte data[]) {
@@ -122,6 +121,12 @@
 final EventThread eventThread;
 
 final Selector selector = Selector.open();
+
+/** Set to true when close is called. Latches the connection such that
+ * we don't attempt to re-connect to the server if in the middle of
+ * closing the connection (client sends session disconnect to server
+ * as part of close operation) */
+volatile boolean closing = false;
 
 public long getSessionId() {
 return sessionId;
@@ -253,7 +258,7 @@
 
 class EventThread extends Thread {
 EventThread() {
-super(EventThread);
+super(currentThread().getName() + -EventThread);
 setUncaughtExceptionHandler(uncaughtExceptionHandler);
 setDaemon(true);
 }
@@ -341,7 +346,10 @@
 }
 }
 } catch (InterruptedException e) {
+LOG.warn(Event thread exiting due to interruption, e);
 }
+
+LOG.info(EventThread shut down);
 }
 }
 
@@ -566,7 +574,7 @@
 }
 
 SendThread() {
-super(SendThread);
+super(currentThread().getName() + -SendThread);
 zooKeeper.state = States.CONNECTING;
 setUncaughtExceptionHandler(uncaughtExceptionHandler);
 setDaemon(true);
@@ -666,6 +674,10 @@
 while (zooKeeper.state.isAlive()) {
 try {
 if (sockKey == null) {
+// don't re-establish connection if we are closing
+if (closing) {
+break;
+}
 startConnect();
 lastSend = now;
 lastHeard = now;
@@ -730,21 +742,34

svn commit: r692164 - in /hadoop/zookeeper/trunk/src/java: jmx/org/apache/zookeeper/jmx/server/ jmx/org/apache/zookeeper/server/ jmx/org/apache/zookeeper/server/quorum/ main/org/apache/zookeeper/ main

2008-09-04 Thread breed
Author: breed
Date: Thu Sep  4 10:02:09 2008
New Revision: 692164

URL: http://svn.apache.org/viewvc?rev=692164view=rev
Log:
ZOOKEEPER-134 findbugs cleanup

Modified:

hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/jmx/server/ConnectionBean.java

hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/jmx/server/DataTreeBean.java

hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/server/ManagedZooKeeperServer.java

hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/server/ManagedZooKeeperServerMain.java

hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/server/ObservableZooKeeperServer.java

hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/server/quorum/ManagedQuorumPeer.java

hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/server/quorum/ManagedQuorumPeerMain.java

hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/server/quorum/ObservableQuorumPeer.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataNode.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/Request.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerStats.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/SyncRequestProcessor.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Follower.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FollowerHandler.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderElection.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumStats.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/version/util/VerGen.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SessionTest.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java

Modified: 
hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/jmx/server/ConnectionBean.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/jmx/server/ConnectionBean.java?rev=692164r1=692163r2=692164view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/jmx/server/ConnectionBean.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/jmx/server/ConnectionBean.java
 Thu Sep  4 10:02:09 2008
@@ -88,6 +88,7 @@
 connection.close();
 }
 
+@Override
 public String toString() {
 return 
ConnectionBean{ClientIP=+getSourceIP()+,SessionId=0x+getSessionId()+};
 }

Modified: 
hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/jmx/server/DataTreeBean.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/jmx/server/DataTreeBean.java?rev=692164r1=692163r2=692164view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/jmx/org/apache/zookeeper/jmx/server

svn commit: r692452 - in /hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test: ClientTest.java QuorumTest.java

2008-09-05 Thread breed
Author: breed
Date: Fri Sep  5 07:39:28 2008
New Revision: 692452

URL: http://svn.apache.org/viewvc?rev=692452view=rev
Log:
ZOOKEEPER-133 hudson tests failing intermittently

Modified:

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/QuorumTest.java

Modified: 
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java?rev=692452r1=692451r2=692452view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java 
(original)
+++ 
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java 
Fri Sep  5 07:39:28 2008
@@ -43,9 +43,6 @@
 public class ClientTest extends ClientBase {
 protected static final Logger LOG = Logger.getLogger(ClientTest.class);
 
-LinkedBlockingQueueWatcherEvent events =
-new LinkedBlockingQueueWatcherEvent();
-
 @Override
 protected void tearDown() throws Exception {
 super.tearDown();
@@ -145,7 +142,10 @@
 }
 }
 
-protected class MyWatcher extends CountdownWatcher {
+private class MyWatcher extends CountdownWatcher {
+LinkedBlockingQueueWatcherEvent events =
+new LinkedBlockingQueueWatcherEvent();
+
 public void process(WatcherEvent event) {
 super.process(event);
 if (event.getType() != Event.EventNone) {
@@ -157,13 +157,132 @@
 }
 }
 }
+
+/**
+ * Register multiple watchers and verify that they all get notified and
+ * in the right order.
+ */
+@Test
+public void testMutipleWatcherObjs()
+throws IOException, InterruptedException, KeeperException
+{
+ZooKeeper zk = createClient(new CountdownWatcher(), hostPort);
+try {
+MyWatcher watchers[] = new MyWatcher[100];
+MyWatcher watchers2[] = new MyWatcher[watchers.length];
+for (int i = 0; i  watchers.length; i++) {
+watchers[i] = new MyWatcher();
+watchers2[i] = new MyWatcher();
+zk.create(/foo- + i, (foodata + i).getBytes(),
+Ids.OPEN_ACL_UNSAFE, 0);
+}
+Stat stat = new Stat();
+
+//
+// test get/exists with single set of watchers
+//   get all, then exists all
+//
+for (int i = 0; i  watchers.length; i++) {
+assertNotNull(zk.getData(/foo- + i, watchers[i], stat));
+}
+for (int i = 0; i  watchers.length; i++) {
+assertNotNull(zk.exists(/foo- + i, watchers[i]));
+}
+// trigger the watches
+for (int i = 0; i  watchers.length; i++) {
+zk.setData(/foo- + i, (foodata2- + i).getBytes(), -1);
+zk.setData(/foo- + i, (foodata3- + i).getBytes(), -1);
+}
+for (int i = 0; i  watchers.length; i++) {
+WatcherEvent event =
+watchers[i].events.poll(10, TimeUnit.SECONDS);
+assertEquals(/foo- + i, event.getPath());
+assertEquals(Event.EventNodeDataChanged, event.getType());
+assertEquals(Event.KeeperStateSyncConnected, event.getState());
+
+// small chance that an unexpected message was delivered
+//  after this check, but we would catch that next time
+//  we check events
+assertEquals(0, watchers[i].events.size());
+}
+
+//
+// test get/exists with single set of watchers
+//  get/exists together
+//
+for (int i = 0; i  watchers.length; i++) {
+assertNotNull(zk.getData(/foo- + i, watchers[i], stat));
+assertNotNull(zk.exists(/foo- + i, watchers[i]));
+}
+// trigger the watches
+for (int i = 0; i  watchers.length; i++) {
+zk.setData(/foo- + i, (foodata4- + i).getBytes(), -1);
+zk.setData(/foo- + i, (foodata5- + i).getBytes(), -1);
+}
+for (int i = 0; i  watchers.length; i++) {
+WatcherEvent event =
+watchers[i].events.poll(10, TimeUnit.SECONDS);
+assertEquals(/foo- + i, event.getPath());
+assertEquals(Event.EventNodeDataChanged, event.getType());
+assertEquals(Event.KeeperStateSyncConnected, event.getState());
+
+// small chance that an unexpected message was delivered
+//  after this check, but we would catch that next time
+//  we check events
+assertEquals

svn commit: r703602 - in /hadoop/zookeeper/trunk: CHANGES.txt src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java

2008-10-10 Thread breed
Author: breed
Date: Fri Oct 10 16:15:55 2008
New Revision: 703602

URL: http://svn.apache.org/viewvc?rev=703602view=rev
Log:
ZOOKEEPER-178 FLE Test

Modified:
hadoop/zookeeper/trunk/CHANGES.txt

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=703602r1=703601r2=703602view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Fri Oct 10 16:15:55 2008
@@ -134,3 +134,5 @@
  ZOOKEEPER-127.  Use of non-standard election ports in config breaks services
  (Mark Harwood and Flavio Junqueira via breed)
 
+ ZOOKEEPER-172. FLE Test (Flavio Junqueira via breed)
+

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java?rev=703602r1=703601r2=703602view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
 Fri Oct 10 16:15:55 2008
@@ -587,6 +587,19 @@
 }
 break;
 case LEADING:
+/*
+ * There is at most one leader for each epoch, so if a peer 
claims to
+ * be the leader for an epoch, then that peer must be the 
leader (no
+ * arbitrary failures assumed). Now, if there is no quorum 
supporting 
+ * this leader, then processes will naturally move to a new 
epoch.
+ */
+if(n.epoch == logicalclock){
+self.setPeerState((n.leader == self.getId()) ? 
+ServerState.LEADING: ServerState.FOLLOWING);
+   
+leaveInstance();
+return new Vote(n.leader, n.zxid);
+}
 case FOLLOWING:
 LOG.info(Notification:  + n.leader + ,  + n.zxid + ,  + 
n.epoch + ,  + self.getId() + ,  + self.getPeerState() + ,  + n.state + 
,  + n.sid);





svn commit: r776797 - /hadoop/zookeeper/trunk/build.xml

2009-05-20 Thread breed
Author: breed
Date: Wed May 20 18:12:24 2009
New Revision: 776797

URL: http://svn.apache.org/viewvc?rev=776797view=rev
Log:
ZOOKEEPER-292. commit configure scripts (autotools) to svn for c projects and 
include in release.

Modified:
hadoop/zookeeper/trunk/build.xml

Modified: hadoop/zookeeper/trunk/build.xml
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/build.xml?rev=776797r1=776796r2=776797view=diff
==
--- hadoop/zookeeper/trunk/build.xml (original)
+++ hadoop/zookeeper/trunk/build.xml Wed May 20 18:12:24 2009
@@ -426,7 +426,8 @@
 !-- == --
 !----
 !-- == --
-target name=package depends=release-jar, api-report, compile-test
+target name=package
+
depends=release-jar,api-report,compile-test,create-cppunit-configure
 description=Build distribution
   mkdir dir=${dist.dir}/
   mkdir dir=${dist.dir}/lib/
@@ -493,14 +494,42 @@
 target name=tar depends=package description=Make release tarball
   macro_tar param.destfile=${build.dir}/${final.name}.tar.gz
 param.listofitems
+  !-- it's unfortunate that the binaries have to be duplicated
+   however that's the only way ant currently supports afaik --
   tarfileset dir=${build.dir} mode=664
 exclude name=${final.name}/bin/* /
 exclude name=${final.name}/contrib/*/bin/* /
+
+exclude name=${final.name}/**/compile /
+exclude name=${final.name}/**/config.guess /
+exclude name=${final.name}/**/config.sub /
+exclude name=${final.name}/**/configure /
+exclude name=${final.name}/**/depcomp /
+exclude name=${final.name}/**/install-sh /
+exclude name=${final.name}/**/ltmain.sh /
+exclude name=${final.name}/**/missing /
+
+exclude name=${final.name}/**/*.sh /
+
 include name=${final.name}/** /
   /tarfileset
   tarfileset dir=${build.dir} mode=755
+!-- be sure to add to exclude list above if you add here --
 include name=${final.name}/bin/* /
 include name=${final.name}/contrib/*/bin/* /
+
+!-- autotools related commands --
+include name=${final.name}/**/compile /
+include name=${final.name}/**/config.guess /
+include name=${final.name}/**/config.sub /
+include name=${final.name}/**/configure /
+include name=${final.name}/**/depcomp /
+include name=${final.name}/**/install-sh /
+include name=${final.name}/**/ltmain.sh /
+include name=${final.name}/**/missing /
+
+!-- any shell scripts --
+include name=${final.name}/**/*.sh /
   /tarfileset
 /param.listofitems
   /macro_tar




svn commit: r776806 - in /hadoop/zookeeper/trunk: ./ src/java/main/org/apache/zookeeper/ src/java/main/org/apache/zookeeper/server/ src/java/main/org/apache/zookeeper/server/quorum/ src/java/test/conf

2009-05-20 Thread breed
Author: breed
Date: Wed May 20 18:48:00 2009
New Revision: 776806

URL: http://svn.apache.org/viewvc?rev=776806view=rev
Log:
ZOOKEEPER-410. address all findbugs warnings in client/server classes.

Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/SyncRequestProcessor.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/ProposalRequestProcessor.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java
hadoop/zookeeper/trunk/src/java/test/config/findbugsExcludeFile.xml

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=776806r1=776805r2=776806view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Wed May 20 18:48:00 2009
@@ -76,6 +76,8 @@
 
   ZOOKEEPER-405. nullpointer exception in zookeeper java shell. (mahadev via 
breed)
 
+  ZOOKEEPER-410. address all findbugs warnings in client/server classes. 
(phunt via breed)
+
 IMPROVEMENTS:
   ZOOKEEPER-308. improve the atomic broadcast performance 3x.
   (breed via mahadev)

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java?rev=776806r1=776805r2=776806view=diff
==
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java 
(original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java 
Wed May 20 18:48:00 2009
@@ -240,6 +240,20 @@
 }
 
 
+/**
+ * Creates a connection object. The actual network connect doesn't get
+ * established until needed. The start() instance method must be called
+ * subsequent to construction.
+ *
+ * @param hosts
+ *a comma separated list of hosts that can be connected to.
+ * @param sessionTimeout
+ *the timeout for connections.
+ * @param zooKeeper
+ *the zookeeper object that this connection is related to.
+ * @param watcher watcher for this connection
+ * @throws IOException
+ */
 public ClientCnxn(String hosts, int sessionTimeout, ZooKeeper zooKeeper,
 ClientWatchManager watcher)
 throws IOException
@@ -249,7 +263,8 @@
 
 /**
  * Creates a connection object. The actual network connect doesn't get
- * established until needed.
+ * established until needed. The start() instance method must be called
+ * subsequent to construction.
  *
  * @param hosts
  *a comma separated list of hosts that can be connected to.
@@ -257,12 +272,15 @@
  *the timeout for connections.
  * @param zooKeeper
  *the zookeeper object that this connection is related to.
- * @throws KeeperException
+ * @param watcher watcher for this connection
+ * @param sessionId session id if re-establishing session
+ * @param sessionPasswd session passwd if re-establishing session
  * @throws IOException
  */
 public ClientCnxn(String hosts, int sessionTimeout, ZooKeeper zooKeeper,
 ClientWatchManager watcher, long sessionId, byte[] sessionPasswd)
-throws IOException {
+throws IOException
+{
 this.zooKeeper = zooKeeper;
 this.watcher = watcher;
 this.sessionId = sessionId;
@@ -286,10 +304,13 @@
 Collections.shuffle(serverAddrs);
 sendThread = new SendThread();
 eventThread = new EventThread();
+}
+
+public void start() {
 sendThread.start();
 eventThread.start();
 }
-
+
 Object eventOfDeath = new Object();
 
 final static

svn commit: r776788 - in /hadoop/zookeeper/trunk: CHANGES.txt src/java/main/org/apache/zookeeper/ZooKeeperMain.java

2009-05-20 Thread breed
Author: breed
Date: Wed May 20 17:58:56 2009
New Revision: 776788

URL: http://svn.apache.org/viewvc?rev=776788view=rev
Log:
ZOOKEEPER-405 nullpointer exception in zookeeper java shell.

Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=776788r1=776787r2=776788view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Wed May 20 17:58:56 2009
@@ -74,6 +74,8 @@
 
   ZOOKEEPER-400. Issues with procedure to close ledger. (flavio)
 
+  ZOOKEEPER-405. nullpointer exception in zookeeper java shell. (mahadev via 
breed)
+
 IMPROVEMENTS:
   ZOOKEEPER-308. improve the atomic broadcast performance 3x.
   (breed via mahadev)

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java?rev=776788r1=776787r2=776788view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java 
(original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java 
Wed May 20 17:58:56 2009
@@ -613,6 +613,7 @@
 } else if (cmd.equals(get)  args.length = 2) {
 path = args[1];
 byte data[] = zk.getData(path, watch, stat);
+data = (data == null)? null.getBytes() : data;
 System.out.println(new String(data));
 printStat(stat);
 } else if (cmd.equals(ls)  args.length = 2) {




svn commit: r777658 - in /hadoop/zookeeper/trunk: CHANGES.txt src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java

2009-05-22 Thread breed
Author: breed
Date: Fri May 22 19:02:04 2009
New Revision: 777658

URL: http://svn.apache.org/viewvc?rev=777658view=rev
Log:
ZOOKEEPER-192.  trailing whitespace in config file can cause number format 
exceptions

Modified:
hadoop/zookeeper/trunk/CHANGES.txt

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=777658r1=777657r2=777658view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Fri May 22 19:02:04 2009
@@ -96,6 +96,8 @@
 
   ZOOKEEPER-385. crctest failed on hudson patch test (mahadev via phunt)
 
+  ZOOKEEPER-192.  trailing whitespace in config file can cause number format 
exceptions (phunt via breed)
+
 IMPROVEMENTS:
   ZOOKEEPER-308. improve the atomic broadcast performance 3x.
   (breed via mahadev)

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java?rev=777658r1=777657r2=777658view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
 Fri May 22 19:02:04 2009
@@ -103,8 +103,8 @@
 protected void parseProperties(Properties zkProp)
 throws IOException, ConfigException {
 for (EntryObject, Object entry : zkProp.entrySet()) {
-String key = entry.getKey().toString();
-String value = entry.getValue().toString();
+String key = entry.getKey().toString().trim();
+String value = entry.getValue().toString().trim();
 if (key.equals(dataDir)) {
 dataDir = value;
 } else if (key.equals(dataLogDir)) {




svn commit: r777662 - in /hadoop/zookeeper/trunk: ./ src/java/main/org/apache/jute/ src/java/main/org/apache/jute/compiler/ src/java/systest/org/apache/zookeeper/test/system/ src/java/test/config/

2009-05-22 Thread breed
Author: breed
Date: Fri May 22 19:06:02 2009
New Revision: 777662

URL: http://svn.apache.org/viewvc?rev=777662view=rev
Log:
ZOOKEEPER-409. address all findbugs warnings in jute related classes 

Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryInputArchive.java

hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryOutputArchive.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/CsvInputArchive.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/CsvOutputArchive.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/Index.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/InputArchive.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/OutputArchive.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/Record.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/RecordReader.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/RecordWriter.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/Utils.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/XmlInputArchive.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/XmlOutputArchive.java

hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/CGenerator.java

hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/CppGenerator.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JBoolean.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JBuffer.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JByte.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JCompType.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JDouble.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JField.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JFile.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JFloat.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JInt.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JLong.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JMap.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JRecord.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JString.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JType.java
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JVector.java

hadoop/zookeeper/trunk/src/java/main/org/apache/jute/compiler/JavaGenerator.java

hadoop/zookeeper/trunk/src/java/systest/org/apache/zookeeper/test/system/InstanceContainer.java
hadoop/zookeeper/trunk/src/java/test/config/findbugsExcludeFile.xml

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=777662r1=777661r2=777662view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Fri May 22 19:06:02 2009
@@ -97,6 +97,8 @@
   ZOOKEEPER-385. crctest failed on hudson patch test (mahadev via phunt)
 
   ZOOKEEPER-192.  trailing whitespace in config file can cause number format 
exceptions (phunt via breed)
+ 
+  ZOOKEEPER-409. address all findbugs warnings in jute related classes (phunt 
via breed)
 
 IMPROVEMENTS:
   ZOOKEEPER-308. improve the atomic broadcast performance 3x.

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryInputArchive.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryInputArchive.java?rev=777662r1=777661r2=777662view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryInputArchive.java 
(original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryInputArchive.java 
Fri May 22 19:06:02 2009
@@ -19,13 +19,12 @@
 package org.apache.jute;
 
 import java.io.DataInput;
-import java.io.IOException;
 import java.io.DataInputStream;
+import java.io.IOException;
 import java.io.InputStream;
 
 /**
  *
- * @author Milind Bhandarkar
  */
 public class BinaryInputArchive implements InputArchive {
 

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryOutputArchive.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryOutputArchive.java?rev=777662r1=777661r2=777662view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryOutputArchive.java 
(original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/jute/BinaryOutputArchive.java 
Fri May 22 19:06:02 2009
@@ -18,19 +18,16 @@
 
 package org.apache.jute;
 
-import java.io.IOException;
-import

svn commit: r781173 - in /hadoop/zookeeper/trunk: ./ src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/ src/contrib/bookkeeper/test/org/apache/bookkeeper/test/

2009-06-02 Thread breed
Author: breed
Date: Tue Jun  2 21:15:23 2009
New Revision: 781173

URL: http://svn.apache.org/viewvc?rev=781173view=rev
Log:
ZOOKEEPER-358. Throw exception when ledger does not exist

Modified:
hadoop/zookeeper/trunk/CHANGES.txt

hadoop/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BKException.java

hadoop/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookKeeper.java

hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=781173r1=781172r2=781173view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Tue Jun  2 21:15:23 2009
@@ -181,6 +181,8 @@
   include in release (phunt via breed)
 
   ZOOKEEPER-383. Asynchronous version of createLedger(). (flavio via mahadev)
+
+  ZOOKEEPER-358. Throw exception when ledger does not exist. (flavio via breed)
  
 NEW FEATURES:
 

Modified: 
hadoop/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BKException.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BKException.java?rev=781173r1=781172r2=781173view=diff
==
--- 
hadoop/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BKException.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BKException.java
 Tue Jun  2 21:15:23 2009
@@ -48,6 +48,8 @@
 return new BKDigestNotInitializedException();
 case Code.DigestMatchException:
 return new BKDigestMatchException();
+case Code.NoSuchLedgerExistsException:
+return new BKNoSuchLedgerExistsException();
 default:
 return new BKIllegalOpException();
 }
@@ -60,6 +62,7 @@
 int NoBookieAvailableException = -3;
 int DigestNotInitializedException = -4;
 int DigestMatchException = -5;
+int NoSuchLedgerExistsException = -6;
 
 int IllegalOpException = -100;
 }
@@ -86,6 +89,8 @@
 return Digest engine not initialized;
 case Code.DigestMatchException:
 return Entry digest does not match;
+case Code.NoSuchLedgerExistsException:
+return No such ledger exists;
 default:
 return Invalid operation;
 }
@@ -126,5 +131,11 @@
 super(Code.IllegalOpException);
 }   
 }
+
+public static class BKNoSuchLedgerExistsException extends BKException {
+public BKNoSuchLedgerExistsException(){
+super(Code.NoSuchLedgerExistsException);
+}   
+}
 }
 

Modified: 
hadoop/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookKeeper.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookKeeper.java?rev=781173r1=781172r2=781173view=diff
==
--- 
hadoop/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookKeeper.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/contrib/bookkeeper/src/java/org/apache/bookkeeper/client/BookKeeper.java
 Tue Jun  2 21:15:23 2009
@@ -303,7 +303,7 @@
  */
 if(zk.exists(BKDefs.prefix + getZKStringId(lId), false) == null){
 LOG.error(Ledger  + getZKStringId(lId) +  doesn't exist.);
-return null;
+throw BKException.create(Code.NoSuchLedgerExistsException);
 }
 
 /*

Modified: 
hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java?rev=781173r1=781172r2=781173view=diff
==
--- 
hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/contrib/bookkeeper/test/org/apache/bookkeeper/test/BookieReadWriteTest.java
 Tue Jun  2 21:15:23 2009
@@ -111,6 +111,18 @@
}   
 }
 
+@Test
+public void testOpenException() 
+throws KeeperException, IOException, InterruptedException {
+bkc = new BookKeeper(127.0.0.1);
+try{
+lh = bkc.openLedger(0, ledgerPassword);
+fail(Haven't thrown exception);
+} catch (BKException e) {
+LOG.warn(Successfully thrown and caught exception:, e

svn commit: r781914 - in /hadoop/zookeeper/trunk: ./ src/java/main/org/apache/zookeeper/server/ src/java/main/org/apache/zookeeper/server/quorum/

2009-06-04 Thread breed
Author: breed
Date: Fri Jun  5 04:44:23 2009
New Revision: 781914

URL: http://svn.apache.org/viewvc?rev=781914view=rev
Log:
ZOOKEEPER-431. Expose methods to ease ZK integration

Modified:
hadoop/zookeeper/trunk/CHANGES.txt

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerConfig.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=781914r1=781913r2=781914view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Fri Jun  5 04:44:23 2009
@@ -185,6 +185,8 @@
   ZOOKEEPER-383. Asynchronous version of createLedger(). (flavio via mahadev)
 
   ZOOKEEPER-358. Throw exception when ledger does not exist. (flavio via breed)
+
+  ZOOKEEPER-431. Expose methods to ease ZK integration. (Jean-Daniel via breed)
  
 NEW FEATURES:
 

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerConfig.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerConfig.java?rev=781914r1=781913r2=781914view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerConfig.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerConfig.java
 Fri Jun  5 04:44:23 2009
@@ -67,10 +67,18 @@
 
 // let qpconfig parse the file and then pull the stuff we are
 // interested in
-clientPort = config.getClientPort();
-dataDir = config.getDataDir();
-dataLogDir = config.getDataLogDir();
-tickTime = config.getTickTime();
+readFrom(config);
+}
+
+/**
+ * Read attributes from a QuorumPeerConfig.
+ * @param config
+ */
+public void readFrom(QuorumPeerConfig config) {
+  clientPort = config.getClientPort();
+  dataDir = config.getDataDir();
+  dataLogDir = config.getDataLogDir();
+  tickTime = config.getTickTime();
 }
 
 public int getClientPort() { return clientPort; }

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java?rev=781914r1=781913r2=781914view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java
 Fri Jun  5 04:44:23 2009
@@ -82,6 +82,15 @@
 config.parse(args);
 }
 
+runFromConfig(config);
+}
+
+/**
+ * Run from a ServerConfig.
+ * @param config ServerConfig to use.
+ * @throws IOException
+ */
+public void runFromConfig(ServerConfig config) throws IOException {
 LOG.info(Starting server);
 try {
 // Note that this thread isn't going to be doing anything else,

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java?rev=781914r1=781913r2=781914view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
 Fri Jun  5 04:44:23 2009
@@ -100,7 +100,13 @@
 }
 }
 
-protected void parseProperties(Properties zkProp)
+/**
+ * Parse config from a Properties.
+ * @param zkProp Properties to parse from.
+ * @throws IOException
+ * @throws ConfigException
+ */
+public void parseProperties(Properties zkProp)
 throws IOException, ConfigException {
 for (EntryObject, Object entry : zkProp.entrySet()) {
 String key = entry.getKey().toString().trim();
@@ -263,4 +269,6 @@
 }
 
 public long getServerId() { return serverId; }
+
+public boolean isDistributed() { return servers.size()  1; }
 }

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java?rev=781914r1=781913r2=781914view=diff

svn commit: r782878 - in /hadoop/zookeeper/trunk: ./ src/c/ src/c/tests/ src/docs/src/documentation/content/xdocs/ src/java/main/org/apache/zookeeper/ src/java/main/org/apache/zookeeper/server/ src/ja

2009-06-08 Thread breed
Author: breed
Date: Tue Jun  9 05:04:36 2009
New Revision: 782878

URL: http://svn.apache.org/viewvc?rev=782878view=rev
Log:
ZOOKEEPER-336. single bad client can cause server to stop accepting connections

Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/src/c/Makefile.am
hadoop/zookeeper/trunk/src/c/tests/zkServer.sh

hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerConfig.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=782878r1=782877r2=782878view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Tue Jun  9 05:04:36 2009
@@ -194,6 +194,8 @@
   ZOOKEEPER-196. doxygen comment for state argument of watcher_fn typedef and
 implementation differ (...one of the *_STATE constants, otherwise -1) (breed
 via mahadev)
+
+  ZOOKEEPER-336. single bad client can cause server to stop accepting 
connections (henry robinson via breed)
  
 NEW FEATURES:
 

Modified: hadoop/zookeeper/trunk/src/c/Makefile.am
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/Makefile.am?rev=782878r1=782877r2=782878view=diff
==
--- hadoop/zookeeper/trunk/src/c/Makefile.am (original)
+++ hadoop/zookeeper/trunk/src/c/Makefile.am Tue Jun  9 05:04:36 2009
@@ -70,9 +70,11 @@
 
 TEST_SOURCES = tests/TestDriver.cc tests/LibCMocks.cc tests/LibCSymTable.cc \
 tests/MocksBase.cc  tests/ZKMocks.cc tests/Util.cc tests/ThreadingUtil.cc \
-tests/TestWatchers.cc \
+tests/TestClientRetry.cc \
 tests/TestOperations.cc tests/TestZookeeperInit.cc \
-tests/TestZookeeperClose.cc tests/TestClient.cc
+tests/TestZookeeperClose.cc tests/TestClient.cc \
+tests/TestWatchers.cc
+
 
 SYMBOL_WRAPPERS=$(shell cat ${srcdir}/tests/wrappers.opt)
 

Modified: hadoop/zookeeper/trunk/src/c/tests/zkServer.sh
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/tests/zkServer.sh?rev=782878r1=782877r2=782878view=diff
==
--- hadoop/zookeeper/trunk/src/c/tests/zkServer.sh (original)
+++ hadoop/zookeeper/trunk/src/c/tests/zkServer.sh Tue Jun  9 05:04:36 2009
@@ -53,12 +53,12 @@
if [ x${base_dir} == x ]
 then
mkdir -p /tmp/zkdata
-java -cp 
../../zookeeper-dev.jar:../../src/java/lib/log4j-1.2.15.jar:../../conf 
org.apache.zookeeper.server.ZooKeeperServerMain 22181 /tmp/zkdata  
/tmp/zk.log 
+java -cp ../../zookeeper-dev.jar:../../src/java/lib/log4j-1.2.15.jar 
org.apache.zookeeper.server.ZooKeeperServerMain 22181 /tmp/zkdata 3000 
$ZKMAXCNXNS  /tmp/zk.log 
 pid=$!
 echo $!  /tmp/zk.pid
 else
 mkdir -p ${base_dir}/build/tmp/zkdata
-java -cp 
${base_dir}/zookeeper-dev.jar:${base_dir}/src/java/lib/log4j-1.2.15.jar:${base_dir}/conf
 org.apache.zookeeper.server.ZooKeeperServerMain 22181 
${base_dir}/build/tmp/zkdata  ${base_dir}/build/tmp/zk.log 
+java -cp 
${base_dir}/zookeeper-dev.jar:${base_dir}/src/java/lib/log4j-1.2.15.jar:${base_dir}/conf
 org.apache.zookeeper.server.ZooKeeperServerMain 22181 
${base_dir}/build/tmp/zkdata 3000 $ZKMAXCNXNS  ${base_dir}/build/tmp/zk.log 
 echo $!  ${base_dir}/build/tmp/zk.pid
fi
 sleep 5

Modified: 
hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml?rev=782878r1=782877r2=782878view=diff
==
--- 
hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
 (original)
+++ 
hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
 Tue Jun  9 05:04:36 2009
@@ -676,6 +676,19 @@
   property. Yes - it's not consistent, and it's annoying.)/para
 /listitem
   /varlistentry
+  varlistentry
+termmaxClientCnxns/term
+listitem
+  para(No Java system property)/para
+
+  paraLimits the number of concurrent connections

svn commit: r782880 - in /hadoop/zookeeper/trunk/src: docs/src/documentation/content/xdocs/zookeeperStarted.xml java/main/org/apache/zookeeper/ZooKeeperMain.java

2009-06-08 Thread breed
Author: breed
Date: Tue Jun  9 05:12:16 2009
New Revision: 782880

URL: http://svn.apache.org/viewvc?rev=782880view=rev
Log:
ZOOKEEPER-434. the java shell should indicate connection status on command 
prompt

Modified:

hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperStarted.xml
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java

Modified: 
hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperStarted.xml
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperStarted.xml?rev=782880r1=782879r2=782880view=diff
==
--- 
hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperStarted.xml
 (original)
+++ 
hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperStarted.xml
 Tue Jun  9 05:12:16 2009
@@ -171,7 +171,7 @@
 listitem
   paraemphasis role=boldJava/emphasis: Use/para
 
-  programlistingbin/zkCli.sh 127.0.0.1:2181/programlisting
+  programlistingbin/zkCli.sh -server 127.0.0.1:2181/programlisting
 
   paraThis lets you perform simple, file-like operations./para
 /listitem

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java?rev=782880r1=782879r2=782880view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java 
(original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java 
Tue Jun  9 05:12:16 2009
@@ -54,12 +54,15 @@
 protected boolean printWatches = true;
 
 protected ZooKeeper zk;
+protected String host = ;
 
 public boolean getPrintWatches( ) {
 return printWatches;
 }
 
 static void populateCommandMap() {
+commandMap.put(connect,  host:port);
+commandMap.put(close,);
 commandMap.put(create, path data acl);
 commandMap.put(delete, path [version]);
 commandMap.put(set, path data [version]);
@@ -80,7 +83,7 @@
 }
 
 static void usage() {
-System.err.println(ZooKeeper host:port cmd args);
+System.err.println(ZooKeeper -server host:port cmd args);
 for (String cmd : commandMap.keySet()) {
 System.err.println(\t+cmd+commandMap.get(cmd));
 }
@@ -90,6 +93,7 @@
 public void process(WatchedEvent event) {
 if (getPrintWatches()) {
 ZooKeeperMain.printMessage(WATCHER::);
+ZooKeeperMain.printMessage(WATCHER::);
 ZooKeeperMain.printMessage(event.toString());
 }
 }
@@ -247,14 +251,24 @@
 return new LinkedListString(commandMap.keySet());
 }
 
-protected String getPrompt() {
-return [zkshell:  + commandCount + ] ;
+protected String getPrompt() {   
+return [zk:  + host + (+zk.getState()+) +   + commandCount + 
] ;
 }
 
 public static void printMessage(String msg) {
 System.out.println(\n+msg);
 }
 
+protected void connectToZK(String newHost) throws InterruptedException, 
IOException {
+if (zk != null  zk.getState().isAlive()) {
+zk.close();
+}
+host = newHost;
+zk = new ZooKeeper(host,
+ Integer.parseInt(cl.getOption(timeout)),
+ new MyWatcher());
+}
+
 public static void main(String args[])
 throws KeeperException, IOException, InterruptedException
 {
@@ -263,11 +277,12 @@
 main.run();
 }
 
-public ZooKeeperMain(String args[]) throws IOException {
+public ZooKeeperMain(String args[]) throws IOException, 
InterruptedException {
 cl.parseOptions(args);
 System.out.println(Connecting to  + cl.getOption(server));
-zk = new ZooKeeper(cl.getOption(server),
-Integer.parseInt(cl.getOption(timeout)), new MyWatcher());
+connectToZK(cl.getOption(server));
+//zk = new ZooKeeper(cl.getOption(server),
+//Integer.parseInt(cl.getOption(timeout)), new MyWatcher());
 }
 
 @SuppressWarnings(unchecked)
@@ -590,10 +605,58 @@
 return false;
 }
 
+if (!commandMap.containsKey(cmd)) {
+usage();
+return false;
+}
+
 boolean watch = args.length  2;
 String path = null;
 ListACL acl = Ids.OPEN_ACL_UNSAFE;
 LOG.debug(Processing  + cmd);
+
+if (cmd.equals(quit)) {
+System.out.println(Quitting...);
+zk.close();
+System.exit(0);
+} else if (cmd.equals(redo)  args.length = 2) {
+Integer i = Integer.decode(args[1]);
+if (commandCount

svn commit: r786317 - in /hadoop/zookeeper/trunk: CHANGES.txt src/java/main/org/apache/zookeeper/ZooKeeperMain.java

2009-06-18 Thread breed
Author: breed
Date: Thu Jun 18 22:22:44 2009
New Revision: 786317

URL: http://svn.apache.org/viewvc?rev=786317view=rev
Log:
ZOOKEEPER-422. Java CLI should support ephemeral and sequential node creation

Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=786317r1=786316r2=786317view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Thu Jun 18 22:22:44 2009
@@ -223,6 +223,8 @@
   ZOOKEEPER-428. logging should be makred as warn rathen than error in
 NIOServerCnxn. (phunt via mahadev)
 
+  ZOOKEEPER-422. Java CLI should support ephemeral and sequential node 
creation (henry via breed)
+
 NEW FEATURES:
 
   ZOOKEEPER-371. jdiff documentation included in build/release (giri via phunt)

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java?rev=786317r1=786316r2=786317view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java 
(original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java 
Thu Jun 18 22:22:44 2009
@@ -63,7 +63,7 @@
 static void populateCommandMap() {
 commandMap.put(connect, host:port);
 commandMap.put(close,);
-commandMap.put(create, path data acl);
+commandMap.put(create, [-s] [-e] path data acl);
 commandMap.put(delete,path [version]);
 commandMap.put(set,path data [version]);
 commandMap.put(get,path [watch]);
@@ -93,7 +93,6 @@
 public void process(WatchedEvent event) {
 if (getPrintWatches()) {
 ZooKeeperMain.printMessage(WATCHER::);
-ZooKeeperMain.printMessage(WATCHER::);
 ZooKeeperMain.printMessage(event.toString());
 }
 }
@@ -661,12 +660,25 @@
 }
 
 if (cmd.equals(create)  args.length = 3) {
-if (args.length == 4) {
-acl = parseACLs(args[3]);
-}
-path = args[1];
-String newPath = zk.create(path, args[2].getBytes(), acl,
-CreateMode.PERSISTENT);
+int first = 0;
+CreateMode flags = CreateMode.PERSISTENT;
+if ((args[1].equals(-e)  args[2].equals(-s))
+|| (args[1]).equals(-s)  (args[2].equals(-e))) {
+first+=2;
+flags = CreateMode.EPHEMERAL_SEQUENTIAL;
+} else if (args[1].equals(-e)) {
+first++;
+flags = CreateMode.EPHEMERAL;
+} else if (args[1].equals(-s)) {
+first++;
+flags = CreateMode.PERSISTENT_SEQUENTIAL;
+}
+if (args.length == first + 4) {
+acl = parseACLs(args[first+3]);
+}
+path = args[first + 1];
+String newPath = zk.create(path, args[first+2].getBytes(), acl,
+flags);
 System.err.println(Created  + newPath);
 } else if (cmd.equals(delete)  args.length = 2) {
 path = args[1];




svn commit: r801747 - in /hadoop/zookeeper/trunk: CHANGES.txt src/c/include/zookeeper.h src/c/src/zookeeper.c src/c/tests/TestClient.cc

2009-08-06 Thread breed
Author: breed
Date: Thu Aug  6 18:03:25 2009
New Revision: 801747

URL: http://svn.apache.org/viewvc?rev=801747view=rev
Log:
ZOOKEEPER-311. handle small path lengths in zoo_create()

Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/src/c/include/zookeeper.h
hadoop/zookeeper/trunk/src/c/src/zookeeper.c
hadoop/zookeeper/trunk/src/c/tests/TestClient.cc

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=801747r1=801746r2=801747view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Thu Aug  6 18:03:25 2009
@@ -46,6 +46,8 @@
 
   ZOOKEEPER-493. patch for command line setquota (steve bendiola via phunt)
 
+  ZOOKEEPER-311. handle small path lengths in zoo_create() (chris barroch via 
breed)
+
 IMPROVEMENTS:
   ZOOKEEPER-473. cleanup junit tests to eliminate false positives due to
   socket reuse and failure to close client (phunt via mahadev)

Modified: hadoop/zookeeper/trunk/src/c/include/zookeeper.h
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/include/zookeeper.h?rev=801747r1=801746r2=801747view=diff
==
--- hadoop/zookeeper/trunk/src/c/include/zookeeper.h (original)
+++ hadoop/zookeeper/trunk/src/c/include/zookeeper.h Thu Aug  6 18:03:25 2009
@@ -958,9 +958,15 @@
  *used.
  * \param flags this parameter can be set to 0 for normal create or an OR
  *of the Create Flags
- * \param realpath the real path that is created (this might be different than 
the
- *path to create because of the ZOO_SEQUENCE flag.
- * \param the maximum length of real path you would want.
+ * \param path_buffer Buffer which will be filled with the path of the
+ *new node (this might be different than the supplied path
+ *because of the ZOO_SEQUENCE flag).  The path string will always be
+ *null-terminated.
+ * \param path_buffer_len Size of path buffer; if the path of the new
+ *node (including space for the null terminator) exceeds the buffer size,
+ *the path string will be truncated to fit.  The actual path of the
+ *new node in the server will not be affected by the truncation.
+ *The path string will always be null-terminated.
  * \return  one of the following codes are returned:
  * ZOK operation completed succesfully
  * ZNONODE the parent node does not exist.
@@ -972,8 +978,8 @@
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  */
 ZOOAPI int zoo_create(zhandle_t *zh, const char *path, const char *value,
-int valuelen, const struct ACL_vector *acl, int flags, char *realpath, 
 
-int max_realpath_len);
+int valuelen, const struct ACL_vector *acl, int flags,
+char *path_buffer, int path_buffer_len);
 
 /**
  * \brief delete a node in zookeeper synchronously.

Modified: hadoop/zookeeper/trunk/src/c/src/zookeeper.c
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/src/zookeeper.c?rev=801747r1=801746r2=801747view=diff
==
--- hadoop/zookeeper/trunk/src/c/src/zookeeper.c (original)
+++ hadoop/zookeeper/trunk/src/c/src/zookeeper.c Thu Aug  6 18:03:25 2009
@@ -1878,14 +1878,13 @@
 struct CreateResponse res;
 int len;
 deserialize_CreateResponse(ia, reply, res);
-if (sc-u.str.str_len  strlen(res.path)) {
-len = strlen(res.path);
-} else {
-len = sc-u.str.str_len-1;
+len = strlen(res.path) + 1;
+if (len  sc-u.str.str_len) {
+len = sc-u.str.str_len;
 }
 if (len  0) {
-memcpy(sc-u.str.str, res.path, len);
-sc-u.str.str[len] = '\0';
+memcpy(sc-u.str.str, res.path, len - 1);
+sc-u.str.str[len - 1] = '\0';
 }
 deallocate_CreateResponse(res);
 }
@@ -2724,16 +2723,16 @@
  * sync API
  */
 int zoo_create(zhandle_t *zh, const char *path, const char *value,
-int valuelen, const struct ACL_vector *acl, int flags, char *realpath,
-int max_realpath_len)
+int valuelen, const struct ACL_vector *acl, int flags,
+char *path_buffer, int path_buffer_len)
 {
 struct sync_completion *sc = alloc_sync_completion();
 int rc;
 if (!sc) {
 return ZSYSTEMERROR;
 }
-sc-u.str.str = realpath;
-sc-u.str.str_len = max_realpath_len;
+sc-u.str.str = path_buffer;
+sc-u.str.str_len = path_buffer_len;
 rc=zoo_acreate(zh, path

svn commit: r801756 - in /hadoop/zookeeper/branches/branch-3.2: CHANGES.txt src/c/include/zookeeper.h src/c/src/zookeeper.c src/c/tests/TestClient.cc

2009-08-06 Thread breed
Author: breed
Date: Thu Aug  6 18:15:08 2009
New Revision: 801756

URL: http://svn.apache.org/viewvc?rev=801756view=rev
Log:
ZOOKEEPER-311. handle small path lengths in zoo_create()

Modified:
hadoop/zookeeper/branches/branch-3.2/CHANGES.txt
hadoop/zookeeper/branches/branch-3.2/src/c/include/zookeeper.h
hadoop/zookeeper/branches/branch-3.2/src/c/src/zookeeper.c
hadoop/zookeeper/branches/branch-3.2/src/c/tests/TestClient.cc

Modified: hadoop/zookeeper/branches/branch-3.2/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.2/CHANGES.txt?rev=801756r1=801755r2=801756view=diff
==
--- hadoop/zookeeper/branches/branch-3.2/CHANGES.txt (original)
+++ hadoop/zookeeper/branches/branch-3.2/CHANGES.txt Thu Aug  6 18:15:08 2009
@@ -43,6 +43,8 @@
 
   ZOOKEEPER-493. patch for command line setquota (steve bendiola via phunt)
 
+  ZOOKEEPER-311. handle small path lengths in zoo_create() (chris barroch via 
breed)
+
 IMPROVEMENTS:
 
 NEW FEATURES:

Modified: hadoop/zookeeper/branches/branch-3.2/src/c/include/zookeeper.h
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.2/src/c/include/zookeeper.h?rev=801756r1=801755r2=801756view=diff
==
--- hadoop/zookeeper/branches/branch-3.2/src/c/include/zookeeper.h (original)
+++ hadoop/zookeeper/branches/branch-3.2/src/c/include/zookeeper.h Thu Aug  6 
18:15:08 2009
@@ -958,9 +958,15 @@
  *used.
  * \param flags this parameter can be set to 0 for normal create or an OR
  *of the Create Flags
- * \param realpath the real path that is created (this might be different than 
the
- *path to create because of the ZOO_SEQUENCE flag.
- * \param the maximum length of real path you would want.
+ * \param path_buffer Buffer which will be filled with the path of the
+ *new node (this might be different than the supplied path
+ *because of the ZOO_SEQUENCE flag).  The path string will always be
+ *null-terminated.
+ * \param path_buffer_len Size of path buffer; if the path of the new
+ *node (including space for the null terminator) exceeds the buffer size,
+ *the path string will be truncated to fit.  The actual path of the
+ *new node in the server will not be affected by the truncation.
+ *The path string will always be null-terminated.
  * \return  one of the following codes are returned:
  * ZOK operation completed succesfully
  * ZNONODE the parent node does not exist.
@@ -972,8 +978,8 @@
  * ZMARSHALLINGERROR - failed to marshall a request; possibly, out of memory
  */
 ZOOAPI int zoo_create(zhandle_t *zh, const char *path, const char *value,
-int valuelen, const struct ACL_vector *acl, int flags, char *realpath, 
 
-int max_realpath_len);
+int valuelen, const struct ACL_vector *acl, int flags,
+char *path_buffer, int path_buffer_len);
 
 /**
  * \brief delete a node in zookeeper synchronously.

Modified: hadoop/zookeeper/branches/branch-3.2/src/c/src/zookeeper.c
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.2/src/c/src/zookeeper.c?rev=801756r1=801755r2=801756view=diff
==
--- hadoop/zookeeper/branches/branch-3.2/src/c/src/zookeeper.c (original)
+++ hadoop/zookeeper/branches/branch-3.2/src/c/src/zookeeper.c Thu Aug  6 
18:15:08 2009
@@ -1878,14 +1878,13 @@
 struct CreateResponse res;
 int len;
 deserialize_CreateResponse(ia, reply, res);
-if (sc-u.str.str_len  strlen(res.path)) {
-len = strlen(res.path);
-} else {
-len = sc-u.str.str_len-1;
+len = strlen(res.path) + 1;
+if (len  sc-u.str.str_len) {
+len = sc-u.str.str_len;
 }
 if (len  0) {
-memcpy(sc-u.str.str, res.path, len);
-sc-u.str.str[len] = '\0';
+memcpy(sc-u.str.str, res.path, len - 1);
+sc-u.str.str[len - 1] = '\0';
 }
 deallocate_CreateResponse(res);
 }
@@ -2724,16 +2723,16 @@
  * sync API
  */
 int zoo_create(zhandle_t *zh, const char *path, const char *value,
-int valuelen, const struct ACL_vector *acl, int flags, char *realpath,
-int max_realpath_len)
+int valuelen, const struct ACL_vector *acl, int flags,
+char *path_buffer, int path_buffer_len)
 {
 struct sync_completion *sc = alloc_sync_completion();
 int rc;
 if (!sc) {
 return ZSYSTEMERROR;
 }
-sc-u.str.str = realpath;
-sc-u.str.str_len = max_realpath_len

svn commit: r833938 - /hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncHammerTest.java

2009-11-08 Thread breed
Author: breed
Date: Sun Nov  8 22:46:04 2009
New Revision: 833938

URL: http://svn.apache.org/viewvc?rev=833938view=rev
Log:
ZOOKEEPER-570. AsyncHammerTest is broken, callbacks need to validate rc 
parameter

Modified:

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncHammerTest.java

Modified: 
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncHammerTest.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncHammerTest.java?rev=833938r1=833937r2=833938view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncHammerTest.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncHammerTest.java
 Sun Nov  8 22:46:04 2009
@@ -27,14 +27,15 @@
 
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.TestableZooKeeper;
 import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.AsyncCallback.DataCallback;
 import org.apache.zookeeper.AsyncCallback.StringCallback;
 import org.apache.zookeeper.AsyncCallback.VoidCallback;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.test.ClientBase.CountdownWatcher;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -56,6 +57,10 @@
 }
 
 protected void restart() throws Exception {
+LOG.info(RESTARTING  + getName());
+qb.tearDown();
+
+// don't call setup - we don't want to reassign ports/dirs, etc...
 JMXEnv.setUp();
 qb.startServers();
 }
@@ -71,21 +76,24 @@
 /**
  * Create /test- sequence nodes asynchronously, max 30 outstanding
  */
-class HammerThread extends Thread
-implements Watcher, StringCallback, VoidCallback
-{
+class HammerThread extends Thread implements StringCallback, VoidCallback {
 private static final int MAX_OUTSTANDING = 30;
 
-private ZooKeeper zk;
+private TestableZooKeeper zk;
 private int outstanding;
 
+private volatile boolean failed = false;
+
 public HammerThread(String name) {
 super(name);
 }
 
 public void run() {
 try {
-zk = new ZooKeeper(qb.hostPort, CONNECTION_TIMEOUT, this);
+CountdownWatcher watcher = new CountdownWatcher();
+zk = new TestableZooKeeper(qb.hostPort, CONNECTION_TIMEOUT,
+watcher);
+watcher.waitForConnected(CONNECTION_TIMEOUT);
 while(bang) {
 incOutstanding(); // before create otw race
 zk.create(/test-, new byte[0], Ids.OPEN_ACL_UNSAFE,
@@ -103,8 +111,12 @@
 if (zk != null) {
 try {
 zk.close();
+if (!zk.testableWaitForShutdown(CONNECTION_TIMEOUT)) {
+failed = true;
+LOG.error(Client did not shutdown);
+}
 } catch (InterruptedException e) {
-LOG.warn(Unexpected, e);
+LOG.info(Interrupted, e);
 }
 }
 }
@@ -128,43 +140,67 @@
 }
 
 public void processResult(int rc, String path, Object ctx, String 
name) {
+if (rc != KeeperException.Code.OK.intValue()) {
+if (bang) {
+failed = true;
+LOG.error(Create failed for 0x
++ Long.toHexString(zk.getSessionId())
++ with rc: + rc +  path: + path);
+}
+decOutstanding();
+return;
+}
 try {
 decOutstanding();
-zk.delete(path, -1, this, null);
+zk.delete(name, -1, this, null);
 } catch (Exception e) {
-LOG.error(Client delete failed, e);
+if (bang) {
+failed = true;
+LOG.error(Client delete failed, e);
+}
 }
 }
 
 public void processResult(int rc, String path, Object ctx) {
-// ignore for purposes of this test
+if (rc != KeeperException.Code.OK.intValue()) {
+if (bang) {
+failed = true;
+LOG.error(Delete failed for 0x
++ Long.toHexString(zk.getSessionId())
++ with rc: + rc +  path: + path);
+}
+}
 }
 }
 
 @Test
 public void testHammer() throws Exception

svn commit: r835515 - in /hadoop/zookeeper/trunk/src: docs/src/documentation/content/xdocs/ java/main/org/apache/zookeeper/server/ java/test/org/apache/zookeeper/test/

2009-11-12 Thread breed
Author: breed
Date: Thu Nov 12 19:53:07 2009
New Revision: 835515

URL: http://svn.apache.org/viewvc?rev=835515view=rev
Log:
ZOOKEEPER-566. reqs four letter word (command port) returns no information

Added:

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FourLetterWordsQuorumTest.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java
Modified:

hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ConnectionBean.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ConnectionMXBean.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxn.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java

Modified: 
hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml?rev=835515r1=835514r2=835515view=diff
==
--- 
hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
 (original)
+++ 
hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
 Thu Nov 12 19:53:07 2009
@@ -930,29 +930,45 @@
   composed of four letters. You issue the commands to ZooKeeper via telnet
   or nc, at the client port./para
 
+  paraThree of the more interesting commands: stat gives some
+  general information about the server and connected clients,
+  while srvr and cons give extended details on server and
+  connections respectively./para
+
   variablelist
 varlistentry
-  termdump/term
+  termcons/term
 
   listitem
-paraLists the outstanding sessions and ephemeral nodes. This
-only works on the leader./para
+paraList full connection/session details for all clients
+connected to this server. Includes information on numbers
+of packets received/sent, session id, operation latencies,
+last operation performed, etc.../para
   /listitem
 /varlistentry
 
 varlistentry
-  termenvi/term
+  termcrst/term
 
   listitem
-paraPrint details about serving environment/para
+paraReset connection/session statistics for all 
connections./para
   /listitem
 /varlistentry
 
 varlistentry
-  termreqs/term
+  termdump/term
 
   listitem
-paraList outstanding requests/para
+paraLists the outstanding sessions and ephemeral nodes. This
+only works on the leader./para
+  /listitem
+/varlistentry
+
+varlistentry
+  termenvi/term
+
+  listitem
+paraPrint details about serving environment/para
   /listitem
 /varlistentry
 
@@ -970,7 +986,15 @@
   termsrst/term
 
   listitem
-paraReset statistics returned by stat command./para
+paraReset server statistics./para
+  /listitem
+/varlistentry
+
+varlistentry
+  termsrvr/term
+
+  listitem
+paraLists full details for the server./para
   /listitem
 /varlistentry
 
@@ -978,7 +1002,7 @@
   termstat/term
 
   listitem
-paraLists statistics about performance and connected
+paraLists brief details for the server and connected
 clients./para
   /listitem
 /varlistentry

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ConnectionBean.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ConnectionBean.java?rev=835515r1=835514r2=835515view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ConnectionBean.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ConnectionBean.java
 Thu Nov 12 19:53:07 2009
@@ -29,20 +29,23 @@
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.jmx.MBeanRegistry;
 import org.apache.zookeeper.jmx.ZKMBeanInfo;
+import org.apache.zookeeper.server.NIOServerCnxn.CnxnStats;
 
 /**
  * Implementation of connection MBean interface.
  */
 public class ConnectionBean implements ConnectionMXBean, ZKMBeanInfo {
 private static final Logger LOG = Logger.getLogger

svn commit: r835519 - in /hadoop/zookeeper/trunk/src: c/include/zookeeper.h java/main/org/apache/zookeeper/ZooKeeper.java

2009-11-12 Thread breed
Author: breed
Date: Thu Nov 12 20:02:35 2009
New Revision: 835519

URL: http://svn.apache.org/viewvc?rev=835519view=rev
Log:
ZOOKEEPER-567. javadoc for getchildren2 needs to mention new in 3.3.0

Modified:
hadoop/zookeeper/trunk/src/c/include/zookeeper.h
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java

Modified: hadoop/zookeeper/trunk/src/c/include/zookeeper.h
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/include/zookeeper.h?rev=835519r1=835518r2=835519view=diff
==
--- hadoop/zookeeper/trunk/src/c/include/zookeeper.h (original)
+++ hadoop/zookeeper/trunk/src/c/include/zookeeper.h Thu Nov 12 20:02:35 2009
@@ -830,6 +830,8 @@
 /**
  * \brief lists the children of a node, and get the parent stat.
  * 
+ * This function is new in version 3.3.0
+ *
  * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
  * \param path the name of the node. Expressed as a file name with slashes 
  * separating ancestors of the node.
@@ -856,6 +858,8 @@
  * This function is similar to \ref zoo_aget_children2 except it allows one 
specify 
  * a watcher object rather than a boolean watch flag.
  *  
+ * This function is new in version 3.3.0
+ *
  * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
  * \param path the name of the node. Expressed as a file name with slashes 
  * separating ancestors of the node.
@@ -1288,6 +1292,8 @@
 /**
  * \brief lists the children of a node and get its stat synchronously.
  * 
+ * This function is new in version 3.3.0
+ *
  * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
  * \param path the name of the node. Expressed as a file name with slashes 
  * separating ancestors of the node.
@@ -1312,6 +1318,8 @@
  * This function is similar to \ref zoo_get_children except it allows one 
specify 
  * a watcher object rather than a boolean watch flag.
  * 
+ * This function is new in version 3.3.0
+ *
  * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
  * \param path the name of the node. Expressed as a file name with slashes 
  * separating ancestors of the node.

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java?rev=835519r1=835518r2=835519view=diff
==
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java 
(original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java 
Thu Nov 12 20:02:35 2009
@@ -1312,6 +1312,8 @@
  * A KeeperException with error code KeeperException.NoNode will be thrown
  * if no node with the given path exists.
  *
+ * @since 3.3.0
+ * 
  * @param path
  * @param watcher explicit watcher
  * @param stat stat of the znode designated by path
@@ -1366,6 +1368,8 @@
  * A KeeperException with error code KeeperException.NoNode will be thrown
  * if no node with the given path exists.
  *
+ * @since 3.3.0
+ * 
  * @param path
  * @param watch
  * @param stat stat of the znode designated by path
@@ -1384,7 +1388,9 @@
  * The Asynchronous version of getChildren. The request doesn't actually
  * until the asynchronous callback is called.
  *
- * @see #getChildren(String, Watcher)
+ * @since 3.3.0
+ * 
+ * @see #getChildren(String, Watcher, Stat)
  */
 public void getChildren(final String path, Watcher watcher,
 Children2Callback cb, Object ctx)
@@ -1414,7 +1420,9 @@
  * The Asynchronous version of getChildren. The request doesn't actually
  * until the asynchronous callback is called.
  *
- * @see #getChildren(String, boolean)
+ * @since 3.3.0
+ * 
+ * @see #getChildren(String, boolean, Stat)
  */
 public void getChildren(String path, boolean watch, Children2Callback cb,
 Object ctx)




svn commit: r881847 - in /hadoop/zookeeper/trunk: CHANGES.txt src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java

2009-11-18 Thread breed
Author: breed
Date: Wed Nov 18 17:44:56 2009
New Revision: 881847

URL: http://svn.apache.org/viewvc?rev=881847view=rev
Log:
ZOOKEEPER-519. Followerhandler should close the socket if it gets an exception 
on a write.

Modified:
hadoop/zookeeper/trunk/CHANGES.txt

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=881847r1=881846r2=881847view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Wed Nov 18 17:44:56 2009
@@ -117,6 +117,9 @@
 
   ZOOKEEPER-532. java compiler should be target Java 1.5 (hiram chirino and 
phunt via breed)
 
+  ZOOKEEPER-519. Followerhandler should close the socket if it gets an 
exception on a write.
+  (mahadev via breed)
+
 IMPROVEMENTS:
   ZOOKEEPER-473. cleanup junit tests to eliminate false positives due to
   socket reuse and failure to close client (phunt via mahadev)

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java?rev=881847r1=881846r2=881847view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java
 Wed Nov 18 17:44:56 2009
@@ -420,6 +420,13 @@
 if (sock != null  !sock.isClosed()) {
 LOG.error(Unexpected exception causing shutdown while sock 
 + still open, e);
+   //close the socket to make sure the 
+   //other side can see it being close
+   try {
+   sock.close();
+   } catch(IOException ie) {
+   // do nothing
+   }
 }
 } catch (InterruptedException e) {
 LOG.error(Unexpected exception causing shutdown, e);




svn commit: r881849 - in /hadoop/zookeeper/trunk: CHANGES.txt src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java

2009-11-18 Thread breed
Author: breed
Date: Wed Nov 18 17:47:38 2009
New Revision: 881849

URL: http://svn.apache.org/viewvc?rev=881849view=rev
Log:
ZOOKEEPER-3. syncLimit has slightly different comments in the class header, and 
 inline with the variable.

Modified:
hadoop/zookeeper/trunk/CHANGES.txt

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=881849r1=881848r2=881849view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Wed Nov 18 17:47:38 2009
@@ -120,6 +120,9 @@
   ZOOKEEPER-519. Followerhandler should close the socket if it gets an 
exception on a write.
   (mahadev via breed)
 
+  ZOOKEEPER-3. syncLimit has slightly different comments in the class header, 
and  inline with the variable.
+  (mahadev via breed)
+
 IMPROVEMENTS:
   ZOOKEEPER-473. cleanup junit tests to eliminate false positives due to
   socket reuse and failure to close client (phunt via mahadev)

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java?rev=881849r1=881848r2=881849view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java
 Wed Nov 18 17:47:38 2009
@@ -670,16 +670,14 @@
 }
 
 /**
- * Get the number of ticks that can pass between sending a request and 
getting
- * an acknowledgement
+ * Get the synclimit
  */
 public int getSyncLimit() {
 return syncLimit;
 }
 
 /**
- * Set the number of ticks that can pass between sending a request and 
getting
- * an acknowledgement
+ * Set the synclimit
  */
 public void setSyncLimit(int syncLimit) {
 this.syncLimit = syncLimit;




svn commit: r884554 - in /hadoop/zookeeper/trunk: CHANGES.txt build.xml

2009-11-26 Thread breed
Author: breed
Date: Thu Nov 26 13:36:46 2009
New Revision: 884554

URL: http://svn.apache.org/viewvc?rev=884554view=rev
Log:
ZOOKEEPER-425. Add OSGi metadata to zookeeper.jar

Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/build.xml

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=884554r1=884553r2=884554view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Thu Nov 26 13:36:46 2009
@@ -157,6 +157,8 @@
   ZOOKEEPER-472.  Making DataNode not instantiate a HashMap when the node is
   ephmeral (Erik Holstad via mahadev)
 
+  ZOOKEEPER-425. Add OSGi metadata to zookeeper.jar (david bosschaert via 
breed)
+
 NEW FEATURES:
   ZOOKEEPER-539. generate eclipse project via ant target. (phunt via mahadev)
 

Modified: hadoop/zookeeper/trunk/build.xml
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/build.xml?rev=884554r1=884553r2=884554view=diff
==
--- hadoop/zookeeper/trunk/build.xml (original)
+++ hadoop/zookeeper/trunk/build.xml Thu Nov 26 13:36:46 2009
@@ -438,6 +438,19 @@
 attribute name=Implementation-Title 
value=org.apache.zookeeper/
 attribute name=Implementation-Version value=${revision}/ 
 attribute name=Implementation-Vendor value=The Apache 
Software Foundation/
+   
+   !-- The following are OSGi manifest headers --
+   !-- currently hardcoded, when things get more complicated we 
could use BND 
+   http://www.aqute.biz/Code/Bnd to generate them --
+attribute name=Bundle-Vendor value=The Apache Software 
Foundation/
+attribute name=Bundle-Name value=ZooKeeper Bundle/
+attribute name=Bundle-SymbolicName 
value=org.apache.hadoop.zookeeper/
+attribute name=Bundle-ManifestVersion value=2/
+attribute name=Bundle-Version value=${version}/
+attribute name=Bundle-License 
value=http://www.apache.org/licenses/LICENSE-2.0.txt/
+attribute name=Bundle-DocURL 
value=http://hadoop.apache.org/zookeeper/
+attribute name=Import-Package 
value='javax.management,org.apache.log4j,org.osgi.framework;version=[1.4,2.0),org.osgi.util.tracker;version=[1.1,2.0)'/
+attribute name=Export-Package 
value='org.apache.zookeeper;version=${version},org.apache.zookeeper.data;version=${version},org.apache.zookeeper.version;version=${version},org.apache.zookeeper.server;version=${version},org.apache.zookeeper.server.quorum;version=${version}'/
 /manifest
 /jar
 /target




svn commit: r886241 - in /hadoop/zookeeper/trunk: CHANGES.txt src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java

2009-12-02 Thread breed
Author: breed
Date: Wed Dec  2 18:36:24 2009
New Revision: 886241

URL: http://svn.apache.org/viewvc?rev=886241view=rev
Log:
ZOOKEEPER-597. ASyncHammerTest is failing intermittently on hudson trunk

Modified:
hadoop/zookeeper/trunk/CHANGES.txt

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=886241r1=886240r2=886241view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Wed Dec  2 18:36:24 2009
@@ -139,6 +139,9 @@
   ZOOKEEPER-598. LearnerHandler is misspelt in the thread's constructor 
   (Henry Robinson via fpj)
 
+  ZOOKEEPER-597. ASyncHammerTest is failing intermittently on hudson trunk 
(take 2)
+  (breed)
+
 IMPROVEMENTS:
   ZOOKEEPER-473. cleanup junit tests to eliminate false positives due to
   socket reuse and failure to close client (phunt via mahadev)

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java?rev=886241r1=886240r2=886241view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
 Wed Dec  2 18:36:24 2009
@@ -955,10 +955,12 @@
 }
 jmxConnectionBean = null;
 
-if (closed) {
-return;
+synchronized(this) {
+if (closed) {
+return;
+}
+closed = true;
 }
-closed = true;
 synchronized (factory.ipMap)
 {
 SetNIOServerCnxn s = 
factory.ipMap.get(sock.socket().getInetAddress());
@@ -1038,42 +1040,46 @@
  *  org.apache.jute.Record, java.lang.String)
  */
 synchronized public void sendResponse(ReplyHeader h, Record r, String tag) 
{
-if (closed) {
-if (LOG.isTraceEnabled()) {
-LOG.trace(send called on closed session 0x
-  + Long.toHexString(sessionId)
-  +  with record  + r);
-}
-return;
-}
-ByteArrayOutputStream baos = new ByteArrayOutputStream();
-// Make space for length
-BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos);
 try {
-baos.write(fourBytes);
-bos.writeRecord(h, header);
-if (r != null) {
-bos.writeRecord(r, tag);
+if (closed) {
+if (LOG.isTraceEnabled()) {
+LOG.trace(send called on closed session 0x
+  + Long.toHexString(sessionId)
+  +  with record  + r);
+}
+return;
 }
-baos.close();
-} catch (IOException e) {
-LOG.error(Error serializing response);
-}
-byte b[] = baos.toByteArray();
-ByteBuffer bb = ByteBuffer.wrap(b);
-bb.putInt(b.length - 4).rewind();
-sendBuffer(bb);
-if (h.getXid()  0) {
-synchronized (this.factory) {
-outstandingRequests--;
-// check throttling
-if (zk.getInProcess()  factory.outstandingLimit
-|| outstandingRequests  1) {
-sk.selector().wakeup();
-enableRecv();
+ByteArrayOutputStream baos = new ByteArrayOutputStream();
+// Make space for length
+BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos);
+try {
+baos.write(fourBytes);
+bos.writeRecord(h, header);
+if (r != null) {
+bos.writeRecord(r, tag);
+}
+baos.close();
+} catch (IOException e) {
+LOG.error(Error serializing response);
+}
+byte b[] = baos.toByteArray();
+ByteBuffer bb = ByteBuffer.wrap(b);
+bb.putInt(b.length - 4).rewind();
+sendBuffer(bb);
+if (h.getXid()  0) {
+synchronized (this.factory) {
+outstandingRequests--;
+// check throttling
+if (zk.getInProcess()  factory.outstandingLimit
+|| outstandingRequests  1) {
+sk.selector().wakeup();
+enableRecv();
+}
 }
 }
-}
+ } catch(Exception e) {
+LOG.error(Unexpected exception. Destruction averted., e);
+ }
 }
 
 /*




svn commit: r889780 - in /hadoop/zookeeper/trunk: CHANGES.txt src/c/include/zookeeper.h src/java/main/org/apache/zookeeper/ZooKeeper.java

2009-12-11 Thread breed
Author: breed
Date: Fri Dec 11 19:38:47 2009
New Revision: 889780

URL: http://svn.apache.org/viewvc?rev=889780view=rev
Log:
ZOOKEEPER-615. wrong javadoc for create with a sequence flag

Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/src/c/include/zookeeper.h
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=889780r1=889779r2=889780view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Fri Dec 11 19:38:47 2009
@@ -178,6 +178,8 @@
 
   ZOOKEEPER-425. Add OSGi metadata to zookeeper.jar (david bosschaert via 
breed)
 
+  ZOOKEEPER-615. wrong javadoc for create with a sequence flag (mahadev via 
breed)
+
 NEW FEATURES:
   ZOOKEEPER-539. generate eclipse project via ant target. (phunt via mahadev)
 

Modified: hadoop/zookeeper/trunk/src/c/include/zookeeper.h
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/include/zookeeper.h?rev=889780r1=889779r2=889780view=diff
==
--- hadoop/zookeeper/trunk/src/c/include/zookeeper.h (original)
+++ hadoop/zookeeper/trunk/src/c/include/zookeeper.h Fri Dec 11 19:38:47 2009
@@ -580,7 +580,8 @@
  * it does not already exists. The Create Flags affect the creation of nodes.
  * If ZOO_EPHEMERAL flag is set, the node will automatically get removed if the
  * client session goes away. If the ZOO_SEQUENCE flag is set, a unique
- * monotonically increasing sequence number is appended to the path name.
+ * monotonically increasing sequence number is appended to the path name. The
+ * sequence number is always fixed length of 10 digits, 0 padded.
  * 
  * \param zh the zookeeper handle obtained by a call to \ref zookeeper_init
  * \param path The name of the node. Expressed as a file name with slashes 

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java?rev=889780r1=889779r2=889780view=diff
==
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java 
(original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java 
Fri Dec 11 19:38:47 2009
@@ -555,7 +555,8 @@
  * p
  * The flags argument can also specify to create a sequential node. The
  * actual path name of a sequential node will be the given path plus a
- * suffix _i where i is the current sequential number of the node. Once
+ * suffix i where i is the current sequential number of the node. The 
sequence
+ * number is always fixed length of 10 digits, 0 padded. Once
  * such a node is created, the sequential number will be incremented by 
one.
  * p
  * If a node with the same actual path already exists in the ZooKeeper, a




svn commit: r889797 - in /hadoop/zookeeper/trunk: CHANGES.txt src/java/main/org/apache/zookeeper/KeeperException.java src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java src/java/main/

2009-12-11 Thread breed
Author: breed
Date: Fri Dec 11 20:24:23 2009
New Revision: 889797

URL: http://svn.apache.org/viewvc?rev=889797view=rev
Log:
ZOOKEEPER-588. remove unnecessary/annoying log of tostring error in 
Request.toString()

Modified:
hadoop/zookeeper/trunk/CHANGES.txt

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/KeeperException.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/Request.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=889797r1=889796r2=889797view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Fri Dec 11 20:24:23 2009
@@ -155,6 +155,10 @@
 
   ZOOKEEPER-611. hudson build failure (take 2) (mahadev)
 
+  ZOOKEEPER-615. wrong javadoc for create with a sequence flag (mahadev via 
breed)
+
+  ZOOKEEPER-588. remove unnecessary/annoying log of tostring error in 
Request.toString() (phunt via breed)
+
 IMPROVEMENTS:
   ZOOKEEPER-473. cleanup junit tests to eliminate false positives due to
   socket reuse and failure to close client (phunt via mahadev)
@@ -178,8 +182,6 @@
 
   ZOOKEEPER-425. Add OSGi metadata to zookeeper.jar (david bosschaert via 
breed)
 
-  ZOOKEEPER-615. wrong javadoc for create with a sequence flag (mahadev via 
breed)
-
 NEW FEATURES:
   ZOOKEEPER-539. generate eclipse project via ant target. (phunt via mahadev)
 

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/KeeperException.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/KeeperException.java?rev=889797r1=889796r2=889797view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/KeeperException.java 
(original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/KeeperException.java 
Fri Dec 11 20:24:23 2009
@@ -484,6 +484,9 @@
 public BadArgumentsException() {
 super(Code.BADARGUMENTS);
 }
+public BadArgumentsException(String path) {
+super(Code.BADARGUMENTS, path);
+}
 }
 
 /**
@@ -493,6 +496,9 @@
 public BadVersionException() {
 super(Code.BADVERSION);
 }
+public BadVersionException(String path) {
+super(Code.BADVERSION, path);
+}
 }
 
 /**
@@ -520,6 +526,9 @@
 public InvalidACLException() {
 super(Code.INVALIDACL);
 }
+public InvalidACLException(String path) {
+super(Code.INVALIDACL, path);
+}
 }
 
 /**
@@ -556,6 +565,9 @@
 public NoChildrenForEphemeralsException() {
 super(Code.NOCHILDRENFOREPHEMERALS);
 }
+public NoChildrenForEphemeralsException(String path) {
+super(Code.NOCHILDRENFOREPHEMERALS, path);
+}
 }
 
 /**
@@ -565,6 +577,9 @@
 public NodeExistsException() {
 super(Code.NODEEXISTS);
 }
+public NodeExistsException(String path) {
+super(Code.NODEEXISTS, path);
+}
 }
 
 /**
@@ -574,6 +589,9 @@
 public NoNodeException() {
 super(Code.NONODE);
 }
+public NoNodeException(String path) {
+super(Code.NONODE, path);
+}
 }
 
 /**
@@ -583,6 +601,9 @@
 public NotEmptyException() {
 super(Code.NOTEMPTY);
 }
+public NotEmptyException(String path) {
+super(Code.NOTEMPTY, path);
+}
 }
 
 /**

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java?rev=889797r1=889796r2=889797view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
 Fri Dec 11 20:24:23 2009
@@ -147,7 +147,7 @@
 }
 }
 if (lastChange == null || lastChange.stat == null) {
-throw new KeeperException.NoNodeException();
+throw new KeeperException.NoNodeException(path);
 }
 return lastChange;
 }
@@ -218,10 +218,10 @@
 if (lastSlash == -1 || path.indexOf('\0') != -1 || failCreate) 
{
 LOG.info(Invalid path  + path +  with session  +
 Long.toHexString(request.sessionId));
-throw new KeeperException.BadArgumentsException();
+throw new

svn commit: r966960 - in /hadoop/zookeeper/trunk: ./ src/java/main/org/apache/zookeeper/server/persistence/ src/java/main/org/apache/zookeeper/server/quorum/ src/java/test/org/apache/zookeeper/test/

2010-07-22 Thread breed
Author: breed
Date: Fri Jul 23 04:22:27 2010
New Revision: 966960

URL: http://svn.apache.org/viewvc?rev=966960view=rev
Log:
ZOOKEEPER-790. Last processed zxid set prematurely while establishing 
leadership (fpj via breed)

Modified:
hadoop/zookeeper/trunk/CHANGES.txt

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Learner.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/QuorumBase.java

hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/QuorumTest.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=966960r1=966959r2=966960view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Fri Jul 23 04:22:27 2010
@@ -86,6 +86,8 @@ IMPROVEMENTS:
   ZOOKEEPER-797 c client source with AI_ADDRCONFIG cannot be compiled with
   early glibc (Qian Ye via phunt)
 
+  ZOOKEEPER-790. Last processed zxid set prematurely while establishing 
leadership (fpj via breed)
+
 NEW FEATURES:
   ZOOKEEPER-729. Java client API to recursively delete a subtree.
   (Kay Kay via henry)

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java?rev=966960r1=966959r2=966960view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java
 Fri Jul 23 04:22:27 2010
@@ -193,7 +193,12 @@ public class FileTxnLog implements TxnLo
 + hdr.getType());
 }
 if (logStream==null) {
-   logFileWrite = new File(logDir, (log. +
+   if(LOG.isInfoEnabled()){
+LOG.info(Creating new log file: log. +  
+Long.toHexString(hdr.getZxid()));
+   }
+   
+   logFileWrite = new File(logDir, (log. + 
Long.toHexString(hdr.getZxid(;
fos = new FileOutputStream(logFileWrite);
logStream=new BufferedOutputStream(fos);

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java?rev=966960r1=966959r2=966960view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
 Fri Jul 23 04:22:27 2010
@@ -274,16 +274,15 @@ public class Leader {
 try {
 self.tick = 0;
 zk.loadData();
-zk.startup();
+
 long epoch = self.getLastLoggedZxid()  32L;
 epoch++;
 zk.setZxid(epoch  32L);
-zk.getZKDatabase().setlastProcessedZxid(zk.getZxid());
 
 synchronized(this){
 lastProposed = zk.getZxid();
 }
-
+
 newLeaderProposal.packet = new QuorumPacket(NEWLEADER, 
zk.getZxid(),
 null, null);
 
@@ -327,6 +326,13 @@ public class Leader {
 Thread.sleep(self.tickTime);
 self.tick++;
 }
+
+if(LOG.isInfoEnabled()){
+LOG.info(Have quorum of supporters; starting up and setting 
last processed zxid:  + zk.getZxid());
+}
+zk.startup();
+zk.getZKDatabase().setlastProcessedZxid(zk.getZxid());
+
 if (!System.getProperty(zookeeper.leaderServes, 
yes).equals(no)) {
 self.cnxnFactory.setZooKeeperServer(zk);
 }
@@ -466,7 +472,7 @@ public class Leader {
 LOG.debug(Count for zxid: 0x + Long.toHexString(zxid)
 +  is  + p.ackSet.size());
 }
-if (self.getQuorumVerifier().containsQuorum(p.ackSet)){
+if (self.getQuorumVerifier().containsQuorum(p.ackSet)){ 
 if (zxid != lastCommitted+1) {
 LOG.warn(Commiting zxid 0x + Long.toHexString(zxid)
 +  from  + followerAddr +  not first!);
@@ -603,6 +609,16 @@ public class Leader {
 
 long lastProposed;
 
+
+/**
+ * Returns the current epoch

svn commit: r966984 - in /hadoop/zookeeper/branches/branch-3.3: ./ src/java/main/org/apache/zookeeper/server/persistence/ src/java/main/org/apache/zookeeper/server/quorum/ src/java/test/org/apache/zoo

2010-07-23 Thread breed
Author: breed
Date: Fri Jul 23 06:31:57 2010
New Revision: 966984

URL: http://svn.apache.org/viewvc?rev=966984view=rev
Log:
ZOOKEEPER-790. Last processed zxid set prematurely while establishing 
leadership (fpj via breed)


Modified:
hadoop/zookeeper/branches/branch-3.3/CHANGES.txt

hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java

hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/quorum/Leader.java

hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/quorum/Learner.java

hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/QuorumBase.java

hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/QuorumTest.java

Modified: hadoop/zookeeper/branches/branch-3.3/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.3/CHANGES.txt?rev=966984r1=966983r2=966984view=diff
==
--- hadoop/zookeeper/branches/branch-3.3/CHANGES.txt (original)
+++ hadoop/zookeeper/branches/branch-3.3/CHANGES.txt Fri Jul 23 06:31:57 2010
@@ -58,6 +58,8 @@ BUGFIXES:
   ZOOKEEPER-766. forrest recipes docs don't mention the lock/queue recipe
   implementations available in the release (phunt via mahadev)
 
+  ZOOKEEPER-790. Last processed zxid set prematurely while establishing 
leadership (fpj via breed)
+
 Release 3.3.0 - 2010-03-24
 
 Non-backward compatible changes:

Modified: 
hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java?rev=966984r1=966983r2=966984view=diff
==
--- 
hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java
 (original)
+++ 
hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java
 Fri Jul 23 06:31:57 2010
@@ -193,7 +193,12 @@ public class FileTxnLog implements TxnLo
 + hdr.getType());
 }
 if (logStream==null) {
-   logFileWrite = new File(logDir, (log. +
+   if(LOG.isInfoEnabled()){
+LOG.info(Creating new log file: log. +  
+Long.toHexString(hdr.getZxid()));
+   }
+   
+   logFileWrite = new File(logDir, (log. + 
Long.toHexString(hdr.getZxid(;
fos = new FileOutputStream(logFileWrite);
logStream=new BufferedOutputStream(fos);

Modified: 
hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/quorum/Leader.java?rev=966984r1=966983r2=966984view=diff
==
--- 
hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
 (original)
+++ 
hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/quorum/Leader.java
 Fri Jul 23 06:31:57 2010
@@ -274,16 +274,15 @@ public class Leader {
 try {
 self.tick = 0;
 zk.loadData();
-zk.startup();
+
 long epoch = self.getLastLoggedZxid()  32L;
 epoch++;
 zk.setZxid(epoch  32L);
-zk.getZKDatabase().setlastProcessedZxid(zk.getZxid());
 
 synchronized(this){
 lastProposed = zk.getZxid();
 }
-
+
 newLeaderProposal.packet = new QuorumPacket(NEWLEADER, 
zk.getZxid(),
 null, null);
 
@@ -327,6 +326,13 @@ public class Leader {
 Thread.sleep(self.tickTime);
 self.tick++;
 }
+
+if(LOG.isInfoEnabled()){
+LOG.info(Have quorum of supporters; starting up and setting 
last processed zxid:  + zk.getZxid());
+}
+zk.startup();
+zk.getZKDatabase().setlastProcessedZxid(zk.getZxid());
+
 if (!System.getProperty(zookeeper.leaderServes, 
yes).equals(no)) {
 self.cnxnFactory.setZooKeeperServer(zk);
 }
@@ -466,7 +472,7 @@ public class Leader {
 LOG.debug(Count for zxid: 0x + Long.toHexString(zxid)
 +  is  + p.ackSet.size());
 }
-if (self.getQuorumVerifier().containsQuorum(p.ackSet)){
+if (self.getQuorumVerifier().containsQuorum(p.ackSet)){ 
 if (zxid != lastCommitted+1

svn commit: r1022824 - in /hadoop/zookeeper/trunk: CHANGES.txt src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java

2010-10-14 Thread breed
Author: breed
Date: Fri Oct 15 05:21:26 2010
New Revision: 1022824

URL: http://svn.apache.org/viewvc?rev=1022824view=rev
Log:
ZOOKEEPER-881. ZooKeeperServer.loadData loads database twice (jared cantwell 
via breed)


Modified:
hadoop/zookeeper/trunk/CHANGES.txt

hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=1022824r1=1022823r2=1022824view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Fri Oct 15 05:21:26 2010
@@ -112,6 +112,8 @@ BUGFIXES: 
 
   ZOOKEEPER-866. Hedwig Server stays in disconnected state when connection 
to ZK dies but gets reconnected (erwin tam via breed)
 
+  ZOOKEEPER-881. ZooKeeperServer.loadData loads database twice (jared cantwell 
via breed)
+
 IMPROVEMENTS:
   ZOOKEEPER-724. Improve junit test integration - log harness information 
   (phunt via mahadev)

Modified: 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java?rev=1022824r1=1022823r2=1022824view=diff
==
--- 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
 (original)
+++ 
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
 Fri Oct 15 05:21:26 2010
@@ -238,7 +238,6 @@ public class ZooKeeperServer implements 
  *  Restore sessions and data
  */
 public void loadData() throws IOException, InterruptedException {
-zkDb.loadDataBase();
 setZxid(zkDb.loadDataBase());
 // Clean up dead sessions
 LinkedListLong deadSessions = new LinkedListLong();




svn commit: r1031453 - in /hadoop/zookeeper/trunk: CHANGES.txt src/contrib/hedwig/client/src/main/cpp/lib/channel.cpp src/contrib/hedwig/client/src/main/cpp/test/pubsubtest.cpp

2010-11-05 Thread breed
Author: breed
Date: Fri Nov  5 06:43:56 2010
New Revision: 1031453

URL: http://svn.apache.org/viewvc?rev=1031453view=rev
Log:
ZOOKEEPER-916. Problem receiving messages from subscribed channels in c++ client

Modified:
hadoop/zookeeper/trunk/CHANGES.txt

hadoop/zookeeper/trunk/src/contrib/hedwig/client/src/main/cpp/lib/channel.cpp

hadoop/zookeeper/trunk/src/contrib/hedwig/client/src/main/cpp/test/pubsubtest.cpp

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=1031453r1=1031452r2=1031453view=diff
==
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Fri Nov  5 06:43:56 2010
@@ -149,6 +149,8 @@ BUGFIXES: 
 
   ZOOKEEPER-884. Remove LedgerSequence references from BookKeeper 
documentation and comments in tests (fpj via breed)
 
+  ZOOKEEPER-916. Problem receiving messages from subscribed channels in c++ 
client (ivan via breed)
+
 IMPROVEMENTS:
   ZOOKEEPER-724. Improve junit test integration - log harness information 
   (phunt via mahadev)
@@ -185,6 +187,8 @@ IMPROVEMENTS:
 
   ZOOKEEPER-864. Hedwig C++ client improvements (Ivan Kelly via breed)
 
+  ZOOKEEPER-862. Hedwig created ledgers with hardcoded Bookkeeper ensemble and 
quorum size. Make these a server config parameter instead. (Erwin T via breed)
+
 NEW FEATURES:
   ZOOKEEPER-729. Java client API to recursively delete a subtree.
   (Kay Kay via henry)

Modified: 
hadoop/zookeeper/trunk/src/contrib/hedwig/client/src/main/cpp/lib/channel.cpp
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/hedwig/client/src/main/cpp/lib/channel.cpp?rev=1031453r1=1031452r2=1031453view=diff
==
--- 
hadoop/zookeeper/trunk/src/contrib/hedwig/client/src/main/cpp/lib/channel.cpp 
(original)
+++ 
hadoop/zookeeper/trunk/src/contrib/hedwig/client/src/main/cpp/lib/channel.cpp 
Fri Nov  5 06:43:56 2010
@@ -218,10 +218,13 @@ void DuplexChannel::connect() {  
 
 void DuplexChannel::startReceiving() {
   if (LOG.isDebugEnabled()) {
-LOG.debugStream()  DuplexChannel::startReceiving channel(  this  
);
+LOG.debugStream()  DuplexChannel::startReceiving channel(  this  
) currently receiving =   receiving;
   }
 
   boost::lock_guardboost::mutex lock(receiving_lock);
+  if (receiving) {
+return;
+  } 
   receiving = true;
   
   DuplexChannel::readSize(shared_from_this());

Modified: 
hadoop/zookeeper/trunk/src/contrib/hedwig/client/src/main/cpp/test/pubsubtest.cpp
URL: 
http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/hedwig/client/src/main/cpp/test/pubsubtest.cpp?rev=1031453r1=1031452r2=1031453view=diff
==
--- 
hadoop/zookeeper/trunk/src/contrib/hedwig/client/src/main/cpp/test/pubsubtest.cpp
 (original)
+++ 
hadoop/zookeeper/trunk/src/contrib/hedwig/client/src/main/cpp/test/pubsubtest.cpp
 Fri Nov  5 06:43:56 2010
@@ -47,6 +47,7 @@ private:
   CPPUNIT_TEST(testPubSubContinuousOverClose);
   //  CPPUNIT_TEST(testPubSubContinuousOverServerDown);
   CPPUNIT_TEST(testMultiTopic);
+  CPPUNIT_TEST(testBigMessage);
   CPPUNIT_TEST(testMultiTopicMultiSubscriber);
   CPPUNIT_TEST_SUITE_END();
 
@@ -183,6 +184,7 @@ public:
 CPPUNIT_ASSERT(pass);
   }
 
+
   /*  void testPubSubContinuousOverServerDown() {
 std::string topic = pubSubTopic;
 std::string sid = MySubscriberid-1;
@@ -328,6 +330,44 @@ public:
 }
 CPPUNIT_ASSERT(passA  passB);
   }
+
+  static const int BIG_MESSAGE_SIZE = 16436*2; // MTU to lo0 is 16436 by 
default on linux
+
+  void testBigMessage() {
+std::string topic = pubSubTopic;
+std::string sid = MySubscriberid-6;
+
+Hedwig::Configuration* conf = new TestServerConfiguration(hw1);
+std::auto_ptrHedwig::Configuration confptr(conf);
+
+Hedwig::Client* client = new Hedwig::Client(*conf);
+std::auto_ptrHedwig::Client clientptr(client);
+
+Hedwig::Subscriber sub = client-getSubscriber();
+Hedwig::Publisher pub = client-getPublisher();
+
+sub.subscribe(topic, sid, Hedwig::SubscribeRequest::CREATE_OR_ATTACH);
+MyMessageHandlerCallback* cb = new MyMessageHandlerCallback(topic, sid);
+Hedwig::MessageHandlerCallbackPtr handler(cb);
+
+sub.startDelivery(topic, sid, handler);
+
+char buf[BIG_MESSAGE_SIZE];
+std::string bigmessage(buf, BIG_MESSAGE_SIZE);
+pub.publish(topic, bigmessage);
+pub.publish(topic, Test Message 1);
+bool pass = false;
+for (int i = 0; i  10; i++) {
+  sleep(3);
+  if (cb-numMessagesReceived()  0) {
+   if (cb-getLastMessage() == Test Message 1) {
+ pass = true;
+ break;
+   }
+  }
+}
+CPPUNIT_ASSERT(pass);
+  }
 };
 
 CPPUNIT_TEST_SUITE_NAMED_REGISTRATION( PubSubTestSuite, PubSub );