You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ma...@apache.org on 2012/02/06 09:34:23 UTC

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

Author: mahadev
Date: Mon Feb  6 08:34:22 2012
New Revision: 1240918

URL: http://svn.apache.org/viewvc?rev=1240918&view=rev
Log:
ZOOKEEPER-1373. Hardcoded SASL login context name clashes with Hadoop security  configuration override. (Eugene Koontz and Thomas Weise via mahadev)

Added:
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthDesignatedClientTest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthFailDesignatedClientTest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthMissingClientConfigTest.java
Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/Login.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1240918&r1=1240917&r2=1240918&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Mon Feb  6 08:34:22 2012
@@ -110,6 +110,9 @@ BUGFIXES:
 
   ZOOKEEPER-1353. C client test suite fails consistently. (Clint Byrum 
   via mahadev)
+
+  ZOOKEEPER-1373. Hardcoded SASL login context name clashes with Hadoop security 
+  configuration override. (Eugene Koontz and Thomas Weise via mahadev)
  
 IMPROVEMENTS:
 

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java?rev=1240918&r1=1240917&r2=1240918&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java Mon Feb  6 08:34:22 2012
@@ -941,20 +941,14 @@ public class ClientCnxn {
 
             setName(getName().replaceAll("\\(.*\\)",
                     "(" + addr.getHostName() + ":" + addr.getPort() + ")"));
-
-            if (System.getProperty("java.security.auth.login.config") != null) {
-                try {
-                    zooKeeperSaslClient = new ZooKeeperSaslClient(ClientCnxn.this, "zookeeper"+"/"+ addr.getHostName());
-                }
-                catch (LoginException e) {
-                    LOG.warn("Zookeeper client cannot authenticate using the Client section of the supplied "
-                      + "configuration file: '" + System.getProperty("java.security.auth.login.config")
-                      + "'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper "
-                      + "server allows it.");
-                    eventThread.queueEvent(new WatchedEvent(
-                            Watcher.Event.EventType.None,
-                            KeeperState.AuthFailed, null));
-                }
+            try {
+                zooKeeperSaslClient = new ZooKeeperSaslClient("zookeeper/"+addr.getHostName());
+            } catch (LoginException e) {
+                LOG.warn("SASL authentication failed: " + e + " Will continue connection to Zookeeper server without "
+                        + "SASL authentication, if Zookeeper server allows it.");
+                eventThread.queueEvent(new WatchedEvent(
+                        Watcher.Event.EventType.None,
+                        Watcher.Event.KeeperState.AuthFailed, null));
             }
             clientCnxnSocket.connect(addr);
         }
@@ -981,9 +975,9 @@ public class ClientCnxn {
                     }
 
                     if (state.isConnected()) {
-                        if ((zooKeeperSaslClient != null) && (zooKeeperSaslClient.isComplete() != true)) {
+                        if ((zooKeeperSaslClient != null) && (zooKeeperSaslClient.isFailed() != true) && (zooKeeperSaslClient.isComplete() != true)) {
                             try {
-                                zooKeeperSaslClient.initialize();
+                                zooKeeperSaslClient.initialize(ClientCnxn.this);
                             }
                             catch (SaslException e) {
                                 LOG.error("SASL authentication with Zookeeper Quorum member failed: " + e);

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/Login.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/Login.java?rev=1240918&r1=1240917&r2=1240918&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/Login.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/Login.java Mon Feb  6 08:34:22 2012
@@ -33,7 +33,7 @@ import javax.security.auth.login.LoginEx
 import javax.security.auth.callback.CallbackHandler;
 
 import org.apache.log4j.Logger;
-
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
 import javax.security.auth.kerberos.KerberosTicket;
 import javax.security.auth.Subject;
 import java.util.Date;
@@ -273,7 +273,10 @@ public class Login {
     private synchronized LoginContext login(final String loginContextName) throws LoginException {
         if (loginContextName == null) {
             throw new LoginException("loginContext name (JAAS file section header) was null. " +
-                    "Please check your java.security.login.auth.config setting.");
+                    "Please check your java.security.login.auth.config (=" +
+                    System.getProperty("java.security.login.auth.config") +
+                    ") and your " + ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY + "(=" + 
+                    System.getProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, "Client") + ")");
         }
         LoginContext loginContext = new LoginContext(loginContextName,callbackHandler);
         loginContext.login();

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java?rev=1240918&r1=1240917&r2=1240918&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java Mon Feb  6 08:34:22 2012
@@ -21,12 +21,12 @@ package org.apache.zookeeper.client;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.ClientCnxn;
 import org.apache.zookeeper.Login;
+import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.proto.GetSASLRequest;
 import org.apache.zookeeper.proto.ReplyHeader;
 import org.apache.zookeeper.proto.RequestHeader;
 import org.apache.zookeeper.proto.SetSASLResponse;
-import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.server.auth.KerberosName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,6 +40,8 @@ import javax.security.auth.callback.Call
 import javax.security.auth.callback.NameCallback;
 import javax.security.auth.callback.PasswordCallback;
 import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
 import javax.security.auth.login.LoginException;
 import javax.security.sasl.AuthorizeCallback;
 import javax.security.sasl.RealmCallback;
@@ -52,12 +54,12 @@ import javax.security.sasl.SaslException
  * allows ClientCnxn to authenticate using SASL with a Zookeeper server.
  */
 public class ZooKeeperSaslClient {
+    public static final String LOGIN_CONTEXT_NAME_KEY = "zookeeper.sasl.clientconfig";
     private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperSaslClient.class);
     private static Login login = null;
     private SaslClient saslClient;
 
     private byte[] saslToken = new byte[0];
-    private ClientCnxn cnxn;
 
     public enum SaslState {
         INITIAL,INTERMEDIATE,COMPLETE,FAILED
@@ -69,15 +71,87 @@ public class ZooKeeperSaslClient {
         return saslState;
     }
 
-    public ZooKeeperSaslClient(ClientCnxn cnxn, String serverPrincipal) throws LoginException {
-        this.cnxn = cnxn;
-        this.saslClient = createSaslClient(serverPrincipal);
+    private String loginContext;
+
+    public String getLoginContext() {
+      return loginContext;
+    }
+
+    public ZooKeeperSaslClient(final String serverPrincipal)
+            throws LoginException {
+        /**
+         * ZOOKEEPER-1373: allow system property to specify the JAAS
+         * configuration section that the zookeeper client should use.
+         * Default to "Client".
+         */
+        String clientSection = System.getProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, "Client");
+        // Note that 'Configuration' here refers to javax.security.auth.login.Configuration.
+        AppConfigurationEntry entries[] = null;
+        SecurityException securityException = null;
+        try {
+            entries = Configuration.getConfiguration().getAppConfigurationEntry(clientSection);
+        } catch (SecurityException e) {
+            // handle below: might be harmless if the user doesn't intend to use JAAS authentication.
+            securityException = e;
+        }
+        if (entries != null) {
+            LOG.info("Found Login Context section '" + clientSection + "': will use it to attempt to SASL-authenticate.");
+            this.saslClient = createSaslClient(serverPrincipal, clientSection);
+        } else {
+            // Handle situation of clientSection's being null: it might simply because the client does not intend to 
+            // use SASL, so not necessarily an error.
+            saslState = SaslState.FAILED;
+            String explicitClientSection = System.getProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY);
+            if (explicitClientSection != null) {
+                // If the user explicitly overrides the default Login Context, they probably expected SASL to
+                // succeed. But if we got here, SASL failed.
+                if (securityException != null) {
+                    throw new LoginException("Zookeeper client cannot authenticate using the " + explicitClientSection +
+                            " section of the supplied JAAS configuration: '" +
+                            System.getProperty("java.security.auth.login.config") + "' because of a " +
+                            "SecurityException: " + securityException);
+                    
+                    
+                } else {
+                    throw new LoginException("Client cannot SASL-authenticate because the specified JAAS configuration " +
+                            "section '" + explicitClientSection + "' could not be found.");
+                }
+            } else {
+                // The user did not override the default context. It might be that they just don't intend to use SASL,
+                // so log at INFO, not WARN, since they don't expect any SASL-related information.
+                if (securityException != null) {
+                    LOG.warn("SecurityException: " + securityException + " occurred when trying to find JAAS configuration.");
+                }
+                LOG.info("Client will not SASL-authenticate because the default JAAS configuration section 'Client' " +
+                        "could not be found. If you are not using SASL, you may ignore this. On the other hand, " +
+                        "if you expected SASL to work, please fix your JAAS configuration.");
+            }
+            if (System.getProperty("java.security.auth.login.config")  != null) {
+                // Again, the user explicitly set something SASL-related, so they probably expected SASL to succeed.
+                if (securityException != null) {
+                    throw new LoginException("Zookeeper client cannot authenticate using the '" +
+                            System.getProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, "Client") +
+                            "' section of the supplied JAAS configuration: '" +
+                            System.getProperty("java.security.auth.login.config") + "' because of a " +
+                            "SecurityException: " + securityException);
+                } else {
+                    throw new LoginException("No JAAS configuration section named '" +
+                            System.getProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, "Client") +
+                            "' was found in specified JAAS configuration file: '" +
+                            System.getProperty("java.security.auth.login.config") + "'.");
+                }
+            }
+        }
     }
 
     public boolean isComplete() {
         return (saslState == SaslState.COMPLETE);
     }
 
+    public boolean isFailed() {
+        return (saslState == SaslState.FAILED);
+    }
+
     public static class ServerSaslResponseCallback implements AsyncCallback.DataCallback {
         public void processResult(int rc, String path, Object ctx, byte data[], Stat stat) {
             // processResult() is used by ClientCnxn's sendThread to respond to
@@ -97,16 +171,21 @@ public class ZooKeeperSaslClient {
                 usedata = new byte[0];
                 LOG.debug("ServerSaslResponseCallback(): using empty data[] as server response (length="+usedata.length+")");
             }
-            client.prepareSaslResponseToServer(usedata);
+            client.prepareSaslResponseToServer(usedata, (ClientCnxn)ctx);
         }
     }
 
-    synchronized private SaslClient createSaslClient(final String servicePrincipal) throws LoginException {
+    synchronized private SaslClient createSaslClient(final String servicePrincipal,
+                                                     final String loginContext) throws LoginException {
         try {
             if (login == null) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("JAAS loginContext is: " + loginContext);
+                }
+                this.loginContext = loginContext;
                 // note that the login object is static: it's shared amongst all zookeeper-related connections.
                 // createSaslClient() must be declared synchronized so that login is initialized only once.
-                login = new Login("Client",new ClientCallbackHandler(null));
+                login = new Login(loginContext, new ClientCallbackHandler(null));
                 login.startThreadIfNeeded();
             }
             Subject subject = login.getSubject();
@@ -163,7 +242,7 @@ public class ZooKeeperSaslClient {
         }
     }
 
-    private void prepareSaslResponseToServer(byte[] serverToken) {
+    private void prepareSaslResponseToServer(byte[] serverToken, ClientCnxn cnxn) {
         saslToken = serverToken;
 
         if (saslClient == null) {
@@ -177,16 +256,17 @@ public class ZooKeeperSaslClient {
                 saslToken = createSaslToken(saslToken);
                 if (saslToken != null) {
                     LOG.debug("saslToken (client) length: " + saslToken.length);
-                    queueSaslPacket(saslToken);
+                    queueSaslPacket(saslToken, cnxn);
                 }
             } catch (SaslException e) {
-                LOG.error("SASL authentication failed.");
+                LOG.error("SASL authentication failed using login context '" +
+                this.getLoginContext() + "'.");
                 saslState = SaslState.FAILED;
             }
         }
     }
 
-    public byte[] createSaslToken() throws SaslException {
+    private byte[] createSaslToken() throws SaslException {
         saslState = SaslState.INTERMEDIATE;
         return createSaslToken(saslToken);
     }
@@ -234,7 +314,7 @@ public class ZooKeeperSaslClient {
         }
     }
 
-    public void queueSaslPacket(byte[] saslToken) {
+    private void queueSaslPacket(byte[] saslToken, ClientCnxn cnxn) {
         LOG.debug("ClientCnxn:sendSaslPacket:length="+saslToken.length);
         RequestHeader h = new RequestHeader();
         h.setType(ZooDefs.OpCode.sasl);
@@ -245,9 +325,9 @@ public class ZooKeeperSaslClient {
         ReplyHeader r = new ReplyHeader();
         cnxn.queuePacket(h,r,request,response,cb);
     }
-
-    public void queueSaslPacket() throws SaslException {
-        queueSaslPacket(createSaslToken());
+    
+    private void queueSaslPacket(ClientCnxn cnxn) throws SaslException {
+        queueSaslPacket(createSaslToken(), cnxn);
     }
 
     // used by ClientCnxn to know when to emit SaslAuthenticated event.
@@ -268,17 +348,17 @@ public class ZooKeeperSaslClient {
         return false;
     }
 
-    public void initialize() throws SaslException {
+    public void initialize(ClientCnxn cnxn) throws SaslException {
         if (saslClient == null) {
             throw new SaslException("saslClient failed to initialize properly: it's null.");
         }
         if (saslState == SaslState.INITIAL) {
             if (saslClient.hasInitialResponse()) {
-                queueSaslPacket();
+                queueSaslPacket(cnxn);
             }
             else {
                 byte[] emptyToken = new byte[0];
-                queueSaslPacket(emptyToken);
+                queueSaslPacket(emptyToken, cnxn);
             }
             saslState = SaslState.INTERMEDIATE;
         }

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthDesignatedClientTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthDesignatedClientTest.java?rev=1240918&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthDesignatedClientTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthDesignatedClientTest.java Mon Feb  6 08:34:22 2012
@@ -0,0 +1,99 @@
+/**
+ * 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 java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class SaslAuthDesignatedClientTest extends ClientBase {
+    static {
+        System.setProperty("zookeeper.authProvider.1","org.apache.zookeeper.server.auth.SASLAuthenticationProvider");
+        System.setProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, "MyZookeeperClient");
+
+        try {
+            File tmpDir = createTmpDir();
+            File saslConfFile = new File(tmpDir, "jaas.conf");
+            FileWriter fwriter = new FileWriter(saslConfFile);
+
+            fwriter.write("" +
+                "Server {\n" +
+                "          org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+                "          user_myuser=\"mypassword\";\n" +
+                "};\n" +
+                "Client {\n" + /* this 'Client' section has an incorrect password, but we're not configured
+                                  to  use it (we're configured by the above System.setProperty(...LOGIN_CONTEXT_NAME_KEY...) to 
+                                  use the 'MyZookeeperClient' section below, which has the correct password).*/
+                "       org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+                "       username=\"myuser\"\n" +
+                "       password=\"wrongpassword\";\n" +
+                "};" +
+                "MyZookeeperClient {\n" +
+                "       org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+                "       username=\"myuser\"\n" +
+                "       password=\"mypassword\";\n" +
+                "};" + "\n");
+            fwriter.close();
+            System.setProperty("java.security.auth.login.config",saslConfFile.getAbsolutePath());
+        }
+        catch (IOException e) {
+            // could not create tmp directory to hold JAAS conf file : test will fail now.
+        }
+    }
+
+    private AtomicInteger authFailed = new AtomicInteger(0);
+
+    private class MyWatcher extends CountdownWatcher {
+        @Override
+        public synchronized void process(WatchedEvent event) {
+            if (event.getState() == KeeperState.AuthFailed) {
+                authFailed.incrementAndGet();
+            }
+            else {
+                super.process(event);
+            }
+        }
+    }
+
+    @Test
+    public void testAuth() throws Exception {
+        ZooKeeper zk = createClient();
+        Thread.sleep(1000);
+        try {
+            zk.create("/path1", null, Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT);
+            Thread.sleep(1000);
+        } catch (KeeperException e) {
+          Assert.fail("test failed :" + e);
+        }
+        finally {
+            zk.close();
+        }
+    }
+}

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthFailDesignatedClientTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthFailDesignatedClientTest.java?rev=1240918&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthFailDesignatedClientTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthFailDesignatedClientTest.java Mon Feb  6 08:34:22 2012
@@ -0,0 +1,100 @@
+/**
+ * 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 java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class SaslAuthFailDesignatedClientTest extends ClientBase {
+    static {
+        System.setProperty("zookeeper.authProvider.1","org.apache.zookeeper.server.auth.SASLAuthenticationProvider");
+        System.setProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, "MyZookeeperClient");
+
+        try {
+            File tmpDir = createTmpDir();
+            File saslConfFile = new File(tmpDir, "jaas.conf");
+            FileWriter fwriter = new FileWriter(saslConfFile);
+
+            fwriter.write("" +
+                "Server {\n" +
+                "          org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+                "          user_myuser=\"mypassword\";\n" +
+                "};\n" +
+                "Client {\n" + /* this 'Client' section has the correct password, but we're not configured
+                                  to  use it (we're configured by the above System.setProperty(...LOGIN_CONTEXT_NAME_KEY...) to 
+                                  use the 'MyZookeeperClient' section, which has an incorrect password).*/
+                "       org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+                "       username=\"myuser\"\n" +
+                "       password=\"mypassword\";\n" +
+                "};" +
+                "MyZookeeperClient {\n" +
+                "       org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+                "       username=\"myuser\"\n" +
+                "       password=\"wrongpassword\";\n" +
+                "};" + "\n");
+            fwriter.close();
+            System.setProperty("java.security.auth.login.config",saslConfFile.getAbsolutePath());
+        }
+        catch (IOException e) {
+            // could not create tmp directory to hold JAAS conf file : test will fail now.
+        }
+    }
+
+    private AtomicInteger authFailed = new AtomicInteger(0);
+
+    private class MyWatcher extends CountdownWatcher {
+        @Override
+        public synchronized void process(WatchedEvent event) {
+            if (event.getState() == KeeperState.AuthFailed) {
+                authFailed.incrementAndGet();
+            }
+            else {
+                super.process(event);
+            }
+        }
+    }
+
+    @Test
+    public void testAuth() throws Exception {
+        ZooKeeper zk = createClient();
+        Thread.sleep(1000);
+        try {
+            zk.create("/path1", null, Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT);
+            Assert.fail("Should have gotten exception.");
+        } catch (KeeperException e) {
+            // ok, exception as expected.
+            LOG.info("Got exception as expected: " + e);
+        }
+        finally {
+            zk.close();
+        }
+    }
+}

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthMissingClientConfigTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthMissingClientConfigTest.java?rev=1240918&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthMissingClientConfigTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthMissingClientConfigTest.java Mon Feb  6 08:34:22 2012
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.test;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class SaslAuthMissingClientConfigTest extends ClientBase {
+    static {
+        System.setProperty("zookeeper.authProvider.1","org.apache.zookeeper.server.auth.SASLAuthenticationProvider");
+        // This configuration section 'MyZookeeperClient', is missing from the JAAS configuration.
+        // As a result, SASL authentication should fail, which is tested by this test (testAuth()).
+        System.setProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, "MyZookeeperClient");
+
+        try {
+            File tmpDir = createTmpDir();
+            File saslConfFile = new File(tmpDir, "jaas.conf");
+            FileWriter fwriter = new FileWriter(saslConfFile);
+
+            fwriter.write("" +
+                "Server {\n" +
+                "          org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+                "          user_myuser=\"mypassword\";\n" +
+                "};\n" +
+                "Client {\n" + /* this 'Client' section has the correct password, but we're not configured
+                                  to  use it - we're configured instead by the above
+                                  System.setProperty(...LOGIN_CONTEXT_NAME_KEY...) to
+                                  use the (nonexistent) 'MyZookeeperClient' section. */
+                "       org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+                "       username=\"myuser\"\n" +
+                "       password=\"mypassword\";\n" +
+                "};\n");
+            fwriter.close();
+            System.setProperty("java.security.auth.login.config",saslConfFile.getAbsolutePath());
+        }
+        catch (IOException e) {
+            // could not create tmp directory to hold JAAS conf file : test will fail now.
+        }
+    }
+
+    private AtomicInteger authFailed = new AtomicInteger(0);
+
+    private class MyWatcher extends CountdownWatcher {
+        @Override
+        public synchronized void process(WatchedEvent event) {
+            if (event.getState() == KeeperState.AuthFailed) {
+                authFailed.incrementAndGet();
+            }
+            else {
+                super.process(event);
+            }
+        }
+    }
+
+    @Test
+    public void testAuth() throws Exception {
+        ZooKeeper zk = createClient();
+        Thread.sleep(1000);
+        try {
+            zk.create("/path1", null, Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT);
+            Assert.fail("Should have gotten exception.");
+        } catch (KeeperException e) {
+            // ok, exception as expected.
+            LOG.info("Got exception as expected: " + e);
+        }
+        finally {
+            zk.close();
+        }
+    }
+}