Explorar o código

ZOOKEEPER-236: SSL Support for Atomic Broadcast protocol (master)

Cherry-picked from branch-3.5.

Author: Andor Molnar <andor@apache.org>
Author: Abraham Fine <afine@apache.org>

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

Closes #655 from anmolnar/ZOOKEEPER-236_master and squashes the following commits:

8c3aa11e [Andor Molnar] ZOOKEEPER-3165: Java 9: X509UtilTest.testCreateSSLContextWithoutTrustStorePassword fails
eb272749 [Abraham Fine] ZOOKEEPER-236: SSL Support for Atomic Broadcast protocol
Andor Molnar %!s(int64=6) %!d(string=hai) anos
pai
achega
7b679bc9dd
Modificáronse 35 ficheiros con 2918 adicións e 220 borrados
  1. 3 1
      build.xml
  2. 3 0
      ivy.xml
  3. 2 2
      zookeeper-client/zookeeper-client-java/src/main/java/org/apache/zookeeper/client/FourLetterWordMain.java
  4. 2 2
      zookeeper-common/src/main/java/org/apache/zookeeper/ClientCnxnSocketNetty.java
  5. 349 0
      zookeeper-common/src/main/java/org/apache/zookeeper/ZKHostnameVerifier.java
  6. 150 0
      zookeeper-common/src/main/java/org/apache/zookeeper/ZKTrustManager.java
  7. 38 0
      zookeeper-common/src/main/java/org/apache/zookeeper/common/ClientX509Util.java
  8. 32 0
      zookeeper-common/src/main/java/org/apache/zookeeper/common/QuorumX509Util.java
  9. 243 82
      zookeeper-common/src/main/java/org/apache/zookeeper/common/X509Util.java
  10. 49 16
      zookeeper-common/src/main/java/org/apache/zookeeper/common/ZKConfig.java
  11. 264 0
      zookeeper-common/src/test/java/org/apache/zookeeper/common/X509UtilTest.java
  12. 248 0
      zookeeper-common/src/test/java/org/apache/zookeeper/common/ZKTrustManagerTest.java
  13. 132 0
      zookeeper-common/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java
  14. 22 18
      zookeeper-common/src/test/java/org/apache/zookeeper/test/SSLAuthTest.java
  15. 0 0
      zookeeper-common/src/test/java/org/apache/zookeeper/test/SSLTest.java.orig
  16. 26 27
      zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxnFactory.java
  17. 21 17
      zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/X509AuthenticationProvider.java
  18. 24 4
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java
  19. 29 7
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java
  20. 49 0
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/PrependableSocket.java
  21. 16 3
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumBean.java
  22. 55 28
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
  23. 10 0
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumMXBean.java
  24. 22 2
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java
  25. 18 3
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
  26. 2 0
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
  27. 79 0
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/UnifiedServerSocket.java
  28. 2 1
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java
  29. 75 0
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumBeanTest.java
  30. 2 2
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerConfigTest.java
  31. 2 1
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
  32. 771 0
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java
  33. 2 1
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/RaceConditionTest.java
  34. 172 0
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java
  35. 4 3
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ZabUtils.java

+ 3 - 1
build.xml

@@ -40,9 +40,11 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant">
 
     <property name="junit.version" value="4.12"/>
     <property name="mockito.version" value="1.8.5"/>
-    <property name="checkstyle.version" value="6.13"/>
+    <property name="checkstyle.version" value="7.1.2"/>
     <property name="commons-collections.version" value="3.2.2"/>
 
+    <property name="bouncycastle.version" value="1.56"/>
+
     <property name="jdiff.version" value="1.0.9"/>
     <property name="xerces.version" value="1.4.4"/>
 

+ 3 - 0
ivy.xml

@@ -78,6 +78,9 @@
     <dependency org="commons-collections" name="commons-collections" 
                 rev="${commons-collections.version}" conf="test->default"/>
 
+    <dependency org="org.bouncycastle" name="bcprov-jdk15on" rev="${bouncycastle.version}" conf="test->default"/>
+    <dependency org="org.bouncycastle" name="bcpkix-jdk15on" rev="${bouncycastle.version}" conf="test->default"/>
+
     <dependency org="jdiff" name="jdiff" rev="${jdiff.version}"
                 conf="jdiff->default"/>
     <dependency org="xerces" name="xerces" rev="${xerces.version}"

+ 2 - 2
zookeeper-client/zookeeper-client-java/src/main/java/org/apache/zookeeper/client/FourLetterWordMain.java

@@ -31,9 +31,9 @@ import javax.net.ssl.SSLContext;
 import javax.net.ssl.SSLSocket;
 import javax.net.ssl.SSLSocketFactory;
 
+import org.apache.zookeeper.common.ClientX509Util;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.common.X509Exception.SSLContextException;
-import org.apache.zookeeper.common.X509Util;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -90,7 +90,7 @@ public class FourLetterWordMain {
             new InetSocketAddress(InetAddress.getByName(null), port);
         if (secure) {
             LOG.info("using secure socket");
-            SSLContext sslContext = X509Util.createSSLContext();
+            SSLContext sslContext = new ClientX509Util().getDefaultSSLContext();
             SSLSocketFactory socketFactory = sslContext.getSocketFactory();
             SSLSocket sslSock = (SSLSocket) socketFactory.createSocket();
             sslSock.connect(hostaddress, timeout);

+ 2 - 2
zookeeper-common/src/main/java/org/apache/zookeeper/ClientCnxnSocketNetty.java

@@ -21,7 +21,7 @@ package org.apache.zookeeper;
 import org.apache.zookeeper.ClientCnxn.EndOfStreamException;
 import org.apache.zookeeper.ClientCnxn.Packet;
 import org.apache.zookeeper.client.ZKClientConfig;
-import org.apache.zookeeper.common.X509Util;
+import org.apache.zookeeper.common.ClientX509Util;
 import org.jboss.netty.bootstrap.ClientBootstrap;
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.buffer.ChannelBuffers;
@@ -370,7 +370,7 @@ public class ClientCnxnSocketNetty extends ClientCnxnSocket {
         // Basically we only need to create it once.
         private synchronized void initSSL(ChannelPipeline pipeline) throws SSLContextException {
             if (sslContext == null || sslEngine == null) {
-                sslContext = X509Util.createSSLContext(clientConfig);
+                sslContext = new ClientX509Util().createSSLContext(clientConfig);
                 sslEngine = sslContext.createSSLEngine(host,port);
                 sslEngine.setUseClientMode(true);
             }

+ 349 - 0
zookeeper-common/src/main/java/org/apache/zookeeper/ZKHostnameVerifier.java

@@ -0,0 +1,349 @@
+/**
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.naming.InvalidNameException;
+import javax.naming.NamingException;
+import javax.naming.directory.Attribute;
+import javax.naming.directory.Attributes;
+import javax.naming.ldap.LdapName;
+import javax.naming.ldap.Rdn;
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.SSLPeerUnverifiedException;
+import javax.net.ssl.SSLSession;
+import javax.security.auth.x500.X500Principal;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.security.cert.Certificate;
+import java.security.cert.CertificateParsingException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.regex.Pattern;
+
+/**
+ * Note: copied from Apache httpclient with some modifications. We want host verification, but depending
+ * on the httpclient jar caused unexplained performance regressions (even when the code was not used).
+ */
+final class ZKHostnameVerifier implements HostnameVerifier {
+
+    /**
+     * Note: copied from Apache httpclient with some minor modifications. We want host verification, but depending
+     * on the httpclient jar caused unexplained performance regressions (even when the code was not used).
+     */
+    private static final class SubjectName {
+        static final int DNS = 2;
+        static final int IP = 7;
+
+        private final String value;
+        private final int type;
+
+        static SubjectName IP(final String value) {
+            return new SubjectName(value, IP);
+        }
+
+        static SubjectName DNS(final String value) {
+            return new SubjectName(value, DNS);
+        }
+
+        SubjectName(final String value, final int type) {
+            if (type != DNS && type != IP) {
+                throw new IllegalArgumentException("Invalid type: " + type);
+            }
+            this.value = Objects.requireNonNull(value);
+            this.type = type;
+        }
+
+        public int getType() {
+            return type;
+        }
+
+        public String getValue() {
+            return value;
+        }
+
+        @Override
+        public String toString() {
+            return value;
+        }
+    }
+
+    /**
+     * Note: copied from Apache httpclient. We want host verification, but depending on the
+     * httpclient jar caused unexplained performance regressions (even when the code was not used).
+     */
+    private static class InetAddressUtils {
+        private InetAddressUtils() {}
+
+        private static final Pattern IPV4_PATTERN = Pattern.compile(
+                "^(25[0-5]|2[0-4]\\d|[0-1]?\\d?\\d)(\\.(25[0-5]|2[0-4]\\d|[0-1]?\\d?\\d)){3}$");
+
+        private static final Pattern IPV6_STD_PATTERN = Pattern.compile(
+                "^(?:[0-9a-fA-F]{1,4}:){7}[0-9a-fA-F]{1,4}$");
+
+        private static final Pattern IPV6_HEX_COMPRESSED_PATTERN = Pattern.compile(
+                "^((?:[0-9A-Fa-f]{1,4}(?::[0-9A-Fa-f]{1,4})*)?)::((?:[0-9A-Fa-f]{1,4}(?::[0-9A-Fa-f]{1,4})*)?)$");
+
+        static boolean isIPv4Address(final String input) {
+            return IPV4_PATTERN.matcher(input).matches();
+        }
+
+        static boolean isIPv6StdAddress(final String input) {
+            return IPV6_STD_PATTERN.matcher(input).matches();
+        }
+
+        static boolean isIPv6HexCompressedAddress(final String input) {
+            return IPV6_HEX_COMPRESSED_PATTERN.matcher(input).matches();
+        }
+
+        static boolean isIPv6Address(final String input) {
+            return isIPv6StdAddress(input) || isIPv6HexCompressedAddress(input);
+        }
+    }
+
+    enum HostNameType {
+
+        IPv4(7), IPv6(7), DNS(2);
+
+        final int subjectType;
+
+        HostNameType(final int subjectType) {
+            this.subjectType = subjectType;
+        }
+
+    }
+
+    private final Logger log = LoggerFactory.getLogger(ZKHostnameVerifier.class);
+
+    @Override
+    public boolean verify(final String host, final SSLSession session) {
+        try {
+            final Certificate[] certs = session.getPeerCertificates();
+            final X509Certificate x509 = (X509Certificate) certs[0];
+            verify(host, x509);
+            return true;
+        } catch (final SSLException ex) {
+            if (log.isDebugEnabled()) {
+                log.debug(ex.getMessage(), ex);
+            }
+            return false;
+        }
+    }
+
+    void verify(final String host, final X509Certificate cert) throws SSLException {
+        final HostNameType hostType = determineHostFormat(host);
+        final List<SubjectName> subjectAlts = getSubjectAltNames(cert);
+        if (subjectAlts != null && !subjectAlts.isEmpty()) {
+            switch (hostType) {
+                case IPv4:
+                    matchIPAddress(host, subjectAlts);
+                    break;
+                case IPv6:
+                    matchIPv6Address(host, subjectAlts);
+                    break;
+                default:
+                    matchDNSName(host, subjectAlts);
+            }
+        } else {
+            // CN matching has been deprecated by rfc2818 and can be used
+            // as fallback only when no subjectAlts are available
+            final X500Principal subjectPrincipal = cert.getSubjectX500Principal();
+            final String cn = extractCN(subjectPrincipal.getName(X500Principal.RFC2253));
+            if (cn == null) {
+                throw new SSLException("Certificate subject for <" + host + "> doesn't contain " +
+                        "a common name and does not have alternative names");
+            }
+            matchCN(host, cn);
+        }
+    }
+
+    private static void matchIPAddress(final String host, final List<SubjectName> subjectAlts) throws SSLException {
+        for (int i = 0; i < subjectAlts.size(); i++) {
+            final SubjectName subjectAlt = subjectAlts.get(i);
+            if (subjectAlt.getType() == SubjectName.IP) {
+                if (host.equals(subjectAlt.getValue())) {
+                    return;
+                }
+            }
+        }
+        throw new SSLPeerUnverifiedException("Certificate for <" + host + "> doesn't match any " +
+                "of the subject alternative names: " + subjectAlts);
+    }
+
+    private static void matchIPv6Address(final String host, final List<SubjectName> subjectAlts) throws SSLException {
+        final String normalisedHost = normaliseAddress(host);
+        for (int i = 0; i < subjectAlts.size(); i++) {
+            final SubjectName subjectAlt = subjectAlts.get(i);
+            if (subjectAlt.getType() == SubjectName.IP) {
+                final String normalizedSubjectAlt = normaliseAddress(subjectAlt.getValue());
+                if (normalisedHost.equals(normalizedSubjectAlt)) {
+                    return;
+                }
+            }
+        }
+        throw new SSLPeerUnverifiedException("Certificate for <" + host + "> doesn't match any " +
+                "of the subject alternative names: " + subjectAlts);
+    }
+
+    private static void matchDNSName(final String host, final List<SubjectName> subjectAlts) throws SSLException {
+        final String normalizedHost = host.toLowerCase(Locale.ROOT);
+        for (int i = 0; i < subjectAlts.size(); i++) {
+            final SubjectName subjectAlt = subjectAlts.get(i);
+            if (subjectAlt.getType() == SubjectName.DNS) {
+                final String normalizedSubjectAlt = subjectAlt.getValue().toLowerCase(Locale.ROOT);
+                if (matchIdentityStrict(normalizedHost, normalizedSubjectAlt)) {
+                    return;
+                }
+            }
+        }
+        throw new SSLPeerUnverifiedException("Certificate for <" + host + "> doesn't match any " +
+                "of the subject alternative names: " + subjectAlts);
+    }
+
+    private static void matchCN(final String host, final String cn) throws SSLException {
+        final String normalizedHost = host.toLowerCase(Locale.ROOT);
+        final String normalizedCn = cn.toLowerCase(Locale.ROOT);
+        if (!matchIdentityStrict(normalizedHost, normalizedCn)) {
+            throw new SSLPeerUnverifiedException("Certificate for <" + host + "> doesn't match " +
+                    "common name of the certificate subject: " + cn);
+        }
+    }
+
+    private static boolean matchIdentity(final String host, final String identity,
+                                         final boolean strict) {
+        // RFC 2818, 3.1. Server Identity
+        // "...Names may contain the wildcard
+        // character * which is considered to match any single domain name
+        // component or component fragment..."
+        // Based on this statement presuming only singular wildcard is legal
+        final int asteriskIdx = identity.indexOf('*');
+        if (asteriskIdx != -1) {
+            final String prefix = identity.substring(0, asteriskIdx);
+            final String suffix = identity.substring(asteriskIdx + 1);
+            if (!prefix.isEmpty() && !host.startsWith(prefix)) {
+                return false;
+            }
+            if (!suffix.isEmpty() && !host.endsWith(suffix)) {
+                return false;
+            }
+            // Additional sanity checks on content selected by wildcard can be done here
+            if (strict) {
+                final String remainder = host.substring(
+                        prefix.length(), host.length() - suffix.length());
+                if (remainder.contains(".")) {
+                    return false;
+                }
+            }
+            return true;
+        }
+        return host.equalsIgnoreCase(identity);
+    }
+
+    private static boolean matchIdentityStrict(final String host, final String identity) {
+        return matchIdentity(host, identity, true);
+    }
+
+    private static String extractCN(final String subjectPrincipal) throws SSLException {
+        if (subjectPrincipal == null) {
+            return null;
+        }
+        try {
+            final LdapName subjectDN = new LdapName(subjectPrincipal);
+            final List<Rdn> rdns = subjectDN.getRdns();
+            for (int i = rdns.size() - 1; i >= 0; i--) {
+                final Rdn rds = rdns.get(i);
+                final Attributes attributes = rds.toAttributes();
+                final Attribute cn = attributes.get("cn");
+                if (cn != null) {
+                    try {
+                        final Object value = cn.get();
+                        if (value != null) {
+                            return value.toString();
+                        }
+                    } catch (final NoSuchElementException ignore) {
+                        // ignore exception
+                    } catch (final NamingException ignore) {
+                        // ignore exception
+                    }
+                }
+            }
+            return null;
+        } catch (final InvalidNameException e) {
+            throw new SSLException(subjectPrincipal + " is not a valid X500 distinguished name");
+        }
+    }
+
+    private static HostNameType determineHostFormat(final String host) {
+        if (InetAddressUtils.isIPv4Address(host)) {
+            return HostNameType.IPv4;
+        }
+        String s = host;
+        if (s.startsWith("[") && s.endsWith("]")) {
+            s = host.substring(1, host.length() - 1);
+        }
+        if (InetAddressUtils.isIPv6Address(s)) {
+            return HostNameType.IPv6;
+        }
+        return HostNameType.DNS;
+    }
+
+    private static List<SubjectName> getSubjectAltNames(final X509Certificate cert) {
+        try {
+            final Collection<List<?>> entries = cert.getSubjectAlternativeNames();
+            if (entries == null) {
+                return Collections.emptyList();
+            }
+            final List<SubjectName> result = new ArrayList<SubjectName>();
+            for (List<?> entry: entries) {
+                final Integer type = entry.size() >= 2 ? (Integer) entry.get(0) : null;
+                if (type != null) {
+                    final String s = (String) entry.get(1);
+                    result.add(new SubjectName(s, type));
+                }
+            }
+            return result;
+        } catch (final CertificateParsingException ignore) {
+            return Collections.emptyList();
+        }
+    }
+
+    /*
+     * Normalize IPv6 or DNS name.
+     */
+    private static String normaliseAddress(final String hostname) {
+        if (hostname == null) {
+            return hostname;
+        }
+        try {
+            final InetAddress inetAddress = InetAddress.getByName(hostname);
+            return inetAddress.getHostAddress();
+        } catch (final UnknownHostException unexpected) { // Should not happen, because we check for IPv6 address above
+            return hostname;
+        }
+    }
+}

+ 150 - 0
zookeeper-common/src/main/java/org/apache/zookeeper/ZKTrustManager.java

@@ -0,0 +1,150 @@
+/**
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.X509ExtendedTrustManager;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+
+/**
+ * A custom TrustManager that supports hostname verification via org.apache.http.conn.ssl.DefaultHostnameVerifier.
+ *
+ * We attempt to perform verification using just the IP address first and if that fails will attempt to perform a
+ * reverse DNS lookup and verify using the hostname.
+ */
+public class ZKTrustManager extends X509ExtendedTrustManager {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ZKTrustManager.class);
+
+    private X509ExtendedTrustManager x509ExtendedTrustManager;
+    private boolean serverHostnameVerificationEnabled;
+    private boolean clientHostnameVerificationEnabled;
+
+    private ZKHostnameVerifier hostnameVerifier;
+
+    /**
+     * Instantiate a new ZKTrustManager.
+     *
+     * @param x509ExtendedTrustManager The trustmanager to use for checkClientTrusted/checkServerTrusted logic
+     * @param serverHostnameVerificationEnabled  If true, this TrustManager should verify hostnames of servers that this
+     *                                 instance connects to.
+     * @param clientHostnameVerificationEnabled  If true, the hostname of a client connecting to this machine will be
+     *                                           verified.
+     */
+    ZKTrustManager(X509ExtendedTrustManager x509ExtendedTrustManager, boolean serverHostnameVerificationEnabled,
+                   boolean clientHostnameVerificationEnabled) {
+        this.x509ExtendedTrustManager = x509ExtendedTrustManager;
+        this.serverHostnameVerificationEnabled = serverHostnameVerificationEnabled;
+        this.clientHostnameVerificationEnabled = clientHostnameVerificationEnabled;
+        hostnameVerifier = new ZKHostnameVerifier();
+    }
+
+    @Override
+    public X509Certificate[] getAcceptedIssuers() {
+        return x509ExtendedTrustManager.getAcceptedIssuers();
+    }
+
+    @Override
+    public void checkClientTrusted(X509Certificate[] chain, String authType, Socket socket) throws CertificateException {
+        x509ExtendedTrustManager.checkClientTrusted(chain, authType, socket);
+        if (clientHostnameVerificationEnabled) {
+            performHostVerification(socket.getInetAddress(), chain[0]);
+        }
+    }
+
+    @Override
+    public void checkServerTrusted(X509Certificate[] chain, String authType, Socket socket) throws CertificateException {
+        x509ExtendedTrustManager.checkServerTrusted(chain, authType, socket);
+        if (serverHostnameVerificationEnabled) {
+            performHostVerification(socket.getInetAddress(), chain[0]);
+        }
+    }
+
+    @Override
+    public void checkClientTrusted(X509Certificate[] chain, String authType, SSLEngine engine) throws CertificateException {
+        x509ExtendedTrustManager.checkServerTrusted(chain, authType, engine);
+        if (clientHostnameVerificationEnabled) {
+            try {
+                performHostVerification(InetAddress.getByName(engine.getPeerHost()), chain[0]);
+            } catch (UnknownHostException e) {
+                throw new CertificateException("Failed to verify host", e);
+            }
+        }
+    }
+
+    @Override
+    public void checkServerTrusted(X509Certificate[] chain, String authType, SSLEngine engine)
+            throws CertificateException {
+        x509ExtendedTrustManager.checkServerTrusted(chain, authType, engine);
+        if (serverHostnameVerificationEnabled) {
+            try {
+                performHostVerification(InetAddress.getByName(engine.getPeerHost()), chain[0]);
+            } catch (UnknownHostException e) {
+                throw new CertificateException("Failed to verify host", e);
+            }
+        }
+    }
+
+    @Override
+    public void checkClientTrusted(X509Certificate[] chain, String authType) throws CertificateException {
+        x509ExtendedTrustManager.checkClientTrusted(chain, authType);
+    }
+
+    @Override
+    public void checkServerTrusted(X509Certificate[] chain, String authType) throws CertificateException {
+        x509ExtendedTrustManager.checkServerTrusted(chain, authType);
+    }
+
+    /**
+     * Compares peer's hostname with the one stored in the provided client certificate. Performs verification
+     * with the help of provided HostnameVerifier.
+     *
+     * @param inetAddress Peer's inet address.
+     * @param certificate Peer's certificate
+     * @throws CertificateException Thrown if the provided certificate doesn't match the peer hostname.
+     */
+    private void performHostVerification(InetAddress inetAddress, X509Certificate certificate)
+            throws CertificateException {
+        String hostAddress = "";
+        String hostName = "";
+        try {
+            hostAddress = inetAddress.getHostAddress();
+            hostnameVerifier.verify(hostAddress, certificate);
+        } catch (SSLException addressVerificationException) {
+            try {
+                LOG.debug("Failed to verify host address: {} attempting to verify host name with reverse dns lookup",
+                        hostAddress, addressVerificationException);
+                hostName = inetAddress.getHostName();
+                hostnameVerifier.verify(hostName, certificate);
+            } catch (SSLException hostnameVerificationException) {
+                LOG.error("Failed to verify host address: {}", hostAddress, addressVerificationException);
+                LOG.error("Failed to verify hostname: {}", hostName, hostnameVerificationException);
+                throw new CertificateException("Failed to verify both host address and host name",
+                        hostnameVerificationException);
+            }
+        }
+    }
+}

+ 38 - 0
zookeeper-common/src/main/java/org/apache/zookeeper/common/ClientX509Util.java

@@ -0,0 +1,38 @@
+/**
+ * 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;
+
+public class ClientX509Util extends X509Util {
+
+    private final String sslAuthProviderProperty = getConfigPrefix() + "authProvider";
+
+    @Override
+    protected String getConfigPrefix() {
+        return "zookeeper.ssl.";
+    }
+
+    @Override
+    protected boolean shouldVerifyClientHostname() {
+        return false;
+    }
+
+    public String getSslAuthProviderProperty() {
+        return sslAuthProviderProperty;
+    }
+}

+ 32 - 0
zookeeper-common/src/main/java/org/apache/zookeeper/common/QuorumX509Util.java

@@ -0,0 +1,32 @@
+/**
+ * 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;
+
+public class QuorumX509Util extends X509Util {
+
+    @Override
+    protected String getConfigPrefix() {
+        return "zookeeper.ssl.quorum.";
+    }
+
+    @Override
+    protected boolean shouldVerifyClientHostname() {
+        return true;
+    }
+}

+ 243 - 82
zookeeper-common/src/main/java/org/apache/zookeeper/common/X509Util.java

@@ -18,64 +18,143 @@
 package org.apache.zookeeper.common;
 
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.ssl.CertPathTrustManagerParameters;
 import javax.net.ssl.KeyManager;
 import javax.net.ssl.KeyManagerFactory;
 import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLParameters;
+import javax.net.ssl.SSLServerSocket;
+import javax.net.ssl.SSLSocket;
 import javax.net.ssl.TrustManager;
 import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509ExtendedTrustManager;
 import javax.net.ssl.X509KeyManager;
 import javax.net.ssl.X509TrustManager;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
+import java.net.Socket;
+import java.security.InvalidAlgorithmParameterException;
+import java.security.KeyManagementException;
 import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.Security;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.PKIXBuilderParameters;
+import java.security.cert.X509CertSelector;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicReference;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.zookeeper.common.X509Exception.KeyManagerException;
-import static org.apache.zookeeper.common.X509Exception.SSLContextException;
-import static org.apache.zookeeper.common.X509Exception.TrustManagerException;
+import org.apache.zookeeper.common.X509Exception.KeyManagerException;
+import org.apache.zookeeper.common.X509Exception.SSLContextException;
+import org.apache.zookeeper.common.X509Exception.TrustManagerException;
 
 /**
  * Utility code for X509 handling
+ *
+ * Default cipher suites:
+ *
+ *   Performance testing done by Facebook engineers shows that on Intel x86_64 machines, Java9 performs better with
+ *   GCM and Java8 performs better with CBC, so these seem like reasonable defaults.
  */
-public class X509Util {
+public abstract class X509Util {
     private static final Logger LOG = LoggerFactory.getLogger(X509Util.class);
 
-    /**
-     * @deprecated Use {@link ZKConfig#SSL_KEYSTORE_LOCATION}
-     *             instead.
-     */
-    @Deprecated
-    public static final String SSL_KEYSTORE_LOCATION = "zookeeper.ssl.keyStore.location";
-    /**
-     * @deprecated Use {@link ZKConfig#SSL_KEYSTORE_PASSWD}
-     *             instead.
-     */
-    @Deprecated
-    public static final String SSL_KEYSTORE_PASSWD = "zookeeper.ssl.keyStore.password";
-    /**
-     * @deprecated Use {@link ZKConfig#SSL_TRUSTSTORE_LOCATION}
-     *             instead.
-     */
-    @Deprecated
-    public static final String SSL_TRUSTSTORE_LOCATION = "zookeeper.ssl.trustStore.location";
-    /**
-     * @deprecated Use {@link ZKConfig#SSL_TRUSTSTORE_PASSWD}
-     *             instead.
-     */
-    @Deprecated
-    public static final String SSL_TRUSTSTORE_PASSWD = "zookeeper.ssl.trustStore.password";
-    /**
-     * @deprecated Use {@link ZKConfig#SSL_AUTHPROVIDER}
-     *             instead.
-     */
-    @Deprecated
-    public static final String SSL_AUTHPROVIDER = "zookeeper.ssl.authProvider";
-
-    public static SSLContext createSSLContext() throws SSLContextException {
-        /**
+    static final String DEFAULT_PROTOCOL = "TLSv1.2";
+    private static final String[] DEFAULT_CIPHERS_JAVA8 = {
+            "TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA256",
+            "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256",
+            "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256",
+            "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256"
+    };
+    private static final String[] DEFAULT_CIPHERS_JAVA9 = {
+            "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256",
+            "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256",
+            "TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA256",
+            "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256"
+    };
+
+    private String sslProtocolProperty = getConfigPrefix() + "protocol";
+    private String cipherSuitesProperty = getConfigPrefix() + "ciphersuites";
+    private String sslKeystoreLocationProperty = getConfigPrefix() + "keyStore.location";
+    private String sslKeystorePasswdProperty = getConfigPrefix() + "keyStore.password";
+    private String sslTruststoreLocationProperty = getConfigPrefix() + "trustStore.location";
+    private String sslTruststorePasswdProperty = getConfigPrefix() + "trustStore.password";
+    private String sslHostnameVerificationEnabledProperty = getConfigPrefix() + "hostnameVerification";
+    private String sslCrlEnabledProperty = getConfigPrefix() + "crl";
+    private String sslOcspEnabledProperty = getConfigPrefix() + "ocsp";
+
+    private String[] cipherSuites;
+
+    private AtomicReference<SSLContext> defaultSSLContext = new AtomicReference<>(null);
+
+    public X509Util() {
+        String cipherSuitesInput = System.getProperty(cipherSuitesProperty);
+        if (cipherSuitesInput == null) {
+            cipherSuites = getDefaultCipherSuites();
+        } else {
+            cipherSuites = cipherSuitesInput.split(",");
+        }
+    }
+
+    protected abstract String getConfigPrefix();
+    protected abstract boolean shouldVerifyClientHostname();
+
+    public String getSslProtocolProperty() {
+        return sslProtocolProperty;
+    }
+
+    public String getCipherSuitesProperty() {
+        return cipherSuitesProperty;
+    }
+
+    public String getSslKeystoreLocationProperty() {
+        return sslKeystoreLocationProperty;
+    }
+
+    public String getSslKeystorePasswdProperty() {
+        return sslKeystorePasswdProperty;
+    }
+
+    public String getSslTruststoreLocationProperty() {
+        return sslTruststoreLocationProperty;
+    }
+
+    public String getSslTruststorePasswdProperty() {
+        return sslTruststorePasswdProperty;
+    }
+
+    public String getSslHostnameVerificationEnabledProperty() {
+        return sslHostnameVerificationEnabledProperty;
+    }
+
+    public String getSslCrlEnabledProperty() {
+        return sslCrlEnabledProperty;
+    }
+
+    public String getSslOcspEnabledProperty() {
+        return sslOcspEnabledProperty;
+    }
+
+    public SSLContext getDefaultSSLContext() throws X509Exception.SSLContextException {
+        SSLContext result = defaultSSLContext.get();
+        if (result == null) {
+            result = createSSLContext();
+            if (!defaultSSLContext.compareAndSet(null, result)) {
+                // lost the race, another thread already set the value
+                result = defaultSSLContext.get();
+            }
+        }
+        return result;
+    }
+
+    private SSLContext createSSLContext() throws SSLContextException {
+        /*
          * Since Configuration initializes the key store and trust store related
          * configuration from system property. Reading property from
          * configuration will be same reading from system property
@@ -84,61 +163,62 @@ public class X509Util {
         return createSSLContext(config);
     }
 
-    public static SSLContext createSSLContext(ZKConfig config) throws SSLContextException {
+    public SSLContext createSSLContext(ZKConfig config) throws SSLContextException {
         KeyManager[] keyManagers = null;
         TrustManager[] trustManagers = null;
 
-        String keyStoreLocationProp = config.getProperty(ZKConfig.SSL_KEYSTORE_LOCATION);
-        String keyStorePasswordProp = config.getProperty(ZKConfig.SSL_KEYSTORE_PASSWD);
+        String keyStoreLocationProp = config.getProperty(sslKeystoreLocationProperty);
+        String keyStorePasswordProp = config.getProperty(sslKeystorePasswdProperty);
 
         // There are legal states in some use cases for null KeyManager or TrustManager.
         // But if a user wanna specify one, location and password are required.
 
         if (keyStoreLocationProp == null && keyStorePasswordProp == null) {
-            LOG.warn("keystore not specified for client connection");
+            LOG.warn(getSslKeystoreLocationProperty() + " not specified");
         } else {
             if (keyStoreLocationProp == null) {
-                throw new SSLContextException("keystore location not specified for client connection");
+                throw new SSLContextException(getSslKeystoreLocationProperty() + " not specified");
             }
             if (keyStorePasswordProp == null) {
-                throw new SSLContextException("keystore password not specified for client connection");
+                throw new SSLContextException(getSslKeystorePasswdProperty() + " not specified");
             }
             try {
                 keyManagers = new KeyManager[]{
                         createKeyManager(keyStoreLocationProp, keyStorePasswordProp)};
-            } catch (KeyManagerException e) {
-                throw new SSLContextException("Failed to create KeyManager", e);
+            } catch (KeyManagerException keyManagerException) {
+                throw new SSLContextException("Failed to create KeyManager", keyManagerException);
             }
         }
 
-        String trustStoreLocationProp = config.getProperty(ZKConfig.SSL_TRUSTSTORE_LOCATION);
-        String trustStorePasswordProp = config.getProperty(ZKConfig.SSL_TRUSTSTORE_PASSWD);
+        String trustStoreLocationProp = config.getProperty(sslTruststoreLocationProperty);
+        String trustStorePasswordProp = config.getProperty(sslTruststorePasswdProperty);
 
-        if (trustStoreLocationProp == null && trustStorePasswordProp == null) {
-            LOG.warn("Truststore not specified for client connection");
+        boolean sslCrlEnabled = config.getBoolean(this.sslCrlEnabledProperty);
+        boolean sslOcspEnabled = config.getBoolean(this.sslOcspEnabledProperty);
+        boolean sslServerHostnameVerificationEnabled =
+                config.getBoolean(this.getSslHostnameVerificationEnabledProperty(),true);
+        boolean sslClientHostnameVerificationEnabled = sslServerHostnameVerificationEnabled && shouldVerifyClientHostname();
+
+        if (trustStoreLocationProp == null) {
+            LOG.warn(getSslTruststoreLocationProperty() + " not specified");
         } else {
-            if (trustStoreLocationProp == null) {
-                throw new SSLContextException("Truststore location not specified for client connection");
-            }
-            if (trustStorePasswordProp == null) {
-                throw new SSLContextException("Truststore password not specified for client connection");
-            }
             try {
                 trustManagers = new TrustManager[]{
-                        createTrustManager(trustStoreLocationProp, trustStorePasswordProp)};
-            } catch (TrustManagerException e) {
-                throw new SSLContextException("Failed to create TrustManager", e);
+                        createTrustManager(trustStoreLocationProp, trustStorePasswordProp, sslCrlEnabled, sslOcspEnabled,
+                                sslServerHostnameVerificationEnabled, sslClientHostnameVerificationEnabled)};
+            } catch (TrustManagerException trustManagerException) {
+                throw new SSLContextException("Failed to create TrustManager", trustManagerException);
             }
         }
 
-        SSLContext sslContext = null;
+        String protocol = System.getProperty(sslProtocolProperty, DEFAULT_PROTOCOL);
         try {
-            sslContext = SSLContext.getInstance("TLSv1");
+            SSLContext sslContext = SSLContext.getInstance(protocol);
             sslContext.init(keyManagers, trustManagers, null);
-        } catch (Exception e) {
-            throw new SSLContextException(e);
+            return sslContext;
+        } catch (NoSuchAlgorithmException|KeyManagementException sslContextInitException) {
+            throw new SSLContextException(sslContextInitException);
         }
-        return sslContext;
     }
 
     public static X509KeyManager createKeyManager(String keyStoreLocation, String keyStorePassword)
@@ -150,7 +230,7 @@ public class X509Util {
             KeyStore ks = KeyStore.getInstance("JKS");
             inputStream = new FileInputStream(keyStoreFile);
             ks.load(inputStream, keyStorePasswordChars);
-            KeyManagerFactory kmf = KeyManagerFactory.getInstance("SunX509");
+            KeyManagerFactory kmf = KeyManagerFactory.getInstance("PKIX");
             kmf.init(ks, keyStorePasswordChars);
 
             for (KeyManager km : kmf.getKeyManagers()) {
@@ -160,43 +240,124 @@ public class X509Util {
             }
             throw new KeyManagerException("Couldn't find X509KeyManager");
 
-        } catch (Exception e) {
-            throw new KeyManagerException(e);
+        } catch (IOException|CertificateException|UnrecoverableKeyException|NoSuchAlgorithmException|KeyStoreException
+                keyManagerCreationException) {
+            throw new KeyManagerException(keyManagerCreationException);
         } finally {
             if (inputStream != null) {
                 try {
                     inputStream.close();
-                } catch (IOException e) {}
+                } catch (IOException ioException) {
+                    LOG.info("Failed to close key store input stream", ioException);
+                }
             }
         }
     }
 
-    public static X509TrustManager createTrustManager(String trustStoreLocation, String trustStorePassword)
+    public static X509TrustManager createTrustManager(String trustStoreLocation, String trustStorePassword,
+                                                      boolean crlEnabled, boolean ocspEnabled,
+                                                      final boolean serverHostnameVerificationEnabled,
+                                                      final boolean clientHostnameVerificationEnabled)
             throws TrustManagerException {
         FileInputStream inputStream = null;
         try {
-            char[] trustStorePasswordChars = trustStorePassword.toCharArray();
             File trustStoreFile = new File(trustStoreLocation);
             KeyStore ts = KeyStore.getInstance("JKS");
             inputStream = new FileInputStream(trustStoreFile);
-            ts.load(inputStream, trustStorePasswordChars);
-            TrustManagerFactory tmf = TrustManagerFactory.getInstance("SunX509");
-            tmf.init(ts);
+            if (trustStorePassword != null) {
+                char[] trustStorePasswordChars = trustStorePassword.toCharArray();
+                ts.load(inputStream, trustStorePasswordChars);
+            } else {
+                ts.load(inputStream, null);
+            }
+
+            PKIXBuilderParameters pbParams = new PKIXBuilderParameters(ts, new X509CertSelector());
+            if (crlEnabled || ocspEnabled) {
+                pbParams.setRevocationEnabled(true);
+                System.setProperty("com.sun.net.ssl.checkRevocation", "true");
+                System.setProperty("com.sun.security.enableCRLDP", "true");
+                if (ocspEnabled) {
+                    Security.setProperty("ocsp.enable", "true");
+                }
+            } else {
+                pbParams.setRevocationEnabled(false);
+            }
 
-            for (TrustManager tm : tmf.getTrustManagers()) {
-                if (tm instanceof X509TrustManager) {
-                    return (X509TrustManager) tm;
+            // Revocation checking is only supported with the PKIX algorithm
+            TrustManagerFactory tmf = TrustManagerFactory.getInstance("PKIX");
+            tmf.init(new CertPathTrustManagerParameters(pbParams));
+
+            for (final TrustManager tm : tmf.getTrustManagers()) {
+                if (tm instanceof X509ExtendedTrustManager) {
+                    return new ZKTrustManager((X509ExtendedTrustManager) tm,
+                            serverHostnameVerificationEnabled, clientHostnameVerificationEnabled);
                 }
             }
             throw new TrustManagerException("Couldn't find X509TrustManager");
-        } catch (Exception e) {
-            throw new TrustManagerException(e);
+        } catch (IOException|CertificateException|NoSuchAlgorithmException|InvalidAlgorithmParameterException|KeyStoreException
+                 trustManagerCreationException) {
+            throw new TrustManagerException(trustManagerCreationException);
         } finally {
             if (inputStream != null) {
                 try {
                     inputStream.close();
-                } catch (IOException e) {}
+                } catch (IOException ioException) {
+                    LOG.info("failed to close TrustStore input stream", ioException);
+                }
             }
         }
     }
-}
+
+    public SSLSocket createSSLSocket() throws X509Exception, IOException {
+        SSLSocket sslSocket = (SSLSocket) getDefaultSSLContext().getSocketFactory().createSocket();
+        configureSSLSocket(sslSocket);
+
+        return sslSocket;
+    }
+
+    public SSLSocket createSSLSocket(Socket socket) throws X509Exception, IOException {
+        SSLSocket sslSocket = (SSLSocket) getDefaultSSLContext().getSocketFactory().createSocket(socket, null, socket.getPort(), true);
+        configureSSLSocket(sslSocket);
+
+        return sslSocket;
+    }
+
+    private void configureSSLSocket(SSLSocket sslSocket) {
+        SSLParameters sslParameters = sslSocket.getSSLParameters();
+        LOG.debug("Setup cipher suites for client socket: {}", Arrays.toString(cipherSuites));
+        sslParameters.setCipherSuites(cipherSuites);
+        sslSocket.setSSLParameters(sslParameters);
+    }
+
+    public SSLServerSocket createSSLServerSocket() throws X509Exception, IOException {
+        SSLServerSocket sslServerSocket = (SSLServerSocket) getDefaultSSLContext().getServerSocketFactory().createServerSocket();
+        configureSSLServerSocket(sslServerSocket);
+
+        return sslServerSocket;
+    }
+
+    public SSLServerSocket createSSLServerSocket(int port) throws X509Exception, IOException {
+        SSLServerSocket sslServerSocket = (SSLServerSocket) getDefaultSSLContext().getServerSocketFactory().createServerSocket(port);
+        configureSSLServerSocket(sslServerSocket);
+
+        return sslServerSocket;
+    }
+
+    private void configureSSLServerSocket(SSLServerSocket sslServerSocket) {
+        SSLParameters sslParameters = sslServerSocket.getSSLParameters();
+        sslParameters.setNeedClientAuth(true);
+        LOG.debug("Setup cipher suites for server socket: {}", Arrays.toString(cipherSuites));
+        sslParameters.setCipherSuites(cipherSuites);
+        sslServerSocket.setSSLParameters(sslParameters);
+    }
+
+    private String[] getDefaultCipherSuites() {
+        String javaVersion = System.getProperty("java.specification.version");
+        if ("9".equals(javaVersion)) {
+            LOG.debug("Using Java9-optimized cipher suites for Java version {}", javaVersion);
+            return DEFAULT_CIPHERS_JAVA9;
+        }
+        LOG.debug("Using Java8-optimized cipher suites for Java version {}", javaVersion);
+        return DEFAULT_CIPHERS_JAVA8;
+    }
+}

+ 49 - 16
zookeeper-common/src/main/java/org/apache/zookeeper/common/ZKConfig.java

@@ -42,17 +42,9 @@ import org.slf4j.LoggerFactory;
 public class ZKConfig {
 
     private static final Logger LOG = LoggerFactory.getLogger(ZKConfig.class);
-    @SuppressWarnings("deprecation")
-    public static final String SSL_KEYSTORE_LOCATION = X509Util.SSL_KEYSTORE_LOCATION;
-    @SuppressWarnings("deprecation")
-    public static final String SSL_KEYSTORE_PASSWD = X509Util.SSL_KEYSTORE_PASSWD;
-    @SuppressWarnings("deprecation")
-    public static final String SSL_TRUSTSTORE_LOCATION = X509Util.SSL_TRUSTSTORE_LOCATION;
-    @SuppressWarnings("deprecation")
-    public static final String SSL_TRUSTSTORE_PASSWD = X509Util.SSL_TRUSTSTORE_PASSWD;
-    @SuppressWarnings("deprecation")
-    public static final String SSL_AUTHPROVIDER = X509Util.SSL_AUTHPROVIDER;
+
     public static final String JUTE_MAXBUFFER = "jute.maxbuffer";
+
     /**
      * Path to a kinit binary: {@value}. Defaults to
      * <code>"/usr/bin/kinit"</code>
@@ -107,14 +99,33 @@ public class ZKConfig {
      * this configuration.
      */
     protected void handleBackwardCompatibility() {
-        properties.put(SSL_KEYSTORE_LOCATION, System.getProperty(SSL_KEYSTORE_LOCATION));
-        properties.put(SSL_KEYSTORE_PASSWD, System.getProperty(SSL_KEYSTORE_PASSWD));
-        properties.put(SSL_TRUSTSTORE_LOCATION, System.getProperty(SSL_TRUSTSTORE_LOCATION));
-        properties.put(SSL_TRUSTSTORE_PASSWD, System.getProperty(SSL_TRUSTSTORE_PASSWD));
-        properties.put(SSL_AUTHPROVIDER, System.getProperty(SSL_AUTHPROVIDER));
         properties.put(JUTE_MAXBUFFER, System.getProperty(JUTE_MAXBUFFER));
         properties.put(KINIT_COMMAND, System.getProperty(KINIT_COMMAND));
         properties.put(JGSS_NATIVE, System.getProperty(JGSS_NATIVE));
+
+        ClientX509Util clientX509Util = new ClientX509Util();
+        putSSLProperties(clientX509Util);
+        properties.put(clientX509Util.getSslAuthProviderProperty(),
+                System.getProperty(clientX509Util.getSslAuthProviderProperty()));
+
+        putSSLProperties(new QuorumX509Util());
+    }
+    
+    private void putSSLProperties(X509Util x509Util) {
+        properties.put(x509Util.getSslKeystoreLocationProperty(),
+                System.getProperty(x509Util.getSslKeystoreLocationProperty()));
+        properties.put(x509Util.getSslKeystorePasswdProperty(),
+                System.getProperty(x509Util.getSslKeystorePasswdProperty()));
+        properties.put(x509Util.getSslTruststoreLocationProperty(),
+                System.getProperty(x509Util.getSslTruststoreLocationProperty()));
+        properties.put(x509Util.getSslTruststorePasswdProperty(),
+                System.getProperty(x509Util.getSslTruststorePasswdProperty()));
+        properties.put(x509Util.getSslHostnameVerificationEnabledProperty(),
+                System.getProperty(x509Util.getSslHostnameVerificationEnabledProperty()));
+        properties.put(x509Util.getSslCrlEnabledProperty(),
+                System.getProperty(x509Util.getSslCrlEnabledProperty()));
+        properties.put(x509Util.getSslOcspEnabledProperty(),
+                System.getProperty(x509Util.getSslOcspEnabledProperty()));
     }
 
     /**
@@ -220,7 +231,29 @@ public class ZKConfig {
      * exists and is equal to the string {@code "true"}.
      */
     public boolean getBoolean(String key) {
-        return Boolean.parseBoolean(getProperty(key));
+        return getBoolean(key, false);
+    }
+
+    /**
+     * Get the value of the <code>key</code> property as a <code>boolean</code>. Returns
+     * {@code true} if and only if the property named by the argument exists and is equal
+     * to the string {@code "true"}. If the property is not set, the provided
+     * <code>defaultValue</code> is returned.
+     *
+     * @param key
+     *            property key.
+     * @param defaultValue
+     *            default value.
+     * @return return property value as an <code>boolean</code>, or
+     *         <code>defaultValue</code>
+     */
+    public boolean getBoolean(String key, boolean defaultValue) {
+        String propertyValue = getProperty(key);
+        if (propertyValue == null) {
+            return defaultValue;
+        } else {
+            return Boolean.parseBoolean(propertyValue);
+        }
     }
 
     /**

+ 264 - 0
zookeeper-common/src/test/java/org/apache/zookeeper/common/X509UtilTest.java

@@ -0,0 +1,264 @@
+/**
+ * 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 org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.ZKTestCase;
+import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.bouncycastle.asn1.x500.X500NameBuilder;
+import org.bouncycastle.asn1.x500.style.BCStyle;
+import org.bouncycastle.asn1.x509.BasicConstraints;
+import org.bouncycastle.asn1.x509.Extension;
+import org.bouncycastle.asn1.x509.KeyUsage;
+import org.bouncycastle.cert.X509v3CertificateBuilder;
+import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter;
+import org.bouncycastle.cert.jcajce.JcaX509v3CertificateBuilder;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.bouncycastle.operator.ContentSigner;
+import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLServerSocket;
+import javax.net.ssl.SSLSocket;
+import java.io.FileOutputStream;
+import java.math.BigInteger;
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+import java.security.KeyStore;
+import java.security.Security;
+import java.security.cert.Certificate;
+import java.security.cert.X509Certificate;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.Random;
+
+import static org.apache.zookeeper.test.ClientBase.createTmpDir;
+
+public class X509UtilTest extends ZKTestCase {
+
+    private static final char[] PASSWORD = "password".toCharArray();
+    private X509Certificate rootCertificate;
+
+    private String truststorePath;
+    private String keystorePath;
+    private static KeyPair rootKeyPair;
+
+    private X509Util x509Util;
+    private String[] customCipherSuites = new String[]{"SSL_DHE_DSS_EXPORT_WITH_DES40_CBC_SHA", "SSL_DH_anon_EXPORT_WITH_DES40_CBC_SHA"};
+
+    @BeforeClass
+    public static void createKeyPair() throws Exception {
+        Security.addProvider(new BouncyCastleProvider());
+        KeyPairGenerator keyPairGenerator = KeyPairGenerator.getInstance("RSA", BouncyCastleProvider.PROVIDER_NAME);
+        keyPairGenerator.initialize(4096);
+        rootKeyPair = keyPairGenerator.genKeyPair();
+    }
+
+    @AfterClass
+    public static void removeBouncyCastleProvider() throws Exception {
+        Security.removeProvider("BC");
+    }
+
+    @Before
+    public void setUp() throws Exception {
+        rootCertificate = createSelfSignedCertifcate(rootKeyPair);
+
+        String tmpDir = createTmpDir().getAbsolutePath();
+        truststorePath = tmpDir + "/truststore.jks";
+        keystorePath = tmpDir + "/keystore.jks";
+
+        x509Util = new ClientX509Util();
+
+        writeKeystore(rootCertificate, rootKeyPair, keystorePath);
+
+        System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory");
+        System.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, "org.apache.zookeeper.ClientCnxnSocketNetty");
+        System.setProperty(x509Util.getSslKeystoreLocationProperty(), keystorePath);
+        System.setProperty(x509Util.getSslKeystorePasswdProperty(), new String(PASSWORD));
+        System.setProperty(x509Util.getSslTruststoreLocationProperty(), truststorePath);
+        System.setProperty(x509Util.getSslTruststorePasswdProperty(), new String(PASSWORD));
+        System.setProperty(x509Util.getSslHostnameVerificationEnabledProperty(), "false");
+
+        writeTrustStore(PASSWORD);
+    }
+
+    private void writeKeystore(X509Certificate certificate, KeyPair keyPair, String path) throws Exception {
+        KeyStore keyStore = KeyStore.getInstance("JKS");
+        keyStore.load(null, PASSWORD);
+        keyStore.setKeyEntry("alias", keyPair.getPrivate(), PASSWORD, new Certificate[] { certificate });
+        FileOutputStream outputStream = new FileOutputStream(path);
+        keyStore.store(outputStream, PASSWORD);
+        outputStream.flush();
+        outputStream.close();
+    }
+
+    private void writeTrustStore(char[] password) throws Exception {
+        KeyStore trustStore = KeyStore.getInstance("JKS");
+        trustStore.load(null, password);
+        trustStore.setCertificateEntry(rootCertificate.getSubjectDN().toString(), rootCertificate);
+        FileOutputStream outputStream = new FileOutputStream(truststorePath);
+        if (password == null) {
+            trustStore.store(outputStream, new char[0]);
+        } else {
+            trustStore.store(outputStream, password);
+        }
+        outputStream.flush();
+        outputStream.close();
+    }
+
+    private X509Certificate createSelfSignedCertifcate(KeyPair keyPair) throws Exception {
+        X500NameBuilder nameBuilder = new X500NameBuilder(BCStyle.INSTANCE);
+        nameBuilder.addRDN(BCStyle.CN, "localhost");
+        Date notBefore = new Date();
+        Calendar cal = Calendar.getInstance();
+        cal.setTime(notBefore);
+        cal.add(Calendar.YEAR, 1);
+        Date notAfter = cal.getTime();
+        BigInteger serialNumber = new BigInteger(128, new Random());
+
+        X509v3CertificateBuilder certificateBuilder =
+                new JcaX509v3CertificateBuilder(nameBuilder.build(), serialNumber, notBefore, notAfter, nameBuilder.build(), keyPair.getPublic())
+                        .addExtension(Extension.basicConstraints, true, new BasicConstraints(0))
+                        .addExtension(Extension.keyUsage, true, new KeyUsage(KeyUsage.digitalSignature | KeyUsage.keyCertSign | KeyUsage.cRLSign));
+
+        ContentSigner contentSigner = new JcaContentSignerBuilder("SHA256WithRSAEncryption").build(keyPair.getPrivate());
+
+        return new JcaX509CertificateConverter().getCertificate(certificateBuilder.build(contentSigner));
+    }
+
+    @After
+    public void cleanUp() throws Exception {
+        System.clearProperty(x509Util.getSslKeystoreLocationProperty());
+        System.clearProperty(x509Util.getSslKeystorePasswdProperty());
+        System.clearProperty(x509Util.getSslTruststoreLocationProperty());
+        System.clearProperty(x509Util.getSslTruststorePasswdProperty());
+        System.clearProperty(x509Util.getSslHostnameVerificationEnabledProperty());
+        System.clearProperty(x509Util.getSslOcspEnabledProperty());
+        System.clearProperty(x509Util.getSslCrlEnabledProperty());
+        System.clearProperty(x509Util.getCipherSuitesProperty());
+        System.clearProperty("com.sun.net.ssl.checkRevocation");
+        System.clearProperty("com.sun.security.enableCRLDP");
+        Security.setProperty("com.sun.security.enableCRLDP", "false");
+    }
+
+    @Test(timeout = 5000)
+    public void testCreateSSLContextWithoutCustomProtocol() throws Exception {
+        SSLContext sslContext = x509Util.getDefaultSSLContext();
+        Assert.assertEquals(X509Util.DEFAULT_PROTOCOL, sslContext.getProtocol());
+    }
+
+    @Test(timeout = 5000)
+    public void testCreateSSLContextWithCustomProtocol() throws Exception {
+        final String protocol = "TLSv1.1";
+        System.setProperty(x509Util.getSslProtocolProperty(), protocol);
+        SSLContext sslContext = x509Util.getDefaultSSLContext();
+        Assert.assertEquals(protocol, sslContext.getProtocol());
+    }
+
+    @Test(timeout = 5000)
+    public void testCreateSSLContextWithoutTrustStorePassword() throws Exception {
+        writeTrustStore(null);
+        System.clearProperty(x509Util.getSslTruststorePasswdProperty());
+        x509Util.getDefaultSSLContext();
+    }
+
+    @Test(timeout = 5000, expected = X509Exception.SSLContextException.class)
+    public void testCreateSSLContextWithoutKeyStoreLocation() throws Exception {
+        System.clearProperty(x509Util.getSslKeystoreLocationProperty());
+        x509Util.getDefaultSSLContext();
+    }
+
+    @Test(timeout = 5000, expected = X509Exception.SSLContextException.class)
+    public void testCreateSSLContextWithoutKeyStorePassword() throws Exception {
+        System.clearProperty(x509Util.getSslKeystorePasswdProperty());
+        x509Util.getDefaultSSLContext();
+    }
+
+    @Test(timeout = 5000)
+    public void testCreateSSLContextWithCustomCipherSuites() throws Exception {
+        setCustomCipherSuites();
+        SSLSocket sslSocket = x509Util.createSSLSocket();
+        Assert.assertArrayEquals(customCipherSuites, sslSocket.getEnabledCipherSuites());
+    }
+
+    // It would be great to test the value of PKIXBuilderParameters#setRevocationEnabled but it does not appear to be
+    // possible
+    @Test(timeout = 5000)
+    public void testCRLEnabled() throws Exception {
+        System.setProperty(x509Util.getSslCrlEnabledProperty(), "true");
+        x509Util.getDefaultSSLContext();
+        Assert.assertTrue(Boolean.valueOf(System.getProperty("com.sun.net.ssl.checkRevocation")));
+        Assert.assertTrue(Boolean.valueOf(System.getProperty("com.sun.security.enableCRLDP")));
+        Assert.assertFalse(Boolean.valueOf(Security.getProperty("ocsp.enable")));
+    }
+
+    @Test(timeout = 5000)
+    public void testCRLDisabled() throws Exception {
+        x509Util.getDefaultSSLContext();
+        Assert.assertFalse(Boolean.valueOf(System.getProperty("com.sun.net.ssl.checkRevocation")));
+        Assert.assertFalse(Boolean.valueOf(System.getProperty("com.sun.security.enableCRLDP")));
+        Assert.assertFalse(Boolean.valueOf(Security.getProperty("ocsp.enable")));
+    }
+
+    @Test(timeout = 5000)
+    public void testOCSPEnabled() throws Exception {
+        System.setProperty(x509Util.getSslOcspEnabledProperty(), "true");
+        x509Util.getDefaultSSLContext();
+        Assert.assertTrue(Boolean.valueOf(System.getProperty("com.sun.net.ssl.checkRevocation")));
+        Assert.assertTrue(Boolean.valueOf(System.getProperty("com.sun.security.enableCRLDP")));
+        Assert.assertTrue(Boolean.valueOf(Security.getProperty("ocsp.enable")));
+    }
+
+    @Test(timeout = 5000)
+    public void testCreateSSLSocket() throws Exception {
+        setCustomCipherSuites();
+        SSLSocket sslSocket = x509Util.createSSLSocket();
+        Assert.assertArrayEquals(customCipherSuites, sslSocket.getEnabledCipherSuites());
+    }
+
+    @Test(timeout = 5000)
+    public void testCreateSSLServerSocketWithoutPort() throws Exception {
+        setCustomCipherSuites();
+        SSLServerSocket sslServerSocket = x509Util.createSSLServerSocket();
+        Assert.assertArrayEquals(customCipherSuites, sslServerSocket.getEnabledCipherSuites());
+        Assert.assertTrue(sslServerSocket.getNeedClientAuth());
+    }
+
+    @Test(timeout = 5000)
+    public void testCreateSSLServerSocketWithPort() throws Exception {
+        int port = PortAssignment.unique();
+        setCustomCipherSuites();
+        SSLServerSocket sslServerSocket = x509Util.createSSLServerSocket(port);
+        Assert.assertEquals(sslServerSocket.getLocalPort(), port);
+        Assert.assertArrayEquals(customCipherSuites, sslServerSocket.getEnabledCipherSuites());
+        Assert.assertTrue(sslServerSocket.getNeedClientAuth());
+    }
+
+    // Warning: this will reset the x509Util
+    private void setCustomCipherSuites() {
+        System.setProperty(x509Util.getCipherSuitesProperty(), customCipherSuites[0] + "," + customCipherSuites[1]);
+        x509Util = new ClientX509Util();
+    }
+}

+ 248 - 0
zookeeper-common/src/test/java/org/apache/zookeeper/common/ZKTrustManagerTest.java

@@ -0,0 +1,248 @@
+/**
+ * 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 org.apache.zookeeper.ZKTestCase;
+import org.bouncycastle.asn1.x500.X500NameBuilder;
+import org.bouncycastle.asn1.x500.style.BCStyle;
+import org.bouncycastle.asn1.x509.BasicConstraints;
+import org.bouncycastle.asn1.x509.Extension;
+import org.bouncycastle.asn1.x509.GeneralName;
+import org.bouncycastle.asn1.x509.GeneralNames;
+import org.bouncycastle.asn1.x509.KeyUsage;
+import org.bouncycastle.cert.X509v3CertificateBuilder;
+import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter;
+import org.bouncycastle.cert.jcajce.JcaX509v3CertificateBuilder;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.bouncycastle.operator.ContentSigner;
+import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import javax.net.ssl.X509ExtendedTrustManager;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+import java.security.Security;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+import java.util.Random;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+// We can only test calls to ZKTrustManager using Sockets (not SSLEngines). This can be fine since the logic is the same.
+public class ZKTrustManagerTest extends ZKTestCase {
+
+    private static KeyPair keyPair;
+
+    private X509ExtendedTrustManager mockX509ExtendedTrustManager;
+    private static final String IP_ADDRESS = "127.0.0.1";
+    private static final String HOSTNAME = "localhost";
+
+    private InetAddress mockInetAddress;
+    private Socket mockSocket;
+
+    @BeforeClass
+    public static void createKeyPair() throws Exception {
+        Security.addProvider(new BouncyCastleProvider());
+        KeyPairGenerator keyPairGenerator = KeyPairGenerator.getInstance("RSA", BouncyCastleProvider.PROVIDER_NAME);
+        keyPairGenerator.initialize(4096);
+        keyPair = keyPairGenerator.genKeyPair();
+    }
+
+    @AfterClass
+    public static void removeBouncyCastleProvider() throws Exception {
+        Security.removeProvider("BC");
+    }
+
+    @Before
+    public void setup() throws Exception {
+        mockX509ExtendedTrustManager = mock(X509ExtendedTrustManager.class);
+
+        mockInetAddress = mock(InetAddress.class);
+        when(mockInetAddress.getHostAddress()).thenAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+                return IP_ADDRESS;
+            }
+        });
+
+        when(mockInetAddress.getHostName()).thenAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+                return HOSTNAME;
+            }
+        });
+
+        mockSocket = mock(Socket.class);
+        when(mockSocket.getInetAddress()).thenAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+                return mockInetAddress;
+            }
+        });
+    }
+
+    private X509Certificate[] createSelfSignedCertifcateChain(String ipAddress, String hostname) throws Exception {
+        X500NameBuilder nameBuilder = new X500NameBuilder(BCStyle.INSTANCE);
+        nameBuilder.addRDN(BCStyle.CN, "NOT_LOCALHOST");
+        Date notBefore = new Date();
+        Calendar cal = Calendar.getInstance();
+        cal.setTime(notBefore);
+        cal.add(Calendar.YEAR, 1);
+        Date notAfter = cal.getTime();
+        BigInteger serialNumber = new BigInteger(128, new Random());
+
+        X509v3CertificateBuilder certificateBuilder =
+                new JcaX509v3CertificateBuilder(nameBuilder.build(), serialNumber, notBefore, notAfter, nameBuilder.build(), keyPair.getPublic())
+                        .addExtension(Extension.basicConstraints, true, new BasicConstraints(0))
+                        .addExtension(Extension.keyUsage, true, new KeyUsage(KeyUsage.digitalSignature | KeyUsage.keyCertSign | KeyUsage.cRLSign));
+
+        List<GeneralName> generalNames = new ArrayList<>();
+        if (ipAddress != null) {
+            generalNames.add(new GeneralName(GeneralName.iPAddress, ipAddress));
+        }
+        if (hostname != null) {
+            generalNames.add(new GeneralName(GeneralName.dNSName, hostname));
+        }
+
+        if (!generalNames.isEmpty()) {
+            certificateBuilder.addExtension(Extension.subjectAlternativeName,  true,  new GeneralNames(generalNames.toArray(new GeneralName[] {})));
+        }
+
+        ContentSigner contentSigner = new JcaContentSignerBuilder("SHA256WithRSAEncryption").build(keyPair.getPrivate());
+
+        return new X509Certificate[] { new JcaX509CertificateConverter().getCertificate(certificateBuilder.build(contentSigner)) };
+    }
+
+    @Test
+    public void testServerHostnameVerificationWithHostnameVerificationDisabled() throws Exception {
+        ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, false, false);
+
+        X509Certificate[] certificateChain = createSelfSignedCertifcateChain(IP_ADDRESS, HOSTNAME);
+        zkTrustManager.checkServerTrusted(certificateChain, null, mockSocket);
+
+        verify(mockInetAddress, times(0)).getHostAddress();
+        verify(mockInetAddress, times(0)).getHostName();
+
+        verify(mockX509ExtendedTrustManager, times(1)).checkServerTrusted(certificateChain, null, mockSocket);
+    }
+
+    @Test
+    public void testServerHostnameVerificationWithHostnameVerificationDisabledAndClientHostnameVerificationEnabled() throws Exception {
+        ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, false, true);
+
+        X509Certificate[] certificateChain = createSelfSignedCertifcateChain(IP_ADDRESS, HOSTNAME);
+        zkTrustManager.checkServerTrusted(certificateChain, null, mockSocket);
+
+        verify(mockInetAddress, times(0)).getHostAddress();
+        verify(mockInetAddress, times(0)).getHostName();
+
+        verify(mockX509ExtendedTrustManager, times(1)).checkServerTrusted(certificateChain, null, mockSocket);
+    }
+
+    @Test
+    public void testServerHostnameVerificationWithIPAddress() throws Exception {
+        ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true, false);
+
+        X509Certificate[] certificateChain = createSelfSignedCertifcateChain(IP_ADDRESS, null);
+        zkTrustManager.checkServerTrusted(certificateChain, null, mockSocket);
+
+        verify(mockInetAddress, times(1)).getHostAddress();
+        verify(mockInetAddress, times(0)).getHostName();
+
+        verify(mockX509ExtendedTrustManager, times(1)).checkServerTrusted(certificateChain, null, mockSocket);
+    }
+
+    @Test
+    public void testServerHostnameVerificationWithHostname() throws Exception {
+        ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true, false);
+
+        X509Certificate[] certificateChain = createSelfSignedCertifcateChain(null, HOSTNAME);
+        zkTrustManager.checkServerTrusted(certificateChain, null, mockSocket);
+
+        verify(mockInetAddress, times(1)).getHostAddress();
+        verify(mockInetAddress, times(1)).getHostName();
+
+        verify(mockX509ExtendedTrustManager, times(1)).checkServerTrusted(certificateChain, null, mockSocket);
+    }
+
+    @Test
+    public void testClientHostnameVerificationWithHostnameVerificationDisabled() throws Exception {
+        ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, false, true);
+
+        X509Certificate[] certificateChain = createSelfSignedCertifcateChain(null, HOSTNAME);
+        zkTrustManager.checkClientTrusted(certificateChain, null, mockSocket);
+
+        verify(mockInetAddress, times(1)).getHostAddress();
+        verify(mockInetAddress, times(1)).getHostName();
+
+        verify(mockX509ExtendedTrustManager, times(1)).checkClientTrusted(certificateChain, null, mockSocket);
+    }
+
+    @Test
+    public void testClientHostnameVerificationWithClientHostnameVerificationDisabled() throws Exception {
+        ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true, false);
+
+        X509Certificate[] certificateChain = createSelfSignedCertifcateChain(null, HOSTNAME);
+        zkTrustManager.checkClientTrusted(certificateChain, null, mockSocket);
+
+        verify(mockInetAddress, times(0)).getHostAddress();
+        verify(mockInetAddress, times(0)).getHostName();
+
+        verify(mockX509ExtendedTrustManager, times(1)).checkClientTrusted(certificateChain, null, mockSocket);
+    }
+
+    @Test
+    public void testClientHostnameVerificationWithIPAddress() throws Exception {
+        ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true, true);
+
+        X509Certificate[] certificateChain = createSelfSignedCertifcateChain(IP_ADDRESS, null);
+        zkTrustManager.checkClientTrusted(certificateChain, null, mockSocket);
+
+        verify(mockInetAddress, times(1)).getHostAddress();
+        verify(mockInetAddress, times(0)).getHostName();
+
+        verify(mockX509ExtendedTrustManager, times(1)).checkClientTrusted(certificateChain, null, mockSocket);
+    }
+
+    @Test
+    public void testClientHostnameVerificationWithHostname() throws Exception {
+        ZKTrustManager zkTrustManager = new ZKTrustManager(mockX509ExtendedTrustManager, true, true);
+
+        X509Certificate[] certificateChain = createSelfSignedCertifcateChain(null, HOSTNAME);
+        zkTrustManager.checkClientTrusted(certificateChain, null, mockSocket);
+
+        verify(mockInetAddress, times(1)).getHostAddress();
+        verify(mockInetAddress, times(1)).getHostName();
+
+        verify(mockX509ExtendedTrustManager, times(1)).checkClientTrusted(certificateChain, null, mockSocket);
+    }
+}

+ 132 - 0
zookeeper-common/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java

@@ -0,0 +1,132 @@
+/**
+ * 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.test;
+
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.common.ClientX509Util;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.quorum.QuorumPeerTestBase;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ClientSSLTest extends QuorumPeerTestBase {
+
+    private ClientX509Util clientX509Util = new ClientX509Util();
+
+    @Before
+    public void setup() {
+        String testDataPath = System.getProperty("test.data.dir", "build/test/data");
+        System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory");
+        System.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, "org.apache.zookeeper.ClientCnxnSocketNetty");
+        System.setProperty(ZKClientConfig.SECURE_CLIENT, "true");
+        System.setProperty(clientX509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
+        System.setProperty(clientX509Util.getSslKeystorePasswdProperty(), "testpass");
+        System.setProperty(clientX509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
+        System.setProperty(clientX509Util.getSslTruststorePasswdProperty(), "testpass");
+    }
+
+    @After
+    public void teardown() throws Exception {
+        System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
+        System.clearProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET);
+        System.clearProperty(ZKClientConfig.SECURE_CLIENT);
+        System.clearProperty(clientX509Util.getSslKeystoreLocationProperty());
+        System.clearProperty(clientX509Util.getSslKeystorePasswdProperty());
+        System.clearProperty(clientX509Util.getSslTruststoreLocationProperty());
+        System.clearProperty(clientX509Util.getSslTruststorePasswdProperty());
+    }
+
+    /**
+     * This test checks that client <-> server SSL works in cluster setup of ZK servers, which includes:
+     * 1. setting "secureClientPort" in "zoo.cfg" file.
+     * 2. setting jvm flags for serverCnxn, keystore, truststore.
+     * Finally, a zookeeper client should be able to connect to the secure port and
+     * communicate with server via secure connection.
+     * <p/>
+     * Note that in this test a ZK server has two ports -- clientPort and secureClientPort.
+     */
+    @Test
+    public void testClientServerSSL() throws Exception {
+        final int SERVER_COUNT = 3;
+        final int clientPorts[] = new int[SERVER_COUNT];
+        final Integer secureClientPorts[] = new Integer[SERVER_COUNT];
+        StringBuilder sb = new StringBuilder();
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            clientPorts[i] = PortAssignment.unique();
+            secureClientPorts[i] = PortAssignment.unique();
+            String server = String.format("server.%d=localhost:%d:%d:participant;localhost:%d",
+                    i, PortAssignment.unique(), PortAssignment.unique(), clientPorts[i]);
+            sb.append(server + "\n");
+        }
+        String quorumCfg = sb.toString();
+
+
+        MainThread[] mt = new MainThread[SERVER_COUNT];
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            mt[i] = new MainThread(i, quorumCfg, secureClientPorts[i], true);
+            mt[i].start();
+        }
+
+        // Servers have been set up. Now go test if secure connection is successful.
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            Assert.assertTrue("waiting for server " + i + " being up",
+                    ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], TIMEOUT));
+
+            ZooKeeper zk = ClientBase.createZKClient("127.0.0.1:" + secureClientPorts[i], TIMEOUT);
+            // Do a simple operation to make sure the connection is fine.
+            zk.create("/test", "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            zk.delete("/test", -1);
+            zk.close();
+        }
+
+        for (int i = 0; i < mt.length; i++) {
+            mt[i].shutdown();
+        }
+    }
+
+
+    /**
+     * Developers might use standalone mode (which is the default for one server).
+     * This test checks SSL works in standalone mode of ZK server.
+     * <p/>
+     * Note that in this test the Zk server has only secureClientPort
+     */
+    @Test
+    public void testSecureStandaloneServer() throws Exception {
+        Integer secureClientPort = PortAssignment.unique();
+        MainThread mt = new MainThread(MainThread.UNSET_MYID, "", secureClientPort, false);
+        mt.start();
+
+        ZooKeeper zk = ClientBase.createZKClient("127.0.0.1:" + secureClientPort, TIMEOUT);
+        zk.create("/test", "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zk.delete("/test", -1);
+        zk.close();
+        mt.shutdown();
+    }
+}

+ 22 - 18
zookeeper-common/src/test/java/org/apache/zookeeper/test/SSLAuthTest.java

@@ -23,7 +23,8 @@ import java.util.concurrent.TimeUnit;
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.TestableZooKeeper;
 import org.apache.zookeeper.client.ZKClientConfig;
-import org.apache.zookeeper.common.ZKConfig;
+import org.apache.zookeeper.common.ClientX509Util;
+import org.apache.zookeeper.common.X509Util;
 import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.junit.After;
 import org.junit.Assert;
@@ -31,17 +32,20 @@ import org.junit.Before;
 import org.junit.Test;
 
 public class SSLAuthTest extends ClientBase {
+    
+    private ClientX509Util clientX509Util = new ClientX509Util();
+    
     @Before
     public void setUp() throws Exception {
         String testDataPath = System.getProperty("test.data.dir", "build/test/data");
         System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory");
         System.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, "org.apache.zookeeper.ClientCnxnSocketNetty");
         System.setProperty(ZKClientConfig.SECURE_CLIENT, "true");
-        System.setProperty(ZKConfig.SSL_AUTHPROVIDER, "x509");
-        System.setProperty(ZKConfig.SSL_KEYSTORE_LOCATION, testDataPath + "/ssl/testKeyStore.jks");
-        System.setProperty(ZKConfig.SSL_KEYSTORE_PASSWD, "testpass");
-        System.setProperty(ZKConfig.SSL_TRUSTSTORE_LOCATION, testDataPath + "/ssl/testTrustStore.jks");
-        System.setProperty(ZKConfig.SSL_TRUSTSTORE_PASSWD, "testpass");
+        System.setProperty(clientX509Util.getSslAuthProviderProperty(), "x509");
+        System.setProperty(clientX509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
+        System.setProperty(clientX509Util.getSslKeystorePasswdProperty(), "testpass");
+        System.setProperty(clientX509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
+        System.setProperty(clientX509Util.getSslTruststorePasswdProperty(), "testpass");
         System.setProperty("javax.net.debug", "ssl");
         System.setProperty("zookeeper.authProvider.x509", "org.apache.zookeeper.server.auth.X509AuthenticationProvider");
 
@@ -60,11 +64,11 @@ public class SSLAuthTest extends ClientBase {
         System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
         System.clearProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET);
         System.clearProperty(ZKClientConfig.SECURE_CLIENT);
-        System.clearProperty(ZKConfig.SSL_AUTHPROVIDER);
-        System.clearProperty(ZKConfig.SSL_KEYSTORE_LOCATION);
-        System.clearProperty(ZKConfig.SSL_KEYSTORE_PASSWD);
-        System.clearProperty(ZKConfig.SSL_TRUSTSTORE_LOCATION);
-        System.clearProperty(ZKConfig.SSL_TRUSTSTORE_PASSWD);
+        System.clearProperty(clientX509Util.getSslAuthProviderProperty());
+        System.clearProperty(clientX509Util.getSslKeystoreLocationProperty());
+        System.clearProperty(clientX509Util.getSslKeystorePasswdProperty());
+        System.clearProperty(clientX509Util.getSslTruststoreLocationProperty());
+        System.clearProperty(clientX509Util.getSslTruststorePasswdProperty());
         System.clearProperty("javax.net.debug");
         System.clearProperty("zookeeper.authProvider.x509");
     }
@@ -74,8 +78,8 @@ public class SSLAuthTest extends ClientBase {
         String testDataPath = System.getProperty("test.data.dir", "build/test/data");
 
         // Replace trusted keys with a valid key that is not trusted by the server
-        System.setProperty(ZKConfig.SSL_KEYSTORE_LOCATION, testDataPath + "/ssl/testUntrustedKeyStore.jks");
-        System.setProperty(ZKConfig.SSL_KEYSTORE_PASSWD, "testpass");
+        System.setProperty(clientX509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testUntrustedKeyStore.jks");
+        System.setProperty(clientX509Util.getSslKeystorePasswdProperty(), "testpass");
 
         CountdownWatcher watcher = new CountdownWatcher();
 
@@ -87,11 +91,11 @@ public class SSLAuthTest extends ClientBase {
 
     @Test
     public void testMisconfiguration() throws Exception {
-        System.clearProperty(ZKConfig.SSL_AUTHPROVIDER);
-        System.clearProperty(ZKConfig.SSL_KEYSTORE_LOCATION);
-        System.clearProperty(ZKConfig.SSL_KEYSTORE_PASSWD);
-        System.clearProperty(ZKConfig.SSL_TRUSTSTORE_LOCATION);
-        System.clearProperty(ZKConfig.SSL_TRUSTSTORE_PASSWD);
+        System.clearProperty(clientX509Util.getSslAuthProviderProperty());
+        System.clearProperty(clientX509Util.getSslKeystoreLocationProperty());
+        System.clearProperty(clientX509Util.getSslKeystorePasswdProperty());
+        System.clearProperty(clientX509Util.getSslTruststoreLocationProperty());
+        System.clearProperty(clientX509Util.getSslTruststorePasswdProperty());
 
         CountdownWatcher watcher = new CountdownWatcher();
         new TestableZooKeeper(hostPort, CONNECTION_TIMEOUT, watcher);

+ 0 - 0
zookeeper-common/src/test/java/org/apache/zookeeper/test/SSLTest.java → zookeeper-common/src/test/java/org/apache/zookeeper/test/SSLTest.java.orig


+ 26 - 27
zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxnFactory.java

@@ -18,31 +18,10 @@
 
 package org.apache.zookeeper.server;
 
-import static org.jboss.netty.buffer.ChannelBuffers.dynamicBuffer;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.security.KeyManagementException;
-import java.security.NoSuchAlgorithmException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Executors;
-
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLEngine;
-import javax.net.ssl.SSLPeerUnverifiedException;
-import javax.net.ssl.SSLSession;
-import javax.net.ssl.X509KeyManager;
-import javax.net.ssl.X509TrustManager;
-
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.common.ZKConfig;
+import org.apache.zookeeper.common.ClientX509Util;
 import org.apache.zookeeper.common.X509Exception;
 import org.apache.zookeeper.common.X509Exception.SSLContextException;
-import org.apache.zookeeper.common.X509Util;
 import org.apache.zookeeper.server.auth.ProviderRegistry;
 import org.apache.zookeeper.server.auth.X509AuthenticationProvider;
 import org.jboss.netty.bootstrap.ServerBootstrap;
@@ -68,6 +47,25 @@ import org.jboss.netty.handler.ssl.SslHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLPeerUnverifiedException;
+import javax.net.ssl.SSLSession;
+import javax.net.ssl.X509KeyManager;
+import javax.net.ssl.X509TrustManager;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.security.KeyManagementException;
+import java.security.NoSuchAlgorithmException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+
+import static org.jboss.netty.buffer.ChannelBuffers.dynamicBuffer;
+
 public class NettyServerCnxnFactory extends ServerCnxnFactory {
     private static final Logger LOG = LoggerFactory.getLogger(NettyServerCnxnFactory.class);
 
@@ -78,6 +76,7 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory {
         new HashMap<InetAddress, Set<NettyServerCnxn>>( );
     InetSocketAddress localAddress;
     int maxClientCnxns = 60;
+    ClientX509Util x509Util;
 
     /**
      * This is an inner class since we need to extend SimpleChannelHandler, but
@@ -302,7 +301,7 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory {
                     cnxn.setClientCertificateChain(session.getPeerCertificates());
 
                     String authProviderProp
-                            = System.getProperty(ZKConfig.SSL_AUTHPROVIDER, "x509");
+                            = System.getProperty(x509Util.getSslAuthProviderProperty(), "x509");
 
                     X509AuthenticationProvider authProvider =
                             (X509AuthenticationProvider)
@@ -358,20 +357,20 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory {
                 return p;
             }
         });
+        x509Util = new ClientX509Util();
     }
 
     private synchronized void initSSL(ChannelPipeline p)
             throws X509Exception, KeyManagementException, NoSuchAlgorithmException {
-        String authProviderProp = System.getProperty(ZKConfig.SSL_AUTHPROVIDER);
+        String authProviderProp = System.getProperty(x509Util.getSslAuthProviderProperty());
         SSLContext sslContext;
         if (authProviderProp == null) {
-            sslContext = X509Util.createSSLContext();
+            sslContext = x509Util.getDefaultSSLContext();
         } else {
             sslContext = SSLContext.getInstance("TLSv1");
             X509AuthenticationProvider authProvider =
                     (X509AuthenticationProvider)ProviderRegistry.getProvider(
-                            System.getProperty(ZKConfig.SSL_AUTHPROVIDER,
-                                    "x509"));
+                            System.getProperty(x509Util.getSslAuthProviderProperty(), "x509"));
 
             if (authProvider == null)
             {

+ 21 - 17
zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/X509AuthenticationProvider.java

@@ -26,6 +26,7 @@ import javax.net.ssl.X509TrustManager;
 import javax.security.auth.x500.X500Principal;
 
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.common.ClientX509Util;
 import org.apache.zookeeper.common.ZKConfig;
 import org.apache.zookeeper.common.X509Exception;
 import org.apache.zookeeper.common.X509Exception.KeyManagerException;
@@ -66,47 +67,50 @@ public class X509AuthenticationProvider implements AuthenticationProvider {
      * <br/><code>zookeeper.ssl.trustStore.password</code>
      */
     public X509AuthenticationProvider() throws X509Exception {
-        String keyStoreLocationProp = System.getProperty(
-                ZKConfig.SSL_KEYSTORE_LOCATION);
-        String keyStorePasswordProp = System.getProperty(
-                ZKConfig.SSL_KEYSTORE_PASSWD);
+        ZKConfig config = new ZKConfig();
+        X509Util x509Util = new ClientX509Util();
+
+        String keyStoreLocation = config.getProperty(x509Util.getSslKeystoreLocationProperty());
+        String keyStorePassword = config.getProperty(x509Util.getSslKeystorePasswdProperty());
+
+        boolean crlEnabled = Boolean.parseBoolean(System.getProperty(x509Util.getSslCrlEnabledProperty()));
+        boolean ocspEnabled = Boolean.parseBoolean(System.getProperty(x509Util.getSslOcspEnabledProperty()));
+        boolean hostnameVerificationEnabled = Boolean.parseBoolean(System.getProperty(x509Util.getSslHostnameVerificationEnabledProperty()));
 
         X509KeyManager km = null;
         X509TrustManager tm = null;
-        if (keyStoreLocationProp == null && keyStorePasswordProp == null) {
+        if (keyStoreLocation == null && keyStorePassword == null) {
             LOG.warn("keystore not specified for client connection");
         } else {
-            if (keyStoreLocationProp == null) {
+            if (keyStoreLocation == null) {
                 throw new X509Exception("keystore location not specified for client connection");
             }
-            if (keyStorePasswordProp == null) {
+            if (keyStorePassword == null) {
                 throw new X509Exception("keystore password not specified for client connection");
             }
             try {
-                km = X509Util.createKeyManager(
-                        keyStoreLocationProp, keyStorePasswordProp);
+                km = X509Util.createKeyManager(keyStoreLocation, keyStorePassword);
             } catch (KeyManagerException e) {
                 LOG.error("Failed to create key manager", e);
             }
         }
         
-        String trustStoreLocationProp = System.getProperty(
-                ZKConfig.SSL_TRUSTSTORE_LOCATION);
-        String trustStorePasswordProp = System.getProperty(
-                ZKConfig.SSL_TRUSTSTORE_PASSWD);
+        String trustStoreLocation = config.getProperty(x509Util.getSslTruststoreLocationProperty());
+        String trustStorePassword = config.getProperty(x509Util.getSslTruststorePasswdProperty());
 
-        if (trustStoreLocationProp == null && trustStorePasswordProp == null) {
+        if (trustStoreLocation == null && trustStorePassword == null) {
             LOG.warn("Truststore not specified for client connection");
         } else {
-            if (trustStoreLocationProp == null) {
+            if (trustStoreLocation == null) {
                 throw new X509Exception("Truststore location not specified for client connection");
             }
-            if (trustStorePasswordProp == null) {
+            if (trustStorePassword == null) {
                 throw new X509Exception("Truststore password not specified for client connection");
             }
             try {
                 tm = X509Util.createTrustManager(
-                        trustStoreLocationProp, trustStorePasswordProp);
+                        trustStoreLocation, trustStorePassword, crlEnabled, ocspEnabled,
+                        hostnameVerificationEnabled, false);
             } catch (TrustManagerException e) {
                 LOG.error("Failed to create trust manager", e);
             }

+ 24 - 4
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java

@@ -42,7 +42,9 @@ import java.util.concurrent.ConcurrentMap;
 import javax.security.sasl.SaslException;
 
 import org.apache.zookeeper.ZooDefs.OpCode;
+import org.apache.zookeeper.common.QuorumX509Util;
 import org.apache.zookeeper.common.Time;
+import org.apache.zookeeper.common.X509Exception;
 import org.apache.zookeeper.server.FinalRequestProcessor;
 import org.apache.zookeeper.server.Request;
 import org.apache.zookeeper.server.RequestProcessor;
@@ -232,19 +234,36 @@ public class Leader {
 
     private final ServerSocket ss;
 
-    Leader(QuorumPeer self,LeaderZooKeeperServer zk) throws IOException {
+    Leader(QuorumPeer self,LeaderZooKeeperServer zk) throws IOException, X509Exception {
         this.self = self;
         this.proposalStats = new BufferStats();
         try {
-            if (self.getQuorumListenOnAllIPs()) {
-                ss = new ServerSocket(self.getQuorumAddress().getPort());
+            if (self.shouldUsePortUnification()) {
+                if (self.getQuorumListenOnAllIPs()) {
+                    ss = new UnifiedServerSocket(new QuorumX509Util(), self.getQuorumAddress().getPort());
+                } else {
+                    ss = new UnifiedServerSocket(new QuorumX509Util());
+                }
+            } else if (self.isSslQuorum()) {
+                if (self.getQuorumListenOnAllIPs()) {
+                    ss = new QuorumX509Util().createSSLServerSocket(self.getQuorumAddress().getPort());
+                } else {
+                    ss = new QuorumX509Util().createSSLServerSocket();
+                }
             } else {
-                ss = new ServerSocket();
+                if (self.getQuorumListenOnAllIPs()) {
+                    ss = new ServerSocket(self.getQuorumAddress().getPort());
+                } else {
+                    ss = new ServerSocket();
+                }
             }
             ss.setReuseAddress(true);
             if (!self.getQuorumListenOnAllIPs()) {
                 ss.bind(self.getQuorumAddress());
             }
+        } catch (X509Exception e) {
+            LOG.error("Failed to setup ssl server socket", e);
+            throw e;
         } catch (BindException e) {
             if (self.getQuorumListenOnAllIPs()) {
                 LOG.error("Couldn't bind to port " + self.getQuorumAddress().getPort(), e);
@@ -382,6 +401,7 @@ public class Leader {
                 while (!stop) {
                     try{
                         Socket s = ss.accept();
+
                         // start with the initLimit, once the ack is processed
                         // in LearnerHandler switch to the syncLimit
                         s.setSoTimeout(self.tickTime * self.initLimit);

+ 29 - 7
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java

@@ -38,6 +38,9 @@ import org.apache.jute.BinaryOutputArchive;
 import org.apache.jute.InputArchive;
 import org.apache.jute.OutputArchive;
 import org.apache.jute.Record;
+import org.apache.zookeeper.common.QuorumX509Util;
+import org.apache.zookeeper.common.X509Exception;
+import org.apache.zookeeper.common.X509Util;
 import org.apache.zookeeper.server.ExitCode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,6 +55,8 @@ import org.apache.zookeeper.server.util.ZxidUtils;
 import org.apache.zookeeper.txn.SetDataTxn;
 import org.apache.zookeeper.txn.TxnHeader;
 
+import javax.net.ssl.SSLSocket;
+
 /**
  * This class is the superclass of two of the three main actors in a ZK
  * ensemble: Followers and Observers. Both Followers and Observers share 
@@ -68,6 +73,8 @@ public class Learner {
     protected BufferedOutputStream bufferedOutput;
     
     protected Socket sock;
+
+    protected X509Util x509Util;
     
     /**
      * Socket getter
@@ -241,9 +248,8 @@ public class Learner {
      * @throws InterruptedException
      */
     protected void connectToLeader(InetSocketAddress addr, String hostname)
-    throws IOException, ConnectException, InterruptedException {
-        sock = new Socket();        
-        sock.setSoTimeout(self.tickTime * self.initLimit);
+            throws IOException, InterruptedException, X509Exception {
+        this.sock = createSocket();
 
         int initLimitTime = self.tickTime * self.initLimit;
         int remainingInitLimitTime = initLimitTime;
@@ -259,6 +265,9 @@ public class Learner {
                 }
 
                 sockConnect(sock, addr, Math.min(self.tickTime * self.syncLimit, remainingInitLimitTime));
+                if (self.isSslQuorum())  {
+                    ((SSLSocket) sock).startHandshake();
+                }
                 sock.setTcpNoDelay(nodelay);
                 break;
             } catch (IOException e) {
@@ -278,8 +287,7 @@ public class Learner {
                     LOG.warn("Unexpected exception, tries=" + tries +
                             ", remaining init limit=" + remainingInitLimitTime +
                             ", connecting to " + addr,e);
-                    sock = new Socket();
-                    sock.setSoTimeout(self.tickTime * self.initLimit);
+                    this.sock = createSocket();
                 }
             }
             Thread.sleep(1000);
@@ -291,8 +299,22 @@ public class Learner {
                 sock.getInputStream()));
         bufferedOutput = new BufferedOutputStream(sock.getOutputStream());
         leaderOs = BinaryOutputArchive.getArchive(bufferedOutput);
-    }   
-    
+    }
+
+    private Socket createSocket() throws X509Exception, IOException {
+        Socket sock;
+        if (self.isSslQuorum()) {
+            if (x509Util == null) {
+                x509Util = new QuorumX509Util();
+            }
+            sock = x509Util.createSSLSocket();
+        } else {
+            sock = new Socket();
+        }
+        sock.setSoTimeout(self.tickTime * self.initLimit);
+        return sock;
+    }
+
     /**
      * Once connected to the leader, perform the handshake protocol to
      * establish a following / observing connection. 

+ 49 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/PrependableSocket.java

@@ -0,0 +1,49 @@
+/**
+ * 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.server.quorum;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.SequenceInputStream;
+import java.net.Socket;
+import java.net.SocketImpl;
+
+public class PrependableSocket extends Socket {
+
+  private SequenceInputStream sequenceInputStream;
+
+  public PrependableSocket(SocketImpl base) throws IOException {
+    super(base);
+  }
+
+  @Override
+  public InputStream getInputStream() throws IOException {
+    if (sequenceInputStream == null) {
+      return super.getInputStream();
+    }
+
+    return sequenceInputStream;
+  }
+
+  public void prependToInputStream(byte[] bytes) throws IOException {
+    sequenceInputStream = new SequenceInputStream(new ByteArrayInputStream(bytes), getInputStream());
+  }
+
+}

+ 16 - 3
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumBean.java

@@ -29,16 +29,29 @@ public class QuorumBean implements QuorumMXBean, ZKMBeanInfo {
         this.peer = peer;
         name = "ReplicatedServer_id" + peer.getId();
     }
-    
+
+    @Override
     public String getName() {
         return name;
     }
-    
+
+    @Override
     public boolean isHidden() {
         return false;
     }
-    
+
+    @Override
     public int getQuorumSize() {
         return peer.getQuorumSize();
     }
+
+    @Override
+    public boolean isSslQuorum() {
+        return peer.isSslQuorum();
+    }
+
+    @Override
+    public boolean isPortUnification() {
+        return peer.shouldUsePortUnification();
+    }
 }

+ 55 - 28
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java

@@ -47,6 +47,9 @@ import java.util.NoSuchElementException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.zookeeper.common.QuorumX509Util;
+import org.apache.zookeeper.common.X509Exception;
+import org.apache.zookeeper.common.X509Util;
 import org.apache.zookeeper.server.ExitCode;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
 import org.apache.zookeeper.server.util.ConfigUtils;
@@ -57,6 +60,7 @@ import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.net.ssl.SSLSocket;
 import static org.apache.zookeeper.common.NetUtils.formatInetAddr;
 
 /**
@@ -172,6 +176,8 @@ public class QuorumCnxManager {
     private final boolean tcpKeepAlive = Boolean.getBoolean("zookeeper.tcpKeepAlive");
 
 
+    private X509Util x509Util;
+
     static public class Message {
         Message(ByteBuffer buffer, long sid) {
             this.buffer = buffer;
@@ -285,6 +291,8 @@ public class QuorumCnxManager {
         // Starts listener thread that waits for connection requests
         listener = new Listener();
         listener.setName("QuorumPeerListener");
+
+        x509Util = new QuorumX509Util();
     }
 
     private void initializeAuth(final long mySid,
@@ -548,14 +556,13 @@ public class QuorumCnxManager {
                 LOG.info("Setting arbitrary identifier to observer: " + sid);
             }
         } catch (IOException e) {
+            LOG.warn("Exception reading or writing challenge: {}", e);
             closeSocket(sock);
-            LOG.warn("Exception reading or writing challenge: {}", e.toString());
             return;
         }
 
         // do authenticating learner
         authServer.authenticate(sock, din);
-
         //If wins the challenge, then close the new connection.
         if (sid < self.getId()) {
             /*
@@ -646,11 +653,19 @@ public class QuorumCnxManager {
 
         Socket sock = null;
         try {
-             LOG.debug("Opening channel to server " + sid);
-             sock = new Socket();
-             setSockOpts(sock);
-             sock.connect(electionAddr, cnxTO);
-             LOG.debug("Connected to server " + sid);
+            LOG.debug("Opening channel to server " + sid);
+            if (self.isSslQuorum()) {
+                SSLSocket sslSock = x509Util.createSSLSocket();
+                setSockOpts(sslSock);
+                sslSock.connect(electionAddr, cnxTO);
+                sslSock.startHandshake();
+                sock = sslSock;
+            } else {
+                sock = new Socket();
+                setSockOpts(sock);
+                sock.connect(electionAddr, cnxTO);
+            }
+            LOG.debug("Connected to server " + sid);
             // Sends connection request asynchronously if the quorum
             // sasl authentication is enabled. This is required because
             // sasl server authentication process may take few seconds to
@@ -660,24 +675,28 @@ public class QuorumCnxManager {
             } else {
                 initiateConnection(sock, sid);
             }
-             return true;
-         } catch (UnresolvedAddressException e) {
-             // Sun doesn't include the address that causes this
-             // exception to be thrown, also UAE cannot be wrapped cleanly
-             // so we log the exception in order to capture this critical
-             // detail.
-             LOG.warn("Cannot open channel to " + sid
-                     + " at election address " + electionAddr, e);
-             closeSocket(sock);
-             throw e;
-         } catch (IOException e) {
-             LOG.warn("Cannot open channel to " + sid
-                     + " at election address " + electionAddr,
-                     e);
-             closeSocket(sock);
-             return false;
-         }
-
+            return true;
+        } catch (UnresolvedAddressException e) {
+            // Sun doesn't include the address that causes this
+            // exception to be thrown, also UAE cannot be wrapped cleanly
+            // so we log the exception in order to capture this critical
+            // detail.
+            LOG.warn("Cannot open channel to " + sid
+                    + " at election address " + electionAddr, e);
+            closeSocket(sock);
+            throw e;
+        } catch (X509Exception e) {
+            LOG.warn("Cannot open secure channel to " + sid
+                    + " at election address " + electionAddr, e);
+            closeSocket(sock);
+            return false;
+        } catch (IOException e) {
+            LOG.warn("Cannot open channel to " + sid
+                            + " at election address " + electionAddr,
+                    e);
+            closeSocket(sock);
+            return false;
+        }
     }
 
     /**
@@ -853,11 +872,19 @@ public class QuorumCnxManager {
             int numRetries = 0;
             InetSocketAddress addr;
             Socket client = null;
-            IOException exitException = null;
+            Exception exitException = null;
             while((!shutdown) && (numRetries < 3)){
                 try {
-                    ss = new ServerSocket();
+                    if (self.shouldUsePortUnification()) {
+                        ss = new UnifiedServerSocket(x509Util);
+                    } else if (self.isSslQuorum()) {
+                        ss = x509Util.createSSLServerSocket();
+                    } else {
+                        ss = new ServerSocket();
+                    }
+
                     ss.setReuseAddress(true);
+
                     if (self.getQuorumListenOnAllIPs()) {
                         int port = self.getElectionAddress().getPort();
                         addr = new InetSocketAddress(port);
@@ -893,7 +920,7 @@ public class QuorumCnxManager {
                                      + "see ZOOKEEPER-2836");
                         }
                     }
-                } catch (IOException e) {
+                } catch (IOException|X509Exception e) {
                     if (shutdown) {
                         break;
                     }

+ 10 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumMXBean.java

@@ -31,4 +31,14 @@ public interface QuorumMXBean {
      * @return configured number of peers in the quorum
      */
     public int getQuorumSize();
+
+    /**
+     * @return SSL communication between quorum members required
+     */
+    public boolean isSslQuorum();
+
+    /**
+     * @return SSL communication between quorum members enabled
+     */
+    public boolean isPortUnification();
 }

+ 22 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java

@@ -48,6 +48,7 @@ import org.apache.zookeeper.KeeperException.BadArgumentsException;
 import org.apache.zookeeper.common.AtomicFileWritingIdiom;
 import org.apache.zookeeper.common.AtomicFileWritingIdiom.WriterStatement;
 import org.apache.zookeeper.common.Time;
+import org.apache.zookeeper.common.X509Exception;
 import org.apache.zookeeper.jmx.MBeanRegistry;
 import org.apache.zookeeper.jmx.ZKMBeanInfo;
 import org.apache.zookeeper.server.ServerCnxnFactory;
@@ -240,7 +241,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
                 throw new ConfigException("Address unresolved: " + serverParts[0] + ":" + serverParts[1]);
             }
             try {
-                electionAddr = new InetSocketAddress(serverParts[0], 
+                electionAddr = new InetSocketAddress(serverParts[0],
                         Integer.parseInt(serverParts[2]));
             } catch (NumberFormatException e) {
                 throw new ConfigException("Address unresolved: " + serverParts[0] + ":" + serverParts[2]);
@@ -467,6 +468,17 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
         this.myid = id;
     }
 
+    private boolean sslQuorum;
+    private boolean shouldUsePortUnification;
+
+    public boolean isSslQuorum() {
+        return sslQuorum;
+    }
+
+    public boolean shouldUsePortUnification() {
+        return shouldUsePortUnification;
+    }
+
     /**
      * This is who I think the leader currently is.
      */
@@ -1009,7 +1021,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
         return new Follower(this, new FollowerZooKeeperServer(logFactory, this, this.zkDb));
     }
 
-    protected Leader makeLeader(FileTxnSnapLog logFactory) throws IOException {
+    protected Leader makeLeader(FileTxnSnapLog logFactory) throws IOException, X509Exception {
         return new Leader(this, new LeaderZooKeeperServer(logFactory, this, this.zkDb));
     }
 
@@ -1691,6 +1703,14 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
         this.rootMetricsContext = rootMetricsContext;
     }
 
+    public void setSslQuorum(boolean sslQuorum) {
+        this.sslQuorum = sslQuorum;
+    }
+
+    public void setUsePortUnification(boolean shouldUsePortUnification) {
+        this.shouldUsePortUnification = shouldUsePortUnification;
+    }
+
     private void startServerCnxnFactory() {
         if (cnxnFactory != null) {
             cnxnFactory.start();

+ 18 - 3
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java

@@ -37,8 +37,8 @@ import java.util.Properties;
 import java.util.Map.Entry;
 
 import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.common.ClientX509Util;
 import org.apache.zookeeper.common.StringUtils;
-import org.apache.zookeeper.common.ZKConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.MDC;
@@ -69,6 +69,8 @@ public class QuorumPeerConfig {
 
     protected InetSocketAddress clientPortAddress;
     protected InetSocketAddress secureClientPortAddress;
+    protected boolean sslQuorum = false;
+    protected boolean shouldUsePortUnification = false;
     protected File dataDir;
     protected File dataLogDir;
     protected String dynamicConfigFileStr = null;
@@ -311,6 +313,11 @@ public class QuorumPeerConfig {
                 } else {
                     throw new ConfigException("Invalid option " + value + " for reconfigEnabled flag. Choose 'true' or 'false.'");
                 }
+            } else if (key.equals("sslQuorum")){
+                sslQuorum = Boolean.parseBoolean(value);
+// TODO: UnifiedServerSocket is currently buggy, will be fixed when @ivmaykov's PRs are merged. Disable port unification until then.
+//            } else if (key.equals("portUnification")){
+//                shouldUsePortUnification = Boolean.parseBoolean(value);
             } else if ((key.startsWith("server.") || key.startsWith("group") || key.startsWith("weight")) && zkProp.containsKey("dynamicConfigFile")) {
                 throw new ConfigException("parameter: " + key + " must be in a separate dynamic config file");
             } else if (key.equals(QuorumAuth.QUORUM_SASL_AUTH_ENABLED)) {
@@ -445,14 +452,15 @@ public class QuorumPeerConfig {
      *             provider is not configured.
      */
     private void configureSSLAuth() throws ConfigException {
-        String sslAuthProp = "zookeeper.authProvider." + System.getProperty(ZKConfig.SSL_AUTHPROVIDER, "x509");
+        ClientX509Util clientX509Util = new ClientX509Util();
+        String sslAuthProp = "zookeeper.authProvider." + System.getProperty(clientX509Util.getSslAuthProviderProperty(), "x509");
         if (System.getProperty(sslAuthProp) == null) {
             if ("zookeeper.authProvider.x509".equals(sslAuthProp)) {
                 System.setProperty("zookeeper.authProvider.x509",
                         "org.apache.zookeeper.server.auth.X509AuthenticationProvider");
             } else {
                 throw new ConfigException("No auth provider configured for the SSL authentication scheme '"
-                        + System.getProperty(ZKConfig.SSL_AUTHPROVIDER) + "'.");
+                        + System.getProperty(clientX509Util.getSslAuthProviderProperty()) + "'.");
             }
         }
     }
@@ -759,6 +767,13 @@ public class QuorumPeerConfig {
     public boolean isLocalSessionsUpgradingEnabled() {
         return localSessionsUpgradingEnabled;
     }
+    public boolean isSslQuorum() {
+        return sslQuorum;
+    }
+
+    public boolean shouldUsePortUnification() {
+        return shouldUsePortUnification;
+    }
 
     public int getInitLimit() { return initLimit; }
     public int getSyncLimit() { return syncLimit; }

+ 2 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java

@@ -197,6 +197,8 @@ public class QuorumPeerMain {
           quorumPeer.initConfigInZKDatabase();
           quorumPeer.setCnxnFactory(cnxnFactory);
           quorumPeer.setSecureCnxnFactory(secureCnxnFactory);
+          quorumPeer.setSslQuorum(config.isSslQuorum());
+          quorumPeer.setUsePortUnification(config.shouldUsePortUnification());
           quorumPeer.setLearnerType(config.getPeerType());
           quorumPeer.setSyncEnabled(config.getSyncEnabled());
           quorumPeer.setQuorumListenOnAllIPs(config.getQuorumListenOnAllIPs());

+ 79 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/UnifiedServerSocket.java

@@ -0,0 +1,79 @@
+/**
+ * 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.server.quorum;
+
+import org.apache.zookeeper.common.X509Exception;
+import org.apache.zookeeper.common.X509Util;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.handler.ssl.SslHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.ssl.SSLSocket;
+import java.io.IOException;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.SocketException;
+
+public class UnifiedServerSocket extends ServerSocket {
+    private static final Logger LOG = LoggerFactory.getLogger(UnifiedServerSocket.class);
+
+    private X509Util x509Util;
+
+    public UnifiedServerSocket(X509Util x509Util) throws IOException {
+        super();
+        this.x509Util = x509Util;
+    }
+
+    public UnifiedServerSocket(X509Util x509Util, int port) throws IOException {
+        super(port);
+        this.x509Util = x509Util;
+    }
+
+    @Override
+    public Socket accept() throws IOException {
+        if (isClosed()) {
+            throw new SocketException("Socket is closed");
+        }
+        if (!isBound()) {
+            throw new SocketException("Socket is not bound yet");
+        }
+        final PrependableSocket prependableSocket = new PrependableSocket(null);
+        implAccept(prependableSocket);
+
+        byte[] litmus = new byte[5];
+        int bytesRead = prependableSocket.getInputStream().read(litmus, 0, 5);
+        prependableSocket.prependToInputStream(litmus);
+
+        if (bytesRead == 5 && SslHandler.isEncrypted(ChannelBuffers.wrappedBuffer(litmus))) {
+            LOG.info(getInetAddress() + " attempting to connect over ssl");
+            SSLSocket sslSocket;
+            try {
+                sslSocket = x509Util.createSSLSocket(prependableSocket);
+            } catch (X509Exception e) {
+                throw new IOException("failed to create SSL context", e);
+            }
+            sslSocket.setUseClientMode(false);
+            return sslSocket;
+        } else {
+            LOG.info(getInetAddress() + " attempting to connect without ssl");
+            return prependableSocket;
+        }
+    }
+}

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

@@ -20,6 +20,7 @@ package org.apache.zookeeper.server.quorum;
 
 import org.apache.jute.OutputArchive;
 import org.apache.jute.Record;
+import org.apache.zookeeper.common.X509Exception;
 import org.apache.zookeeper.server.Request;
 import org.apache.zookeeper.server.ZKDatabase;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
@@ -53,7 +54,7 @@ public class LeaderBeanTest {
     private QuorumPeer qp;
 
     @Before
-    public void setUp() throws IOException {
+    public void setUp() throws IOException, X509Exception {
         qp = new QuorumPeer();
         QuorumVerifier quorumVerifierMock = mock(QuorumVerifier.class);
         qp.setQuorumVerifier(quorumVerifierMock, false);

+ 75 - 0
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumBeanTest.java

@@ -0,0 +1,75 @@
+/**
+ * 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.quorum;
+
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class QuorumBeanTest {
+    @Test
+    public void testGetNameProperty() {
+        QuorumPeer qpMock = mock(QuorumPeer.class);
+        when(qpMock.getId()).thenReturn(1L);
+        QuorumBean qb = new QuorumBean(qpMock);
+
+        assertThat("getName property should return Bean name in the right format", qb.getName(), equalTo("ReplicatedServer_id1"));
+    }
+
+    @Test
+    public void testIsHiddenProperty() {
+        QuorumPeer qpMock = mock(QuorumPeer.class);
+        QuorumBean qb = new QuorumBean(qpMock);
+        assertThat("isHidden should return false", qb.isHidden(), equalTo(false));
+    }
+
+    @Test
+    public void testGetQuorumSizeProperty() {
+        QuorumPeer qpMock = mock(QuorumPeer.class);
+        QuorumBean qb = new QuorumBean(qpMock);
+
+        when(qpMock.getQuorumSize()).thenReturn(5);
+        assertThat("getQuorumSize property should return value of peet.getQuorumSize()", qb.getQuorumSize(), equalTo(5));
+    }
+
+    @Test
+    public void testSslQuorumProperty() {
+        QuorumPeer qpMock = mock(QuorumPeer.class);
+        QuorumBean qb = new QuorumBean(qpMock);
+
+        when(qpMock.isSslQuorum()).thenReturn(true);
+        assertThat("isSslQuorum property should return value of peer.isSslQuorum()", qb.isSslQuorum(), equalTo(true));
+        when(qpMock.isSslQuorum()).thenReturn(false);
+        assertThat("isSslQuorum property should return value of peer.isSslQuorum()", qb.isSslQuorum(), equalTo(false));
+    }
+
+    @Test
+    public void testPortUnificationProperty() {
+        QuorumPeer qpMock = mock(QuorumPeer.class);
+        QuorumBean qb = new QuorumBean(qpMock);
+
+        when(qpMock.shouldUsePortUnification()).thenReturn(true);
+        assertThat("isPortUnification property should return value of peer.shouldUsePortUnification()", qb.isPortUnification(), equalTo(true));
+        when(qpMock.shouldUsePortUnification()).thenReturn(false);
+        assertThat("isPortUnification property should return value of peer.shouldUsePortUnification()", qb.isPortUnification(), equalTo(false));
+    }
+}

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

@@ -27,7 +27,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.Properties;
 
-import org.apache.zookeeper.common.ZKConfig;
+import org.apache.zookeeper.common.ClientX509Util;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
 import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
 import org.junit.Test;
@@ -93,7 +93,7 @@ public class QuorumPeerConfigTest {
     @Test
     public void testCustomSSLAuth()
             throws IOException{
-        System.setProperty(ZKConfig.SSL_AUTHPROVIDER, "y509");
+        System.setProperty(new ClientX509Util().getSslAuthProviderProperty(), "y509");
         QuorumPeerConfig quorumPeerConfig = new QuorumPeerConfig();
         try {
             Properties zkProp = getDefaultZKProperties();

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

@@ -55,6 +55,7 @@ import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooKeeper.States;
 import org.apache.zookeeper.common.Time;
+import org.apache.zookeeper.common.X509Exception;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.metrics.BaseTestMetricsProvider;
 import org.apache.zookeeper.metrics.impl.NullMetricsProvider;
@@ -1801,7 +1802,7 @@ public class QuorumPeerMainTest extends QuorumPeerTestBase {
         }
 
         @Override
-        protected Leader makeLeader(FileTxnSnapLog logFactory) throws IOException {
+        protected Leader makeLeader(FileTxnSnapLog logFactory) throws IOException, X509Exception {
             return new Leader(this, new LeaderZooKeeperServer(logFactory,
                     this, this.getZkDb())) {
                 @Override

+ 771 - 0
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java

@@ -0,0 +1,771 @@
+/**
+ * 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.server.quorum;
+
+import com.sun.net.httpserver.Headers;
+import com.sun.net.httpserver.HttpHandler;
+import com.sun.net.httpserver.HttpServer;
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.common.QuorumX509Util;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.test.ClientBase;
+import org.bouncycastle.asn1.ocsp.OCSPResponse;
+import org.bouncycastle.asn1.ocsp.OCSPResponseStatus;
+import org.bouncycastle.asn1.x500.X500Name;
+import org.bouncycastle.asn1.x500.X500NameBuilder;
+import org.bouncycastle.asn1.x500.style.BCStyle;
+import org.bouncycastle.asn1.x509.AuthorityInformationAccess;
+import org.bouncycastle.asn1.x509.BasicConstraints;
+import org.bouncycastle.asn1.x509.CRLDistPoint;
+import org.bouncycastle.asn1.x509.CRLNumber;
+import org.bouncycastle.asn1.x509.CRLReason;
+import org.bouncycastle.asn1.x509.DistributionPoint;
+import org.bouncycastle.asn1.x509.DistributionPointName;
+import org.bouncycastle.asn1.x509.Extension;
+import org.bouncycastle.asn1.x509.GeneralName;
+import org.bouncycastle.asn1.x509.GeneralNames;
+import org.bouncycastle.asn1.x509.KeyUsage;
+import org.bouncycastle.asn1.x509.SubjectPublicKeyInfo;
+import org.bouncycastle.asn1.x509.X509ObjectIdentifiers;
+import org.bouncycastle.cert.X509CRLHolder;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.cert.X509ExtensionUtils;
+import org.bouncycastle.cert.X509v2CRLBuilder;
+import org.bouncycastle.cert.X509v3CertificateBuilder;
+import org.bouncycastle.cert.bc.BcX509ExtensionUtils;
+import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter;
+import org.bouncycastle.cert.jcajce.JcaX509CertificateHolder;
+import org.bouncycastle.cert.jcajce.JcaX509ExtensionUtils;
+import org.bouncycastle.cert.jcajce.JcaX509v2CRLBuilder;
+import org.bouncycastle.cert.jcajce.JcaX509v3CertificateBuilder;
+import org.bouncycastle.cert.ocsp.BasicOCSPResp;
+import org.bouncycastle.cert.ocsp.BasicOCSPRespBuilder;
+import org.bouncycastle.cert.ocsp.CertificateID;
+import org.bouncycastle.cert.ocsp.CertificateStatus;
+import org.bouncycastle.cert.ocsp.OCSPException;
+import org.bouncycastle.cert.ocsp.OCSPReq;
+import org.bouncycastle.cert.ocsp.OCSPResp;
+import org.bouncycastle.cert.ocsp.OCSPRespBuilder;
+import org.bouncycastle.cert.ocsp.Req;
+import org.bouncycastle.cert.ocsp.UnknownStatus;
+import org.bouncycastle.cert.ocsp.jcajce.JcaBasicOCSPRespBuilder;
+import org.bouncycastle.cert.ocsp.jcajce.JcaCertificateID;
+import org.bouncycastle.crypto.util.PublicKeyFactory;
+import org.bouncycastle.crypto.util.SubjectPublicKeyInfoFactory;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.bouncycastle.openssl.MiscPEMGenerator;
+import org.bouncycastle.operator.ContentSigner;
+import org.bouncycastle.operator.DigestCalculator;
+import org.bouncycastle.operator.OperatorException;
+import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder;
+import org.bouncycastle.operator.jcajce.JcaDigestCalculatorProviderBuilder;
+import org.bouncycastle.util.io.pem.PemWriter;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import javax.net.ssl.SSLServerSocketFactory;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.net.InetSocketAddress;
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.PrivateKey;
+import java.security.Security;
+import java.security.cert.Certificate;
+import java.security.cert.CertificateEncodingException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
+import static org.apache.zookeeper.test.ClientBase.createTmpDir;
+import static org.junit.Assert.fail;
+
+public class QuorumSSLTest extends QuorumPeerTestBase {
+
+    private static final String SSL_QUORUM_ENABLED = "sslQuorum=true\n";
+    private static final String PORT_UNIFICATION_ENABLED = "portUnification=true\n";
+    private static final String PORT_UNIFICATION_DISABLED = "portUnification=false\n";
+
+    private static final char[] PASSWORD = "testpass".toCharArray();
+    private static final String HOSTNAME = "localhost";
+
+    private QuorumX509Util quorumX509Util = new QuorumX509Util();
+
+    private MainThread q1;
+    private MainThread q2;
+    private MainThread q3;
+
+    private int clientPortQp1;
+    private int clientPortQp2;
+    private int clientPortQp3;
+
+    private String tmpDir;
+
+    private String quorumConfiguration;
+    private String validKeystorePath;
+    private String truststorePath;
+
+    private KeyPair rootKeyPair;
+    private X509Certificate rootCertificate;
+
+    private KeyPair defaultKeyPair;
+
+    private ContentSigner contentSigner;
+
+    private Date certStartTime;
+    private Date certEndTime;
+
+    @Rule
+    public Timeout timeout = Timeout.builder().withTimeout(5, TimeUnit.MINUTES)
+            .withLookingForStuckThread(true).build();
+
+    @Before
+    public void setup() throws Exception {
+        ClientBase.setupTestEnv();
+
+        tmpDir = createTmpDir().getAbsolutePath();
+
+        clientPortQp1 = PortAssignment.unique();
+        clientPortQp2 = PortAssignment.unique();
+        clientPortQp3 = PortAssignment.unique();
+
+        validKeystorePath = tmpDir + "/valid.jks";
+        truststorePath = tmpDir + "/truststore.jks";
+
+        quorumConfiguration = generateQuorumConfiguration();
+
+        Security.addProvider(new BouncyCastleProvider());
+
+        certStartTime = new Date();
+        Calendar cal = Calendar.getInstance();
+        cal.setTime(certStartTime);
+        cal.add(Calendar.YEAR, 1);
+        certEndTime = cal.getTime();
+
+        rootKeyPair = createKeyPair();
+        contentSigner = new JcaContentSignerBuilder("SHA256WithRSAEncryption").build(rootKeyPair.getPrivate());
+        rootCertificate = createSelfSignedCertifcate(rootKeyPair);
+
+        // Write the truststore
+        KeyStore trustStore = KeyStore.getInstance(KeyStore.getDefaultType());
+        trustStore.load(null, PASSWORD);
+        trustStore.setCertificateEntry(rootCertificate.getSubjectDN().toString(), rootCertificate);
+        FileOutputStream outputStream = new FileOutputStream(truststorePath);
+        trustStore.store(outputStream, PASSWORD);
+        outputStream.flush();
+        outputStream.close();
+
+        defaultKeyPair = createKeyPair();
+        X509Certificate validCertificate = buildEndEntityCert(defaultKeyPair, rootCertificate, rootKeyPair.getPrivate(),
+                HOSTNAME, "127.0.0.1", null, null);
+        writeKeystore(validCertificate, defaultKeyPair, validKeystorePath);
+
+        setSSLSystemProperties();
+    }
+
+    private void writeKeystore(X509Certificate certificate, KeyPair entityKeyPair, String path) throws Exception {
+        KeyStore keyStore = KeyStore.getInstance(KeyStore.getDefaultType());
+        keyStore.load(null, PASSWORD);
+        keyStore.setKeyEntry("alias", entityKeyPair.getPrivate(), PASSWORD, new Certificate[] { certificate });
+        FileOutputStream outputStream = new FileOutputStream(path);
+        keyStore.store(outputStream, PASSWORD);
+        outputStream.flush();
+        outputStream.close();
+    }
+
+    private class OCSPHandler implements HttpHandler {
+
+        private X509Certificate revokedCert;
+
+        // Builds an OCSPHandler that responds with a good status for all certificates
+        // except revokedCert.
+        public OCSPHandler(X509Certificate revokedCert) {
+            this.revokedCert = revokedCert;
+        }
+
+        @Override
+        public void handle(com.sun.net.httpserver.HttpExchange httpExchange) throws IOException {
+            byte[] responseBytes;
+            try {
+                InputStream request = httpExchange.getRequestBody();
+                byte[] requestBytes = new byte[10000];
+                request.read(requestBytes);
+
+                OCSPReq ocspRequest = new OCSPReq(requestBytes);
+                Req[] requestList = ocspRequest.getRequestList();
+
+                DigestCalculator digestCalculator = new JcaDigestCalculatorProviderBuilder().build()
+                        .get(CertificateID.HASH_SHA1);
+
+                BasicOCSPRespBuilder responseBuilder =
+                        new JcaBasicOCSPRespBuilder(rootKeyPair.getPublic(), digestCalculator);
+                for ( Req req : requestList ) {
+                    CertificateID certId = req.getCertID();
+                    CertificateID revokedCertId =
+                            new JcaCertificateID(digestCalculator, rootCertificate, revokedCert.getSerialNumber());
+                    CertificateStatus certificateStatus;
+                    if (revokedCertId.equals(certId)) {
+                        certificateStatus = new UnknownStatus();
+                    } else {
+                        certificateStatus = CertificateStatus.GOOD;
+                    }
+
+                    responseBuilder.addResponse(certId, certificateStatus,null);
+                }
+
+                X509CertificateHolder[] chain = new X509CertificateHolder[] {
+                        new JcaX509CertificateHolder(rootCertificate)
+                };
+                ContentSigner signer = new JcaContentSignerBuilder("SHA1withRSA").setProvider("BC")
+                        .build(rootKeyPair.getPrivate());
+                BasicOCSPResp ocspResponse = responseBuilder.build(signer, chain, Calendar.getInstance().getTime() );
+
+                responseBytes = new OCSPRespBuilder().build(OCSPRespBuilder.SUCCESSFUL, ocspResponse).getEncoded();
+            } catch (OperatorException | CertificateEncodingException | OCSPException exception) {
+                responseBytes = new OCSPResp(new OCSPResponse(
+                        new OCSPResponseStatus(OCSPRespBuilder.INTERNAL_ERROR), null)).getEncoded();
+            }
+
+            Headers rh = httpExchange.getResponseHeaders();
+            rh.set("Content-Type", "application/ocsp-response");
+            httpExchange.sendResponseHeaders(200, responseBytes.length);
+
+            OutputStream os = httpExchange.getResponseBody();
+            os.write(responseBytes);
+            os.close();
+        }
+    }
+
+    private X509Certificate createSelfSignedCertifcate(KeyPair keyPair) throws Exception {
+        X500NameBuilder nameBuilder = new X500NameBuilder(BCStyle.INSTANCE);
+        nameBuilder.addRDN(BCStyle.CN, HOSTNAME);
+        BigInteger serialNumber = new BigInteger(128, new Random());
+
+        X509v3CertificateBuilder certificateBuilder =
+                new JcaX509v3CertificateBuilder(nameBuilder.build(), serialNumber, certStartTime, certEndTime,
+                        nameBuilder.build(), keyPair.getPublic())
+                .addExtension(Extension.basicConstraints, true, new BasicConstraints(0))
+                .addExtension(Extension.keyUsage, true,
+                        new KeyUsage(KeyUsage.digitalSignature | KeyUsage.keyCertSign | KeyUsage.cRLSign));
+
+        return new JcaX509CertificateConverter().getCertificate(certificateBuilder.build(contentSigner));
+    }
+
+    private void buildCRL(X509Certificate x509Certificate, String crlPath) throws Exception {
+        X509v2CRLBuilder builder = new JcaX509v2CRLBuilder(x509Certificate.getIssuerX500Principal(), certStartTime);
+        builder.addCRLEntry(x509Certificate.getSerialNumber(), certStartTime, CRLReason.cACompromise);
+        builder.setNextUpdate(certEndTime);
+        builder.addExtension(Extension.authorityKeyIdentifier, false, new JcaX509ExtensionUtils()
+                .createAuthorityKeyIdentifier(rootCertificate));
+        builder.addExtension(Extension.cRLNumber, false, new CRLNumber(new BigInteger("1000")));
+
+        X509CRLHolder cRLHolder = builder.build(contentSigner);
+
+        PemWriter pemWriter = new PemWriter(new FileWriter(crlPath));
+        pemWriter.writeObject(new MiscPEMGenerator(cRLHolder));
+        pemWriter.flush();
+        pemWriter.close();
+    }
+
+    public X509Certificate buildEndEntityCert(KeyPair keyPair, X509Certificate caCert, PrivateKey caPrivateKey,
+                                              String hostname, String ipAddress, String crlPath, Integer ocspPort)
+            throws Exception {
+        X509CertificateHolder holder = new JcaX509CertificateHolder(caCert);
+        ContentSigner signer =new JcaContentSignerBuilder("SHA256WithRSAEncryption").build(caPrivateKey);
+
+        List<GeneralName> generalNames = new ArrayList<>();
+        if (hostname != null) {
+            generalNames.add(new GeneralName(GeneralName.dNSName, hostname));
+        }
+
+        if (ipAddress != null) {
+            generalNames.add(new GeneralName(GeneralName.iPAddress, ipAddress));
+        }
+
+        SubjectPublicKeyInfo entityKeyInfo =
+                SubjectPublicKeyInfoFactory.createSubjectPublicKeyInfo(PublicKeyFactory.createKey(keyPair.getPublic()
+                        .getEncoded()));
+        X509ExtensionUtils extensionUtils = new BcX509ExtensionUtils();
+        X509v3CertificateBuilder certificateBuilder =
+                new JcaX509v3CertificateBuilder(holder.getSubject(), new BigInteger(128, new Random()),
+                    certStartTime, certEndTime, new X500Name("CN=Test End Entity Certificate"), keyPair.getPublic())
+                .addExtension(Extension.authorityKeyIdentifier, false,
+                        extensionUtils.createAuthorityKeyIdentifier(holder))
+                .addExtension(Extension.subjectKeyIdentifier, false,
+                        extensionUtils.createSubjectKeyIdentifier(entityKeyInfo))
+                .addExtension(Extension.basicConstraints, true, new BasicConstraints(false))
+                .addExtension(Extension.keyUsage, true,
+                        new KeyUsage(KeyUsage.digitalSignature | KeyUsage.keyEncipherment));
+
+        if (!generalNames.isEmpty()) {
+            certificateBuilder.addExtension(Extension.subjectAlternativeName,  true,
+                    new GeneralNames(generalNames.toArray(new GeneralName[] {})));
+        }
+
+        if (crlPath != null) {
+            DistributionPointName distPointOne = new DistributionPointName(new GeneralNames(
+                    new GeneralName(GeneralName.uniformResourceIdentifier,"file://" + crlPath)));
+
+            certificateBuilder.addExtension(Extension.cRLDistributionPoints, false,
+                    new CRLDistPoint(new DistributionPoint[] {
+                            new DistributionPoint(distPointOne, null, null)
+                    }));
+        }
+
+        if (ocspPort != null) {
+            certificateBuilder.addExtension(Extension.authorityInfoAccess, false,
+                    new AuthorityInformationAccess(X509ObjectIdentifiers.ocspAccessMethod,
+                    new GeneralName(GeneralName.uniformResourceIdentifier, "http://" + hostname + ":" + ocspPort)));
+        }
+
+        return new JcaX509CertificateConverter().getCertificate(certificateBuilder.build(signer));
+    }
+
+
+    private KeyPair createKeyPair() throws NoSuchProviderException, NoSuchAlgorithmException {
+        KeyPairGenerator keyPairGenerator = KeyPairGenerator.getInstance("RSA",
+                BouncyCastleProvider.PROVIDER_NAME);
+        keyPairGenerator.initialize(4096);
+        KeyPair keyPair = keyPairGenerator.genKeyPair();
+        return keyPair;
+    }
+
+    private String generateQuorumConfiguration() {
+        int portQp1 = PortAssignment.unique();
+        int portQp2 = PortAssignment.unique();
+        int portQp3 = PortAssignment.unique();
+
+        int portLe1 = PortAssignment.unique();
+        int portLe2 = PortAssignment.unique();
+        int portLe3 = PortAssignment.unique();
+
+
+
+        return "server.1=127.0.0.1:" + (portQp1) + ":" + (portLe1) + ";" +  clientPortQp1 + "\n" +
+               "server.2=127.0.0.1:" + (portQp2) + ":" + (portLe2) + ";" + clientPortQp2 + "\n" +
+               "server.3=127.0.0.1:" + (portQp3) + ":" + (portLe3) + ";" + clientPortQp3;
+    }
+
+
+    public void setSSLSystemProperties() {
+        System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
+                "org.apache.zookeeper.server.NettyServerCnxnFactory");
+        System.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, "org.apache.zookeeper.ClientCnxnSocketNetty");
+        System.setProperty(quorumX509Util.getSslKeystoreLocationProperty(), validKeystorePath);
+        System.setProperty(quorumX509Util.getSslKeystorePasswdProperty(), "testpass");
+        System.setProperty(quorumX509Util.getSslTruststoreLocationProperty(), truststorePath);
+        System.setProperty(quorumX509Util.getSslTruststorePasswdProperty(), "testpass");
+    }
+
+    @After
+    public void cleanUp() throws Exception {
+        clearSSLSystemProperties();
+        if (q1 != null) {
+            q1.shutdown();
+        }
+        if (q2 != null) {
+            q2.shutdown();
+        }
+        if (q3 != null) {
+            q3.shutdown();
+        }
+
+        Security.removeProvider("BC");
+    }
+
+    private void clearSSLSystemProperties() {
+        System.clearProperty(quorumX509Util.getSslKeystoreLocationProperty());
+        System.clearProperty(quorumX509Util.getSslKeystorePasswdProperty());
+        System.clearProperty(quorumX509Util.getSslTruststoreLocationProperty());
+        System.clearProperty(quorumX509Util.getSslTruststorePasswdProperty());
+        System.clearProperty(quorumX509Util.getSslHostnameVerificationEnabledProperty());
+        System.clearProperty(quorumX509Util.getSslOcspEnabledProperty());
+        System.clearProperty(quorumX509Util.getSslCrlEnabledProperty());
+        System.clearProperty(quorumX509Util.getCipherSuitesProperty());
+        System.clearProperty(quorumX509Util.getSslProtocolProperty());
+    }
+
+    @Test
+    public void testQuorumSSL() throws Exception {
+        q1 = new MainThread(1, clientPortQp1, quorumConfiguration, SSL_QUORUM_ENABLED);
+        q2 = new MainThread(2, clientPortQp2, quorumConfiguration, SSL_QUORUM_ENABLED);
+
+
+        q1.start();
+        q2.start();
+
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp1, CONNECTION_TIMEOUT));
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp2, CONNECTION_TIMEOUT));
+
+        clearSSLSystemProperties();
+
+        // This server should fail to join the quorum as it is not using ssl.
+        q3 = new MainThread(3, clientPortQp3, quorumConfiguration);
+        q3.start();
+
+        Assert.assertFalse(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
+    }
+
+    @Ignore("portUnification is currently broken and disabled")
+    @Test
+    public void testRollingUpgrade() throws Exception {
+        // Form a quorum without ssl
+        q1 = new MainThread(1, clientPortQp1, quorumConfiguration);
+        q2 = new MainThread(2, clientPortQp2, quorumConfiguration);
+        q3 = new MainThread(3, clientPortQp3, quorumConfiguration);
+
+
+        Map<Integer, MainThread> members = new HashMap<>();
+        members.put(clientPortQp1, q1);
+        members.put(clientPortQp2, q2);
+        members.put(clientPortQp3, q3);
+
+        for (MainThread member : members.values()) {
+            member.start();
+        }
+
+        for (int clientPort : members.keySet()) {
+            Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPort, CONNECTION_TIMEOUT));
+        }
+
+        // Set SSL system properties and port unification, begin restarting servers
+        setSSLSystemProperties();
+
+        stopAppendConfigRestartAll(members, PORT_UNIFICATION_ENABLED);
+        stopAppendConfigRestartAll(members, SSL_QUORUM_ENABLED);
+        stopAppendConfigRestartAll(members, PORT_UNIFICATION_DISABLED);
+    }
+
+    private void stopAppendConfigRestartAll(Map<Integer, MainThread> members, String config) throws Exception {
+        for (Map.Entry<Integer, MainThread> entry : members.entrySet()) {
+            int clientPort = entry.getKey();
+            MainThread member = entry.getValue();
+
+            member.shutdown();
+            Assert.assertTrue(ClientBase.waitForServerDown("127.0.0.1:" + clientPort, CONNECTION_TIMEOUT));
+
+            FileWriter fileWriter = new FileWriter(member.getConfFile(), true);
+            fileWriter.write(config);
+            fileWriter.flush();
+            fileWriter.close();
+
+            member.start();
+
+            Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPort, CONNECTION_TIMEOUT));
+        }
+    }
+
+    @Test
+    public void testHostnameVerificationWithInvalidHostname() throws Exception {
+        String badhostnameKeystorePath = tmpDir + "/badhost.jks";
+        X509Certificate badHostCert = buildEndEntityCert(defaultKeyPair, rootCertificate, rootKeyPair.getPrivate(),
+                "bleepbloop", null, null, null);
+        writeKeystore(badHostCert, defaultKeyPair, badhostnameKeystorePath);
+
+        testHostnameVerification(badhostnameKeystorePath, false);
+    }
+
+    @Test
+    public void testHostnameVerificationWithInvalidIPAddress() throws Exception {
+        String badhostnameKeystorePath = tmpDir + "/badhost.jks";
+        X509Certificate badHostCert = buildEndEntityCert(defaultKeyPair, rootCertificate, rootKeyPair.getPrivate(),
+                null, "140.211.11.105",null, null);
+        writeKeystore(badHostCert, defaultKeyPair, badhostnameKeystorePath);
+
+        testHostnameVerification(badhostnameKeystorePath, false);
+    }
+
+    @Test
+    public void testHostnameVerificationWithInvalidIpAddressAndInvalidHostname() throws Exception {
+        String badhostnameKeystorePath = tmpDir + "/badhost.jks";
+        X509Certificate badHostCert = buildEndEntityCert(defaultKeyPair, rootCertificate, rootKeyPair.getPrivate(),
+                "bleepbloop", "140.211.11.105", null, null);
+        writeKeystore(badHostCert, defaultKeyPair, badhostnameKeystorePath);
+
+        testHostnameVerification(badhostnameKeystorePath, false);
+    }
+
+    @Test
+    public void testHostnameVerificationWithInvalidIpAddressAndValidHostname() throws Exception {
+        String badhostnameKeystorePath = tmpDir + "/badhost.jks";
+        X509Certificate badHostCert = buildEndEntityCert(defaultKeyPair, rootCertificate, rootKeyPair.getPrivate(),
+                "localhost", "140.211.11.105", null, null);
+        writeKeystore(badHostCert, defaultKeyPair, badhostnameKeystorePath);
+
+        testHostnameVerification(badhostnameKeystorePath, true);
+    }
+
+    @Test
+    public void testHostnameVerificationWithValidIpAddressAndInvalidHostname() throws Exception {
+        String badhostnameKeystorePath = tmpDir + "/badhost.jks";
+        X509Certificate badHostCert = buildEndEntityCert(defaultKeyPair, rootCertificate, rootKeyPair.getPrivate(),
+                "bleepbloop", "127.0.0.1", null, null);
+        writeKeystore(badHostCert, defaultKeyPair, badhostnameKeystorePath);
+
+        testHostnameVerification(badhostnameKeystorePath, true);
+    }
+
+    /**
+     * @param keystorePath The keystore to use
+     * @param expectSuccess True for expecting the keystore to pass hostname verification, false for expecting failure
+     * @throws Exception
+     */
+    private void testHostnameVerification(String keystorePath, boolean expectSuccess) throws Exception {
+        System.setProperty(quorumX509Util.getSslHostnameVerificationEnabledProperty(), "false");
+
+        q1 = new MainThread(1, clientPortQp1, quorumConfiguration, SSL_QUORUM_ENABLED);
+        q2 = new MainThread(2, clientPortQp2, quorumConfiguration, SSL_QUORUM_ENABLED);
+
+        q1.start();
+        q2.start();
+
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp1, CONNECTION_TIMEOUT));
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp2, CONNECTION_TIMEOUT));
+
+        System.setProperty(quorumX509Util.getSslKeystoreLocationProperty(), keystorePath);
+
+        // This server should join successfully
+        q3 = new MainThread(3, clientPortQp3, quorumConfiguration, SSL_QUORUM_ENABLED);
+        q3.start();
+
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
+
+        q1.shutdown();
+        q2.shutdown();
+        q3.shutdown();
+
+        Assert.assertTrue(ClientBase.waitForServerDown("127.0.0.1:" + clientPortQp1, CONNECTION_TIMEOUT));
+        Assert.assertTrue(ClientBase.waitForServerDown("127.0.0.1:" + clientPortQp2, CONNECTION_TIMEOUT));
+        Assert.assertTrue(ClientBase.waitForServerDown("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
+
+        setSSLSystemProperties();
+        System.clearProperty(quorumX509Util.getSslHostnameVerificationEnabledProperty());
+
+        q1.start();
+        q2.start();
+
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp1, CONNECTION_TIMEOUT));
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp2, CONNECTION_TIMEOUT));
+
+        System.setProperty(quorumX509Util.getSslKeystoreLocationProperty(), keystorePath);
+        q3.start();
+
+        Assert.assertEquals(expectSuccess, ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3,
+                CONNECTION_TIMEOUT));
+    }
+
+
+    @Test
+    public void testCertificateRevocationList() throws Exception {
+        q1 = new MainThread(1, clientPortQp1, quorumConfiguration, SSL_QUORUM_ENABLED);
+        q2 = new MainThread(2, clientPortQp2, quorumConfiguration, SSL_QUORUM_ENABLED);
+
+        q1.start();
+        q2.start();
+
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp1, CONNECTION_TIMEOUT));
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp2, CONNECTION_TIMEOUT));
+
+        String revokedInCRLKeystorePath = tmpDir + "/crl_revoked.jks";
+        String crlPath = tmpDir + "/crl.pem";
+        X509Certificate revokedInCRLCert = buildEndEntityCert(defaultKeyPair, rootCertificate, rootKeyPair.getPrivate(),
+                HOSTNAME, null, crlPath, null);
+        writeKeystore(revokedInCRLCert, defaultKeyPair, revokedInCRLKeystorePath);
+        buildCRL(revokedInCRLCert, crlPath);
+
+        System.setProperty(quorumX509Util.getSslKeystoreLocationProperty(), revokedInCRLKeystorePath);
+
+        // This server should join successfully
+        q3 = new MainThread(3, clientPortQp3, quorumConfiguration, SSL_QUORUM_ENABLED);
+        q3.start();
+
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
+
+
+        q1.shutdown();
+        q2.shutdown();
+        q3.shutdown();
+
+        Assert.assertTrue(ClientBase.waitForServerDown("127.0.0.1:" + clientPortQp1, CONNECTION_TIMEOUT));
+        Assert.assertTrue(ClientBase.waitForServerDown("127.0.0.1:" + clientPortQp2, CONNECTION_TIMEOUT));
+        Assert.assertTrue(ClientBase.waitForServerDown("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
+
+        setSSLSystemProperties();
+        System.setProperty(quorumX509Util.getSslCrlEnabledProperty(), "true");
+
+        X509Certificate validCertificate = buildEndEntityCert(defaultKeyPair, rootCertificate, rootKeyPair.getPrivate(),
+                HOSTNAME, null, crlPath, null);
+        writeKeystore(validCertificate, defaultKeyPair, validKeystorePath);
+
+        q1.start();
+        q2.start();
+
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp1, CONNECTION_TIMEOUT));
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp2, CONNECTION_TIMEOUT));
+
+        System.setProperty(quorumX509Util.getSslKeystoreLocationProperty(), revokedInCRLKeystorePath);
+        q3.start();
+
+        Assert.assertFalse(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
+    }
+
+    @Test
+    public void testOCSP() throws Exception {
+        Integer ocspPort = PortAssignment.unique();
+
+        q1 = new MainThread(1, clientPortQp1, quorumConfiguration, SSL_QUORUM_ENABLED);
+        q2 = new MainThread(2, clientPortQp2, quorumConfiguration, SSL_QUORUM_ENABLED);
+
+        q1.start();
+        q2.start();
+
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp1, CONNECTION_TIMEOUT));
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp2, CONNECTION_TIMEOUT));
+
+        String revokedInOCSPKeystorePath = tmpDir + "/ocsp_revoked.jks";
+        X509Certificate revokedInOCSPCert = buildEndEntityCert(defaultKeyPair, rootCertificate, rootKeyPair.getPrivate(),
+                HOSTNAME, null,null, ocspPort);
+        writeKeystore(revokedInOCSPCert, defaultKeyPair, revokedInOCSPKeystorePath);
+
+        HttpServer ocspServer = HttpServer.create(new InetSocketAddress(ocspPort), 0);
+        try {
+            ocspServer.createContext("/", new OCSPHandler(revokedInOCSPCert));
+            ocspServer.start();
+
+            System.setProperty(quorumX509Util.getSslKeystoreLocationProperty(), revokedInOCSPKeystorePath);
+
+            // This server should join successfully
+            q3 = new MainThread(3, clientPortQp3, quorumConfiguration, SSL_QUORUM_ENABLED);
+            q3.start();
+
+            Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
+
+            q1.shutdown();
+            q2.shutdown();
+            q3.shutdown();
+
+            Assert.assertTrue(ClientBase.waitForServerDown("127.0.0.1:" + clientPortQp1, CONNECTION_TIMEOUT));
+            Assert.assertTrue(ClientBase.waitForServerDown("127.0.0.1:" + clientPortQp2, CONNECTION_TIMEOUT));
+            Assert.assertTrue(ClientBase.waitForServerDown("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
+
+            setSSLSystemProperties();
+            System.setProperty(quorumX509Util.getSslOcspEnabledProperty(), "true");
+
+            X509Certificate validCertificate = buildEndEntityCert(defaultKeyPair, rootCertificate,
+                    rootKeyPair.getPrivate(),
+                    HOSTNAME, null,null, ocspPort);
+            writeKeystore(validCertificate, defaultKeyPair, validKeystorePath);
+
+            q1.start();
+            q2.start();
+
+            Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp1, CONNECTION_TIMEOUT));
+            Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp2, CONNECTION_TIMEOUT));
+
+            System.setProperty(quorumX509Util.getSslKeystoreLocationProperty(), revokedInOCSPKeystorePath);
+            q3.start();
+
+            Assert.assertFalse(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
+        } finally {
+            ocspServer.stop(0);
+        }
+    }
+
+    @Test
+    public void testCipherSuites() throws Exception {
+        // Get default cipher suites from JDK
+        SSLServerSocketFactory ssf = (SSLServerSocketFactory)SSLServerSocketFactory.getDefault();
+        List<String> defaultCiphers = new ArrayList<String>();
+        for (String cipher : ssf.getDefaultCipherSuites()) {
+            if (!cipher.matches(".*EMPTY.*") && cipher.startsWith("TLS") && cipher.contains("RSA")) {
+                defaultCiphers.add(cipher);
+            }
+        }
+
+        if (defaultCiphers.size() < 2) {
+            fail("JDK has to support at least 2 valid (RSA) cipher suites for this test to run");
+        }
+
+        // Use them all except one to build the ensemble
+        String suitesOfEnsemble = String.join(",", defaultCiphers.subList(1, defaultCiphers.size()));
+        System.setProperty(quorumX509Util.getCipherSuitesProperty(), suitesOfEnsemble);
+
+        q1 = new MainThread(1, clientPortQp1, quorumConfiguration, SSL_QUORUM_ENABLED);
+        q2 = new MainThread(2, clientPortQp2, quorumConfiguration, SSL_QUORUM_ENABLED);
+
+        q1.start();
+        q2.start();
+
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp1, CONNECTION_TIMEOUT));
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp2, CONNECTION_TIMEOUT));
+
+        // Use the odd one out for the client
+        String suiteOfClient = defaultCiphers.get(0);
+        System.setProperty(quorumX509Util.getCipherSuitesProperty(), suiteOfClient);
+
+        // This server should fail to join the quorum as it is not using one of the supported suites from the other
+        // quorum members
+        q3 = new MainThread(3, clientPortQp3, quorumConfiguration, SSL_QUORUM_ENABLED);
+        q3.start();
+
+        Assert.assertFalse(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
+    }
+
+    @Test
+    public void testProtocolVersion() throws Exception {
+        System.setProperty(quorumX509Util.getSslProtocolProperty(), "TLSv1.2");
+
+        q1 = new MainThread(1, clientPortQp1, quorumConfiguration, SSL_QUORUM_ENABLED);
+        q2 = new MainThread(2, clientPortQp2, quorumConfiguration, SSL_QUORUM_ENABLED);
+
+        q1.start();
+        q2.start();
+
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp1, CONNECTION_TIMEOUT));
+        Assert.assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp2, CONNECTION_TIMEOUT));
+
+        System.setProperty(quorumX509Util.getSslProtocolProperty(), "TLSv1.1");
+
+        // This server should fail to join the quorum as it is not using TLSv1.2
+        q3 = new MainThread(3, clientPortQp3, quorumConfiguration, SSL_QUORUM_ENABLED);
+        q3.start();
+
+        Assert.assertFalse(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
+    }
+}

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

@@ -27,6 +27,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.common.X509Exception;
 import org.apache.zookeeper.server.FinalRequestProcessor;
 import org.apache.zookeeper.server.PrepRequestProcessor;
 import org.apache.zookeeper.server.Request;
@@ -182,7 +183,7 @@ public class RaceConditionTest extends QuorumPeerTestBase {
         }
 
         @Override
-        protected Leader makeLeader(FileTxnSnapLog logFactory) throws IOException {
+        protected Leader makeLeader(FileTxnSnapLog logFactory) throws IOException, X509Exception {
             LeaderZooKeeperServer zk = new LeaderZooKeeperServer(logFactory, this, this.getZkDb()) {
                 @Override
                 protected void setupRequestProcessors() {

+ 172 - 0
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/UnifiedServerSocketTest.java

@@ -0,0 +1,172 @@
+/**
+ * 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.server.quorum;
+
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.client.ZKClientConfig;
+import org.apache.zookeeper.common.ClientX509Util;
+import org.apache.zookeeper.common.Time;
+import org.apache.zookeeper.common.X509Util;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.net.ssl.HandshakeCompletedEvent;
+import javax.net.ssl.HandshakeCompletedListener;
+import javax.net.ssl.SSLSocket;
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertThat;
+
+public class UnifiedServerSocketTest {
+
+    private static final int MAX_RETRIES = 5;
+    private static final int TIMEOUT = 1000;
+
+    private X509Util x509Util;
+    private int port;
+    private volatile boolean handshakeCompleted;
+
+    @Before
+    public void setUp() throws Exception {
+        handshakeCompleted = false;
+
+        port = PortAssignment.unique();
+
+        String testDataPath = System.getProperty("test.data.dir", "build/test/data");
+        System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory");
+        System.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, "org.apache.zookeeper.ClientCnxnSocketNetty");
+        System.setProperty(ZKClientConfig.SECURE_CLIENT, "true");
+
+        x509Util = new ClientX509Util();
+
+        System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
+        System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass");
+        System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
+        System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass");
+        System.setProperty(x509Util.getSslHostnameVerificationEnabledProperty(), "false");
+    }
+
+    @Test
+    public void testConnectWithSSL() throws Exception {
+        class ServerThread extends Thread {
+            public void run() {
+                try {
+                    Socket unifiedSocket = new UnifiedServerSocket(x509Util, port).accept();
+                    ((SSLSocket)unifiedSocket).getSession(); // block until handshake completes
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+        ServerThread serverThread = new ServerThread();
+        serverThread.start();
+
+        SSLSocket sslSocket = null;
+        int retries = 0;
+        while (retries < MAX_RETRIES) {
+            try {
+                sslSocket = x509Util.createSSLSocket();
+                sslSocket.setSoTimeout(TIMEOUT);
+                sslSocket.connect(new InetSocketAddress(port), TIMEOUT);
+                break;
+            } catch (ConnectException connectException) {
+                connectException.printStackTrace();
+                Thread.sleep(TIMEOUT);
+            }
+            retries++;
+        }
+
+        sslSocket.addHandshakeCompletedListener(new HandshakeCompletedListener() {
+            @Override
+            public void handshakeCompleted(HandshakeCompletedEvent handshakeCompletedEvent) {
+                completeHandshake();
+            }
+        });
+        sslSocket.startHandshake();
+
+        serverThread.join(TIMEOUT);
+
+        long start = Time.currentElapsedTime();
+        while (Time.currentElapsedTime() < start + TIMEOUT) {
+            if (handshakeCompleted) {
+                return;
+            }
+        }
+
+        Assert.fail("failed to complete handshake");
+    }
+
+    private void completeHandshake() {
+        handshakeCompleted = true;
+    }
+
+    @Test
+    public void testConnectWithoutSSL() throws Exception {
+        final byte[] testData = "hello there".getBytes();
+        final String[] dataReadFromClient = {null};
+
+        class ServerThread extends Thread {
+            public void run() {
+                try {
+                    Socket unifiedSocket = new UnifiedServerSocket(x509Util, port).accept();
+                    unifiedSocket.getOutputStream().write(testData);
+                    unifiedSocket.getOutputStream().flush();
+                    byte[] inputbuff = new byte[5];
+                    unifiedSocket.getInputStream().read(inputbuff, 0, 5);
+                    dataReadFromClient[0] = new String(inputbuff);
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+        ServerThread serverThread = new ServerThread();
+        serverThread.start();
+
+        Socket socket = null;
+        int retries = 0;
+        while (retries < MAX_RETRIES) {
+            try {
+                socket = new Socket();
+                socket.setSoTimeout(TIMEOUT);
+                socket.connect(new InetSocketAddress(port), TIMEOUT);
+                break;
+            } catch (ConnectException connectException) {
+                connectException.printStackTrace();
+                Thread.sleep(TIMEOUT);
+            }
+            retries++;
+        }
+
+        socket.getOutputStream().write("hellobello".getBytes());
+        socket.getOutputStream().flush();
+
+        byte[] readBytes = new byte[testData.length];
+        socket.getInputStream().read(readBytes, 0, testData.length);
+
+        serverThread.join(TIMEOUT);
+
+        Assert.assertArrayEquals(testData, readBytes);
+        assertThat("Data sent by the client is invalid", dataReadFromClient[0], equalTo("hello"));
+    }
+}

+ 4 - 3
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ZabUtils.java

@@ -19,6 +19,7 @@
 package org.apache.zookeeper.server.quorum;
 
 import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.common.X509Exception;
 import org.apache.zookeeper.server.ServerCnxn;
 import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.ZKDatabase;
@@ -76,13 +77,13 @@ public class ZabUtils {
     }
 
     public static Leader createLeader(File tmpDir, QuorumPeer peer)
-            throws IOException, NoSuchFieldException, IllegalAccessException {
+            throws IOException, NoSuchFieldException, IllegalAccessException, X509Exception {
         LeaderZooKeeperServer zk = prepareLeader(tmpDir, peer);
         return new Leader(peer, zk);
     }
 
     public static Leader createMockLeader(File tmpDir, QuorumPeer peer)
-            throws IOException, NoSuchFieldException, IllegalAccessException {
+            throws IOException, NoSuchFieldException, IllegalAccessException, X509Exception {
         LeaderZooKeeperServer zk = prepareLeader(tmpDir, peer);
         return new MockLeader(peer, zk);
     }
@@ -148,7 +149,7 @@ public class ZabUtils {
     public static final class MockLeader extends Leader {
 
         MockLeader(QuorumPeer qp, LeaderZooKeeperServer zk)
-                throws IOException {
+                throws IOException, X509Exception {
             super(qp, zk);
         }