Browse Source

ZOOKEEPER-473. cleanup junit tests to eliminate false positives due to socket reuse and failure to close client (phunt via mahadev)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/zookeeper/trunk@794887 13f79535-47bb-0310-9956-ffa450edef68
Mahadev Konar 16 năm trước cách đây
mục cha
commit
a44fae6243
43 tập tin đã thay đổi với 560 bổ sung365 xóa
  1. 2 0
      CHANGES.txt
  2. 27 25
      src/java/main/org/apache/zookeeper/ClientCnxn.java
  3. 23 22
      src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
  4. 34 0
      src/java/test/org/apache/zookeeper/PortAssignment.java
  5. 26 19
      src/java/test/org/apache/zookeeper/server/CRCTest.java
  6. 4 5
      src/java/test/org/apache/zookeeper/server/DataTreeUnitTest.java
  7. 13 1
      src/java/test/org/apache/zookeeper/server/DeserializationPerfTest.java
  8. 34 17
      src/java/test/org/apache/zookeeper/server/InvalidSnapshotTest.java
  9. 13 1
      src/java/test/org/apache/zookeeper/server/SerializationPerfTest.java
  10. 6 2
      src/java/test/org/apache/zookeeper/server/ZooKeeperServerTest.java
  11. 2 2
      src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
  12. 24 19
      src/java/test/org/apache/zookeeper/test/ACLTest.java
  13. 1 0
      src/java/test/org/apache/zookeeper/test/AsyncTest.java
  14. 1 0
      src/java/test/org/apache/zookeeper/test/ChrootClientTest.java
  15. 0 8
      src/java/test/org/apache/zookeeper/test/ChrootTest.java
  16. 51 12
      src/java/test/org/apache/zookeeper/test/ClientBase.java
  17. 20 12
      src/java/test/org/apache/zookeeper/test/ClientRetry.java
  18. 3 1
      src/java/test/org/apache/zookeeper/test/DataTreeTest.java
  19. 1 1
      src/java/test/org/apache/zookeeper/test/EventTypeTest.java
  20. 10 12
      src/java/test/org/apache/zookeeper/test/FLENewEpochTest.java
  21. 17 17
      src/java/test/org/apache/zookeeper/test/FLERestartTest.java
  22. 19 15
      src/java/test/org/apache/zookeeper/test/FLETest.java
  23. 12 8
      src/java/test/org/apache/zookeeper/test/HierarchicalQuorumTest.java
  24. 4 5
      src/java/test/org/apache/zookeeper/test/IntegrityCheck.java
  25. 22 16
      src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java
  26. 8 4
      src/java/test/org/apache/zookeeper/test/LETest.java
  27. 1 1
      src/java/test/org/apache/zookeeper/test/NullDataTest.java
  28. 27 17
      src/java/test/org/apache/zookeeper/test/OOMTest.java
  29. 16 13
      src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java
  30. 46 26
      src/java/test/org/apache/zookeeper/test/QuorumBase.java
  31. 2 2
      src/java/test/org/apache/zookeeper/test/QuorumQuotaTest.java
  32. 11 10
      src/java/test/org/apache/zookeeper/test/QuorumTest.java
  33. 11 7
      src/java/test/org/apache/zookeeper/test/RecoveryTest.java
  34. 12 12
      src/java/test/org/apache/zookeeper/test/RepeatStartupTest.java
  35. 21 19
      src/java/test/org/apache/zookeeper/test/SessionTest.java
  36. 1 1
      src/java/test/org/apache/zookeeper/test/SyncCallTest.java
  37. 1 1
      src/java/test/org/apache/zookeeper/test/TestHammer.java
  38. 19 15
      src/java/test/org/apache/zookeeper/test/UpgradeTest.java
  39. 2 2
      src/java/test/org/apache/zookeeper/test/WatchedEventTest.java
  40. 3 3
      src/java/test/org/apache/zookeeper/test/WatcherFuncTest.java
  41. 4 4
      src/java/test/org/apache/zookeeper/test/WatcherTest.java
  42. 1 0
      src/java/test/org/apache/zookeeper/test/ZooKeeperQuotaTest.java
  43. 5 8
      src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java

+ 2 - 0
CHANGES.txt

@@ -9,6 +9,8 @@ Backward compatible changes:
 BUGFIXES: 
 
 IMPROVEMENTS:
+  ZOOKEEPER-473. cleanup junit tests to eliminate false positives due to
+"socket reuse" and failure to close client (phunt via mahadev)
 
 NEW FEATURES:
 

+ 27 - 25
src/java/main/org/apache/zookeeper/ClientCnxn.java

@@ -76,18 +76,18 @@ import org.apache.zookeeper.server.ZooTrace;
  */
 public class ClientCnxn {
     private static final Logger LOG = Logger.getLogger(ClientCnxn.class);
-    
+
     /** This controls whether automatic watch resetting is enabled.
      * Clients automatically reset watches during session reconnect, this
      * option allows the client to turn off this behavior by setting
      * the environment variable "zookeeper.disableAutoWatchReset" to "true" */
     private static boolean disableAutoWatchReset;
-   
+
     public static final int packetLen;
     static {
-        // this var should not be public, but otw there is no easy way 
+        // this var should not be public, but otw there is no easy way
         // to test
-        disableAutoWatchReset = 
+        disableAutoWatchReset =
             Boolean.getBoolean("zookeeper.disableAutoWatchReset");
         LOG.info("zookeeper.disableAutoWatchReset is " + disableAutoWatchReset);
         packetLen = Integer.getInteger("jute.maxbuffer", 4096 * 1024);
@@ -105,7 +105,7 @@ public class ClientCnxn {
 
         byte data[];
     }
-    
+
     private ArrayList<AuthData> authInfo = new ArrayList<AuthData>();
 
     /**
@@ -141,7 +141,7 @@ public class ClientCnxn {
     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
@@ -153,7 +153,7 @@ public class ClientCnxn {
     public long getSessionId() {
         return sessionId;
     }
-    
+
     public byte[] getSessionPasswd() {
         return sessionPasswd;
     }
@@ -329,7 +329,7 @@ public class ClientCnxn {
         sendThread = new SendThread();
         eventThread = new EventThread();
     }
-    
+
     /**
      * tests use this to check on reset of watches
      * @return if the auto reset of watches are disabled
@@ -360,15 +360,15 @@ public class ClientCnxn {
     private static class WatcherSetEventPair {
         private final Set<Watcher> watchers;
         private final WatchedEvent event;
-        
+
         public WatcherSetEventPair(Set<Watcher> watchers, WatchedEvent event) {
             this.watchers = watchers;
             this.event = event;
         }
     }
-    
+
     class EventThread extends Thread {
-        private final LinkedBlockingQueue<Object> waitingEvents = 
+        private final LinkedBlockingQueue<Object> waitingEvents =
             new LinkedBlockingQueue<Object>();
 
         /** This is really the queued session state until the event
@@ -382,9 +382,9 @@ public class ClientCnxn {
             setUncaughtExceptionHandler(uncaughtExceptionHandler);
             setDaemon(true);
         }
-        
+
         public void queueEvent(WatchedEvent event) {
-            if (event.getType() == EventType.None 
+            if (event.getType() == EventType.None
                     && sessionState == event.getState()) {
                 return;
             }
@@ -398,7 +398,7 @@ public class ClientCnxn {
             // queue the pair (watch set & event) for later processing
             waitingEvents.add(pair);
         }
-        
+
         public void queuePacket(Packet packet) {
             waitingEvents.add(packet);
         }
@@ -510,7 +510,7 @@ public class ClientCnxn {
             } catch (InterruptedException e) {
                 LOG.error("Event thread exiting due to interruption", e);
             }
-            
+
             LOG.info("EventThread shut down");
         }
     }
@@ -609,7 +609,7 @@ public class ClientCnxn {
                 if (LOG.isDebugEnabled()) {
                     LOG.debug("Got ping response for sessionid:0x"
                             + Long.toHexString(sessionId)
-                            + " after " 
+                            + " after "
                             + ((System.nanoTime() - lastPingSentNs) / 1000000)
                             + "ms");
                 }
@@ -640,7 +640,7 @@ public class ClientCnxn {
                     LOG.debug("Got " + we + " for sessionid 0x"
                             + Long.toHexString(sessionId));
                 }
-                
+
                 eventThread.queueEvent( we );
                 return;
             }
@@ -775,7 +775,7 @@ public class ClientCnxn {
         }
 
         SendThread() {
-            super(currentThread().getName() + "-SendThread");
+            super(currentThread().getName() + "-SendThread()");
             zooKeeper.state = States.CONNECTING;
             setUncaughtExceptionHandler(uncaughtExceptionHandler);
             setDaemon(true);
@@ -867,6 +867,8 @@ public class ClientCnxn {
             sock.socket().setSoLinger(false, -1);
             sock.socket().setTcpNoDelay(true);
             LOG.info("Attempting connection to server " + addr);
+            setName(getName().replaceAll("\\(.*\\)",
+                    "(" + addr.getHostName() + ":" + addr.getPort() + ")"));
             sockKey = sock.register(selector, SelectionKey.OP_CONNECT);
             if (sock.connect(addr)) {
                 primeConnection(sockKey);
@@ -964,7 +966,7 @@ public class ClientCnxn {
                                 + " : " + e.getMessage());
                         break;
                     } else {
-                        LOG.warn("Exception closing session 0x" 
+                        LOG.warn("Exception closing session 0x"
                                 + Long.toHexString(getSessionId()) + " to "
                                 + sockKey, e);
                         cleanup();
@@ -974,7 +976,7 @@ public class ClientCnxn {
                                     Event.KeeperState.Disconnected,
                                     null));
                         }
-    
+
                         now = System.currentTimeMillis();
                         lastHeard = now;
                         lastSend = now;
@@ -1051,7 +1053,7 @@ public class ClientCnxn {
      * behavior.
      */
     public void disconnect() {
-        LOG.info("Disconnecting ClientCnxn for session: 0x" 
+        LOG.info("Disconnecting ClientCnxn for session: 0x"
                 + Long.toHexString(getSessionId()));
 
         sendThread.close();
@@ -1061,19 +1063,19 @@ public class ClientCnxn {
     /**
      * Close the connection, which includes; send session disconnect to the
      * server, shutdown the send/event threads.
-     * 
+     *
      * @throws IOException
      */
     public void close() throws IOException {
-        LOG.info("Closing ClientCnxn for session: 0x" 
+        LOG.info("Closing ClientCnxn for session: 0x"
                 + Long.toHexString(getSessionId()));
 
         closing = true;
-        
+
         try {
             RequestHeader h = new RequestHeader();
             h.setType(ZooDefs.OpCode.closeSession);
-            
+
             submitRequest(h, null, null, null);
         } catch (InterruptedException e) {
             // ignore, close the send/event threads

+ 23 - 22
src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java

@@ -89,13 +89,13 @@ public class NIOServerCnxn implements Watcher, ServerCnxn {
         ByteBuffer directBuffer = ByteBuffer.allocateDirect(64 * 1024);
 
         HashSet<NIOServerCnxn> cnxns = new HashSet<NIOServerCnxn>();
-        HashMap<InetAddress, Set<NIOServerCnxn>> ipMap = new HashMap<InetAddress, Set<NIOServerCnxn>>( ); 
+        HashMap<InetAddress, Set<NIOServerCnxn>> ipMap = new HashMap<InetAddress, Set<NIOServerCnxn>>( );
 
         int outstandingLimit = 1;
 
-        int maxClientCnxns = 10;  
-        
-        
+        int maxClientCnxns = 10;
+
+
         /**
          * Construct a new server connection factory which will accept an unlimited number
          * of concurrent connections from each client (up to the file descriptor
@@ -106,13 +106,13 @@ public class NIOServerCnxn implements Watcher, ServerCnxn {
         public Factory(int port) throws IOException {
             this(port,0);
         }
-        
-        
+
+
         /**
          * Constructs a new server connection factory where the number of concurrent connections
          * from a single IP address is limited to maxcc (or unlimited if 0).
          * startup(zks) must be called subsequently.
-         * @param port - the port to listen on for connections. 
+         * @param port - the port to listen on for connections.
          * @param maxcc - the number of concurrent connections allowed from a single client.
          * @throws IOException
          */
@@ -122,9 +122,10 @@ public class NIOServerCnxn implements Watcher, ServerCnxn {
             maxClientCnxns = maxcc;
             this.ss = ServerSocketChannel.open();
             ss.socket().setReuseAddress(true);
+            LOG.info("binding to port " + port);
             ss.socket().bind(new InetSocketAddress(port));
             ss.configureBlocking(false);
-            ss.register(selector, SelectionKey.OP_ACCEPT);         
+            ss.register(selector, SelectionKey.OP_ACCEPT);
         }
 
         @Override
@@ -175,7 +176,7 @@ public class NIOServerCnxn implements Watcher, ServerCnxn {
                     }
                     s.add(cnxn);
                     ipMap.put(addr,s);
-                }                
+                }
             }
         }
 
@@ -189,8 +190,8 @@ public class NIOServerCnxn implements Watcher, ServerCnxn {
             if (s == null) return 0;
             return s.size();
         }
-        
-        public void run() {         
+
+        public void run() {
             while (!ss.socket().isClosed()) {
                 try {
                     selector.select(1000);
@@ -204,13 +205,13 @@ public class NIOServerCnxn implements Watcher, ServerCnxn {
                     for (SelectionKey k : selectedList) {
                         if ((k.readyOps() & SelectionKey.OP_ACCEPT) != 0) {
                             SocketChannel sc = ((ServerSocketChannel) k
-                                    .channel()).accept();     
+                                    .channel()).accept();
                             InetAddress ia = sc.socket().getInetAddress();
-                            int cnxncount = getClientCnxnCount(ia); 
-                            if (maxClientCnxns > 0 && cnxncount >= maxClientCnxns){                                  
-                                LOG.warn("Too many connections from " + ia 
+                            int cnxncount = getClientCnxnCount(ia);
+                            if (maxClientCnxns > 0 && cnxncount >= maxClientCnxns){
+                                LOG.warn("Too many connections from " + ia
                                          + " - max is " + maxClientCnxns );
-                                sc.close();                 
+                                sc.close();
                             } else {
                                 sc.configureBlocking(false);
                                 SelectionKey sk = sc.register(selector,
@@ -218,7 +219,7 @@ public class NIOServerCnxn implements Watcher, ServerCnxn {
                                 NIOServerCnxn cnxn = createConnection(sc, sk);
                                 sk.attach(cnxn);
                                 addCnxn(cnxn);
-                            }                            
+                            }
                         } else if ((k.readyOps() & (SelectionKey.OP_READ | SelectionKey.OP_WRITE)) != 0) {
                             NIOServerCnxn c = (NIOServerCnxn) k.attachment();
                             c.doIO(k);
@@ -288,8 +289,8 @@ public class NIOServerCnxn implements Watcher, ServerCnxn {
         }
 
 
-		private void closeSessionWithoutWakeup(long sessionId) {
-			synchronized (cnxns) {
+        private void closeSessionWithoutWakeup(long sessionId) {
+            synchronized (cnxns) {
                 for (Iterator<NIOServerCnxn> it = cnxns.iterator(); it
                         .hasNext();) {
                     NIOServerCnxn cnxn = it.next();
@@ -609,7 +610,7 @@ public class NIOServerCnxn implements Watcher, ServerCnxn {
         // session is setup
         disableRecv();
         if (connReq.getSessionId() != 0) {
-        	factory.closeSessionWithoutWakeup(connReq.getSessionId());
+            factory.closeSessionWithoutWakeup(connReq.getSessionId());
             setSessionId(connReq.getSessionId());
             zk.reopenSession(this, sessionId, passwd, sessionTimeout);
             LOG.info("Renewing session 0x" + Long.toHexString(sessionId));
@@ -816,7 +817,7 @@ public class NIOServerCnxn implements Watcher, ServerCnxn {
             LOG.warn("Failed to unregister with JMX", e);
         }
         jmxConnectionBean = null;
-        
+
         if (closed) {
             return;
         }
@@ -824,7 +825,7 @@ public class NIOServerCnxn implements Watcher, ServerCnxn {
         synchronized (factory.ipMap)
         {
             Set<NIOServerCnxn> s = factory.ipMap.get(sock.socket().getInetAddress());
-            s.remove(this);           
+            s.remove(this);
         }
         synchronized (factory.cnxns) {
             factory.cnxns.remove(this);

+ 34 - 0
src/java/test/org/apache/zookeeper/PortAssignment.java

@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper;
+
+import org.apache.log4j.Logger;
+
+/** Assign ports to tests */
+public class PortAssignment {
+    private static final Logger LOG = Logger.getLogger(PortAssignment.class);
+
+    private static int nextPort = 11221;
+
+    /** Assign a new, unique port to the test */
+    public synchronized static int unique() {
+        LOG.info("assigning port " + nextPort);
+        return nextPort++;
+    }
+}

+ 26 - 19
src/java/test/org/apache/zookeeper/server/CRCTest.java

@@ -39,6 +39,7 @@ import org.apache.jute.BinaryInputArchive;
 import org.apache.jute.InputArchive;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
@@ -48,14 +49,15 @@ import org.apache.zookeeper.server.persistence.FileSnap;
 import org.apache.zookeeper.server.persistence.FileTxnLog;
 import org.apache.zookeeper.server.persistence.TxnLog.TxnIterator;
 import org.apache.zookeeper.test.ClientBase;
+import org.junit.Test;
 
 public class CRCTest extends TestCase implements Watcher{
-    
     private static final Logger LOG = Logger.getLogger(CRCTest.class);
-    private static String HOSTPORT = "127.0.0.1:2357";
-    ZooKeeperServer zks;
-    private CountDownLatch startSignal;
-    
+
+    private static final String HOSTPORT =
+        "127.0.0.1:" + PortAssignment.unique();
+    private volatile CountDownLatch startSignal;
+
     @Override
     protected void setUp() throws Exception {
         LOG.info("STARTING " + getName());
@@ -64,7 +66,7 @@ public class CRCTest extends TestCase implements Watcher{
     protected void tearDown() throws Exception {
         LOG.info("FINISHED " + getName());
     }
-    
+
     /**
      * corrupt a file by writing m at 500 b
      * offset
@@ -77,7 +79,7 @@ public class CRCTest extends TestCase implements Watcher{
         byte[] b = "mahadev".getBytes();
         long writeLen = 500L;
         raf.seek(writeLen);
-        //corruptting the data
+        //corrupting the data
         raf.write(b);
         raf.close();
     }
@@ -108,28 +110,33 @@ public class CRCTest extends TestCase implements Watcher{
         crcIn.close();
         return (val != checksum);
     }
-    
+
     /** test checksums for the logs and snapshots.
-     * the reader should fail on reading 
+     * the reader should fail on reading
      * a corrupt snapshot and a corrupt log
      * file
      * @throws Exception
      */
-   public void testChecksums() throws Exception {
+    @Test
+    public void testChecksums() throws Exception {
         File tmpDir = ClientBase.createTmpDir();
         ClientBase.setupTestEnv();
-        zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+        ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         SyncRequestProcessor.setSnapCount(150);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
         f.startup(zks);
         LOG.info("starting up the zookeeper server .. waiting");
-        assertTrue("waiting for server being up", 
+        assertTrue("waiting for server being up",
                 ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
         ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
-        for (int i =0; i < 2000; i++) {
-            zk.create("/crctest- " + i , ("/crctest- " + i).getBytes(), 
-                    Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        try {
+            for (int i =0; i < 2000; i++) {
+                zk.create("/crctest- " + i , ("/crctest- " + i).getBytes(),
+                        Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            }
+        } finally {
+            zk.close();
         }
         f.shutdown();
         assertTrue("waiting for server down",
@@ -138,7 +145,7 @@ public class CRCTest extends TestCase implements Watcher{
 
         File versionDir = new File(tmpDir, "version-2");
         File[] list = versionDir.listFiles();
-        //there should be only two files 
+        //there should be only two files
         // one the snapshot and the other logFile
         File snapFile = null;
         File logFile = null;
@@ -178,13 +185,13 @@ public class CRCTest extends TestCase implements Watcher{
         }
         assertTrue(cfile);
    }
-    
+
     public void process(WatchedEvent event) {
         LOG.info("Event:" + event.getState() + " " + event.getType() + " " + event.getPath());
         if (event.getState() == KeeperState.SyncConnected
                 && startSignal != null && startSignal.getCount() > 0)
-        {              
-            startSignal.countDown();      
+        {
+            startSignal.countDown();
         }
     }
 }

+ 4 - 5
src/java/test/org/apache/zookeeper/server/DataTreeUnitTest.java

@@ -20,14 +20,13 @@ package org.apache.zookeeper.server;
 
 import junit.framework.TestCase;
 
+import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.server.DataTree;
 
 public class DataTreeUnitTest extends TestCase {
     DataTree dt;
-    
+
     public void setUp() throws Exception {
         dt=new DataTree();
     }
@@ -36,7 +35,7 @@ public class DataTreeUnitTest extends TestCase {
         dt=null;
     }
 
-   
+
     public void testRootWatchTriggered() throws Exception {
         class MyWatcher implements Watcher{
             boolean fired=false;
@@ -44,7 +43,7 @@ public class DataTreeUnitTest extends TestCase {
                 if(event.getPath().equals("/"))
                     fired=true;
             }
-        };
+        }
         MyWatcher watcher=new MyWatcher();
         // set a watch on the root node
         dt.getChildren("/", new Stat(), watcher);

+ 13 - 1
src/java/test/org/apache/zookeeper/server/DeserializationPerfTest.java

@@ -26,9 +26,13 @@ import junit.framework.TestCase;
 
 import org.apache.jute.BinaryInputArchive;
 import org.apache.jute.BinaryOutputArchive;
+import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
+import org.junit.Test;
 
 public class DeserializationPerfTest extends TestCase {
+    protected static final Logger LOG = Logger.getLogger(DeserializationPerfTest.class);
+
     private static void deserializeTree(int depth, int width, int len)
             throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         BinaryInputArchive ia;
@@ -58,46 +62,54 @@ public class DeserializationPerfTest extends TestCase {
 
         assertEquals(count, dserTree.getNodeCount());
 
-        System.out.println("Deserialized " + count + " nodes in " + durationms
+        LOG.info("Deserialized " + count + " nodes in " + durationms
                 + " ms (" + pernodeus + "us/node), depth=" + depth + " width="
                 + width + " datalen=" + len);
     }
 
+    @Test
     public void testSingleDeserialize() throws
             InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(1, 0, 20);
     }
 
+    @Test
     public void testWideDeserialize() throws
             InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(2, 50000, 20);
     }
 
+    @Test
     public void testDeepDeserialize() throws
             InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(1000, 1, 20);
     }
 
+    @Test
     public void test10Wide5DeepDeserialize() throws
             InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(5, 10, 20);
     }
 
+    @Test
     public void test15Wide5DeepDeserialize() throws
             InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(5, 15, 20);
     }
 
+    @Test
     public void test25Wide4DeepDeserialize() throws
             InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(4, 25, 20);
     }
 
+    @Test
     public void test40Wide4DeepDeserialize() throws
             InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(4, 40, 20);
     }
 
+    @Test
     public void test300Wide3DeepDeserialize() throws
             InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         deserializeTree(3, 300, 20);

+ 34 - 17
src/java/test/org/apache/zookeeper/server/InvalidSnapshotTest.java

@@ -25,43 +25,52 @@ import junit.framework.TestCase;
 
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.test.ClientBase;
+import org.junit.Test;
 
 /**
- * this test checks that the server works 
+ * this test checks that the server works
  * even if the last snapshot is invalidated
- * by corruption or if the server crashes 
+ * by corruption or if the server crashes
  * while generating the snapshot.
  */
 public class InvalidSnapshotTest extends TestCase implements Watcher {
-    private static final Logger LOG = Logger.getLogger(InvalidSnapshotTest.class);
-    private static String HOSTPORT = "127.0.0.1:2357";
-    ZooKeeperServer zks = null;
+    private static final Logger LOG =
+        Logger.getLogger(InvalidSnapshotTest.class);
+
+    private static final String HOSTPORT =
+        "127.0.0.1:" + PortAssignment.unique();
     private static final int CONNECTION_TIMEOUT = 3000;
-    
+
     /**
-     * this test does the main work of testing 
+     * this test does the main work of testing
      * an invalid snapshot
      * @throws Exception
      */
+    @Test
     public void testInvalidSnapshot() throws Exception {
        File tmpDir = ClientBase.createTmpDir();
        ClientBase.setupTestEnv();
-       zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+       ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
        SyncRequestProcessor.setSnapCount(100);
        final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
        NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
        f.startup(zks);
-       assertTrue("waiting for server being up ", 
+       assertTrue("waiting for server being up ",
                ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
        ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
-       for (int i=0; i< 2000; i++) {
-           zk.create("/invalidsnap-" + i, new byte[0], Ids.OPEN_ACL_UNSAFE, 
-                   CreateMode.PERSISTENT);
+       try {
+           for (int i=0; i< 2000; i++) {
+               zk.create("/invalidsnap-" + i, new byte[0], Ids.OPEN_ACL_UNSAFE,
+                       CreateMode.PERSISTENT);
+           }
+       } finally {
+           zk.close();
        }
        f.shutdown();
        assertTrue("waiting for server to shutdown",
@@ -76,19 +85,27 @@ public class InvalidSnapshotTest extends TestCase implements Watcher {
        SyncRequestProcessor.setSnapCount(100);
        f = new NIOServerCnxn.Factory(PORT);
        f.startup(zks);
-       assertTrue("waiting for server being up ", 
+       assertTrue("waiting for server being up ",
                ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
        // the server should come up
        zk = new ZooKeeper(HOSTPORT, 20000, this);
-       assertTrue("the node should exist", (zk.exists("/invalidsnap-1999", false) != null));
+       try {
+           assertTrue("the node should exist",
+                   (zk.exists("/invalidsnap-1999", false) != null));
+           f.shutdown();
+           assertTrue("waiting for server to shutdown",
+                   ClientBase.waitForServerDown(HOSTPORT, CONNECTION_TIMEOUT));
+       } finally {
+           zk.close();
+       }
+
        f.shutdown();
        assertTrue("waiting for server to shutdown",
                ClientBase.waitForServerDown(HOSTPORT, CONNECTION_TIMEOUT));
-       
     }
-    
+
     public void process(WatchedEvent event) {
         // do nothing for now
     }
-    
+
 }

+ 13 - 1
src/java/test/org/apache/zookeeper/server/SerializationPerfTest.java

@@ -24,9 +24,13 @@ import java.io.OutputStream;
 import junit.framework.TestCase;
 
 import org.apache.jute.BinaryOutputArchive;
+import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
+import org.junit.Test;
 
 public class SerializationPerfTest extends TestCase {
+    protected static final Logger LOG = Logger.getLogger(SerializationPerfTest.class);
+
     private static class NullOutputStream extends OutputStream {
         public void write(int b) {
             // do nothing - exclude persistence from perf
@@ -66,46 +70,54 @@ public class SerializationPerfTest extends TestCase {
         long end = System.nanoTime();
         long durationms = (end - start)/1000000L;
         long pernodeus = ((end - start)/1000L)/count;
-        System.out.println("Serialized " + count + " nodes in "
+        LOG.info("Serialized " + count + " nodes in "
                 + durationms + " ms (" + pernodeus + "us/node), depth="
                 + depth + " width=" + width + " datalen=" + len);
     }
 
+    @Test
     public void testSingleSerialize()
             throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(1, 0, 20);
     }
 
+    @Test
     public void testWideSerialize()
             throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(2, 50000, 20);
     }
 
+    @Test
     public void testDeepSerialize()
             throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(1000, 1, 20);
     }
 
+    @Test
     public void test10Wide5DeepSerialize()
             throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(5, 10, 20);
     }
 
+    @Test
     public void test15Wide5DeepSerialize()
             throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(5, 15, 20);
     }
 
+    @Test
     public void test25Wide4DeepSerialize()
             throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(4, 25, 20);
     }
 
+    @Test
     public void test40Wide4DeepSerialize()
             throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(4, 40, 20);
     }
 
+    @Test
     public void test300Wide3DeepSerialize()
             throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
         serializeTree(3, 300, 20);

+ 6 - 2
src/java/test/org/apache/zookeeper/server/ZooKeeperServerTest.java

@@ -21,12 +21,14 @@ package org.apache.zookeeper.server;
 import java.io.File;
 import java.util.List;
 
+import junit.framework.TestCase;
+
 import org.apache.zookeeper.server.persistence.FileTxnLog;
 import org.apache.zookeeper.server.persistence.Util;
-
-import junit.framework.TestCase;
+import org.junit.Test;
 
 public class ZooKeeperServerTest extends TestCase {
+    @Test
     public void testSortDataDirAscending() {
         File[] files = new File[5];
 
@@ -47,6 +49,7 @@ public class ZooKeeperServerTest extends TestCase {
         assertEquals(orig[4], filelist.get(4));
     }
 
+    @Test
     public void testSortDataDirDescending() {
         File[] files = new File[5];
 
@@ -67,6 +70,7 @@ public class ZooKeeperServerTest extends TestCase {
         assertEquals(orig[2], filelist.get(4));
     }
 
+    @Test
     public void testGetLogFiles() {
         File[] files = new File[5];
 

+ 2 - 2
src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java

@@ -202,13 +202,13 @@ public class QuorumPeerMainTest extends TestCase implements Watcher {
                         5000);
 
             assertFalse("Server never came up", isup);
-            
+
             q1.shutdown();
 
             assertTrue("waiting for server 1 down",
                     ClientBase.waitForServerDown("localhost:" + CLIENT_PORT_QP1,
                             ClientBase.CONNECTION_TIMEOUT));
-            
+
         } finally {
             Logger.getLogger(org.apache.zookeeper.server.quorum.QuorumPeer.class)
                 .removeAppender(appender);

+ 24 - 19
src/java/test/org/apache/zookeeper/test/ACLTest.java

@@ -29,6 +29,7 @@ import junit.framework.TestCase;
 
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
@@ -42,9 +43,9 @@ import org.apache.zookeeper.server.ZooKeeperServer;
 
 public class ACLTest extends TestCase implements Watcher {
     private static final Logger LOG = Logger.getLogger(ACLTest.class);
-    private static String HOSTPORT = "127.0.0.1:2355";
-    ZooKeeperServer zks;
-    private CountDownLatch startSignal;
+    private static final String HOSTPORT =
+        "127.0.0.1:" + PortAssignment.unique();
+    private volatile CountDownLatch startSignal;
 
     @Override
     protected void setUp() throws Exception {
@@ -58,14 +59,14 @@ public class ACLTest extends TestCase implements Watcher {
     public void testDisconnectedAddAuth() throws Exception {
         File tmpDir = ClientBase.createTmpDir();
         ClientBase.setupTestEnv();
-        zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+        ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         SyncRequestProcessor.setSnapCount(1000);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
         f.startup(zks);
         LOG.info("starting up the zookeeper server .. waiting");
-        assertTrue("waiting for server being up", 
-                ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
+        assertTrue("waiting for server being up",
+                ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
         ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
         try {
             zk.addAuthInfo("digest", "pat:test".getBytes());
@@ -80,7 +81,7 @@ public class ACLTest extends TestCase implements Watcher {
                    ClientBase.waitForServerDown(HOSTPORT,
                            ClientBase.CONNECTION_TIMEOUT));
     }
-    
+
     /**
      * Verify that acl optimization of storing just
      * a few acls and there references in the data
@@ -89,14 +90,14 @@ public class ACLTest extends TestCase implements Watcher {
     public void testAcls() throws Exception {
         File tmpDir = ClientBase.createTmpDir();
         ClientBase.setupTestEnv();
-        zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+        ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         SyncRequestProcessor.setSnapCount(1000);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
         f.startup(zks);
         LOG.info("starting up the zookeeper server .. waiting");
         assertTrue("waiting for server being up",
-                ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
+                ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
         ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
         String path;
         LOG.info("starting creating acls");
@@ -122,8 +123,7 @@ public class ACLTest extends TestCase implements Watcher {
         // now shutdown the server and restart it
         f.shutdown();
         assertTrue("waiting for server down",
-                ClientBase.waitForServerDown(HOSTPORT,
-                        CONNECTION_TIMEOUT));
+                ClientBase.waitForServerDown(HOSTPORT, CONNECTION_TIMEOUT));
         startSignal = new CountDownLatch(1);
 
         zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
@@ -132,15 +132,14 @@ public class ACLTest extends TestCase implements Watcher {
         f.startup(zks);
 
         assertTrue("waiting for server up",
-                   ClientBase.waitForServerUp(HOSTPORT,
-                                       CONNECTION_TIMEOUT));
+                   ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
 
         startSignal.await(CONNECTION_TIMEOUT,
                 TimeUnit.MILLISECONDS);
         assertTrue("count == 0", startSignal.getCount() == 0);
 
         assertTrue("acl map ", (101 == zks.dataTree.longKeyMap.size()));
-        for (int j =200; j < 205; j++) {
+        for (int j = 200; j < 205; j++) {
             path = "/" + j;
             ACL acl = new ACL();
             acl.setPerms(0);
@@ -154,6 +153,8 @@ public class ACLTest extends TestCase implements Watcher {
         }
         assertTrue("acl map ", (106 == zks.dataTree.longKeyMap.size()));
 
+        zk.close();
+
         f.shutdown();
 
         assertTrue("waiting for server down",
@@ -168,11 +169,15 @@ public class ACLTest extends TestCase implements Watcher {
      * @see org.apache.zookeeper.Watcher#process(org.apache.zookeeper.WatcherEvent)
      */
     public void process(WatchedEvent event) {
-        LOG.info("Event:" + event.getState() + " " + event.getType() + " " + event.getPath());
-        if (event.getState() == KeeperState.SyncConnected
-                && startSignal != null && startSignal.getCount() > 0)
-        {
-            startSignal.countDown();
+        LOG.info("Event:" + event.getState() + " " + event.getType() + " "
+                 + event.getPath());
+        if (event.getState() == KeeperState.SyncConnected) {
+            if (startSignal != null && startSignal.getCount() > 0) {
+                LOG.info("startsignal.countDown()");
+                startSignal.countDown();
+            } else {
+                LOG.warn("startsignal " + startSignal);
+            }
         }
     }
 }

+ 1 - 0
src/java/test/org/apache/zookeeper/test/AsyncTest.java

@@ -190,6 +190,7 @@ public class AsyncTest extends TestCase
         }
         // before restart
         QuorumBase qt = new QuorumBase();
+        qt.setUp();
         qt.verifyRootOfAllServersMatch(qb.hostPort);
         tearDown();
 

+ 1 - 0
src/java/test/org/apache/zookeeper/test/ChrootClientTest.java

@@ -33,6 +33,7 @@ public class ChrootClientTest extends ClientTest {
         String hp = hostPort;
         hostPort = hostPort + "/chrootclienttest";
 
+        System.out.println(hostPort);
         super.setUp();
 
         LOG.info("STARTING " + getName());

+ 0 - 8
src/java/test/org/apache/zookeeper/test/ChrootTest.java

@@ -27,16 +27,8 @@ import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.test.AsyncOps.ACLCB;
-import org.apache.zookeeper.test.AsyncOps.ChildrenCB;
-import org.apache.zookeeper.test.AsyncOps.DataCB;
-import org.apache.zookeeper.test.AsyncOps.StatCB;
-import org.apache.zookeeper.test.AsyncOps.StringCB;
-import org.apache.zookeeper.test.AsyncOps.VoidCB;
-import org.junit.Test;
 
 public class ChrootTest extends ClientBase {
     private class MyWatcher implements Watcher {

+ 51 - 12
src/java/test/org/apache/zookeeper/test/ClientBase.java

@@ -25,6 +25,7 @@ import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.net.Socket;
 import java.util.Arrays;
+import java.util.LinkedList;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -35,6 +36,7 @@ import junit.framework.TestCase;
 
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.TestableZooKeeper;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
@@ -51,7 +53,7 @@ public abstract class ClientBase extends TestCase {
     static final File BASETEST =
         new File(System.getProperty("build.test.dir", "build"));
 
-    protected String hostPort = "127.0.0.1:33221";
+    protected String hostPort = "127.0.0.1:" + PortAssignment.unique();
     protected int maxCnxns = 0;
     protected NIOServerCnxn.Factory serverFactory = null;
     protected File tmpDir = null;
@@ -59,10 +61,6 @@ public abstract class ClientBase extends TestCase {
         super();
     }
 
-    public ClientBase(String name) {
-        super(name);
-    }
-
     /**
      * In general don't use this. Only use in the special case that you
      * want to ignore results (for whatever reason) in your test. Don't
@@ -137,6 +135,9 @@ public abstract class ClientBase extends TestCase {
         return createClient(watcher, hp);
     }
 
+    private LinkedList<ZooKeeper> allClients;
+    private boolean allClientsSetup = false;
+
     protected TestableZooKeeper createClient(CountdownWatcher watcher, String hp)
         throws IOException, InterruptedException
     {
@@ -148,6 +149,19 @@ public abstract class ClientBase extends TestCase {
         {
             fail("Unable to connect to server");
         }
+        synchronized(this) {
+            if (!allClientsSetup) {
+                LOG.error("allClients never setup");
+                fail("allClients never setup");
+            }
+            if (allClients != null) {
+                allClients.add(zk);
+            } else {
+                // test done - close the zk, not needed
+                zk.close();
+            }
+        }
+
         JMXEnv.ensureAll("0x" + Long.toHexString(zk.getSessionId()));
 
         return zk;
@@ -247,11 +261,11 @@ public abstract class ClientBase extends TestCase {
         File tmpFile = File.createTempFile("test", ".junit", parentDir);
         // don't delete tmpFile - this ensures we don't attempt to create
         // a tmpDir with a duplicate name
-        
+
         File tmpDir = new File(tmpFile + ".dir");
         assertFalse(tmpDir.exists()); // never true if tmpfile does it's job
         assertTrue(tmpDir.mkdirs());
-        
+
         return tmpDir;
     }
     private static int getPort(String hostPort) {
@@ -262,10 +276,10 @@ public abstract class ClientBase extends TestCase {
         }
         return Integer.parseInt(portstr);
     }
-    
+
     static NIOServerCnxn.Factory createNewServerInstance(File dataDir,
             NIOServerCnxn.Factory factory, String hostPort, int maxCnxns)
-        throws IOException, InterruptedException 
+        throws IOException, InterruptedException
     {
         ZooKeeperServer zks = new ZooKeeperServer(dataDir, dataDir, 3000);
         final int PORT = getPort(hostPort);
@@ -306,6 +320,11 @@ public abstract class ClientBase extends TestCase {
         FileTxnLog.setPreallocSize(100 * 1024);
     }
 
+    protected void setUpAll() throws Exception {
+        allClients = new LinkedList<ZooKeeper>();
+        allClientsSetup = true;
+    }
+
     @Override
     protected void setUp() throws Exception {
         LOG.info("STARTING " + getName());
@@ -313,6 +332,8 @@ public abstract class ClientBase extends TestCase {
 
         JMXEnv.setUp();
 
+        setUpAll();
+
         tmpDir = createTmpDir(BASETEST);
 
         startServer();
@@ -334,19 +355,37 @@ public abstract class ClientBase extends TestCase {
         // ensure no beans are leftover
         JMXEnv.ensureOnly();
     }
-        
+
+    protected void tearDownAll() throws Exception {
+        synchronized (this) {
+            for (ZooKeeper zk : allClients) {
+                try {
+                    if (zk != null)
+                        zk.close();
+                } catch (InterruptedException e) {
+                    LOG.warn("ignoring interrupt", e);
+                }
+            }
+            allClients = null;
+        }
+    }
+
     @Override
     protected void tearDown() throws Exception {
         LOG.info("tearDown starting");
+        tearDownAll();
 
         stopServer();
-        
+
         if (tmpDir != null) {
             //assertTrue("delete " + tmpDir.toString(), recursiveDelete(tmpDir));
             // FIXME see ZOOKEEPER-121 replace following line with previous
             recursiveDelete(tmpDir);
         }
-        
+
+        // This has to be set to null when the same instance of this class is reused between test cases
+        serverFactory = null;
+
         JMXEnv.tearDown();
 
         LOG.info("FINISHED " + getName());

+ 20 - 12
src/java/test/org/apache/zookeeper/test/ClientRetry.java

@@ -19,10 +19,11 @@ package org.apache.zookeeper.test;
 
 import java.io.IOException;
 import java.util.concurrent.TimeoutException;
+
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper.States;
 
-public class ClientRetry extends ClientBase{
+public class ClientRetry extends ClientBase {
 
     public void setUp() throws Exception {
         maxCnxns = 1;
@@ -44,17 +45,24 @@ public class ClientRetry extends ClientBase{
         CountdownWatcher cdw1 = new CountdownWatcher();
         CountdownWatcher cdw2 = new CountdownWatcher();
         ZooKeeper zk = new ZooKeeper(hostPort, 10000, cdw1);
-        cdw1.waitForConnected(CONNECTION_TIMEOUT);
-        ZooKeeper zk2 = new ZooKeeper(hostPort, 10000, cdw2);
-        States s1 = zk.getState();
-        States s2 = zk2.getState();
-        assertSame(s1,States.CONNECTED);
-        assertSame(s2,States.CONNECTING);
-        cdw1.reset();
-        zk.close();
-        cdw1.waitForDisconnected(CONNECTION_TIMEOUT);
-        cdw2.waitForConnected(CONNECTION_TIMEOUT);
-        assertSame(zk2.getState(),States.CONNECTED);
+        try {
+            cdw1.waitForConnected(CONNECTION_TIMEOUT);
+            ZooKeeper zk2 = new ZooKeeper(hostPort, 10000, cdw2);
+            try {
+                States s1 = zk.getState();
+                States s2 = zk2.getState();
+                assertSame(s1,States.CONNECTED);
+                assertSame(s2,States.CONNECTING);
+                cdw1.reset();
+                cdw1.waitForDisconnected(CONNECTION_TIMEOUT);
+                cdw2.waitForConnected(CONNECTION_TIMEOUT);
+                assertSame(zk2.getState(),States.CONNECTED);
+            } finally {
+                zk2.close();
+            }
+        } finally {
+            zk.close();
+        }
     }
 }
            

+ 3 - 1
src/java/test/org/apache/zookeeper/test/DataTreeTest.java

@@ -21,10 +21,11 @@ package org.apache.zookeeper.test;
 import junit.framework.TestCase;
 
 import org.apache.log4j.Logger;
+import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.server.DataTree;
+import org.junit.Test;
 
 public class DataTreeTest extends TestCase {
     protected static final Logger LOG = Logger.getLogger(DataTreeTest.class);
@@ -43,6 +44,7 @@ public class DataTreeTest extends TestCase {
         LOG.info("FINISHED " + getName());
     }
 
+    @Test
     public void testRootWatchTriggered() throws Exception {
         class MyWatcher implements Watcher{
             boolean fired=false;

+ 1 - 1
src/java/test/org/apache/zookeeper/test/EventTypeTest.java

@@ -21,9 +21,9 @@ package org.apache.zookeeper.test;
 import java.util.EnumSet;
 
 import junit.framework.TestCase;
-import org.junit.Test;
 
 import org.apache.zookeeper.Watcher.Event.EventType;
+import org.junit.Test;
 
 public class EventTypeTest extends TestCase {
     

+ 10 - 12
src/java/test/org/apache/zookeeper/test/FLENewEpochTest.java

@@ -27,6 +27,7 @@ import java.util.concurrent.Semaphore;
 import junit.framework.TestCase;
 
 import org.apache.log4j.Logger;
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.server.quorum.FastLeaderElection;
 import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.Vote;
@@ -38,8 +39,6 @@ public class FLENewEpochTest extends TestCase {
     protected static final Logger LOG = Logger.getLogger(FLENewEpochTest.class);
 
     int count;
-    int baseport;
-    int baseLEport;
     HashMap<Long,QuorumServer> peers;
     ArrayList<LEThread> threads;
     File tmpdir[];
@@ -52,8 +51,6 @@ public class FLENewEpochTest extends TestCase {
     @Override
     public void setUp() throws Exception {
         count = 3;
-        baseport= 33303;
-        baseLEport = 43303;
 
         peers = new HashMap<Long,QuorumServer>(count);
         threads = new ArrayList<LEThread>(count);
@@ -101,7 +98,7 @@ public class FLENewEpochTest extends TestCase {
                     LOG.info("Going to call leader election again: " + i);
                     v = peer.getElectionAlg().lookForLeader();
 
-                    if(v == null){
+                    if (v == null){
                         fail("Thread " + i + " got a null vote");
                     }
 
@@ -114,7 +111,7 @@ public class FLENewEpochTest extends TestCase {
                     LOG.info("Finished election: " + i + ", " + v.id);
                     //votes[i] = v;
 
-                    switch(i){
+                    switch (i) {
                     case 0:
                         LOG.info("First peer, do nothing, just join");
                         if(finish0.tryAcquire(1000, java.util.concurrent.TimeUnit.MILLISECONDS)){
@@ -128,8 +125,7 @@ public class FLENewEpochTest extends TestCase {
                         if(round[1] != 0){
                             finish0.release();
                             flag = false;
-                        }
-                        else{
+                        } else {
                             finish3.acquire();
                             start0.release();
                         }
@@ -147,7 +143,7 @@ public class FLENewEpochTest extends TestCase {
                         break;
                     }
                 }
-            } catch (Exception e){
+            } catch (Exception e) {
                 e.printStackTrace();
             }
         }
@@ -161,10 +157,12 @@ public class FLENewEpochTest extends TestCase {
 
           LOG.info("TestLE: " + getName()+ ", " + count);
           for(int i = 0; i < count; i++) {
-              peers.put(Long.valueOf(i), new QuorumServer(i, new InetSocketAddress(baseport+100+i),
-                      new InetSocketAddress(baseLEport+100+i)));
+              peers.put(Long.valueOf(i),
+                      new QuorumServer(i,
+                              new InetSocketAddress(PortAssignment.unique()),
+                      new InetSocketAddress(PortAssignment.unique())));
               tmpdir[i] = ClientBase.createTmpDir();
-              port[i] = baseport+i;
+              port[i] = PortAssignment.unique();
           }
 
           for(int i = 1; i < le.length; i++) {

+ 17 - 17
src/java/test/org/apache/zookeeper/test/FLERestartTest.java

@@ -26,10 +26,10 @@ import java.util.HashSet;
 import java.util.Random;
 import java.util.concurrent.Semaphore;
 
-
 import junit.framework.TestCase;
 
 import org.apache.log4j.Logger;
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.server.quorum.FastLeaderElection;
 import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.Vote;
@@ -58,8 +58,8 @@ public class FLERestartTest extends TestCase {
     }
 
     int count;
-    int baseport;
-    int baseLEport;
+    //    int baseport;
+    //    int baseLEport;
     HashMap<Long,QuorumServer> peers;
     ArrayList<FLERestartThread> restartThreads;
     HashMap<Integer, HashSet<TestVote> > voteMap;
@@ -67,7 +67,7 @@ public class FLERestartTest extends TestCase {
     int port[];
     int successCount;
     Semaphore finish;
-    
+
     volatile Vote votes[];
     volatile boolean leaderDies;
     volatile long leader = -1;
@@ -77,8 +77,6 @@ public class FLERestartTest extends TestCase {
     @Override
     public void setUp() throws Exception {
         count = 3;
-        baseport= 33003;
-        baseLEport = 43003;
 
         peers = new HashMap<Long,QuorumServer>(count);
         restartThreads = new ArrayList<FLERestartThread>(count);
@@ -137,23 +135,23 @@ public class FLERestartTest extends TestCase {
                             LOG.info("First peer, shutting it down");
                             peer.shutdown();
                             ((FastLeaderElection) restartThreads.get(i).peer.getElectionAlg()).shutdown();
-                            
+
                             peer = new QuorumPeer(peers, tmpdir[i], tmpdir[i], port[i], 3, i, 2, 2, 2);
                             peer.startLeaderElection();
                             peerRound++;
-                        } else { 
+                        } else {
                             finish.release(2);
                             return;
-                        }    
-                        
+                        }
+
                         break;
                     case 1:
                         LOG.info("Second entering case");
                         finish.acquire();
                         //if(threads.get(0).peer.getPeerState() == ServerState.LEADING ){
                         LOG.info("Release");
-                        
-                        return;                   
+
+                        return;
                     case 2:
                         LOG.info("First peer, do nothing, just join");
                         finish.acquire();
@@ -168,8 +166,8 @@ public class FLERestartTest extends TestCase {
             }
         }
     }
-    
-    
+
+
     @Test
     public void testLERestart() throws Exception {
 
@@ -179,10 +177,12 @@ public class FLERestartTest extends TestCase {
 
         LOG.info("TestLE: " + getName()+ ", " + count);
         for(int i = 0; i < count; i++) {
-            peers.put(Long.valueOf(i), new QuorumServer(i, new InetSocketAddress(baseport+100+i),
-                    new InetSocketAddress(baseLEport+100+i)));
+            peers.put(Long.valueOf(i),
+                    new QuorumServer(i,
+                            new InetSocketAddress(PortAssignment.unique()),
+                    new InetSocketAddress(PortAssignment.unique())));
             tmpdir[i] = ClientBase.createTmpDir();
-            port[i] = baseport+i;
+            port[i] = PortAssignment.unique();
         }
 
         for(int i = 0; i < count; i++) {

+ 19 - 15
src/java/test/org/apache/zookeeper/test/FLETest.java

@@ -28,6 +28,7 @@ import java.util.Random;
 import junit.framework.TestCase;
 
 import org.apache.log4j.Logger;
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.server.quorum.FastLeaderElection;
 import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.Vote;
@@ -37,6 +38,7 @@ import org.junit.Test;
 
 public class FLETest extends TestCase {
     protected static final Logger LOG = Logger.getLogger(FLETest.class);
+    private FLETest.LEThread leThread;
 
     static class TestVote {
         TestVote(int id, long leader) {
@@ -56,8 +58,6 @@ public class FLETest extends TestCase {
     }
 
     int count;
-    int baseport;
-    int baseLEport;
     HashMap<Long,QuorumServer> peers;
     ArrayList<LEThread> threads;
     HashMap<Integer, HashSet<TestVote> > voteMap;
@@ -75,8 +75,6 @@ public class FLETest extends TestCase {
     @Override
     public void setUp() throws Exception {
         count = 7;
-        baseport= 33003;
-        baseLEport = 43003;
 
         peers = new HashMap<Long,QuorumServer>(count);
         threads = new ArrayList<LEThread>(count);
@@ -92,8 +90,10 @@ public class FLETest extends TestCase {
 
     @Override
     public void tearDown() throws Exception {
-        for(int i = 0; i < threads.size(); i++) {
-            ((FastLeaderElection) threads.get(i).peer.getElectionAlg()).shutdown();
+        for (int i = 0; i < threads.size(); i++) {
+            leThread = threads.get(i);
+            ((FastLeaderElection) leThread.peer.getElectionAlg()).shutdown();
+            leThread.peer.shutdown();
         }
         LOG.info("FINISHED " + getName());
     }
@@ -264,14 +264,17 @@ public class FLETest extends TestCase {
 
         LOG.info("TestLE: " + getName()+ ", " + count);
         for(int i = 0; i < count; i++) {
-            peers.put(Long.valueOf(i), new QuorumServer(i, new InetSocketAddress(baseport+100+i),
-                    new InetSocketAddress(baseLEport+100+i)));
+            peers.put(Long.valueOf(i),
+                    new QuorumServer(i,
+                            new InetSocketAddress(PortAssignment.unique()),
+                    new InetSocketAddress(PortAssignment.unique())));
             tmpdir[i] = ClientBase.createTmpDir();
-            port[i] = baseport+i;
+            port[i] = PortAssignment.unique();
         }
 
         for(int i = 0; i < le.length; i++) {
-            QuorumPeer peer = new QuorumPeer(peers, tmpdir[i], tmpdir[i], port[i], 3, i, 2, 2, 2);
+            QuorumPeer peer = new QuorumPeer(peers, tmpdir[i], tmpdir[i],
+                    port[i], 3, i, 2, 2, 2);
             peer.startLeaderElection();
             LEThread thread = new LEThread(peer, i);
             thread.start();
@@ -288,12 +291,13 @@ public class FLETest extends TestCase {
             }
         }
 
-       /*
-        * Lists what threads haven-t joined. A thread doesn't join if it hasn't decided
-        * upon a leader yet. It can happen that a peer is slow or disconnected, and it can
-        * take longer to nominate and connect to the current leader.
+        /*
+        * Lists what threads haven-t joined. A thread doesn't join if
+        * it hasn't decided upon a leader yet. It can happen that a
+        * peer is slow or disconnected, and it can take longer to
+        * nominate and connect to the current leader.
         */
-       for(int i = 0; i < threads.size(); i++) {
+       for (int i = 0; i < threads.size(); i++) {
             if (threads.get(i).isAlive()) {
                 LOG.info("Threads didn't join: " + i);
             }

+ 12 - 8
src/java/test/org/apache/zookeeper/test/HierarchicalQuorumTest.java

@@ -28,6 +28,7 @@ import java.util.Random;
 import junit.framework.TestCase;
 
 import org.apache.log4j.Logger;
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.server.quorum.FastLeaderElection;
 import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.Vote;
@@ -43,8 +44,6 @@ public class HierarchicalQuorumTest extends TestCase {
     Properties qp;
 
     int count;
-    int baseport;
-    int baseLEport;
     HashMap<Long,QuorumServer> peers;
     ArrayList<LEThread> threads;
     File tmpdir[];
@@ -61,8 +60,6 @@ public class HierarchicalQuorumTest extends TestCase {
     @Override
     protected void setUp() throws Exception {
         count = 9;
-        baseport= 33003;
-        baseLEport = 43003;
 
         peers = new HashMap<Long,QuorumServer>(count);
         threads = new ArrayList<LEThread>(count);
@@ -93,7 +90,12 @@ public class HierarchicalQuorumTest extends TestCase {
 
     protected void tearDown() throws Exception {
         for(int i = 0; i < threads.size(); i++) {
-            ((FastLeaderElection) threads.get(i).peer.getElectionAlg()).shutdown();
+            LEThread leThread = threads.get(i);
+            ((FastLeaderElection) leThread.peer.getElectionAlg()).shutdown();
+            // shutdown() has to be explicitly called for every thread to
+            // make sure that resources are freed properly and all fixed network ports
+            // are available for other test cases
+            leThread.peer.shutdown();
         }
         LOG.info("FINISHED " + getName());
     }
@@ -148,10 +150,12 @@ public class HierarchicalQuorumTest extends TestCase {
 
         LOG.info("TestHierarchicalQuorum: " + getName()+ ", " + count);
         for(int i = 0; i < count; i++) {
-            peers.put(Long.valueOf(i), new QuorumServer(i, new InetSocketAddress(baseport+100+i),
-                    new InetSocketAddress(baseLEport+100+i)));
+            peers.put(Long.valueOf(i),
+                    new QuorumServer(i,
+                            new InetSocketAddress(PortAssignment.unique()),
+                    new InetSocketAddress(PortAssignment.unique())));
             tmpdir[i] = ClientBase.createTmpDir();
-            port[i] = baseport+i;
+            port[i] = PortAssignment.unique();
         }
 
         for(int i = 0; i < le.length; i++) {

+ 4 - 5
src/java/test/org/apache/zookeeper/test/IntegrityCheck.java

@@ -36,17 +36,16 @@ import java.util.Date;
 import java.util.HashMap;
 
 import org.apache.log4j.Logger;
-
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.AsyncCallback.DataCallback;
 import org.apache.zookeeper.AsyncCallback.StatCallback;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.WatchedEvent;
 
 public class IntegrityCheck implements Watcher, StatCallback, DataCallback {
     private static final Logger LOG = Logger.getLogger(IntegrityCheck.class);

+ 22 - 16
src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java

@@ -26,6 +26,7 @@ import java.util.concurrent.CountDownLatch;
 import junit.framework.TestCase;
 
 import org.apache.log4j.Logger;
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
@@ -36,12 +37,13 @@ import org.apache.zookeeper.server.ZooKeeperServer;
 
 public class InvalidSnapshotTest extends TestCase implements Watcher {
     private final static Logger LOG = Logger.getLogger(UpgradeTest.class);
-    private static String HOSTPORT = "127.0.0.1:2359";
-    ZooKeeperServer zks;
+    private static final String HOSTPORT =
+            "127.0.0.1:" + PortAssignment.unique();
+
     private static final File testData = new File(
             System.getProperty("test.data.dir", "build/test/data"));
     private CountDownLatch startSignal;
-    
+
     @Override
     protected void setUp() throws Exception {
         LOG.info("STARTING " + getName());
@@ -50,39 +52,43 @@ public class InvalidSnapshotTest extends TestCase implements Watcher {
     protected void tearDown() throws Exception {
         LOG.info("FINISHED " + getName());
     }
-    
+
     /**
      * test the snapshot
-     * @throws Exception
+     * @throws Exception an exception could be expected
      */
     public void testSnapshot() throws Exception {
         File snapDir = new File(testData, "invalidsnap");
-        zks = new ZooKeeperServer(snapDir, snapDir, 3000);
+        ZooKeeperServer zks = new ZooKeeperServer(snapDir, snapDir, 3000);
         SyncRequestProcessor.setSnapCount(1000);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
         f.startup(zks);
         LOG.info("starting up the zookeeper server .. waiting");
-        assertTrue("waiting for server being up", 
-                ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
+        assertTrue("waiting for server being up",
+                ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
         ZooKeeper zk = new ZooKeeper(HOSTPORT, 20000, this);
-        // we know this from the data files
-        // this node is the last node in the snapshot
-        
-        assertTrue(zk.exists("/9/9/8", false) != null);
+        try {
+            // we know this from the data files
+            // this node is the last node in the snapshot
+
+            assertTrue(zk.exists("/9/9/8", false) != null);
+        } finally {
+            zk.close();
+        }
         f.shutdown();
         assertTrue("waiting for server down",
                    ClientBase.waitForServerDown(HOSTPORT,
                            ClientBase.CONNECTION_TIMEOUT));
-        
+
     }
-    
+
     public void process(WatchedEvent event) {
         LOG.info("Event:" + event.getState() + " " + event.getType() + " " + event.getPath());
         if (event.getState() == KeeperState.SyncConnected
                 && startSignal != null && startSignal.getCount() > 0)
-        {              
-            startSignal.countDown();      
+        {
+            startSignal.countDown();
         }
     }
 }

+ 8 - 4
src/java/test/org/apache/zookeeper/test/LETest.java

@@ -26,6 +26,7 @@ import java.util.Random;
 
 import junit.framework.TestCase;
 
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.server.quorum.LeaderElection;
 import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.Vote;
@@ -84,22 +85,25 @@ public class LETest extends TestCase {
     }
     public void testLE() throws Exception {
         int count = 30;
-        int baseport= 33003;
         HashMap<Long,QuorumServer> peers = new HashMap<Long,QuorumServer>(count);
         ArrayList<LEThread> threads = new ArrayList<LEThread>(count);
         File tmpdir[] = new File[count];
         int port[] = new int[count];
         votes = new Vote[count];
         for(int i = 0; i < count; i++) {
-            peers.put(Long.valueOf(i), new QuorumServer(i, new InetSocketAddress("127.0.0.1", baseport+100+i)));
+            peers.put(Long.valueOf(i),
+                    new QuorumServer(i,
+                            new InetSocketAddress("127.0.0.1",
+                                    PortAssignment.unique())));
             tmpdir[i] = ClientBase.createTmpDir();
-            port[i] = baseport+i;
+            port[i] = PortAssignment.unique();
         }
         LeaderElection le[] = new LeaderElection[count];
         leaderDies = true;
         boolean allowOneBadLeader = leaderDies;
         for(int i = 0; i < le.length; i++) {
-            QuorumPeer peer = new QuorumPeer(peers, tmpdir[i], tmpdir[i], port[i], 0, i, 2, 2, 2);
+            QuorumPeer peer = new QuorumPeer(peers, tmpdir[i], tmpdir[i],
+                    port[i], 0, i, 2, 2, 2);
             peer.startLeaderElection();
             le[i] = new LeaderElection(peer);
             LEThread thread = new LEThread(le[i], peer, i);

+ 1 - 1
src/java/test/org/apache/zookeeper/test/NullDataTest.java

@@ -25,9 +25,9 @@ import java.util.concurrent.TimeUnit;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.AsyncCallback.StatCallback;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.AsyncCallback.StatCallback;
 
 public class NullDataTest extends ClientBase implements StatCallback {
     String snapCount;

+ 27 - 17
src/java/test/org/apache/zookeeper/test/OOMTest.java

@@ -26,20 +26,23 @@ import java.util.ArrayList;
 
 import junit.framework.TestCase;
 
+import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.server.NIOServerCnxn;
 import org.apache.zookeeper.server.ZooKeeperServer;
+import org.junit.Test;
 
 /**
  *
  */
 public class OOMTest extends TestCase implements Watcher {
+    @Test
     public void testOOM() throws IOException, InterruptedException, KeeperException {
         // This test takes too long to run!
         if (true)
@@ -58,8 +61,8 @@ public class OOMTest extends TestCase implements Watcher {
         }
         ClientBase.setupTestEnv();
         ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
-        
-        final int PORT = 33221;
+
+        final int PORT = PortAssignment.unique();
         NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
         f.startup(zks);
         assertTrue("waiting for server up",
@@ -67,14 +70,14 @@ public class OOMTest extends TestCase implements Watcher {
                                               CONNECTION_TIMEOUT));
 
         System.err.println("OOM Stage 0");
-        utestPrep();
+        utestPrep(PORT);
         System.out.println("Free = " + Runtime.getRuntime().freeMemory()
                 + " total = " + Runtime.getRuntime().totalMemory() + " max = "
                 + Runtime.getRuntime().maxMemory());
         System.err.println("OOM Stage 1");
         for (int i = 0; i < 1000; i++) {
             System.out.println(i);
-            utestExists();
+            utestExists(PORT);
         }
         System.out.println("Free = " + Runtime.getRuntime().freeMemory()
                 + " total = " + Runtime.getRuntime().totalMemory() + " max = "
@@ -82,7 +85,7 @@ public class OOMTest extends TestCase implements Watcher {
         System.err.println("OOM Stage 2");
         for (int i = 0; i < 1000; i++) {
             System.out.println(i);
-            utestGet();
+            utestGet(PORT);
         }
         System.out.println("Free = " + Runtime.getRuntime().freeMemory()
                 + " total = " + Runtime.getRuntime().totalMemory() + " max = "
@@ -90,7 +93,7 @@ public class OOMTest extends TestCase implements Watcher {
         System.err.println("OOM Stage 3");
         for (int i = 0; i < 1000; i++) {
             System.out.println(i);
-            utestChildren();
+            utestChildren(PORT);
         }
         System.out.println("Free = " + Runtime.getRuntime().freeMemory()
                 + " total = " + Runtime.getRuntime().totalMemory() + " max = "
@@ -103,28 +106,33 @@ public class OOMTest extends TestCase implements Watcher {
                                                 CONNECTION_TIMEOUT));
     }
 
-    private void utestExists() throws IOException, InterruptedException, KeeperException {
+    private void utestExists(int port)
+        throws IOException, InterruptedException, KeeperException
+    {
         ZooKeeper zk =
-            new ZooKeeper("127.0.0.1:33221", CONNECTION_TIMEOUT, this);
+            new ZooKeeper("127.0.0.1:" + port, CONNECTION_TIMEOUT, this);
         for (int i = 0; i < 10000; i++) {
             zk.exists("/this/path/doesnt_exist!", true);
         }
         zk.close();
     }
 
-    private void utestPrep() throws IOException,
-            InterruptedException, KeeperException {
+    private void utestPrep(int port)
+        throws IOException, InterruptedException, KeeperException
+    {
         ZooKeeper zk =
-            new ZooKeeper("127.0.0.1:33221", CONNECTION_TIMEOUT, this);
+            new ZooKeeper("127.0.0.1:" + port, CONNECTION_TIMEOUT, this);
         for (int i = 0; i < 10000; i++) {
             zk.create("/" + i, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         }
         zk.close();
     }
 
-    private void utestGet() throws IOException, InterruptedException, KeeperException {
+    private void utestGet(int port)
+        throws IOException, InterruptedException, KeeperException
+    {
         ZooKeeper zk =
-            new ZooKeeper("127.0.0.1:33221", CONNECTION_TIMEOUT, this);
+            new ZooKeeper("127.0.0.1:" + port, CONNECTION_TIMEOUT, this);
         for (int i = 0; i < 10000; i++) {
             Stat stat = new Stat();
             zk.getData("/" + i, true, stat);
@@ -132,9 +140,11 @@ public class OOMTest extends TestCase implements Watcher {
         zk.close();
     }
 
-    private void utestChildren() throws IOException, InterruptedException, KeeperException {
+    private void utestChildren(int port)
+        throws IOException, InterruptedException, KeeperException
+    {
         ZooKeeper zk =
-            new ZooKeeper("127.0.0.1:33221", CONNECTION_TIMEOUT, this);
+            new ZooKeeper("127.0.0.1:" + port, CONNECTION_TIMEOUT, this);
         for (int i = 0; i < 10000; i++) {
             zk.getChildren("/" + i, true);
         }

+ 16 - 13
src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java

@@ -23,8 +23,8 @@ import java.util.List;
 
 import junit.framework.TestCase;
 
-import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
@@ -35,33 +35,36 @@ import org.apache.zookeeper.server.SyncRequestProcessor;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
 
-/** 
+/**
  * test the purging of the logs
  * and purging of the snapshots.
  */
 public class PurgeTxnTest extends TestCase implements  Watcher {
-    private static final Logger LOG = Logger.getLogger(PurgeTxnTest.class);
-    private static String HOSTPORT = "127.0.0.1:2357";
-    ZooKeeperServer zks = null;
+    //private static final Logger LOG = Logger.getLogger(PurgeTxnTest.class);
+    private static String HOSTPORT = "127.0.0.1:" + PortAssignment.unique();
     private static final int CONNECTION_TIMEOUT = 3000;
     /**
      * test the purge
-     * @throws Exception
+     * @throws Exception an exception might be thrown here
      */
     public void testPurge() throws Exception {
         File tmpDir = ClientBase.createTmpDir();
         ClientBase.setupTestEnv();
-        zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+        ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         SyncRequestProcessor.setSnapCount(100);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
         f.startup(zks);
-        assertTrue("waiting for server being up ", 
+        assertTrue("waiting for server being up ",
                 ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
         ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
-        for (int i=0; i< 2000; i++) {
-            zk.create("/invalidsnap-" + i, new byte[0], Ids.OPEN_ACL_UNSAFE, 
-                    CreateMode.PERSISTENT);
+        try {
+            for (int i = 0; i< 2000; i++) {
+                zk.create("/invalidsnap-" + i, new byte[0], Ids.OPEN_ACL_UNSAFE,
+                        CreateMode.PERSISTENT);
+            }
+        } finally {
+            zk.close();
         }
         f.shutdown();
         assertTrue("waiting for server to shutdown",
@@ -78,9 +81,9 @@ public class PurgeTxnTest extends TestCase implements  Watcher {
         }
         assertTrue("exactly 3 snapshots ", (numSnaps == 3));
     }
-    
+
     public void process(WatchedEvent event) {
         // do nothing
     }
-    
+
 }

+ 46 - 26
src/java/test/org/apache/zookeeper/test/QuorumBase.java

@@ -26,6 +26,7 @@ import java.util.LinkedHashSet;
 import java.util.Set;
 
 import org.apache.log4j.Logger;
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
@@ -36,14 +37,32 @@ public class QuorumBase extends ClientBase {
 
     File s1dir, s2dir, s3dir, s4dir, s5dir;
     QuorumPeer s1, s2, s3, s4, s5;
+    private int port1;
+    private int port2;
+    private int port3;
+    private int port4;
+    private int port5;
 
+    @Override
     protected void setUp() throws Exception {
         LOG.info("STARTING " + getName());
         setupTestEnv();
 
         JMXEnv.setUp();
 
-        hostPort = "127.0.0.1:2181,127.0.0.1:2182,127.0.0.1:2183,127.0.0.1:2184,127.0.0.1:2185";
+        setUpAll();
+
+        port1 = PortAssignment.unique();
+        port2 = PortAssignment.unique();
+        port3 = PortAssignment.unique();
+        port4 = PortAssignment.unique();
+        port5 = PortAssignment.unique();
+        hostPort = "127.0.0.1:" + port1
+            + ",127.0.0.1:" + port2
+            + ",127.0.0.1:" + port3
+            + ",127.0.0.1:" + port4
+            + ",127.0.0.1:" + port5;
+        LOG.info("Ports are: " + hostPort);
 
         s1dir = ClientBase.createTmpDir();
         s2dir = ClientBase.createTmpDir();
@@ -60,27 +79,27 @@ public class QuorumBase extends ClientBase {
         int initLimit = 3;
         int syncLimit = 3;
         HashMap<Long,QuorumServer> peers = new HashMap<Long,QuorumServer>();
-        peers.put(Long.valueOf(1), new QuorumServer(1, new InetSocketAddress("127.0.0.1", 3181)));
-        peers.put(Long.valueOf(2), new QuorumServer(2, new InetSocketAddress("127.0.0.1", 3182)));
-        peers.put(Long.valueOf(3), new QuorumServer(3, new InetSocketAddress("127.0.0.1", 3183)));
-        peers.put(Long.valueOf(4), new QuorumServer(4, new InetSocketAddress("127.0.0.1", 3184)));
-        peers.put(Long.valueOf(5), new QuorumServer(5, new InetSocketAddress("127.0.0.1", 3185)));
-
-        LOG.info("creating QuorumPeer 1");
-        s1 = new QuorumPeer(peers, s1dir, s1dir, 2181, 0, 1, tickTime, initLimit, syncLimit);
-        assertEquals(2181, s1.getClientPort());
-        LOG.info("creating QuorumPeer 2");
-        s2 = new QuorumPeer(peers, s2dir, s2dir, 2182, 0, 2, tickTime, initLimit, syncLimit);
-        assertEquals(2182, s2.getClientPort());
-        LOG.info("creating QuorumPeer 3");
-        s3 = new QuorumPeer(peers, s3dir, s3dir, 2183, 0, 3, tickTime, initLimit, syncLimit);
-        assertEquals(2183, s3.getClientPort());
-        LOG.info("creating QuorumPeer 4");
-        s4 = new QuorumPeer(peers, s4dir, s4dir, 2184, 0, 4, tickTime, initLimit, syncLimit);
-        assertEquals(2184, s4.getClientPort());
-        LOG.info("creating QuorumPeer 5");
-        s5 = new QuorumPeer(peers, s5dir, s5dir, 2185, 0, 5, tickTime, initLimit, syncLimit);
-        assertEquals(2185, s5.getClientPort());
+        peers.put(Long.valueOf(1), new QuorumServer(1, new InetSocketAddress("127.0.0.1", port1 + 1000)));
+        peers.put(Long.valueOf(2), new QuorumServer(2, new InetSocketAddress("127.0.0.1", port2 + 1000)));
+        peers.put(Long.valueOf(3), new QuorumServer(3, new InetSocketAddress("127.0.0.1", port3 + 1000)));
+        peers.put(Long.valueOf(4), new QuorumServer(4, new InetSocketAddress("127.0.0.1", port4 + 1000)));
+        peers.put(Long.valueOf(5), new QuorumServer(5, new InetSocketAddress("127.0.0.1", port5 + 1000)));
+
+        LOG.info("creating QuorumPeer 1 port " + port1);
+        s1 = new QuorumPeer(peers, s1dir, s1dir, port1, 0, 1, tickTime, initLimit, syncLimit);
+        assertEquals(port1, s1.getClientPort());
+        LOG.info("creating QuorumPeer 2 port " + port2);
+        s2 = new QuorumPeer(peers, s2dir, s2dir, port2, 0, 2, tickTime, initLimit, syncLimit);
+        assertEquals(port2, s2.getClientPort());
+        LOG.info("creating QuorumPeer 3 port " + port3);
+        s3 = new QuorumPeer(peers, s3dir, s3dir, port3, 0, 3, tickTime, initLimit, syncLimit);
+        assertEquals(port3, s3.getClientPort());
+        LOG.info("creating QuorumPeer 4 port " + port4);
+        s4 = new QuorumPeer(peers, s4dir, s4dir, port4, 0, 4, tickTime, initLimit, syncLimit);
+        assertEquals(port4, s4.getClientPort());
+        LOG.info("creating QuorumPeer 5 port " + port5);
+        s5 = new QuorumPeer(peers, s5dir, s5dir, port5, 0, 5, tickTime, initLimit, syncLimit);
+        assertEquals(port5, s5.getClientPort());
         LOG.info("start QuorumPeer 1");
         s1.start();
         LOG.info("start QuorumPeer 2");
@@ -93,6 +112,7 @@ public class QuorumBase extends ClientBase {
         s5.start();
         LOG.info("started QuorumPeer 5");
 
+        LOG.info ("Closing ports " + hostPort);
         for (String hp : hostPort.split(",")) {
             assertTrue("waiting for server up",
                        ClientBase.waitForServerUp(hp,
@@ -159,14 +179,14 @@ public class QuorumBase extends ClientBase {
 
     protected ZooKeeper createClient()
         throws IOException, InterruptedException
-        {
+    {
         return createClient(hostPort);
-        }
+    }
 
     protected ZooKeeper createClient(String hp)
         throws IOException, InterruptedException
-        {
+    {
         CountdownWatcher watcher = new CountdownWatcher();
         return createClient(watcher, hp);
-        }
+    }
 }

+ 2 - 2
src/java/test/org/apache/zookeeper/test/QuorumQuotaTest.java

@@ -34,8 +34,8 @@ import org.apache.zookeeper.data.Stat;
  */
 
 public class QuorumQuotaTest extends QuorumBase {
-    private static final Logger LOG = Logger.getLogger(
-            QuorumQuotaTest.class);
+    private static final Logger LOG =
+        Logger.getLogger(QuorumQuotaTest.class);
 
     @Override
     protected void setUp() throws Exception {

+ 11 - 10
src/java/test/org/apache/zookeeper/test/QuorumTest.java

@@ -20,8 +20,6 @@ package org.apache.zookeeper.test;
 import java.io.IOException;
 import java.util.ArrayList;
 
-import junit.framework.TestCase;
-
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -32,20 +30,23 @@ import org.apache.zookeeper.ZooKeeper;
 import org.junit.Before;
 import org.junit.Test;
 
-public class QuorumTest extends TestCase {
+public class QuorumTest extends QuorumBase {
     private static final Logger LOG = Logger.getLogger(QuorumTest.class);
     public static final long CONNECTION_TIMEOUT = ClientTest.CONNECTION_TIMEOUT;
-    private QuorumBase qb = new QuorumBase();
+
+    private final QuorumBase qb = new QuorumBase();
     private final ClientTest ct = new ClientTest();
-    
+
     @Before
     @Override
     protected void setUp() throws Exception {
-        qb.setUp();        
+        qb.setUp();
         ct.hostPort = qb.hostPort;
+        ct.setUpAll();
     }
-    
+
     protected void tearDown() throws Exception {
+        ct.tearDownAll();
         qb.tearDown();
     }
 
@@ -95,7 +96,7 @@ public class QuorumTest extends TestCase {
     {
         ct.testMutipleWatcherObjs();
     }
-    
+
     @Test
     /**
      * Connect to two different servers with two different handles using the same session and
@@ -117,7 +118,7 @@ public class QuorumTest extends TestCase {
             fail("Should have lost the connection");
         } catch(KeeperException.SessionMovedException e) {
         }
-        
+
         ArrayList<ZooKeeper> toClose = new ArrayList<ZooKeeper>();
         toClose.add(zknew);
         // Let's just make sure it can still move
@@ -128,7 +129,7 @@ public class QuorumTest extends TestCase {
             toClose.add(zknew);
             zknew.create("/t-"+i, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
         }
-	for(ZooKeeper z: toClose) {
+    for(ZooKeeper z: toClose) {
             z.close();
         }
         zk.close();

+ 11 - 7
src/java/test/org/apache/zookeeper/test/RecoveryTest.java

@@ -28,6 +28,7 @@ import junit.framework.TestCase;
 
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
@@ -42,7 +43,8 @@ import org.junit.Test;
 public class RecoveryTest extends TestCase implements Watcher {
     protected static final Logger LOG = Logger.getLogger(RecoveryTest.class);
 
-    private static String HOSTPORT = "127.0.0.1:2344";
+    private static final String HOSTPORT =
+        "127.0.0.1:" + PortAssignment.unique();
 
     private volatile CountDownLatch startSignal;
 
@@ -72,14 +74,14 @@ public class RecoveryTest extends TestCase implements Watcher {
         File tmpDir = ClientBase.createTmpDir();
 
         ClientBase.setupTestEnv();
-        ZooKeeperServer zs = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+        ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
 
         int oldSnapCount = SyncRequestProcessor.getSnapCount();
         SyncRequestProcessor.setSnapCount(1000);
         try {
             final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
             NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
-            f.startup(zs);
+            f.startup(zks);
             LOG.info("starting up the the server, waiting");
 
             assertTrue("waiting for server up",
@@ -115,12 +117,12 @@ public class RecoveryTest extends TestCase implements Watcher {
                        ClientBase.waitForServerDown(HOSTPORT,
                                           CONNECTION_TIMEOUT));
 
-            zs = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+            zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
             f = new NIOServerCnxn.Factory(PORT);
 
             startSignal = new CountDownLatch(1);
 
-            f.startup(zs);
+            f.startup(zks);
 
             assertTrue("waiting for server up",
                        ClientBase.waitForServerUp(HOSTPORT,
@@ -153,12 +155,12 @@ public class RecoveryTest extends TestCase implements Watcher {
                        ClientBase.waitForServerDown(HOSTPORT,
                                           ClientBase.CONNECTION_TIMEOUT));
 
-            zs = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+            zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
             f = new NIOServerCnxn.Factory(PORT);
 
             startSignal = new CountDownLatch(1);
 
-            f.startup(zs);
+            f.startup(zks);
 
             assertTrue("waiting for server up",
                        ClientBase.waitForServerUp(HOSTPORT,
@@ -184,6 +186,8 @@ public class RecoveryTest extends TestCase implements Watcher {
                     }
                 }
             }
+            zk.close();
+
             f.shutdown();
 
             assertTrue("waiting for server down",

+ 12 - 12
src/java/test/org/apache/zookeeper/test/RepeatStartupTest.java

@@ -25,21 +25,24 @@ import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.server.NIOServerCnxn;
 import org.apache.zookeeper.server.ZooKeeperServer;
+import org.junit.Test;
 
 /**
- * this test fails quorum peers 
- * and then brings up one of the
- * node as a standalone server
+ * this test fails quorum peers and then brings up one of the node as
+ * a standalone server
  */
 public class RepeatStartupTest extends TestCase {
-    
+
     /** bring up 5 quorum peers and then shut them down
      * and then bring one of the nodes as server
-     * @throws Exception
+     *
+     * @throws Exception might be thrown here
      */
+    @Test
     public void testFail() throws Exception {
         QuorumBase qb = new QuorumBase();
         qb.setUp();
+
         System.out.println("Comment: the servers are at " + qb.hostPort);
         ZooKeeper zk = qb.createClient();
         zk.create("/test", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
@@ -50,14 +53,11 @@ public class RepeatStartupTest extends TestCase {
         qb.shutdown(qb.s4);
         qb.shutdown(qb.s5);
         String hp = qb.hostPort.split(",")[0];
-        ZooKeeperServer zks = new ZooKeeperServer(qb.s1.getTxnFactory().getSnapDir(), 
+        ZooKeeperServer zks = new ZooKeeperServer(qb.s1.getTxnFactory().getSnapDir(),
                 qb.s1.getTxnFactory().getDataDir(), 3000);
         final int PORT = Integer.parseInt(hp.split(":")[1]);
-        NIOServerCnxn.Factory factory = null;
-        if (factory == null) {
-            factory = new NIOServerCnxn.Factory(PORT);
-        }
-        
+        NIOServerCnxn.Factory factory = new NIOServerCnxn.Factory(PORT);
+
         factory.startup(zks);
         System.out.println("Comment: starting factory");
         assertTrue("waiting for server up",
@@ -67,6 +67,6 @@ public class RepeatStartupTest extends TestCase {
         assertTrue("waiting for server down",
                    ClientBase.waitForServerDown("127.0.0.1:" + PORT,
                                                 QuorumTest.CONNECTION_TIMEOUT));
-        System.out.println("Comment: shutting doen standalone");
+        System.out.println("Comment: shutting down standalone");
     }
 }

+ 21 - 19
src/java/test/org/apache/zookeeper/test/SessionTest.java

@@ -31,6 +31,7 @@ import junit.framework.TestCase;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
@@ -45,13 +46,14 @@ import org.junit.Test;
 public class SessionTest extends TestCase implements Watcher {
     protected static final Logger LOG = Logger.getLogger(SessionTest.class);
 
-    private static final String HOSTPORT = "127.0.0.1:33299";
+    private static final String HOSTPORT = "127.0.0.1:" +
+            PortAssignment.unique();
     private NIOServerCnxn.Factory serverFactory;
-    
+
     private CountDownLatch startSignal;
 
     File tmpDir;
-    
+
     @Override
     protected void setUp() throws Exception {
         LOG.info("STARTING " + getName());
@@ -62,7 +64,7 @@ public class SessionTest extends TestCase implements Watcher {
 
         ClientBase.setupTestEnv();
         ZooKeeperServer zs = new ZooKeeperServer(tmpDir, tmpDir, 3000);
-        
+
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         serverFactory = new NIOServerCnxn.Factory(PORT);
         serverFactory.startup(zs);
@@ -211,31 +213,31 @@ public class SessionTest extends TestCase implements Watcher {
             CONNECTION_TIMEOUT = oldTimeout;
         }
     }
-    
+
     /**
      * Make sure that we cannot have two connections with the same
      * session id.
-     * 
+     *
      * @throws IOException
      * @throws InterruptedException
      * @throws KeeperException
      */
     @Test
     public void testSessionMove() throws IOException, InterruptedException, KeeperException {
-    	ZooKeeper zk = createClient();
-    	zk.getChildren("/", false);
-    	// This should stomp the zk handle
-    	ZooKeeper zknew = new DisconnectableZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this,
+        ZooKeeper zk = createClient();
+        zk.getChildren("/", false);
+        // This should stomp the zk handle
+        ZooKeeper zknew = new DisconnectableZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this,
                    zk.getSessionId(),
                    zk.getSessionPasswd());
-    	zknew.getChildren("/", false);
-    	try {
-    	    zk.getChildren("/", false);
-    	    fail("Should have lost the connection");
-    	} catch(KeeperException.ConnectionLossException e) {
-    	}
-    	zknew.close();
-    	zk.close();
+        zknew.getChildren("/", false);
+        try {
+            zk.getChildren("/", false);
+            fail("Should have lost the connection");
+        } catch(KeeperException.ConnectionLossException e) {
+        }
+        zknew.close();
+        zk.close();
     }
     @Test
     /**
@@ -269,7 +271,7 @@ public class SessionTest extends TestCase implements Watcher {
 
         zk.close();
     }
-    
+
     private class DupWatcher extends CountdownWatcher {
         public LinkedList<WatchedEvent> states = new LinkedList<WatchedEvent>();
         public void process(WatchedEvent event) {

+ 1 - 1
src/java/test/org/apache/zookeeper/test/SyncCallTest.java

@@ -25,8 +25,8 @@ import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.AsyncCallback.ChildrenCallback;
 import org.apache.zookeeper.AsyncCallback.StringCallback;
 import org.apache.zookeeper.AsyncCallback.VoidCallback;

+ 1 - 1
src/java/test/org/apache/zookeeper/test/TestHammer.java

@@ -20,9 +20,9 @@ package org.apache.zookeeper.test;
 
 import java.io.IOException;
 
+import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.AsyncCallback.VoidCallback;
-import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooDefs.Ids;
 
 public class TestHammer implements VoidCallback {

+ 19 - 15
src/java/test/org/apache/zookeeper/test/UpgradeTest.java

@@ -29,6 +29,7 @@ import junit.framework.TestCase;
 
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
@@ -42,12 +43,12 @@ import org.apache.zookeeper.server.upgrade.UpgradeMain;
 
 public class UpgradeTest extends TestCase implements Watcher {
     private final static Logger LOG = Logger.getLogger(UpgradeTest.class);
-    private static String HOSTPORT = "127.0.0.1:2359";
-    ZooKeeperServer zks;
+
+    private static String HOSTPORT = "127.0.0.1:" + PortAssignment.unique();
     private static final File testData = new File(
             System.getProperty("test.data.dir", "build/test/data"));
     private CountDownLatch startSignal;
-    
+
     @Override
     protected void setUp() throws Exception {
         LOG.info("STARTING " + getName());
@@ -56,7 +57,7 @@ public class UpgradeTest extends TestCase implements Watcher {
     protected void tearDown() throws Exception {
         LOG.info("FINISHED " + getName());
     }
-    
+
     /**
      * test the upgrade
      * @throws Exception
@@ -65,43 +66,46 @@ public class UpgradeTest extends TestCase implements Watcher {
         File upgradeDir = new File(testData, "upgrade");
         UpgradeMain upgrade = new UpgradeMain(upgradeDir, upgradeDir);
         upgrade.runUpgrade();
-        zks = new ZooKeeperServer(upgradeDir, upgradeDir, 3000);
+        ZooKeeperServer zks = new ZooKeeperServer(upgradeDir, upgradeDir, 3000);
         SyncRequestProcessor.setSnapCount(1000);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
         f.startup(zks);
         LOG.info("starting up the zookeeper server .. waiting");
-        assertTrue("waiting for server being up", 
-                ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
+        assertTrue("waiting for server being up",
+                ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
         ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
         Stat stat = zk.exists("/", false);
         List<String> children = zk.getChildren("/", false);
         Collections.sort(children);
-        for (int i=0; i < 10; i++) {
+        for (int i = 0; i < 10; i++) {
             assertTrue("data tree sanity check",
-                    ("test-"+ i).equals(children.get(i)));
+                    ("test-" + i).equals(children.get(i)));
         }
         //try creating one node
-        zk.create("/upgrade","upgrade".getBytes(), Ids.OPEN_ACL_UNSAFE,
+        zk.create("/upgrade", "upgrade".getBytes(), Ids.OPEN_ACL_UNSAFE,
                 CreateMode.PERSISTENT);
         // check if its there
-        if (zk.exists("/upgrade",false) == null) {
+        if (zk.exists("/upgrade", false) == null) {
             assertTrue(false);
         }
+
+        zk.close();
+
         // bring down the server
         f.shutdown();
         assertTrue("waiting for server down",
                    ClientBase.waitForServerDown(HOSTPORT,
                            ClientBase.CONNECTION_TIMEOUT));
-        
+
     }
-    
+
     public void process(WatchedEvent event) {
         LOG.info("Event:" + event.getState() + " " + event.getType() + " " + event.getPath());
         if (event.getState() == KeeperState.SyncConnected
                 && startSignal != null && startSignal.getCount() > 0)
-        {              
-            startSignal.countDown();      
+        {
+            startSignal.countDown();
         }
     }
 }

+ 2 - 2
src/java/test/org/apache/zookeeper/test/WatchedEventTest.java

@@ -21,12 +21,12 @@ package org.apache.zookeeper.test;
 import java.util.EnumSet;
 
 import junit.framework.TestCase;
-import org.junit.Test;
 
+import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
-import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.proto.WatcherEvent;
+import org.junit.Test;
 
 public class WatchedEventTest extends TestCase {
     

+ 3 - 3
src/java/test/org/apache/zookeeper/test/WatcherFuncTest.java

@@ -25,14 +25,14 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.WatchedEvent;
 
 public class WatcherFuncTest extends ClientBase {
     private static class SimpleWatcher implements Watcher {

+ 4 - 4
src/java/test/org/apache/zookeeper/test/WatcherTest.java

@@ -132,7 +132,7 @@ public class WatcherTest extends ClientBase {
     /**
      * This test checks that watches for pending requests do not get triggered,
      * but watches set by previous requests do.
-     * 
+     *
      * @throws Exception
      */
     @Test
@@ -154,14 +154,14 @@ public class WatcherTest extends ClientBase {
        Thread.sleep(50);
        zk2.close();
        stopServer();
-       watches[0].waitForDisconnected(3000);
+       watches[0].waitForDisconnected(60000);
        for(int i = COUNT/2; i < COUNT; i++) {
            watches[i] = new MyWatcher();
            cbs[i] = new MyStatCallback();
            zk.exists("/test", watches[i], cbs[i], count);
        }
        startServer();
-       watches[49].waitForConnected(4000);
+       watches[COUNT/2-1].waitForConnected(60000);
        assertEquals(null, zk.exists("/test", false));
        Thread.sleep(10);
        for(int i = 0; i < COUNT/2; i++) {
@@ -177,7 +177,7 @@ public class WatcherTest extends ClientBase {
        assertEquals(COUNT, count[0]);
        zk.close();
     }
-    
+
     @Test
     public void testWatcherAutoResetWithGlobal() throws Exception {
         ZooKeeper zk = null;

+ 1 - 0
src/java/test/org/apache/zookeeper/test/ZooKeeperQuotaTest.java

@@ -19,6 +19,7 @@
 package org.apache.zookeeper.test;
 
 import java.io.IOException;
+
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;

+ 5 - 8
src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java

@@ -23,19 +23,17 @@ import java.util.List;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 
-import junit.framework.AssertionFailedError;
 import junit.framework.TestCase;
 
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.KeeperException.Code;
+import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.WatchedEvent;
-import org.junit.Test;
 
 public class ZooKeeperTestClient extends TestCase implements Watcher {
   protected String hostPort = "127.0.0.1:22801";
@@ -133,7 +131,7 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
     if (stat == null) {
       fail("node " + nodeName + " should exist");
     }
-    System.out.println("Closing client with sessionid: 0x" 
+    System.out.println("Closing client with sessionid: 0x"
             + Long.toHexString(zk.getSessionId()));
     zk.close();
     zk = new ZooKeeper(hostPort, 10000, this);
@@ -262,7 +260,7 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
 
     event = this.getEvent(10);
     if (event == null) {
-      throw new AssertionFailedError("First event was not delivered promptly");
+      throw new Error("First event was not delivered promptly");
     }
     if (!((event.getType() == EventType.NodeChildrenChanged &&
            event.getPath().equalsIgnoreCase(parentName)) ||
@@ -276,7 +274,7 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
     event = this.getEvent(10);
 
     if (event == null) {
-      throw new AssertionFailedError("Second event was not delivered promptly");
+      throw new Error("Second event was not delivered promptly");
     }
     if (!((event.getType() == EventType.NodeChildrenChanged &&
         event.getPath().equalsIgnoreCase(parentName)) ||
@@ -364,7 +362,6 @@ public class ZooKeeperTestClient extends TestCase implements Watcher {
     zk.close();
   }
 
-  @Test
   public void my_test_1() throws IOException,
           InterruptedException, KeeperException {
     enode_test_1();