You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by br...@apache.org on 2010/11/30 21:16:36 UTC

svn commit: r1040752 - in /zookeeper/trunk: ./ src/java/main/org/apache/zookeeper/ src/java/main/org/apache/zookeeper/client/ src/java/test/org/apache/zookeeper/test/

Author: breed
Date: Tue Nov 30 20:16:35 2010
New Revision: 1040752

URL: http://svn.apache.org/viewvc?rev=1040752&view=rev
Log:
ZOOKEEPER-836. hostlist as string.

Added:
    zookeeper/trunk/src/java/main/org/apache/zookeeper/client/
    zookeeper/trunk/src/java/main/org/apache/zookeeper/client/ConnectStringParser.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/client/HostProvider.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/client/StaticHostProvider.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ConnectStringParserTest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java
Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1040752&r1=1040751&r2=1040752&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Tue Nov 30 20:16:35 2010
@@ -203,6 +203,8 @@ IMPROVEMENTS:
   ZOOKEEPER-908. Remove code duplication and inconsistent naming in
   ClientCnxn.Packet creation (Thomas Koch via phunt)
 
+  ZOOKEEPER-836. hostlist as string. (Thomas Koch via breed)
+
 NEW FEATURES:
   ZOOKEEPER-729. Java client API to recursively delete a subtree.
   (Kay Kay via henry)

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java?rev=1040752&r1=1040751&r2=1040752&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java Tue Nov 30 20:16:35 2010
@@ -21,12 +21,9 @@ package org.apache.zookeeper;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.lang.Thread.UncaughtExceptionHandler;
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
 import java.util.LinkedList;
 import java.util.Random;
 import java.util.Set;
@@ -50,7 +47,7 @@ import org.apache.zookeeper.Watcher.Even
 import org.apache.zookeeper.ZooDefs.OpCode;
 import org.apache.zookeeper.ZooKeeper.States;
 import org.apache.zookeeper.ZooKeeper.WatchRegistration;
-import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.client.HostProvider;
 import org.apache.zookeeper.proto.AuthPacket;
 import org.apache.zookeeper.proto.ConnectRequest;
 import org.apache.zookeeper.proto.CreateResponse;
@@ -93,9 +90,6 @@ public class ClientCnxn {
         }
     }
 
-    private final ArrayList<InetSocketAddress> serverAddrs =
-        new ArrayList<InetSocketAddress>();
-
     static class AuthData {
         AuthData(String scheme, byte data[]) {
             this.scheme = scheme;
@@ -119,8 +113,6 @@ public class ClientCnxn {
      */
     private final LinkedList<Packet> outgoingQueue = new LinkedList<Packet>();
 
-    private int nextAddrToTry = 0;
-
     private int connectTimeout;
 
     /**
@@ -156,6 +148,11 @@ public class ClientCnxn {
      * operation)
      */
     private volatile boolean closing = false;
+    
+    /**
+     * A set of ZooKeeper hosts this client could connect to.
+     */
+    private final HostProvider hostProvider;
 
     public long getSessionId() {
         return sessionId;
@@ -270,8 +267,9 @@ public class ClientCnxn {
      * established until needed. The start() instance method must be called
      * subsequent to construction.
      *
-     * @param hosts
-     *                a comma separated list of hosts that can be connected to.
+     * @param chrootPath - the chroot of this client. Should be removed from this Class in ZOOKEEPER-838
+     * @param hostProvider
+     *                the list of ZooKeeper servers to connect to
      * @param sessionTimeout
      *                the timeout for connections.
      * @param zooKeeper
@@ -281,10 +279,10 @@ public class ClientCnxn {
      *                the socket implementation used (e.g. NIO/Netty)
      * @throws IOException
      */
-    public ClientCnxn(String hosts, int sessionTimeout, ZooKeeper zooKeeper,
+    public ClientCnxn(String chrootPath, HostProvider hostProvider, int sessionTimeout, ZooKeeper zooKeeper,
             ClientWatchManager watcher, ClientCnxnSocket clientCnxnSocket)
             throws IOException {
-        this(hosts, sessionTimeout, zooKeeper, watcher, clientCnxnSocket, 0, new byte[16]);
+        this(chrootPath, hostProvider, sessionTimeout, zooKeeper, watcher, clientCnxnSocket, 0, new byte[16]);
     }
 
     /**
@@ -292,8 +290,9 @@ public class ClientCnxn {
      * established until needed. The start() instance method must be called
      * subsequent to construction.
      *
-     * @param hosts
-     *                a comma separated list of hosts that can be connected to.
+     * @param chrootPath - the chroot of this client. Should be removed from this Class in ZOOKEEPER-838
+     * @param hostProvider
+     *                the list of ZooKeeper servers to connect to
      * @param sessionTimeout
      *                the timeout for connections.
      * @param zooKeeper
@@ -305,50 +304,20 @@ public class ClientCnxn {
      * @param sessionPasswd session passwd if re-establishing session
      * @throws IOException
      */
-    public ClientCnxn(String hosts, int sessionTimeout, ZooKeeper zooKeeper,
+    public ClientCnxn(String chrootPath, HostProvider hostProvider, int sessionTimeout, ZooKeeper zooKeeper,
             ClientWatchManager watcher, ClientCnxnSocket clientCnxnSocket,
-            long sessionId, byte[] sessionPasswd) throws IOException {
+            long sessionId, byte[] sessionPasswd) {
         this.zooKeeper = zooKeeper;
         this.watcher = watcher;
         this.sessionId = sessionId;
         this.sessionPasswd = sessionPasswd;
-
-        // parse out chroot, if any
-        int off = hosts.indexOf('/');
-        if (off >= 0) {
-            String chrootPath = hosts.substring(off);
-            // ignore "/" chroot spec, same as null
-            if (chrootPath.length() == 1) {
-                this.chrootPath = null;
-            } else {
-                PathUtils.validatePath(chrootPath);
-                this.chrootPath = chrootPath;
-            }
-            hosts = hosts.substring(0,  off);
-        } else {
-            this.chrootPath = null;
-        }
-
-        String hostsList[] = hosts.split(",");
-        for (String host : hostsList) {
-            int port = 2181;
-            int pidx = host.lastIndexOf(':');
-            if (pidx >= 0) {
-                // otherwise : is at the end of the string, ignore
-                if (pidx < host.length() - 1) {
-                    port = Integer.parseInt(host.substring(pidx + 1));
-                }
-                host = host.substring(0, pidx);
-            }
-            InetAddress addrs[] = InetAddress.getAllByName(host);
-            for (InetAddress addr : addrs) {
-                serverAddrs.add(new InetSocketAddress(addr, port));
-            }
-        }
         this.sessionTimeout = sessionTimeout;
-        connectTimeout = sessionTimeout / hostsList.length;
+        this.hostProvider = hostProvider;
+        this.chrootPath = chrootPath;
+
+        connectTimeout = sessionTimeout / hostProvider.size();
         readTimeout = sessionTimeout * 2 / 3;
-        Collections.shuffle(serverAddrs);
+
         sendThread = new SendThread(clientCnxnSocket);
         eventThread = new EventThread();
     }
@@ -655,9 +624,8 @@ public class ClientCnxn {
     class SendThread extends Thread {
         private long lastPingSentNs;
         private final ClientCnxnSocket clientCnxnSocket;
-        private int lastConnectIndex = -1;
-        private int currentConnectIndex;
-        private Random r = new Random(System.nanoTime());
+        private Random r = new Random(System.nanoTime());        
+        private boolean isFirstConnect = true;
 
         void readResponse(ByteBuffer incomingBuffer) throws IOException {
             ByteBufferInputStream bbis = new ByteBufferInputStream(
@@ -789,7 +757,7 @@ public class ClientCnxn {
         void primeConnection() throws IOException {
             LOG.info("Socket connection established to "
                     + clientCnxnSocket.getRemoteSocketAddress() + ", initiating session");
-            lastConnectIndex = currentConnectIndex;
+            isFirstConnect = false;
             ConnectRequest conReq = new ConnectRequest(0, lastZxid,
                     sessionTimeout, sessionId, sessionPasswd);
             synchronized (outgoingQueue) {
@@ -834,32 +802,16 @@ public class ClientCnxn {
         }
 
         private void startConnect() throws IOException {
-            if (lastConnectIndex == -1) {
-                // We don't want to delay the first try at a connect, so we
-                // start with -1 the first time around
-                lastConnectIndex = 0;
-            } else {
+            if(!isFirstConnect){
                 try {
                     Thread.sleep(r.nextInt(1000));
                 } catch (InterruptedException e1) {
                     LOG.warn("Unexpected exception", e1);
                 }
-                if (nextAddrToTry == lastConnectIndex) {
-                    try {
-                        // Try not to spin too fast!
-                        Thread.sleep(1000);
-                    } catch (InterruptedException e) {
-                        LOG.warn("Unexpected exception", e);
-                    }
-                }
             }
             state = States.CONNECTING;
-            currentConnectIndex = nextAddrToTry;
-            InetSocketAddress addr = serverAddrs.get(nextAddrToTry);
-            nextAddrToTry++;
-            if (nextAddrToTry == serverAddrs.size()) {
-                nextAddrToTry = 0;
-            }
+            InetSocketAddress addr = hostProvider.next(1000);
+
             LOG.info("Opening socket connection to server " + addr);
 
             setName(getName().replaceAll("\\(.*\\)",
@@ -876,6 +828,7 @@ public class ClientCnxn {
             clientCnxnSocket.introduce(this,sessionId);
             clientCnxnSocket.updateNow();
             clientCnxnSocket.updateLastSendAndHeard();
+            int to;
             while (state.isAlive()) {
                 try {
                     if (!clientCnxnSocket.isConnected()) {
@@ -886,11 +839,13 @@ public class ClientCnxn {
                         startConnect();
                         clientCnxnSocket.updateLastSendAndHeard();
                     }
-
-                    int to = readTimeout - clientCnxnSocket.getIdleRecv();
-                    if (state != States.CONNECTED) {
+                   
+                    if (state == States.CONNECTED) {
+                        to = readTimeout - clientCnxnSocket.getIdleRecv();
+                    } else {
                         to = connectTimeout - clientCnxnSocket.getIdleRecv();
                     }
+                    
                     if (to <= 0) {
                         throw new SessionTimeoutException(
                                 "Client session timed out, have not heard from server in "
@@ -1002,7 +957,8 @@ public class ClientCnxn {
                                 + Long.toHexString(sessionId) + " has expired");
             }
             readTimeout = negotiatedSessionTimeout * 2 / 3;
-            connectTimeout = negotiatedSessionTimeout / serverAddrs.size();
+            connectTimeout = negotiatedSessionTimeout / hostProvider.size();
+            hostProvider.onConnected();
             sessionId = _sessionId;
             sessionPasswd = _sessionPasswd;
             state = States.CONNECTED;

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java?rev=1040752&r1=1040751&r2=1040752&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java Tue Nov 30 20:16:35 2010
@@ -37,6 +37,9 @@ import org.apache.zookeeper.AsyncCallbac
 import org.apache.zookeeper.AsyncCallback.StatCallback;
 import org.apache.zookeeper.AsyncCallback.StringCallback;
 import org.apache.zookeeper.AsyncCallback.VoidCallback;
+import org.apache.zookeeper.client.ConnectStringParser;
+import org.apache.zookeeper.client.HostProvider;
+import org.apache.zookeeper.client.StaticHostProvider;
 import org.apache.zookeeper.common.PathUtils;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
@@ -376,8 +379,14 @@ public class ZooKeeper {
                 + " sessionTimeout=" + sessionTimeout + " watcher=" + watcher);
 
         watchManager.defaultWatcher = watcher;
-        cnxn = new ClientCnxn(connectString, sessionTimeout, this,
-                watchManager, getClientCnxnSocket());
+
+        ConnectStringParser connectStringParser = new ConnectStringParser(
+                connectString);
+        HostProvider hostProvider = new StaticHostProvider(
+                connectStringParser.getServerAddresses());
+        cnxn = new ClientCnxn(connectStringParser.getChrootPath(),
+                hostProvider, sessionTimeout, this, watchManager,
+                getClientCnxnSocket());
         cnxn.start();
     }
 
@@ -431,6 +440,7 @@ public class ZooKeeper {
      *
      * @throws IOException in cases of network failure
      * @throws IllegalArgumentException if an invalid chroot path is specified
+     * @throws IllegalArgumentException for an invalid list of ZooKeeper hosts
      */
     public ZooKeeper(String connectString, int sessionTimeout, Watcher watcher,
             long sessionId, byte[] sessionPasswd)
@@ -444,8 +454,14 @@ public class ZooKeeper {
                 + (sessionPasswd == null ? "<null>" : "<hidden>"));
 
         watchManager.defaultWatcher = watcher;
-        cnxn = new ClientCnxn(connectString, sessionTimeout, this,
-                watchManager, getClientCnxnSocket(), sessionId, sessionPasswd);
+
+        ConnectStringParser connectStringParser = new ConnectStringParser(
+                connectString);
+        HostProvider hostProvider = new StaticHostProvider(
+                connectStringParser.getServerAddresses());
+        cnxn = new ClientCnxn(connectStringParser.getChrootPath(),
+                hostProvider, sessionTimeout, this, watchManager,
+                getClientCnxnSocket(), sessionId, sessionPasswd);
         cnxn.start();
     }
 

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/client/ConnectStringParser.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/client/ConnectStringParser.java?rev=1040752&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/client/ConnectStringParser.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/client/ConnectStringParser.java Tue Nov 30 20:16:35 2010
@@ -0,0 +1,87 @@
+/**
+ * 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.client;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+
+import org.apache.zookeeper.common.PathUtils;
+
+/**
+ * A parser for ZooKeeper Client connect strings.
+ * 
+ * This class is not meant to be seen or used outside of ZooKeeper itself.
+ * 
+ * The chrootPath member should be replaced by a Path object in issue
+ * ZOOKEEPER-849.
+ * 
+ * @see org.apache.zookeeper.ZooKeeper
+ */
+public final class ConnectStringParser {
+    private static final int DEFAULT_PORT = 2181;
+
+    private final String chrootPath;
+
+    private final ArrayList<InetSocketAddress> serverAddresses = new ArrayList<InetSocketAddress>();
+
+    /**
+     * 
+     * @throws IllegalArgumentException
+     *             for an invalid chroot path.
+     */
+    public ConnectStringParser(String connectString) {
+        // parse out chroot, if any
+        int off = connectString.indexOf('/');
+        if (off >= 0) {
+            String chrootPath = connectString.substring(off);
+            // ignore "/" chroot spec, same as null
+            if (chrootPath.length() == 1) {
+                this.chrootPath = null;
+            } else {
+                PathUtils.validatePath(chrootPath);
+                this.chrootPath = chrootPath;
+            }
+            connectString = connectString.substring(0, off);
+        } else {
+            this.chrootPath = null;
+        }
+
+        String hostsList[] = connectString.split(",");
+        for (String host : hostsList) {
+            int port = DEFAULT_PORT;
+            int pidx = host.lastIndexOf(':');
+            if (pidx >= 0) {
+                // otherwise : is at the end of the string, ignore
+                if (pidx < host.length() - 1) {
+                    port = Integer.parseInt(host.substring(pidx + 1));
+                }
+                host = host.substring(0, pidx);
+            }
+            serverAddresses.add(InetSocketAddress.createUnresolved(host, port));
+        }
+    }
+
+    public String getChrootPath() {
+        return chrootPath;
+    }
+
+    public ArrayList<InetSocketAddress> getServerAddresses() {
+        return serverAddresses;
+    }
+}
\ No newline at end of file

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/client/HostProvider.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/client/HostProvider.java?rev=1040752&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/client/HostProvider.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/client/HostProvider.java Tue Nov 30 20:16:35 2010
@@ -0,0 +1,61 @@
+/**
+ * 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.client;
+
+import java.net.InetSocketAddress;
+
+/**
+ * A set of hosts a ZooKeeper client should connect to.
+ * 
+ * Classes implementing this interface must guarantee the following:
+ * 
+ * * Every call to next() returns an InetSocketAddress. So the iterator never
+ * ends.
+ * 
+ * * The size() of a HostProvider may never be zero.
+ * 
+ * A HostProvider must return resolved InetSocketAddress instances on next(),
+ * but it's up to the HostProvider, when it wants to do the resolving.
+ * 
+ * Different HostProvider could be imagined:
+ * 
+ * * A HostProvider that loads the list of Hosts from an URL or from DNS 
+ * * A HostProvider that re-resolves the InetSocketAddress after a timeout. 
+ * * A HostProvider that prefers nearby hosts.
+ */
+public interface HostProvider {
+    public int size();
+
+    /**
+     * The next host to try to connect to.
+     * 
+     * For a spinDelay of 0 there should be no wait.
+     * 
+     * @param spinDelay
+     *            Milliseconds to wait if all hosts have been tried once.
+     */
+    public InetSocketAddress next(long spinDelay);
+
+    /**
+     * Notify the HostProvider of a successful connection.
+     * 
+     * The HostProvider may use this notification to reset it's inner state.
+     */
+    public void onConnected();
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/client/StaticHostProvider.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/client/StaticHostProvider.java?rev=1040752&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/client/StaticHostProvider.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/client/StaticHostProvider.java Tue Nov 30 20:16:35 2010
@@ -0,0 +1,98 @@
+/**
+ * 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.client;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.log4j.Logger;
+
+/**
+ * Most simple HostProvider, resolves only on instantiation.
+ * 
+ */
+public final class StaticHostProvider implements HostProvider {
+    private static final Logger LOG = Logger
+            .getLogger(StaticHostProvider.class);
+    private final List<InetSocketAddress> serverAddresses = new ArrayList<InetSocketAddress>(
+            5);
+
+    private int lastIndex = -1;
+
+    private int currentIndex = -1;
+
+    /**
+     * Constructs a SimpleHostSet.
+     * 
+     * @param serverAddresses
+     *            possibly unresolved ZooKeeper server addresses
+     * @throws UnknownHostException
+     * @throws IllegalArgumentException
+     *             if serverAddresses is empty or resolves to an empty list
+     */
+    public StaticHostProvider(Collection<InetSocketAddress> serverAddresses)
+            throws UnknownHostException {
+        for (InetSocketAddress address : serverAddresses) {
+            InetAddress resolvedAddresses[] = InetAddress.getAllByName(address
+                    .getHostName());
+            for (InetAddress resolvedAddress : resolvedAddresses) {
+                this.serverAddresses.add(new InetSocketAddress(resolvedAddress
+                        .getHostAddress(), address.getPort()));
+            }
+        }
+
+        if (this.serverAddresses.isEmpty()) {
+            throw new IllegalArgumentException(
+                    "A HostProvider may not be empty!");
+        }
+        Collections.shuffle(this.serverAddresses);
+    }
+
+    public int size() {
+        return serverAddresses.size();
+    }
+
+    public InetSocketAddress next(long spinDelay) {
+        ++currentIndex;
+        if (currentIndex == serverAddresses.size()) {
+            currentIndex = 0;
+        }
+        if (currentIndex == lastIndex && spinDelay > 0) {
+            try {
+                Thread.sleep(spinDelay);
+            } catch (InterruptedException e) {
+                LOG.warn("Unexpected exception", e);
+            }
+        } else if (lastIndex == -1) {
+            // We don't want to sleep on the first ever connect attempt.
+            lastIndex = 0;
+        }
+
+        return serverAddresses.get(currentIndex);
+    }
+
+    public void onConnected() {
+        lastIndex = currentIndex;
+    }
+}
\ No newline at end of file

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ConnectStringParserTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ConnectStringParserTest.java?rev=1040752&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ConnectStringParserTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ConnectStringParserTest.java Tue Nov 30 20:16:35 2010
@@ -0,0 +1,68 @@
+/**
+ * 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.test;
+
+import org.apache.zookeeper.ZKTestCase;
+import org.apache.zookeeper.client.ConnectStringParser;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ConnectStringParserTest extends ZKTestCase{
+
+    @Test
+    public void testSingleServerChrootPath(){
+        String chrootPath = "/hallo/welt";
+        String servers = "10.10.10.1";
+        assertChrootPath(chrootPath,
+                new ConnectStringParser(servers+chrootPath));
+    }
+
+    @Test
+    public void testMultipleServersChrootPath(){
+        String chrootPath = "/hallo/welt";
+        String servers = "10.10.10.1,10.10.10.2";
+        assertChrootPath(chrootPath,
+                new ConnectStringParser(servers+chrootPath));
+    }
+
+    @Test
+    public void testParseServersWithoutPort(){
+        String servers = "10.10.10.1,10.10.10.2";
+        ConnectStringParser parser = new ConnectStringParser(servers);
+
+        Assert.assertEquals("10.10.10.1", parser.getServerAddresses().get(0).getHostName());
+        Assert.assertEquals("10.10.10.2", parser.getServerAddresses().get(1).getHostName());
+    }
+
+    @Test
+    public void testParseServersWithPort(){
+        String servers = "10.10.10.1:112,10.10.10.2:110";
+        ConnectStringParser parser = new ConnectStringParser(servers);
+
+        Assert.assertEquals("10.10.10.1", parser.getServerAddresses().get(0).getHostName());
+        Assert.assertEquals("10.10.10.2", parser.getServerAddresses().get(1).getHostName());
+
+        Assert.assertEquals(112, parser.getServerAddresses().get(0).getPort());
+        Assert.assertEquals(110, parser.getServerAddresses().get(1).getPort());
+    }
+
+    private void assertChrootPath(String expected, ConnectStringParser parser){
+        Assert.assertEquals(expected, parser.getChrootPath());
+    }
+}

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java?rev=1040752&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/StaticHostProviderTest.java Tue Nov 30 20:16:35 2010
@@ -0,0 +1,98 @@
+/**
+ * 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.test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.zookeeper.ZKTestCase;
+import org.apache.zookeeper.client.HostProvider;
+import org.apache.zookeeper.client.StaticHostProvider;
+import org.junit.Test;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+
+public class StaticHostProviderTest extends ZKTestCase {
+
+    @Test
+    public void testNextGoesRound() throws UnknownHostException {
+        HostProvider hostProvider = getHostProvider(2);
+        InetSocketAddress first = hostProvider.next(0);
+        assertTrue(first instanceof InetSocketAddress);
+        hostProvider.next(0);
+        assertEquals(first, hostProvider.next(0));
+    }
+
+    @Test
+    public void testNextGoesRoundAndSleeps() throws UnknownHostException {
+        int size = 2;
+        HostProvider hostProvider = getHostProvider(size);
+        while (size > 0) {
+            hostProvider.next(0);
+            --size;
+        }
+        long start = System.currentTimeMillis();
+        hostProvider.next(1000);
+        long stop = System.currentTimeMillis();
+        assertTrue(900 <= stop - start);
+    }
+
+    @Test
+    public void testNextDoesNotSleepForZero() throws UnknownHostException {
+        int size = 2;
+        HostProvider hostProvider = getHostProvider(size);
+        while (size > 0) {
+            hostProvider.next(0);
+            --size;
+        }
+        long start = System.currentTimeMillis();
+        hostProvider.next(0);
+        long stop = System.currentTimeMillis();
+        assertTrue(10000 > stop - start);
+    }
+
+    @Test
+    public void testTwoConsequitiveCallsToNextReturnDifferentElement()
+            throws UnknownHostException {
+        HostProvider hostProvider = getHostProvider(2);
+        assertNotSame(hostProvider.next(0), hostProvider.next(0));
+    }
+
+    @Test
+    public void testOnConnectDoesNotReset() throws UnknownHostException {
+        HostProvider hostProvider = getHostProvider(2);
+        InetSocketAddress first = hostProvider.next(0);
+        hostProvider.onConnected();
+        InetSocketAddress second = hostProvider.next(0);
+        assertNotSame(first, second);
+    }
+
+    private StaticHostProvider getHostProvider(int size)
+            throws UnknownHostException {
+        ArrayList<InetSocketAddress> list = new ArrayList<InetSocketAddress>(
+                size);
+        while (size > 0) {
+            list.add(new InetSocketAddress("10.10.10." + size, 1234));
+            --size;
+        }
+        return new StaticHostProvider(list);
+    }
+}