You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by an...@apache.org on 2018/07/27 10:32:51 UTC

zookeeper git commit: ZOOKEEPER-3057: Fix IPv6 literal usage

Repository: zookeeper
Updated Branches:
  refs/heads/master 7b7c65877 -> ba8932dcc


ZOOKEEPER-3057: Fix IPv6 literal usage

This patch contains fixes for IPv6 literal usage and corresponding unit test changes.

As per discussion in ZOOKEEPER-3057 - The issue/problem is the same as ZOOKEEPER-2989, but we changed the code to pass IPv6 literal [%s]:%s, also we changed the logging and the LocalPeerBean to show this IPv6 literal as well, which makes it easier to check when using Ipv6 and we added detailed tests for this change, sending out for review to see if it's better or not.

ZKPatch: 88e94e6f3665353446bf70a042c8f0cd50834f7c (extract)

Author: Mohamed Jeelani <mj...@fb.com>
Author: Jeelani Mohamed Abdul Khader <mj...@devvm3360.prn2.facebook.com>

Reviewers: breed@apache.org, andor@apache.org

Closes #548 from mjeelanimsft/fix-ipv6-literal and squashes the following commits:

69c60b74 [Mohamed Jeelani] Fix method rename from splitServerConfig(s) to getHostAndPort(s)
47f7a854 [Mohamed Jeelani] Incorporating review feedback from @maoling
ff470498 [Mohamed Jeelani] Incorporating Review Feedback - Adding Unit Tests
f932e16e [Mohamed Jeelani] Added Unit Test and JavaDoc for splitServerConfig()
43e19e93 [Mohamed Jeelani] Incorporating code review comments
33f695f2 [Jeelani Mohamed Abdul Khader] Fix IPv6 literal usage


Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/ba8932dc
Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/ba8932dc
Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/ba8932dc

Branch: refs/heads/master
Commit: ba8932dccb227b5b52de98e33c46054014f951b7
Parents: 7b7c658
Author: Mohamed Jeelani <mj...@fb.com>
Authored: Fri Jul 27 12:32:45 2018 +0200
Committer: Andor Molnar <an...@apache.org>
Committed: Fri Jul 27 12:32:45 2018 +0200

----------------------------------------------------------------------
 .../org/apache/zookeeper/common/NetUtils.java   | 43 ++++++++++++
 .../zookeeper/server/quorum/LocalPeerBean.java  | 10 ++-
 .../server/quorum/QuorumCnxManager.java         | 21 ++++--
 .../zookeeper/server/quorum/QuorumPeer.java     | 36 +++-------
 .../server/quorum/QuorumPeerConfig.java         | 11 +--
 .../zookeeper/server/util/ConfigUtils.java      | 27 ++++++++
 .../apache/zookeeper/common/NetUtilsTest.java   | 55 +++++++++++++++
 .../server/quorum/LocalPeerBeanTest.java        |  2 +-
 .../server/quorum/QuorumPeerMainTest.java       | 70 +++++++++++++-------
 .../zookeeper/server/util/ConfigUtilsTest.java  | 52 +++++++++++++++
 10 files changed, 258 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/zookeeper/blob/ba8932dc/src/java/main/org/apache/zookeeper/common/NetUtils.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/common/NetUtils.java b/src/java/main/org/apache/zookeeper/common/NetUtils.java
new file mode 100644
index 0000000..4779003
--- /dev/null
+++ b/src/java/main/org/apache/zookeeper/common/NetUtils.java
@@ -0,0 +1,43 @@
+/**
+ * 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.common;
+
+import java.net.Inet6Address;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+
+/**
+ * This class contains common utilities for netstuff. Like printing IPv6 literals correctly
+ */
+public class NetUtils {
+
+    public static String formatInetAddr(InetSocketAddress addr) {
+        InetAddress ia = addr.getAddress();
+
+        if (ia == null) {
+            return String.format("%s:%s", addr.getHostString(), addr.getPort());
+        }
+
+        if (ia instanceof Inet6Address) {
+            return String.format("[%s]:%s", ia.getHostAddress(), addr.getPort());
+        } else {
+            return String.format("%s:%s", ia.getHostAddress(), addr.getPort());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/ba8932dc/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java b/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java
index 91d779b..5bbd80e 100644
--- a/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java
+++ b/src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java
@@ -18,7 +18,7 @@
 
 package org.apache.zookeeper.server.quorum;
 
-
+import static org.apache.zookeeper.common.NetUtils.formatInetAddr;
 
 /**
  * Implementation of the local peer MBean interface.
@@ -71,7 +71,7 @@ public class LocalPeerBean extends ServerBean implements LocalPeerMXBean {
     }
     
     public String getQuorumAddress() {
-        return peer.getQuorumAddress().toString();
+        return formatInetAddr(peer.getQuorumAddress());
     }
     
     public int getElectionType() {
@@ -79,14 +79,12 @@ public class LocalPeerBean extends ServerBean implements LocalPeerMXBean {
     }
 
     public String getElectionAddress() {
-        return peer.getElectionAddress().getHostString() + ":" +
-            peer.getElectionAddress().getPort();
+        return formatInetAddr(peer.getElectionAddress());
     }
 
     public String getClientAddress() {
         if (null != peer.cnxnFactory) {
-            return String.format("%s:%d", peer.cnxnFactory.getLocalAddress()
-                    .getHostString(), peer.getClientPort());
+            return formatInetAddr(peer.cnxnFactory.getLocalAddress());
         } else {
             return "";
         }

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/ba8932dc/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
index 705b846..645cfa6 100644
--- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
+++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
@@ -47,6 +47,8 @@ import java.util.NoSuchElementException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
+import org.apache.zookeeper.server.util.ConfigUtils;
 import org.apache.zookeeper.server.ZooKeeperThread;
 import org.apache.zookeeper.server.quorum.auth.QuorumAuthLearner;
 import org.apache.zookeeper.server.quorum.auth.QuorumAuthServer;
@@ -54,6 +56,8 @@ import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.zookeeper.common.NetUtils.formatInetAddr;
+
 /**
  * This class implements a connection manager for leader election using TCP. It
  * maintains one connection for every pair of servers. The tricky part is to
@@ -223,10 +227,13 @@ public class QuorumCnxManager {
                         num_read, remaining, sid);
             }
 
-            // FIXME: IPv6 is not supported. Using something like Guava's HostAndPort
-            //        parser would be good.
             String addr = new String(b);
-            String[] host_port = addr.split(":");
+            String[] host_port;
+            try {
+                host_port = ConfigUtils.getHostAndPort(addr);
+            } catch (ConfigException e) {
+                throw new InitialMessageException("Badly formed address: %s", addr);
+            }
 
             if (host_port.length != 2) {
                 throw new InitialMessageException("Badly formed address: %s", addr);
@@ -237,6 +244,8 @@ public class QuorumCnxManager {
                 port = Integer.parseInt(host_port[1]);
             } catch (NumberFormatException e) {
                 throw new InitialMessageException("Bad port number: %s", host_port[1]);
+            } catch (ArrayIndexOutOfBoundsException e) {
+                throw new InitialMessageException("No port number in: %s", addr);
             }
 
             return new InitialMessage(sid, new InetSocketAddress(host_port[0], port));
@@ -404,7 +413,7 @@ public class QuorumCnxManager {
             // represents protocol version (in other words - message type)
             dout.writeLong(PROTOCOL_VERSION);
             dout.writeLong(self.getId());
-            String addr = self.getElectionAddress().getHostString() + ":" + self.getElectionAddress().getPort();
+            String addr = formatInetAddr(self.getElectionAddress());
             byte[] addr_bytes = addr.getBytes();
             dout.writeInt(addr_bytes.length);
             dout.write(addr_bytes);
@@ -865,7 +874,7 @@ public class QuorumCnxManager {
                             client = ss.accept();
                             setSockOpts(client);
                             LOG.info("Received connection request "
-                                     + client.getRemoteSocketAddress());
+                                     + formatInetAddr((InetSocketAddress)client.getRemoteSocketAddress()));
                             // Receive and handle the connection request
                             // asynchronously if the quorum sasl authentication is
                             // enabled. This is required because sasl server
@@ -907,7 +916,7 @@ public class QuorumCnxManager {
                 LOG.error("As I'm leaving the listener thread, "
                         + "I won't be able to participate in leader "
                         + "election any longer: "
-                        + self.getElectionAddress());
+                        + formatInetAddr(self.getElectionAddress()));
                 if (exitException instanceof BindException) {
                     // After leaving listener thread, the host cannot join the
                     // quorum anymore, this is a severe error that we cannot

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/ba8932dc/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java
index 61ed573..1a5acdd 100644
--- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java
+++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java
@@ -68,10 +68,13 @@ import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
 import org.apache.zookeeper.server.quorum.flexible.QuorumMaj;
 import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
+import org.apache.zookeeper.server.util.ConfigUtils;
 import org.apache.zookeeper.server.util.ZxidUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.zookeeper.common.NetUtils.formatInetAddr;
+
 /**
  * This class manages the quorum protocol. There are three states this server
  * can be in:
@@ -193,27 +196,6 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
             }
         }
 
-        private static String[] splitWithLeadingHostname(String s)
-                throws ConfigException
-        {
-            /* Does it start with an IPv6 literal? */
-            if (s.startsWith("[")) {
-                int i = s.indexOf("]:");
-                if (i < 0) {
-                    throw new ConfigException(s + " starts with '[' but has no matching ']:'");
-                }
-
-                String[] sa = s.substring(i + 2).split(":");
-                String[] nsa = new String[sa.length + 1];
-                nsa[0] = s.substring(1, i);
-                System.arraycopy(sa, 0, nsa, 1, sa.length);
-
-                return nsa;
-            } else {
-                return s.split(":");
-            }
-        }
-
         private static final String wrongFormat = " does not have the form server_config or server_config;client_config"+
         " where server_config is host:port:port or host:port:port:type and client_config is port or host:port";
 
@@ -221,7 +203,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
             // LOG.warn("sid = " + sid + " addressStr = " + addressStr);
             this.id = sid;
             String serverClientParts[] = addressStr.split(";");
-            String serverParts[] = splitWithLeadingHostname(serverClientParts[0]);
+            String serverParts[] = ConfigUtils.getHostAndPort(serverClientParts[0]);
             if ((serverClientParts.length > 2) || (serverParts.length < 3)
                     || (serverParts.length > 4)) {
                 throw new ConfigException(addressStr + wrongFormat);
@@ -229,7 +211,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
 
             if (serverClientParts.length == 2) {
                 //LOG.warn("ClientParts: " + serverClientParts[1]);
-                String clientParts[] = splitWithLeadingHostname(serverClientParts[1]);
+                String clientParts[] = ConfigUtils.getHostAndPort(serverClientParts[1]);
                 if (clientParts.length > 2) {
                     throw new ConfigException(addressStr + wrongFormat);
                 }
@@ -1354,14 +1336,14 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
             if (leader != null) {
                 for (LearnerHandler fh : leader.getLearners()) {
                     if (fh.getSocket() != null) {
-                        String s = fh.getSocket().getRemoteSocketAddress().toString();
+                        String s = formatInetAddr((InetSocketAddress)fh.getSocket().getRemoteSocketAddress());
                         if (leader.isLearnerSynced(fh))
                             s += "*";
                         l.add(s);
                     }
                 }
             } else if (follower != null) {
-                l.add(follower.sock.getRemoteSocketAddress().toString());
+                l.add(formatInetAddr((InetSocketAddress)follower.sock.getRemoteSocketAddress()));
             }
         }
         return l.toArray(new String[0]);
@@ -1992,8 +1974,8 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
     private void updateThreadName() {
         String plain = cnxnFactory != null ?
                 cnxnFactory.getLocalAddress() != null ?
-                        cnxnFactory.getLocalAddress().toString() : "disabled" : "disabled";
-        String secure = secureCnxnFactory != null ? secureCnxnFactory.getLocalAddress().toString() : "disabled";
+                        formatInetAddr(cnxnFactory.getLocalAddress()) : "disabled" : "disabled";
+        String secure = secureCnxnFactory != null ? formatInetAddr(secureCnxnFactory.getLocalAddress()) : "disabled";
         setName(String.format("QuorumPeer[myid=%d](plain=%s)(secure=%s)", getId(), plain, secure));
     }
 

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/ba8932dc/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
index 7159139..ab7b9f6 100644
--- a/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
+++ b/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
@@ -54,6 +54,9 @@ import org.apache.zookeeper.server.quorum.flexible.QuorumHierarchical;
 import org.apache.zookeeper.server.quorum.flexible.QuorumMaj;
 import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
 import org.apache.zookeeper.server.util.VerifyingFileFactory;
+import org.apache.zookeeper.server.util.ConfigUtils;
+
+import static org.apache.zookeeper.common.NetUtils.formatInetAddr;
 
 @InterfaceAudience.Public
 public class QuorumPeerConfig {
@@ -373,10 +376,10 @@ public class QuorumPeerConfig {
         } else if (clientPortAddress != null) {
             this.clientPortAddress = new InetSocketAddress(
                     InetAddress.getByName(clientPortAddress), clientPort);
-            LOG.info("clientPortAddress is {}", this.clientPortAddress.toString());
+            LOG.info("clientPortAddress is {}", formatInetAddr(this.clientPortAddress));
         } else {
             this.clientPortAddress = new InetSocketAddress(clientPort);
-            LOG.info("clientPortAddress is {}", this.clientPortAddress.toString());
+            LOG.info("clientPortAddress is {}", formatInetAddr(this.clientPortAddress));
         }
 
         if (secureClientPort == 0) {
@@ -387,10 +390,10 @@ public class QuorumPeerConfig {
         } else if (secureClientPortAddress != null) {
             this.secureClientPortAddress = new InetSocketAddress(
                     InetAddress.getByName(secureClientPortAddress), secureClientPort);
-            LOG.info("secureClientPortAddress is {}", this.secureClientPortAddress.toString());
+            LOG.info("secureClientPortAddress is {}", formatInetAddr(this.secureClientPortAddress));
         } else {
             this.secureClientPortAddress = new InetSocketAddress(secureClientPort);
-            LOG.info("secureClientPortAddress is {}", this.secureClientPortAddress.toString());
+            LOG.info("secureClientPortAddress is {}", formatInetAddr(this.secureClientPortAddress));
         }
         if (this.secureClientPortAddress != null) {
             configureSSLAuth();

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/ba8932dc/src/java/main/org/apache/zookeeper/server/util/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/util/ConfigUtils.java b/src/java/main/org/apache/zookeeper/server/util/ConfigUtils.java
index 1ca37d1..18b2891 100644
--- a/src/java/main/org/apache/zookeeper/server/util/ConfigUtils.java
+++ b/src/java/main/org/apache/zookeeper/server/util/ConfigUtils.java
@@ -60,4 +60,31 @@ public class ConfigUtils {
         }
         return version + " " + sb.toString();
     }
+
+    /**
+     * Gets host and port by spliting server config
+     * with support for IPv6 literals
+     * @return String[] first element being the
+     *  IP address and the next being the port
+     * @param s server config, server:port
+     */
+    public static String[] getHostAndPort(String s)
+        throws ConfigException
+    {
+        if (s.startsWith("[")) {
+            int i = s.indexOf("]:");
+            if (i < 0) {
+                throw new ConfigException(s + " starts with '[' but has no matching ']:'");
+            }
+
+            String[] sa = s.substring(i + 2).split(":");
+            String[] nsa = new String[sa.length + 1];
+            nsa[0] = s.substring(1, i);
+            System.arraycopy(sa, 0, nsa, 1, sa.length);
+
+            return nsa;
+        } else {
+            return s.split(":");
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/ba8932dc/src/java/test/org/apache/zookeeper/common/NetUtilsTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/common/NetUtilsTest.java b/src/java/test/org/apache/zookeeper/common/NetUtilsTest.java
new file mode 100644
index 0000000..969777f
--- /dev/null
+++ b/src/java/test/org/apache/zookeeper/common/NetUtilsTest.java
@@ -0,0 +1,55 @@
+package org.apache.zookeeper.common;
+
+import org.apache.zookeeper.common.NetUtils;
+import org.apache.zookeeper.ZKTestCase;
+import org.hamcrest.core.AnyOf;
+import org.hamcrest.core.IsEqual;
+import org.junit.Assert;
+import org.junit.Test;
+import java.net.InetSocketAddress;
+
+public class NetUtilsTest extends ZKTestCase {
+
+    private Integer port = 1234;
+    private String v4addr = "127.0.0.1";
+    private String v6addr = "[0:0:0:0:0:0:0:1]";
+    private String v6addr2 = "[2600:0:0:0:0:0:0:0]";
+    private String v4local = v4addr + ":" + port.toString();
+    private String v6local = v6addr + ":" + port.toString();
+    private String v6ext = v6addr2 + ":" + port.toString();
+
+    @Test
+    public void testFormatInetAddrGoodIpv4() {
+        InetSocketAddress isa = new InetSocketAddress(v4addr, port);
+        Assert.assertEquals("127.0.0.1:1234", NetUtils.formatInetAddr(isa));
+    }
+
+    @Test
+    public void testFormatInetAddrGoodIpv6Local() {
+        // Have to use the expanded address here, hence not using v6addr in instantiation
+        InetSocketAddress isa = new InetSocketAddress("::1", port);
+        Assert.assertEquals(v6local, NetUtils.formatInetAddr(isa));
+    }
+
+    @Test
+    public void testFormatInetAddrGoodIpv6Ext() {
+        // Have to use the expanded address here, hence not using v6addr in instantiation
+        InetSocketAddress isa = new InetSocketAddress("2600::", port);
+        Assert.assertEquals(v6ext, NetUtils.formatInetAddr(isa));
+    }
+
+    @Test
+    public void testFormatInetAddrGoodHostname() {
+        InetSocketAddress isa = new InetSocketAddress("localhost", 1234);
+
+        Assert.assertThat(NetUtils.formatInetAddr(isa),
+            AnyOf.anyOf(IsEqual.equalTo(v4local), IsEqual.equalTo(v6local)
+        ));
+    }
+
+    @Test
+    public void testFormatAddrUnresolved() {
+        InetSocketAddress isa = InetSocketAddress.createUnresolved("doesnt.exist.com", 1234);
+        Assert.assertEquals("doesnt.exist.com:1234", NetUtils.formatInetAddr(isa));
+    }
+}

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/ba8932dc/src/java/test/org/apache/zookeeper/server/quorum/LocalPeerBeanTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/server/quorum/LocalPeerBeanTest.java b/src/java/test/org/apache/zookeeper/server/quorum/LocalPeerBeanTest.java
index 563a501..9b4a9c1 100644
--- a/src/java/test/org/apache/zookeeper/server/quorum/LocalPeerBeanTest.java
+++ b/src/java/test/org/apache/zookeeper/server/quorum/LocalPeerBeanTest.java
@@ -60,7 +60,7 @@ public class LocalPeerBeanTest {
 
         result = remotePeerBean.getClientAddress();
         String ipv4 = "0.0.0.0:" + clientPort;
-        String ipv6 = "0:0:0:0:0:0:0:0:" + clientPort;
+        String ipv6 = "[0:0:0:0:0:0:0:0]:" + clientPort;
         assertTrue(result.equals(ipv4) || result.equals(ipv6));
         // cleanup
         cnxnFactory.shutdown();

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/ba8932dc/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
index 0356ed2..dff7cfc 100644
--- a/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
+++ b/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
@@ -85,18 +85,22 @@ public class QuorumPeerMainTest extends QuorumPeerTestBase {
     /**
      * Verify the ability to start a cluster.
      */
-    @Test
-    public void testQuorum() throws Exception {
+    public void testQuorumInternal(String addr) throws Exception {
         ClientBase.setupTestEnv();
 
         final int CLIENT_PORT_QP1 = PortAssignment.unique();
         final int CLIENT_PORT_QP2 = PortAssignment.unique();
 
-        String quorumCfgSection =
-            "server.1=127.0.0.1:" + PortAssignment.unique()
-            + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP1
-            + "\nserver.2=127.0.0.1:" + PortAssignment.unique() 
-            + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP2;
+        String quorumCfgSection = String.format("server.1=%1$s:%2$s:%3$s;%4$s",
+                addr,
+                PortAssignment.unique(),
+                PortAssignment.unique(),
+                CLIENT_PORT_QP1) + "\n" +
+            String.format("server.2=%1$s:%2$s:%3$s;%4$s",
+                    addr,
+                    PortAssignment.unique(),
+                    PortAssignment.unique(),
+                    CLIENT_PORT_QP2);
 
         MainThread q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection);
         MainThread q2 = new MainThread(2, CLIENT_PORT_QP2, quorumCfgSection);
@@ -104,34 +108,34 @@ public class QuorumPeerMainTest extends QuorumPeerTestBase {
         q2.start();
 
         Assert.assertTrue("waiting for server 1 being up",
-                        ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1,
-                        CONNECTION_TIMEOUT));
+            ClientBase.waitForServerUp(addr + ":" + CLIENT_PORT_QP1,
+                CONNECTION_TIMEOUT));
         Assert.assertTrue("waiting for server 2 being up",
-                        ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2,
-                        CONNECTION_TIMEOUT));
+            ClientBase.waitForServerUp(addr + ":" + CLIENT_PORT_QP2,
+                CONNECTION_TIMEOUT));
         QuorumPeer quorumPeer = q1.main.quorumPeer;
 
         int tickTime = quorumPeer.getTickTime();
         Assert.assertEquals(
-                "Default value of minimumSessionTimeOut is not considered",
-                tickTime * 2, quorumPeer.getMinSessionTimeout());
+            "Default value of minimumSessionTimeOut is not considered",
+            tickTime * 2, quorumPeer.getMinSessionTimeout());
         Assert.assertEquals(
-                "Default value of maximumSessionTimeOut is not considered",
-                tickTime * 20, quorumPeer.getMaxSessionTimeout());
+            "Default value of maximumSessionTimeOut is not considered",
+            tickTime * 20, quorumPeer.getMaxSessionTimeout());
 
-        ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT_QP1,
-                ClientBase.CONNECTION_TIMEOUT, this);
+        ZooKeeper zk = new ZooKeeper(addr + ":" + CLIENT_PORT_QP1,
+            ClientBase.CONNECTION_TIMEOUT, this);
         waitForOne(zk, States.CONNECTED);
         zk.create("/foo_q1", "foobar1".getBytes(), Ids.OPEN_ACL_UNSAFE,
-                CreateMode.PERSISTENT);
+            CreateMode.PERSISTENT);
         Assert.assertEquals(new String(zk.getData("/foo_q1", null, null)), "foobar1");
         zk.close();
 
-        zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT_QP2,
-                ClientBase.CONNECTION_TIMEOUT, this);
+        zk = new ZooKeeper(addr + ":" + CLIENT_PORT_QP2,
+            ClientBase.CONNECTION_TIMEOUT, this);
         waitForOne(zk, States.CONNECTED);
         zk.create("/foo_q2", "foobar2".getBytes(), Ids.OPEN_ACL_UNSAFE,
-                CreateMode.PERSISTENT);
+            CreateMode.PERSISTENT);
         Assert.assertEquals(new String(zk.getData("/foo_q2", null, null)), "foobar2");
         zk.close();
 
@@ -139,11 +143,27 @@ public class QuorumPeerMainTest extends QuorumPeerTestBase {
         q2.shutdown();
 
         Assert.assertTrue("waiting for server 1 down",
-                ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP1,
-                        ClientBase.CONNECTION_TIMEOUT));
+            ClientBase.waitForServerDown(addr + ":" + CLIENT_PORT_QP1,
+                ClientBase.CONNECTION_TIMEOUT));
         Assert.assertTrue("waiting for server 2 down",
-                ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP2,
-                        ClientBase.CONNECTION_TIMEOUT));
+            ClientBase.waitForServerDown(addr + ":" + CLIENT_PORT_QP2,
+                ClientBase.CONNECTION_TIMEOUT));
+    }
+
+    /**
+     * Verify the ability to start a cluster.
+     */
+    @Test
+    public void testQuorum() throws Exception {
+        testQuorumInternal("127.0.0.1");
+    }
+
+    /**
+     * Verify the ability to start a cluster. IN V6!!!!
+     */
+    @Test
+    public void testQuorumV6() throws Exception {
+        testQuorumInternal("[::1]");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/zookeeper/blob/ba8932dc/src/java/test/org/apache/zookeeper/server/util/ConfigUtilsTest.java
----------------------------------------------------------------------
diff --git a/src/java/test/org/apache/zookeeper/server/util/ConfigUtilsTest.java b/src/java/test/org/apache/zookeeper/server/util/ConfigUtilsTest.java
new file mode 100644
index 0000000..d852d0f
--- /dev/null
+++ b/src/java/test/org/apache/zookeeper/server/util/ConfigUtilsTest.java
@@ -0,0 +1,52 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.server.util;
+
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
+
+public class ConfigUtilsTest {
+
+    @Test
+    public void testSplitServerConfig() throws ConfigException {
+        String[] nsa = ConfigUtils.getHostAndPort("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:443");
+        System.out.println(nsa[0]);
+        assertEquals(nsa[0], "2001:db8:85a3:8d3:1319:8a2e:370:7348");
+        assertEquals(nsa[1], "443");
+    }
+
+    @Test
+    public void testSplitServerConfig2() throws ConfigException {
+        String[] nsa = ConfigUtils.getHostAndPort("127.0.0.1:443");
+        assertEquals(nsa.length, 2, 0);
+    }
+
+    @Test(expected = ConfigException.class)
+    public void testSplitServerConfig3() throws ConfigException {
+        String[] nsa = ConfigUtils.getHostAndPort("[2001:db8:85a3:8d3:1319:8a2e:370:7348");
+    }
+
+    @Test
+    public void testSplitServerConfig4() throws ConfigException {
+        String[] nsa = ConfigUtils.getHostAndPort("2001:db8:85a3:8d3:1319:8a2e:370:7348:443");
+        assertFalse(nsa.length == 2);
+    }
+}