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 2011/08/19 00:05:36 UTC

svn commit: r1159432 [2/2] - in /zookeeper/trunk: ./ src/ src/java/main/org/apache/zookeeper/ src/java/main/org/apache/zookeeper/client/ src/java/main/org/apache/zookeeper/server/ src/java/main/org/apache/zookeeper/server/auth/ src/java/test/org/apache...

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/auth/KerberosName.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/auth/KerberosName.java?rev=1159432&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/auth/KerberosName.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/auth/KerberosName.java Thu Aug 18 22:05:35 2011
@@ -0,0 +1,418 @@
+/**
+ * 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.
+ */
+
+ /* This file copied from Hadoop's security branch,
+  * with the following changes:
+  * 1. package changed from org.apache.hadoop.security to
+  *    org.apache.zookeeper.server.auth.
+  * 2. Usage of Hadoop's Configuration class removed since
+  *    it is not available in Zookeeper: instead, system property
+  *    "zookeeper.security.auth_to_local" is used.
+  */
+
+package org.apache.zookeeper.server.auth;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import sun.security.krb5.Config;
+import sun.security.krb5.KrbException;
+
+/**
+ * This class implements parsing and handling of Kerberos principal names. In 
+ * particular, it splits them apart and translates them down into local
+ * operating system names.
+ */
+public class KerberosName {
+  /** The first component of the name */
+  private final String serviceName;
+  /** The second component of the name. It may be null. */
+  private final String hostName;
+  /** The realm of the name. */
+  private final String realm;
+
+  /**
+   * A pattern that matches a Kerberos name with at most 2 components.
+   */
+  private static final Pattern nameParser = 
+    Pattern.compile("([^/@]*)(/([^/@]*))?@([^/@]*)");
+
+  /** 
+   * A pattern that matches a string with out '$' and then a single
+   * parameter with $n.
+   */
+  private static Pattern parameterPattern = 
+    Pattern.compile("([^$]*)(\\$(\\d*))?");
+
+  /**
+   * A pattern for parsing a auth_to_local rule.
+   */
+  private static final Pattern ruleParser =
+    Pattern.compile("\\s*((DEFAULT)|(RULE:\\[(\\d*):([^\\]]*)](\\(([^)]*)\\))?"+
+                    "(s/([^/]*)/([^/]*)/(g)?)?))");
+  
+  /**
+   * A pattern that recognizes simple/non-simple names.
+   */
+  private static final Pattern nonSimplePattern = Pattern.compile("[/@]");
+  
+  /**
+   * The list of translation rules.
+   */
+  private static List<Rule> rules;
+
+  private static String defaultRealm;
+  private static Config kerbConf;
+  
+  static {
+    try {
+      kerbConf = Config.getInstance();
+      defaultRealm = kerbConf.getDefaultRealm();
+    } catch (KrbException ke) {
+      if ((System.getProperty("zookeeper.requireKerberosConfig") != null) &&
+          (System.getProperty("zookeeper.requireKerberosConfig").equals("true"))) {
+        throw new IllegalArgumentException("Can't get Kerberos configuration",ke);
+      }
+    }
+    try {
+      // setConfiguration() will work even if the above try() fails due
+      // to a missing Kerberos configuration (unless zookeeper.requireKerberosConfig
+      // is set to true, which would not allow execution to reach here due to the
+      // throwing of an IllegalArgumentException above).
+      setConfiguration();
+    }
+    catch (IOException e) {
+      throw new IllegalArgumentException("Could not configure Kerberos principal name mapping.");
+    }
+  }
+
+  /**
+   * Create a name from the full Kerberos principal name.
+   * @param name
+   */
+  public KerberosName(String name) {
+    Matcher match = nameParser.matcher(name);
+    if (!match.matches()) {
+      if (name.contains("@")) {
+        throw new IllegalArgumentException("Malformed Kerberos name: " + name);
+      } else {
+        serviceName = name;
+        hostName = null;
+        realm = null;
+      }
+    } else {
+      serviceName = match.group(1);
+      hostName = match.group(3);
+      realm = match.group(4);
+    }
+  }
+
+  /**
+   * Get the configured default realm.
+   * @return the default realm from the krb5.conf
+   */
+  public String getDefaultRealm() {
+    return defaultRealm;
+  }
+
+  /**
+   * Put the name back together from the parts.
+   */
+  @Override
+  public String toString() {
+    StringBuilder result = new StringBuilder();
+    result.append(serviceName);
+    if (hostName != null) {
+      result.append('/');
+      result.append(hostName);
+    }
+    if (realm != null) {
+      result.append('@');
+      result.append(realm);
+    }
+    return result.toString();
+  }
+
+  /**
+   * Get the first component of the name.
+   * @return the first section of the Kerberos principal name
+   */
+  public String getServiceName() {
+    return serviceName;
+  }
+
+  /**
+   * Get the second component of the name.
+   * @return the second section of the Kerberos principal name, and may be null
+   */
+  public String getHostName() {
+    return hostName;
+  }
+  
+  /**
+   * Get the realm of the name.
+   * @return the realm of the name, may be null
+   */
+  public String getRealm() {
+    return realm;
+  }
+  
+  /**
+   * An encoding of a rule for translating kerberos names.
+   */
+  private static class Rule {
+    private final boolean isDefault;
+    private final int numOfComponents;
+    private final String format;
+    private final Pattern match;
+    private final Pattern fromPattern;
+    private final String toPattern;
+    private final boolean repeat;
+
+    Rule() {
+      isDefault = true;
+      numOfComponents = 0;
+      format = null;
+      match = null;
+      fromPattern = null;
+      toPattern = null;
+      repeat = false;
+    }
+
+    Rule(int numOfComponents, String format, String match, String fromPattern,
+         String toPattern, boolean repeat) {
+      isDefault = false;
+      this.numOfComponents = numOfComponents;
+      this.format = format;
+      this.match = match == null ? null : Pattern.compile(match);
+      this.fromPattern = 
+        fromPattern == null ? null : Pattern.compile(fromPattern);
+      this.toPattern = toPattern;
+      this.repeat = repeat;
+    }
+    
+    @Override
+    public String toString() {
+      StringBuilder buf = new StringBuilder();
+      if (isDefault) {
+        buf.append("DEFAULT");
+      } else {
+        buf.append("RULE:[");
+        buf.append(numOfComponents);
+        buf.append(':');
+        buf.append(format);
+        buf.append(']');
+        if (match != null) {
+          buf.append('(');
+          buf.append(match);
+          buf.append(')');
+        }
+        if (fromPattern != null) {
+          buf.append("s/");
+          buf.append(fromPattern);
+          buf.append('/');
+          buf.append(toPattern);
+          buf.append('/');
+          if (repeat) {
+            buf.append('g');
+          }
+        }
+      }
+      return buf.toString();
+    }
+    
+    /**
+     * Replace the numbered parameters of the form $n where n is from 1 to 
+     * the length of params. Normal text is copied directly and $n is replaced
+     * by the corresponding parameter.
+     * @param format the string to replace parameters again
+     * @param params the list of parameters
+     * @return the generated string with the parameter references replaced.
+     * @throws BadFormatString
+     */
+    static String replaceParameters(String format, 
+                                    String[] params) throws BadFormatString {
+      Matcher match = parameterPattern.matcher(format);
+      int start = 0;
+      StringBuilder result = new StringBuilder();
+      while (start < format.length() && match.find(start)) {
+        result.append(match.group(1));
+        String paramNum = match.group(3);
+        if (paramNum != null) {
+          try {
+            int num = Integer.parseInt(paramNum);
+            if (num < 0 || num > params.length) {
+              throw new BadFormatString("index " + num + " from " + format +
+                                        " is outside of the valid range 0 to " +
+                                        (params.length - 1));
+            }
+            result.append(params[num]);
+          } catch (NumberFormatException nfe) {
+            throw new BadFormatString("bad format in username mapping in " + 
+                                      paramNum, nfe);
+          }
+          
+        }
+        start = match.end();
+      }
+      return result.toString();
+    }
+
+    /**
+     * Replace the matches of the from pattern in the base string with the value
+     * of the to string.
+     * @param base the string to transform
+     * @param from the pattern to look for in the base string
+     * @param to the string to replace matches of the pattern with
+     * @param repeat whether the substitution should be repeated
+     * @return
+     */
+    static String replaceSubstitution(String base, Pattern from, String to, 
+                                      boolean repeat) {
+      Matcher match = from.matcher(base);
+      if (repeat) {
+        return match.replaceAll(to);
+      } else {
+        return match.replaceFirst(to);
+      }
+    }
+
+    /**
+     * Try to apply this rule to the given name represented as a parameter
+     * array.
+     * @param params first element is the realm, second and later elements are
+     *        are the components of the name "a/b@FOO" -> {"FOO", "a", "b"}
+     * @return the short name if this rule applies or null
+     * @throws IOException throws if something is wrong with the rules
+     */
+    String apply(String[] params) throws IOException {
+      String result = null;
+      if (isDefault) {
+        if (defaultRealm.equals(params[0])) {
+          result = params[1];
+        }
+      } else if (params.length - 1 == numOfComponents) {
+        String base = replaceParameters(format, params);
+        if (match == null || match.matcher(base).matches()) {
+          if (fromPattern == null) {
+            result = base;
+          } else {
+            result = replaceSubstitution(base, fromPattern, toPattern,  repeat);
+          }
+        }
+      }
+      if (result != null && nonSimplePattern.matcher(result).find()) {
+        throw new NoMatchingRule("Non-simple name " + result +
+                                 " after auth_to_local rule " + this);
+      }
+      return result;
+    }
+  }
+
+  static List<Rule> parseRules(String rules) {
+    List<Rule> result = new ArrayList<Rule>();
+    String remaining = rules.trim();
+    while (remaining.length() > 0) {
+      Matcher matcher = ruleParser.matcher(remaining);
+      if (!matcher.lookingAt()) {
+        throw new IllegalArgumentException("Invalid rule: " + remaining);
+      }
+      if (matcher.group(2) != null) {
+        result.add(new Rule());
+      } else {
+        result.add(new Rule(Integer.parseInt(matcher.group(4)),
+                            matcher.group(5),
+                            matcher.group(7),
+                            matcher.group(9),
+                            matcher.group(10),
+                            "g".equals(matcher.group(11))));
+      }
+      remaining = remaining.substring(matcher.end());
+    }
+    return result;
+  }
+
+  /**
+   * Set the static configuration to get the rules.
+   * @param conf the new configuration
+   * @throws IOException
+   */
+  public static void setConfiguration() throws IOException {
+    String ruleString = System.getProperty("zookeeper.security.auth_to_local", "DEFAULT");
+    rules = parseRules(ruleString);
+  }
+
+  @SuppressWarnings("serial")
+  public static class BadFormatString extends IOException {
+    BadFormatString(String msg) {
+      super(msg);
+    }
+    BadFormatString(String msg, Throwable err) {
+      super(msg, err);
+    }
+  }
+
+  @SuppressWarnings("serial")
+  public static class NoMatchingRule extends IOException {
+    NoMatchingRule(String msg) {
+      super(msg);
+    }
+  }
+
+  /**
+   * Get the translation of the principal name into an operating system
+   * user name.
+   * @return the short name
+   * @throws IOException
+   */
+  public String getShortName() throws IOException {
+    String[] params;
+    if (hostName == null) {
+      // if it is already simple, just return it
+      if (realm == null) {
+        return serviceName;
+      }
+      params = new String[]{realm, serviceName};
+    } else {
+      params = new String[]{realm, serviceName, hostName};
+    }
+    for(Rule r: rules) {
+      String result = r.apply(params);
+      if (result != null) {
+        return result;
+      }
+    }
+    throw new NoMatchingRule("No rules applied to " + toString());
+  }
+
+  static void printRules() throws IOException {
+    int i = 0;
+    for(Rule r: rules) {
+      System.out.println(++i + " " + r);
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    for(String arg: args) {
+      KerberosName name = new KerberosName(arg);
+      System.out.println("Name: " + name + " to " + name.getShortName());
+    }
+  }
+}
\ No newline at end of file

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/auth/SASLAuthenticationProvider.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/auth/SASLAuthenticationProvider.java?rev=1159432&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/auth/SASLAuthenticationProvider.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/auth/SASLAuthenticationProvider.java Thu Aug 18 22:05:35 2011
@@ -0,0 +1,70 @@
+/**
+ * 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.auth;
+
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.server.ServerCnxn;
+
+public class SASLAuthenticationProvider implements AuthenticationProvider {
+
+    public String getScheme() {
+        return "sasl";
+    }
+
+    public KeeperException.Code
+        handleAuthentication(ServerCnxn cnxn, byte[] authData)
+    {
+        // Should never call this: SASL authentication is negotiated at session initiation.
+        // TODO: consider substituting current implementation of direct ClientCnxn manipulation with
+        // a call to this method (SASLAuthenticationProvider:handleAuthentication()) at session initiation.
+        return KeeperException.Code.AUTHFAILED;
+
+    }
+
+    public boolean matches(String id,String aclExpr) {
+        if (System.getProperty("zookeeper.superUser") != null) {
+            return (id.equals(System.getProperty("zookeeper.superUser")) || id.equals(aclExpr));
+        }
+        return (id.equals("super") || id.equals(aclExpr));
+    }
+
+    public boolean isAuthenticated() {
+        return true;
+    }
+
+    public boolean isValid(String id) {
+        // Since the SASL authenticator will usually be used with Kerberos authentication,
+        // it should enforce that these names are valid according to Kerberos's
+        // syntax for principals.
+        //
+        // Use the KerberosName(id) constructor to define validity:
+        // if KerberosName(id) throws IllegalArgumentException, then id is invalid.
+        // otherwise, it is valid.
+        //
+        try {
+            new KerberosName(id);
+            return true;
+        }
+        catch (IllegalArgumentException e) {
+            return false;
+        }
+   }
+
+
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/auth/SaslServerCallbackHandler.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/auth/SaslServerCallbackHandler.java?rev=1159432&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/auth/SaslServerCallbackHandler.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/auth/SaslServerCallbackHandler.java Thu Aug 18 22:05:35 2011
@@ -0,0 +1,160 @@
+/**
+ * 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.auth;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+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.sasl.AuthorizeCallback;
+import javax.security.sasl.RealmCallback;
+
+public class SaslServerCallbackHandler implements CallbackHandler {
+    private String userName = null;
+    private Map<String,String> credentials = new HashMap<String,String>();
+    Logger LOG = LoggerFactory.getLogger(SaslServerCallbackHandler.class);
+
+    public SaslServerCallbackHandler(Configuration configuration) throws IOException {
+        AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry("Server");
+
+        if (configurationEntries == null) {
+            String errorMessage = "could not find a 'Server' entry in this configuration: server cannot start.";
+            LOG.error(errorMessage);
+            throw(new IOException(errorMessage));
+        }
+        credentials.clear();
+        for(AppConfigurationEntry entry: configurationEntries) {
+            Map<String,?> options = entry.getOptions();
+            // Populate DIGEST-MD5 user -> password map with JAAS configuration entries from the "Server" section.
+            // Usernames are distinguished from other options by prefixing the username with a "user_" prefix.
+            Iterator it = options.entrySet().iterator();
+            while (it.hasNext()) {
+            Map.Entry pair = (Map.Entry)it.next();
+                    String key = (String)pair.getKey();
+                    if (key.substring(0,5).equals("user_")) {
+                        String userName = key.substring(5);
+                        credentials.put(userName,(String)pair.getValue());
+                    }
+                }
+            }
+            return;
+        }
+
+        public void handle(Callback[] callbacks) throws
+                UnsupportedCallbackException {
+            for (Callback callback : callbacks) {
+                if (callback instanceof NameCallback) {
+                    NameCallback nc = (NameCallback) callback;
+                    // check to see if this user is in the user password database.
+                    if (credentials.get(nc.getDefaultName()) != null) {
+                        nc.setName(nc.getDefaultName());
+                        this.userName = nc.getDefaultName();
+                    }
+                    else { // no such user.
+                        LOG.warn("User '" + nc.getDefaultName() + "' not found in list of DIGEST-MD5 authenticateable users.");
+                    }
+                }
+                else {
+                    if (callback instanceof PasswordCallback) {
+                        PasswordCallback pc = (PasswordCallback) callback;
+
+                        if ((this.userName.equals("super")
+                              &&
+                              (System.getProperty("zookeeper.SASLAuthenticationProvider.superPassword") != null))) {
+                            // superuser: use Java system property for password, if available.
+                            pc.setPassword(System.getProperty("zookeeper.SASLAuthenticationProvider.superPassword").toCharArray());
+                        }
+                        else {
+                            if (this.credentials.get(this.userName) != null) {
+                                pc.setPassword(this.credentials.get(this.userName).toCharArray());
+                            }
+                            else {
+                                LOG.warn("No password found for user: " + this.userName);
+                            }
+                        }
+                    }
+                    else {
+                        if (callback instanceof RealmCallback) {
+                            RealmCallback rc = (RealmCallback) callback;
+                            LOG.debug("client supplied realm: " + rc.getDefaultText());
+                            rc.setText(rc.getDefaultText());
+                        }
+                        else {
+                            if (callback instanceof AuthorizeCallback) {
+                                AuthorizeCallback ac = (AuthorizeCallback) callback;
+
+                                String authenticationID = ac.getAuthenticationID();
+                                String authorizationID = ac.getAuthorizationID();
+
+                                LOG.info("Successfully authenticated client: authenticationID=" + authenticationID + ";  authorizationID=" + authorizationID + ".");
+                                if (authenticationID.equals(authorizationID)) {
+                                    LOG.debug("setAuthorized(true) since " + authenticationID + "==" + authorizationID);
+                                    ac.setAuthorized(true);
+                                } else {
+                                    LOG.debug("setAuthorized(true), even though " + authenticationID + "!=" + authorizationID + ".");
+                                    ac.setAuthorized(true);
+                                }
+                                if (ac.isAuthorized()) {
+                                    LOG.debug("isAuthorized() since ac.isAuthorized() == true");
+                                    // canonicalize authorization id according to system properties:
+                                    // kerberos.removeRealmFromPrincipal(={true,false})
+                                    // kerberos.removeHostFromPrincipal(={true,false})
+                                    KerberosName kerberosName = new KerberosName(authenticationID);
+                                    try {
+                                        String userName = kerberosName.getShortName();
+                                        if (!removeHost() && (kerberosName.getHostName() != null)) {
+                                            userName += "/" + kerberosName.getServiceName();
+                                        }
+                                        if (!removeRealm() && (kerberosName.getRealm() != null)) {
+                                            userName += "@" + kerberosName.getRealm();
+                                        }
+                                        LOG.info("Setting authorizedID: " + userName);
+                                        ac.setAuthorizedID(userName);
+                                    }
+                                    catch (IOException e) {
+                                        LOG.error("Failed to set name based on Kerberos authentication rules.");
+                                    }
+                                }
+                            }
+                        }
+                    }
+                }
+            }
+        }
+
+    private boolean removeRealm() {
+        return ((System.getProperty("zookeeper.kerberos.removeRealmFromPrincipal") != null) &&
+          (System.getProperty("zookeeper.kerberos.removeRealmFromPrincipal").equals("true")));
+    }
+
+    private boolean removeHost() {
+        return ((System.getProperty("zookeeper.kerberos.removeHostFromPrincipal") != null) &&
+          (System.getProperty("zookeeper.kerberos.removeHostFromPrincipal").equals("true")));
+    }
+}

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthFailTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthFailTest.java?rev=1159432&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthFailTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthFailTest.java Thu Aug 18 22:05:35 2011
@@ -0,0 +1,123 @@
+/**
+ * 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.TestableZooKeeper;
+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.junit.Test;
+import org.junit.Assert;
+
+public class SaslAuthFailTest extends ClientBase {
+    static {
+        System.setProperty("zookeeper.authProvider.1","org.apache.zookeeper.server.auth.SASLAuthenticationProvider");
+        System.setProperty("zookeeper.allowSaslFailedClients","true");
+
+        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_super=\"test\";\n" +
+                    "};\n" +
+                    "Client {\n" +
+                    "       org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+                    "       username=\"super\"\n" +
+                    "       password=\"test1\";\n" + // NOTE: wrong password ('test' != 'test1') : this is to test SASL authentication failure.
+                    "};" + "\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.
+        }
+    }
+
+    private AtomicInteger authFailed = new AtomicInteger(0);
+    
+    @Override
+    protected TestableZooKeeper createClient(String hp)
+    throws IOException, InterruptedException
+    {
+        File tmpDir = ClientBase.createTmpDir();
+        File saslConfFile = new File(tmpDir, "jaas_bad_password.conf");
+        FileWriter fwriter = new FileWriter(saslConfFile);
+
+        fwriter.write("" +
+                "Server {\n" +
+                "          org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+                "          user_super=\"test\";\n" +
+                "};\n" +
+                "Client {\n" +
+                "       org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+                "       username=\"super\"\n" +
+                "       password=\"test1\";\n" + // NOTE: wrong password to cause authentication failure : 'test' != 'test1'.
+                "};" + "\n");
+        fwriter.close();
+        System.setProperty("java.security.auth.login.config",saslConfFile.getAbsolutePath());
+        MyWatcher watcher = new MyWatcher();
+        return createClient(watcher, hp);
+    }
+
+    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 testBadSaslAuthNotifiesWatch() throws Exception {
+        ZooKeeper zk = createClient();
+        Thread.sleep(1000);
+        zk.close();
+    }
+
+    
+    @Test
+    public void testAuthFail() 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(Exception 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/SaslAuthTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthTest.java?rev=1159432&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SaslAuthTest.java Thu Aug 18 22:05:35 2011
@@ -0,0 +1,173 @@
+/**
+ * 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.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.TestableZooKeeper;
+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.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class SaslAuthTest extends ClientBase {
+    static {
+        System.setProperty("zookeeper.authProvider.1","org.apache.zookeeper.server.auth.SASLAuthenticationProvider");
+
+        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_super=\"test\";\n" +
+                    "};\n" +
+                    "Client {\n" +
+                    "       org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+                    "       username=\"super\"\n" +
+                    "       password=\"test\";\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);
+    
+    @Override
+    protected TestableZooKeeper createClient(String hp)
+    throws IOException, InterruptedException
+    {
+        File tmpDir = ClientBase.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_super=\"test\";\n" +
+                "};\n" +
+                "Client {\n" +
+                "       org.apache.zookeeper.server.auth.DigestLoginModule required\n" +
+                "       username=\"super\"\n" +
+                "       password=\"test\";\n" +
+                "};" + "\n");
+        fwriter.close();
+        System.setProperty("java.security.auth.login.config",saslConfFile.getAbsolutePath());
+        MyWatcher watcher = new MyWatcher();
+        return createClient(watcher, hp);
+    }
+
+    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 testBadSaslAuthNotifiesWatch() throws Exception {
+        ZooKeeper zk = createClient();
+        Thread.sleep(1000);
+        zk.close();
+    }
+
+    
+    @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);
+        } finally {
+            zk.close();
+        }
+    }
+
+    @Test
+    public void testValidSaslIds() throws Exception {
+        ZooKeeper zk = createClient();
+        Thread.sleep(1000);
+
+        List<String> validIds = new ArrayList<String>();
+        validIds.add("user");
+        validIds.add("service/host.name.com");
+        validIds.add("user@KERB.REALM");
+        validIds.add("service/host.name.com@KERB.REALM");
+
+        int i = 0;
+        for(String validId: validIds) {
+            List<ACL> aclList = new ArrayList<ACL>();
+            ACL acl = new ACL(0,new Id("sasl",validId));
+            aclList.add(acl);
+            zk.create("/valid"+i,null,aclList,CreateMode.PERSISTENT);
+            i++;
+        }
+    }
+
+    @Test
+    public void testInvalidSaslIds() throws Exception {
+        ZooKeeper zk = createClient();
+        Thread.sleep(1000);
+
+        List<String> invalidIds = new ArrayList<String>();
+        invalidIds.add("user@KERB.REALM/server.com");
+        invalidIds.add("user@KERB.REALM1@KERB.REALM2");
+
+        int i = 0;
+        for(String invalidId: invalidIds) {
+            List<ACL> aclList = new ArrayList<ACL>();
+            try {
+                ACL acl = new ACL(0,new Id("sasl",invalidId));
+                aclList.add(acl);
+                zk.create("/invalid"+i,null,aclList,CreateMode.PERSISTENT);
+                Assert.fail("SASLAuthenticationProvider.isValid() failed to catch invalid Id.");
+            }
+            catch (KeeperException.InvalidACLException e) {
+                // ok.
+            }
+            finally {
+                i++;
+            }
+        }
+    }
+
+}

Modified: zookeeper/trunk/src/zookeeper.jute
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/zookeeper.jute?rev=1159432&r1=1159431&r2=1159432&view=diff
==============================================================================
--- zookeeper/trunk/src/zookeeper.jute (original)
+++ zookeeper/trunk/src/zookeeper.jute Thu Aug 18 22:05:35 2011
@@ -116,6 +116,15 @@ module org.apache.zookeeper.proto {
     class SetDataResponse {
         org.apache.zookeeper.data.Stat stat;
     }
+    class GetSASLRequest {
+        buffer token;
+    }
+    class SetSASLRequest {
+        buffer token;
+    }
+    class SetSASLResponse {
+        buffer token;
+    }
     class CreateRequest {
         ustring path;
         buffer data;