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

[zookeeper] branch master updated: ZOOKEEPER-3482: add unit tests for client SASL authentication over SSL

This is an automated email from the ASF dual-hosted git repository.

andor pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/zookeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new b7dd0e4  ZOOKEEPER-3482: add unit tests for client SASL authentication over SSL
b7dd0e4 is described below

commit b7dd0e49c5b25fffc1be8e619a3164bc680ef8bd
Author: Mate Szalay-Beko <sz...@gmail.com>
AuthorDate: Wed Jan 22 11:08:42 2020 +0100

    ZOOKEEPER-3482: add unit tests for client SASL authentication over SSL
    
    We had some issues about configuring Kerberos authentication with SSL, so I created some unit tests to verify that these features work together. The problem was originally reported on 3.5.5. There was some conflicts to backport the tests to 3.5, I will prepare a separate PR on that branch.
    
    Author: Mate Szalay-Beko <sz...@gmail.com>
    Author: Mate Szalay-Beko <ms...@cloudera.com>
    
    Reviewers: eolivelli@apache.org, andor@apache.org
    
    Closes #1204 from symat/ZOOKEEPER-3482-master and squashes the following commits:
    
    53194f564 [Mate Szalay-Beko] ZOOKEEPER-3482: update SASL related documentation
    576ac0b21 [Mate Szalay-Beko] Merge remote-tracking branch 'apache/master' into ZOOKEEPER-3482-master
    28da105c2 [Mate Szalay-Beko] ZOOKEEPER-3482: reload SASL and Kerberos configs before executing the tests
    396c73e8a [Mate Szalay-Beko] ZOOKEEPER-3482: add unit tests for client SASL authentication over SSL
---
 .../src/main/resources/markdown/zookeeperAdmin.md  |  27 ++-
 .../resources/markdown/zookeeperProgrammers.md     |  27 ++-
 .../server/quorum/auth/KerberosTestUtils.java      |   8 +
 .../server/quorum/auth/QuorumAuthTestBase.java     |   5 +
 .../server/quorum/auth/QuorumKerberosAuthTest.java |   6 +
 .../auth/QuorumKerberosHostBasedAuthTest.java      |   7 +
 .../zookeeper/test/SaslDigestAuthOverSSLTest.java  | 152 +++++++++++++
 .../test/SaslKerberosAuthOverSSLTest.java          | 246 +++++++++++++++++++++
 8 files changed, 468 insertions(+), 10 deletions(-)

diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
index 7e28cc7..3fa57ae 100644
--- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
+++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
@@ -1377,6 +1377,11 @@ As an example, this will enable all four letter word commands:
 The options in this section allow control over
 encryption/authentication/authorization performed by the service.
 
+Beside this page, you can also find useful information about client side configuration in the 
+[Programmers Guide](zookeeperProgrammers.html#sc_java_client_configuration). 
+The ZooKeeper Wiki also has useful pages about [ZooKeeper SSL support](https://cwiki.apache.org/confluence/display/ZOOKEEPER/ZooKeeper+SSL+User+Guide), 
+and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/display/ZOOKEEPER/ZooKeeper+and+SASL).
+
 * *DigestAuthenticationProvider.superDigest* :
     (Java system property: **zookeeper.DigestAuthenticationProvider.superDigest**)
     By default this feature is **disabled**
@@ -1540,11 +1545,29 @@ encryption/authentication/authorization performed by the service.
     TBD
 
 * *client.portUnification*:
-    (Java system properties: **zookeeper.client.portUnification**)
+    (Java system property: **zookeeper.client.portUnification**)
     Specifies that the client port should accept SSL connections
     (using the same configuration as the secure client port).
     Default: false
-
+    
+* *authProvider*:
+    (Java system property: **zookeeper.authProvider**)
+    You can specify multiple authentication provider classes for ZooKeeper.
+    Usually you use this parameter to specify the SASL authentication provider
+    like: `authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider`
+    
+* *kerberos.removeHostFromPrincipal*
+    (Java system property: **zookeeper.kerberos.removeHostFromPrincipal**)
+    You can instruct ZooKeeper to remove the host from the client principal name during authentication.
+    (e.g. the zk/myhost@EXAMPLE.COM client principal will be authenticated in ZooKeeper as zk@EXAMPLE.COM)
+    Default: false
+    
+* *kerberos.removeRealmFromPrincipal*
+    (Java system property: **zookeeper.kerberos.removeRealmFromPrincipal**)
+    You can instruct ZooKeeper to remove the realm from the client principal name during authentication.
+    (e.g. the zk/myhost@EXAMPLE.COM client principal will be authenticated in ZooKeeper as zk/myhost)
+    Default: false
+    
 
 <a name="Experimental+Options%2FFeatures"></a>
 
diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperProgrammers.md b/zookeeper-docs/src/main/resources/markdown/zookeeperProgrammers.md
index 08d30c8..5d07159 100644
--- a/zookeeper-docs/src/main/resources/markdown/zookeeperProgrammers.md
+++ b/zookeeper-docs/src/main/resources/markdown/zookeeperProgrammers.md
@@ -1276,8 +1276,11 @@ handle is undefined behavior and should be avoided.
 
 The following list contains configuration properties for the Java client. You can set any
 of these properties using Java system properties. For server properties, please check the
-following reference
-[Server configuration section.](zookeeperAdmin.html#sc_configuration)
+[Server configuration section of the Admin Guide](zookeeperAdmin.html#sc_configuration).
+The ZooKeeper Wiki also has useful pages about
+[ZooKeeper SSL support](https://cwiki.apache.org/confluence/display/ZOOKEEPER/ZooKeeper+SSL+User+Guide), 
+and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/display/ZOOKEEPER/ZooKeeper+and+SASL).
+
 
 * *zookeeper.sasl.client* :
     Set the value to **false** to disable
@@ -1286,6 +1289,13 @@ following reference
 * *zookeeper.sasl.clientconfig* :
     Specifies the context key in the JAAS login file. Default is "Client".
 
+* *zookeeper.server.principal* :
+    Specifies the server principal to be used by the client for authentication, while connecting to the zookeeper
+    server, when Kerberos authentication is enabled. If this configuration is provided, then 
+    the ZooKeeper client will NOT USE any of the following parameters to determine the server principal: 
+    zookeeper.sasl.client.username, zookeeper.sasl.client.canonicalize.hostname, zookeeper.server.realm
+    Note: this config parameter is working only for ZooKeeper 3.5.7+, 3.6.0+
+
 * *zookeeper.sasl.client.username* :
     Traditionally, a principal is divided into three parts: the primary, the instance, and the realm.
     The format of a typical Kerberos V5 principal is primary/instance@REALM.
@@ -1294,6 +1304,13 @@ following reference
     username/IP@realm, where username is the value of zookeeper.sasl.client.username, IP is
     the server IP, and realm is the value of zookeeper.server.realm.
 
+* *zookeeper.sasl.client.canonicalize.hostname* :
+    Expecting the zookeeper.server.principal parameter is not provided, the ZooKeeper client will try to
+    determine the 'instance' (host) part of the ZooKeeper server principal. First it takes the hostname provided 
+    as the ZooKeeper server connection string. Then it tries to 'canonicalize' the address by getting
+    the fully qualified domain name belonging to the address. You can disable this 'canonicalization'
+    by setting: zookeeper.sasl.client.canonicalize.hostname=false
+
 * *zookeeper.server.realm* :
     Realm part of the server principal. By default it is the client principal realm.
 
@@ -1337,12 +1354,6 @@ following reference
 * *zookeeper.kinit* :
     Specifies path to kinit binary. Default is "/usr/bin/kinit".
 
-* *zookeeper.server.principal* :
-    Specifies the server principal to be used by the client for authentication, while connecting to the zookeeper
-    server, when Kerberos authentication is enabled. A couple of ways to specify the server principal can be as
-    "zookeeper.server.principal = **zookeeper/zookeeper.apache.org@APACHE.ORG**" or
-    "zookeeper.server.principal = **zookeeper/zookeeper.apache.org**"
-
 <a name="C+Binding"></a>
 
 ### C Binding
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java
index 30dd3e4..755712b 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java
@@ -38,6 +38,14 @@ public class KerberosTestUtils {
         return "zkquorum/localhost@EXAMPLE.COM";
     }
 
+    public static String getClientPrincipal() {
+        return getClientUsername() + "/localhost@EXAMPLE.COM";
+    }
+
+    public static String getClientUsername() {
+        return "zkclient";
+    }
+
     public static String getHostLearnerPrincipal() {
         return "learner/_HOST@EXAMPLE.COM";
     }
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthTestBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthTestBase.java
index 3b52cc0..97650d0 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthTestBase.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthTestBase.java
@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import javax.security.auth.login.Configuration;
 import org.apache.commons.io.FileUtils;
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.ZKTestCase;
@@ -56,6 +57,10 @@ public class QuorumAuthTestBase extends ZKTestCase {
             // could not create tmp directory to hold JAAS conf file : test will
             // fail now.
         }
+
+        // refresh the SASL configuration in this JVM (making sure that we use the latest config
+        // even if other tests already have been executed and initialized the SASL configs before)
+        Configuration.getConfiguration().refresh();
     }
 
     public static void cleanupJaasConfig() {
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java
index b9d662a..e4534df 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java
@@ -40,6 +40,10 @@ public class QuorumKerberosAuthTest extends KerberosSecurityTestcase {
 
     static {
         String keytabFilePath = FilenameUtils.normalize(KerberosTestUtils.getKeytabFile(), true);
+
+        // note: we use "refreshKrb5Config=true" to refresh the kerberos config in the JVM,
+        // making sure that we use the latest config even if other tests already have been executed
+        // and initialized the kerberos client configs before)
         String jaasEntries = ""
                                      + "QuorumServer {\n"
                                      + "       com.sun.security.auth.module.Krb5LoginModule required\n"
@@ -50,6 +54,7 @@ public class QuorumKerberosAuthTest extends KerberosSecurityTestcase {
                                      + "       storeKey=true\n"
                                      + "       useTicketCache=false\n"
                                      + "       debug=false\n"
+                                     + "       refreshKrb5Config=true\n"
                                      + "       principal=\""
                                      + KerberosTestUtils.getServerPrincipal()
                                      + "\";\n"
@@ -63,6 +68,7 @@ public class QuorumKerberosAuthTest extends KerberosSecurityTestcase {
                                      + "       storeKey=true\n"
                                      + "       useTicketCache=false\n"
                                      + "       debug=false\n"
+                                     + "       refreshKrb5Config=true\n"
                                      + "       principal=\""
                                      + KerberosTestUtils.getLearnerPrincipal()
                                      + "\";\n"
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java
index 9386755..fc15049 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java
@@ -53,6 +53,10 @@ public class QuorumKerberosHostBasedAuthTest extends KerberosSecurityTestcase {
         String hostLearnerPrincipal,
         String hostNamedLearnerPrincipal) {
         String keytabFilePath = FilenameUtils.normalize(KerberosTestUtils.getKeytabFile(), true);
+
+        // note: we use "refreshKrb5Config=true" to refresh the kerberos config in the JVM,
+        // making sure that we use the latest config even if other tests already have been executed
+        // and initialized the kerberos client configs before)
         String jaasEntries = "QuorumServer {\n"
                              + "       com.sun.security.auth.module.Krb5LoginModule required\n"
                              + "       useKeyTab=true\n"
@@ -61,6 +65,7 @@ public class QuorumKerberosHostBasedAuthTest extends KerberosSecurityTestcase {
                              + "       storeKey=true\n"
                              + "       useTicketCache=false\n"
                              + "       debug=false\n"
+                             + "       refreshKrb5Config=true\n"
                              + "       principal=\"" + KerberosTestUtils.replaceHostPattern(hostServerPrincipal)
                              + "\";\n"
                              + "};\n"
@@ -72,6 +77,7 @@ public class QuorumKerberosHostBasedAuthTest extends KerberosSecurityTestcase {
                              + "       storeKey=true\n"
                              + "       useTicketCache=false\n"
                              + "       debug=false\n"
+                             + "       refreshKrb5Config=true\n"
                              + "       principal=\"" + KerberosTestUtils.replaceHostPattern(hostLearnerPrincipal)
                              + "\";\n"
                              + "};\n"
@@ -83,6 +89,7 @@ public class QuorumKerberosHostBasedAuthTest extends KerberosSecurityTestcase {
                              + "       storeKey=true\n"
                              + "       useTicketCache=false\n"
                              + "       debug=false\n"
+                             + "       refreshKrb5Config=true\n"
                              + "       principal=\"" + hostNamedLearnerPrincipal
                              + "\";\n"
                              + "};\n";
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslDigestAuthOverSSLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslDigestAuthOverSSLTest.java
new file mode 100644
index 0000000..09d9297
--- /dev/null
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslDigestAuthOverSSLTest.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.test;
+
+import static org.apache.zookeeper.client.ZKClientConfig.LOGIN_CONTEXT_NAME_KEY;
+import static org.junit.Assert.fail;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.net.InetSocketAddress;
+import javax.security.auth.login.Configuration;
+import org.apache.commons.io.FileUtils;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.Environment;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.ZooDefs.Ids;
+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.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class SaslDigestAuthOverSSLTest extends ClientBase {
+
+    private ClientX509Util clientX509Util;
+    private File saslConfFile;
+
+    @Before
+    @Override
+    public void setUp() throws Exception {
+        initSaslConfig();
+        clientX509Util = setUpSSLWithNoAuth();
+
+        String host = "localhost";
+        int port = PortAssignment.unique();
+        hostPort = host + ":" + port;
+
+        serverFactory = ServerCnxnFactory.createFactory();
+        serverFactory.configure(new InetSocketAddress(host, port), maxCnxns, -1, true);
+
+        super.setUp();
+    }
+
+
+    @After
+    @Override
+    public void tearDown() throws Exception {
+        super.tearDown();
+        clearSslSetting(clientX509Util);
+        clearSaslConfig();
+    }
+
+
+    @Test
+    public void testAuth() throws Exception {
+        ZooKeeper zk = createClient();
+        try {
+            zk.create("/path1", null, Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT);
+            Thread.sleep(1000);
+        } catch (KeeperException e) {
+            fail("test failed :" + e);
+        } finally {
+            zk.close();
+        }
+    }
+
+
+    public void initSaslConfig() {
+        System.setProperty("zookeeper.authProvider.1", "org.apache.zookeeper.server.auth.SASLAuthenticationProvider");
+        System.setProperty(LOGIN_CONTEXT_NAME_KEY, "ClientUsingDigest");
+        try {
+            File tmpDir = createTmpDir();
+            saslConfFile = new File(tmpDir, "jaas.conf");
+            PrintWriter saslConf = new PrintWriter(new FileWriter(saslConfFile));
+            saslConf.println("Server {");
+            saslConf.println("org.apache.zookeeper.server.auth.DigestLoginModule required");
+            saslConf.println("user_super=\"test\";");
+            saslConf.println("};");
+            saslConf.println("ClientUsingDigest {");
+            saslConf.println("org.apache.zookeeper.server.auth.DigestLoginModule required");
+            saslConf.println("username=\"super\"");
+            saslConf.println("password=\"test\";");
+            saslConf.println("};");
+            saslConf.close();
+            System.setProperty(Environment.JAAS_CONF_KEY, saslConfFile.getAbsolutePath());
+        } catch (IOException e) {
+            LOG.error("could not create tmp directory to hold JAAS conf file, test will fail...", e);
+        }
+
+        // refresh the SASL configuration in this JVM (making sure that we use the latest config
+        // even if other tests already have been executed and initialized the SASL configs before)
+        Configuration.getConfiguration().refresh();
+    }
+
+    public void clearSaslConfig() {
+        FileUtils.deleteQuietly(saslConfFile);
+        System.clearProperty(Environment.JAAS_CONF_KEY);
+        System.clearProperty("zookeeper.authProvider.1");
+    }
+
+    public ClientX509Util setUpSSLWithNoAuth() {
+        String testDataPath = System.getProperty("test.data.dir", "src/test/resources/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("zookeeper.ssl.clientAuth", "none");
+        System.setProperty("zookeeper.ssl.quorum.clientAuth", "none");
+
+        ClientX509Util x509Util = new ClientX509Util();
+        System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
+        System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass");
+        System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
+        System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass");
+
+        return x509Util;
+    }
+
+    public void clearSslSetting(ClientX509Util clientX509Util) {
+        System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
+        System.clearProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET);
+        System.clearProperty(ZKClientConfig.SECURE_CLIENT);
+        System.clearProperty(clientX509Util.getSslTruststoreLocationProperty());
+        System.clearProperty(clientX509Util.getSslTruststorePasswdProperty());
+        System.clearProperty(clientX509Util.getSslKeystoreLocationProperty());
+        System.clearProperty(clientX509Util.getSslKeystorePasswdProperty());
+        System.clearProperty("zookeeper.ssl.clientAuth");
+        System.clearProperty("zookeeper.ssl.quorum.clientAuth");
+        clientX509Util.close();
+    }
+
+}
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslKerberosAuthOverSSLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslKerberosAuthOverSSLTest.java
new file mode 100644
index 0000000..d466c8b
--- /dev/null
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslKerberosAuthOverSSLTest.java
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.test;
+
+import static org.apache.zookeeper.client.ZKClientConfig.ENABLE_CLIENT_SASL_KEY;
+import static org.apache.zookeeper.client.ZKClientConfig.LOGIN_CONTEXT_NAME_KEY;
+import static org.apache.zookeeper.client.ZKClientConfig.ZK_SASL_CLIENT_USERNAME;
+import static org.apache.zookeeper.client.ZKClientConfig.ZOOKEEPER_SERVER_PRINCIPAL;
+import static org.apache.zookeeper.client.ZKClientConfig.ZOOKEEPER_SERVER_REALM;
+import static org.junit.Assert.fail;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.net.InetSocketAddress;
+import java.util.Properties;
+import javax.security.auth.login.Configuration;
+import org.apache.commons.io.FileUtils;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.Environment;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.ZooDefs.Ids;
+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.auth.KerberosTestUtils;
+import org.apache.zookeeper.server.quorum.auth.MiniKdc;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class SaslKerberosAuthOverSSLTest extends ClientBase {
+
+    private ClientX509Util clientX509Util;
+    private File keytabFileForKerberosPrincipals;
+    private File saslConfFile;
+
+    private static MiniKdc kdc;
+    private static File kdcWorkDir;
+    private static Properties conf;
+
+
+
+    @BeforeClass
+    public static void setupKdc() {
+        startMiniKdc();
+    }
+
+    @AfterClass
+    public static void tearDownKdc() {
+        stopMiniKdc();
+        FileUtils.deleteQuietly(kdcWorkDir);
+    }
+
+
+
+    @Before
+    @Override
+    public void setUp() throws Exception {
+        initSaslConfig();
+        clientX509Util = setUpSSLWithNoAuth();
+
+        String host = "localhost";
+        int port = PortAssignment.unique();
+        hostPort = host + ":" + port;
+
+        serverFactory = ServerCnxnFactory.createFactory();
+        serverFactory.configure(new InetSocketAddress(host, port), maxCnxns, -1, true);
+
+        super.setUp();
+    }
+
+
+    @After
+    @Override
+    public void tearDown() throws Exception {
+        super.tearDown();
+        clearSslSetting(clientX509Util);
+        clearSaslConfig();
+    }
+
+
+    @Test
+    public void testAuth() throws Exception {
+        ZooKeeper zk = createClient();
+        try {
+            zk.create("/path1", null, Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT);
+            Thread.sleep(1000);
+        } catch (KeeperException e) {
+            fail("test failed :" + e);
+        } finally {
+            zk.close();
+        }
+    }
+
+
+    public void initSaslConfig() throws Exception {
+
+        // registering the server and client users in the KDC mini server
+        keytabFileForKerberosPrincipals = new File(KerberosTestUtils.getKeytabFile());
+        String clientPrincipal = KerberosTestUtils.getClientPrincipal();
+        String serverPrincipal = KerberosTestUtils.getServerPrincipal();
+        clientPrincipal = clientPrincipal.substring(0, clientPrincipal.lastIndexOf("@"));
+        serverPrincipal = serverPrincipal.substring(0, serverPrincipal.lastIndexOf("@"));
+        kdc.createPrincipal(keytabFileForKerberosPrincipals, clientPrincipal, serverPrincipal);
+
+        // client-side SASL config
+        System.setProperty(ZOOKEEPER_SERVER_PRINCIPAL, KerberosTestUtils.getServerPrincipal());
+        System.setProperty(ENABLE_CLIENT_SASL_KEY, "true");
+        System.setProperty(ZOOKEEPER_SERVER_REALM, KerberosTestUtils.getRealm());
+        System.setProperty(LOGIN_CONTEXT_NAME_KEY, "ClientUsingKerberos");
+
+        // server side SASL config
+        System.setProperty("zookeeper.authProvider.1", "org.apache.zookeeper.server.auth.SASLAuthenticationProvider");
+        System.setProperty(SaslTestUtil.requireSASLAuthProperty, "true");
+
+        // generating the SASL config to use (contains sections both for the client and the server)
+        // note: we use "refreshKrb5Config=true" to refresh the kerberos config in the JVM,
+        // making sure that we use the latest config even if other tests already have been executed
+        // and initialized the kerberos client configs before)
+        try {
+            File tmpDir = createTmpDir();
+            saslConfFile = new File(tmpDir, "jaas.conf");
+            PrintWriter saslConf = new PrintWriter(new FileWriter(saslConfFile));
+            saslConf.println("Server {");
+            saslConf.println("  com.sun.security.auth.module.Krb5LoginModule required");
+            saslConf.println("  storeKey=\"true\"");
+            saslConf.println("  useTicketCache=\"false\"");
+            saslConf.println("  useKeyTab=\"true\"");
+            saslConf.println("  doNotPrompt=\"true\"");
+            saslConf.println("  debug=\"true\"");
+            saslConf.println("  refreshKrb5Config=\"true\"");
+            saslConf.println("  keyTab=\"" + keytabFileForKerberosPrincipals.getAbsolutePath() + "\"");
+            saslConf.println("  principal=\"" + KerberosTestUtils.getServerPrincipal() + "\";");
+            saslConf.println("};");
+            saslConf.println("ClientUsingKerberos {");
+            saslConf.println("  com.sun.security.auth.module.Krb5LoginModule required");
+            saslConf.println("  storeKey=\"false\"");
+            saslConf.println("  useTicketCache=\"false\"");
+            saslConf.println("  useKeyTab=\"true\"");
+            saslConf.println("  doNotPrompt=\"true\"");
+            saslConf.println("  debug=\"true\"");
+            saslConf.println("  refreshKrb5Config=\"true\"");
+            saslConf.println("  keyTab=\"" + keytabFileForKerberosPrincipals.getAbsolutePath() + "\"");
+            saslConf.println("  principal=\"" + KerberosTestUtils.getClientPrincipal() + "\";");
+            saslConf.println("};");
+            saslConf.close();
+            System.setProperty(Environment.JAAS_CONF_KEY, saslConfFile.getAbsolutePath());
+
+        } catch (IOException e) {
+            LOG.error("could not create tmp directory to hold JAAS conf file, test will fail...", e);
+        }
+
+        // refresh the SASL configuration in this JVM (making sure that we use the latest config
+        // even if other tests already have been executed and initialized the SASL configs before)
+        Configuration.getConfiguration().refresh();
+    }
+
+    public void clearSaslConfig() {
+        FileUtils.deleteQuietly(keytabFileForKerberosPrincipals);
+        FileUtils.deleteQuietly(saslConfFile);
+
+        System.clearProperty(Environment.JAAS_CONF_KEY);
+        System.clearProperty(ZK_SASL_CLIENT_USERNAME);
+        System.clearProperty(ENABLE_CLIENT_SASL_KEY);
+        System.clearProperty(LOGIN_CONTEXT_NAME_KEY);
+        System.clearProperty("zookeeper.authProvider.1");
+
+        System.clearProperty(SaslTestUtil.requireSASLAuthProperty);
+        System.clearProperty(ZOOKEEPER_SERVER_PRINCIPAL);
+    }
+
+    public ClientX509Util setUpSSLWithNoAuth() {
+        String testDataPath = System.getProperty("test.data.dir", "src/test/resources/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("zookeeper.ssl.clientAuth", "none");
+        System.setProperty("zookeeper.ssl.quorum.clientAuth", "none");
+
+        ClientX509Util x509Util = new ClientX509Util();
+        System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks");
+        System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass");
+        System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks");
+        System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass");
+
+        return x509Util;
+    }
+
+    public void clearSslSetting(ClientX509Util clientX509Util) {
+        System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
+        System.clearProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET);
+        System.clearProperty(ZKClientConfig.SECURE_CLIENT);
+        System.clearProperty(clientX509Util.getSslTruststoreLocationProperty());
+        System.clearProperty(clientX509Util.getSslTruststorePasswdProperty());
+        System.clearProperty(clientX509Util.getSslKeystoreLocationProperty());
+        System.clearProperty(clientX509Util.getSslKeystorePasswdProperty());
+        System.clearProperty("javax.net.debug");
+        System.clearProperty("zookeeper.ssl.clientAuth");
+        System.clearProperty("zookeeper.ssl.quorum.clientAuth");
+        clientX509Util.close();
+    }
+
+
+
+    public static void startMiniKdc() {
+        try {
+            kdcWorkDir = createEmptyTestDir();
+            conf = MiniKdc.createConf();
+            conf.setProperty("debug", "true");
+
+            kdc = new MiniKdc(conf, kdcWorkDir);
+            kdc.start();
+        } catch (Exception e) {
+            throw new RuntimeException("failed to start MiniKdc", e);
+        }
+
+    }
+
+    public static void stopMiniKdc() {
+        if (kdc != null) {
+            kdc.stop();
+        }
+    }
+
+}