You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ae...@apache.org on 2016/10/21 16:44:47 UTC

[31/50] [abbrv] hadoop git commit: HADOOP-12082 Support multiple authentication schemes via AuthenticationFilter

HADOOP-12082 Support multiple authentication schemes via AuthenticationFilter


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

Branch: refs/heads/HDFS-7240
Commit: 4bca385241c0fc8ff168c7b0f2984a7aed2c7492
Parents: d75cbc5
Author: Benoy Antony <be...@apache.org>
Authored: Tue Oct 18 18:32:01 2016 -0700
Committer: Benoy Antony <be...@apache.org>
Committed: Tue Oct 18 18:32:01 2016 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-auth/pom.xml       |  42 +++
 .../client/KerberosAuthenticator.java           |   8 +-
 .../server/AuthenticationFilter.java            |  47 ++-
 .../server/AuthenticationHandler.java           |   2 +-
 .../server/AuthenticationHandlerUtil.java       | 105 ++++++
 .../server/CompositeAuthenticationHandler.java  |  30 ++
 .../authentication/server/HttpConstants.java    |  55 +++
 .../server/LdapAuthenticationHandler.java       | 339 +++++++++++++++++++
 .../MultiSchemeAuthenticationHandler.java       | 209 ++++++++++++
 .../authentication/server/package-info.java     |  27 ++
 .../src/site/markdown/Configuration.md          | 137 ++++++++
 .../client/TestKerberosAuthenticator.java       |  71 +++-
 .../authentication/server/LdapConstants.java    |  31 ++
 .../server/TestLdapAuthenticationHandler.java   | 159 +++++++++
 .../TestMultiSchemeAuthenticationHandler.java   | 189 +++++++++++
 .../DelegationTokenAuthenticationFilter.java    |   9 +-
 .../DelegationTokenAuthenticationHandler.java   |  25 +-
 ...emeDelegationTokenAuthenticationHandler.java | 182 ++++++++++
 hadoop-project/pom.xml                          |   4 +
 19 files changed, 1649 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/pom.xml b/hadoop-common-project/hadoop-auth/pom.xml
index 4cbdc49..0b37715 100644
--- a/hadoop-common-project/hadoop-auth/pom.xml
+++ b/hadoop-common-project/hadoop-auth/pom.xml
@@ -135,6 +135,48 @@
       <groupId>org.apache.kerby</groupId>
       <artifactId>kerb-simplekdc</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.directory.server</groupId>
+      <artifactId>apacheds-core</artifactId>
+      <version>${apacheds.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.directory.server</groupId>
+      <artifactId>apacheds-protocol-ldap</artifactId>
+      <version>${apacheds.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.directory.server</groupId>
+      <artifactId>apacheds-ldif-partition</artifactId>
+      <version>${apacheds.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.directory.api</groupId>
+      <artifactId>api-ldap-codec-core</artifactId>
+      <version>${ldap-api.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.directory.api</groupId>
+      <artifactId>api-ldap-model</artifactId>
+      <version>${ldap-api.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.directory.server</groupId>
+      <artifactId>apacheds-server-integ</artifactId>
+      <version>${apacheds.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.directory.server</groupId>
+      <artifactId>apacheds-core-integ</artifactId>
+      <version>${apacheds.version}</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
index a69ee46..ceec927 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
@@ -14,6 +14,7 @@
 package org.apache.hadoop.security.authentication.client;
 
 import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.security.authentication.server.HttpConstants;
 import org.apache.hadoop.security.authentication.util.AuthToken;
 import org.apache.hadoop.security.authentication.util.KerberosUtil;
 import org.ietf.jgss.GSSContext;
@@ -57,17 +58,18 @@ public class KerberosAuthenticator implements Authenticator {
   /**
    * HTTP header used by the SPNEGO server endpoint during an authentication sequence.
    */
-  public static final String WWW_AUTHENTICATE = "WWW-Authenticate";
+  public static final String WWW_AUTHENTICATE =
+      HttpConstants.WWW_AUTHENTICATE_HEADER;
 
   /**
    * HTTP header used by the SPNEGO client endpoint during an authentication sequence.
    */
-  public static final String AUTHORIZATION = "Authorization";
+  public static final String AUTHORIZATION = HttpConstants.AUTHORIZATION_HEADER;
 
   /**
    * HTTP header prefix used by the SPNEGO client/server endpoints during an authentication sequence.
    */
-  public static final String NEGOTIATE = "Negotiate";
+  public static final String NEGOTIATE = HttpConstants.NEGOTIATE;
 
   private static final String AUTH_HTTP_METHOD = "OPTIONS";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
index 5262fdc..264d991 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
@@ -165,15 +165,9 @@ public class AuthenticationFilter implements Filter {
           PseudoAuthenticationHandler.TYPE + "|" + 
           KerberosAuthenticationHandler.TYPE + "|<class>");
     }
-    if (authHandlerName.toLowerCase(Locale.ENGLISH).equals(
-        PseudoAuthenticationHandler.TYPE)) {
-      authHandlerClassName = PseudoAuthenticationHandler.class.getName();
-    } else if (authHandlerName.toLowerCase(Locale.ENGLISH).equals(
-        KerberosAuthenticationHandler.TYPE)) {
-      authHandlerClassName = KerberosAuthenticationHandler.class.getName();
-    } else {
-      authHandlerClassName = authHandlerName;
-    }
+    authHandlerClassName =
+        AuthenticationHandlerUtil
+            .getAuthenticationHandlerClassName(authHandlerName);
     maxInactiveInterval = Long.parseLong(config.getProperty(
         AUTH_TOKEN_MAX_INACTIVE_INTERVAL, "-1")); // By default, disable.
     if (maxInactiveInterval > 0) {
@@ -452,7 +446,8 @@ public class AuthenticationFilter implements Filter {
     }
     if (tokenStr != null) {
       token = AuthenticationToken.parse(tokenStr);
-      if (!token.getType().equals(authHandler.getType())) {
+      boolean match = verifyTokenType(getAuthenticationHandler(), token);
+      if (!match) {
         throw new AuthenticationException("Invalid AuthenticationToken type");
       }
       if (token.isExpired()) {
@@ -463,6 +458,38 @@ public class AuthenticationFilter implements Filter {
   }
 
   /**
+   * This method verifies if the specified token type matches one of the the
+   * token types supported by a specified {@link AuthenticationHandler}. This
+   * method is specifically designed to work with
+   * {@link CompositeAuthenticationHandler} implementation which supports
+   * multiple authentication schemes while the {@link AuthenticationHandler}
+   * interface supports a single type via
+   * {@linkplain AuthenticationHandler#getType()} method.
+   *
+   * @param handler The authentication handler whose supported token types
+   *                should be used for verification.
+   * @param token   The token whose type needs to be verified.
+   * @return true   If the token type matches one of the supported token types
+   *         false  Otherwise
+   */
+  protected boolean verifyTokenType(AuthenticationHandler handler,
+      AuthenticationToken token) {
+    if(!(handler instanceof CompositeAuthenticationHandler)) {
+      return handler.getType().equals(token.getType());
+    }
+    boolean match = false;
+    Collection<String> tokenTypes =
+        ((CompositeAuthenticationHandler) handler).getTokenTypes();
+    for (String tokenType : tokenTypes) {
+      if (tokenType.equals(token.getType())) {
+        match = true;
+        break;
+      }
+    }
+    return match;
+  }
+
+  /**
    * If the request has a valid authentication token it allows the request to continue to the target resource,
    * otherwise it triggers an authentication sequence using the configured {@link AuthenticationHandler}.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationHandler.java
index 797e95a..4a95853 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationHandler.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationHandler.java
@@ -29,7 +29,7 @@ import java.util.Properties;
  */
 public interface AuthenticationHandler {
 
-  public static final String WWW_AUTHENTICATE = "WWW-Authenticate";
+  String WWW_AUTHENTICATE = HttpConstants.WWW_AUTHENTICATE_HEADER;
 
   /**
    * Returns the authentication type of the authentication handler.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationHandlerUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationHandlerUtil.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationHandlerUtil.java
new file mode 100644
index 0000000..d881c65
--- /dev/null
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationHandlerUtil.java
@@ -0,0 +1,105 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.security.authentication.server;
+
+import static org.apache.hadoop.security.authentication.server.HttpConstants.NEGOTIATE;
+import static org.apache.hadoop.security.authentication.server.HttpConstants.BASIC;
+import static org.apache.hadoop.security.authentication.server.HttpConstants.DIGEST;
+
+import java.util.Locale;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * This is a utility class designed to provide functionality related to
+ * {@link AuthenticationHandler}.
+ */
+public final class AuthenticationHandlerUtil {
+
+  /**
+   * This class should only contain the static utility methods. Hence it is not
+   * intended to be instantiated.
+   */
+  private AuthenticationHandlerUtil() {
+  }
+
+  /**
+   * This method provides an instance of {@link AuthenticationHandler} based on
+   * specified <code>authHandlerName</code>.
+   *
+   * @param authHandler The short-name (or fully qualified class name) of the
+   *          authentication handler.
+   * @return an instance of AuthenticationHandler implementation.
+   */
+  public static String getAuthenticationHandlerClassName(String authHandler) {
+    String handlerName =
+        Preconditions.checkNotNull(authHandler).toLowerCase(Locale.ENGLISH);
+
+    String authHandlerClassName = null;
+
+    if (handlerName.equals(PseudoAuthenticationHandler.TYPE)) {
+      authHandlerClassName = PseudoAuthenticationHandler.class.getName();
+    } else if (handlerName.equals(KerberosAuthenticationHandler.TYPE)) {
+      authHandlerClassName = KerberosAuthenticationHandler.class.getName();
+    } else if (handlerName.equals(LdapAuthenticationHandler.TYPE)) {
+      authHandlerClassName = LdapAuthenticationHandler.class.getName();
+    } else if (handlerName.equals(MultiSchemeAuthenticationHandler.TYPE)) {
+      authHandlerClassName = MultiSchemeAuthenticationHandler.class.getName();
+    } else {
+      authHandlerClassName = authHandler;
+    }
+
+    return authHandlerClassName;
+  }
+
+  /**
+   * This method checks if the specified HTTP authentication <code>scheme</code>
+   * value is valid.
+   *
+   * @param scheme HTTP authentication scheme to be checked
+   * @return Canonical representation of HTTP authentication scheme
+   * @throws IllegalArgumentException In case the specified value is not a valid
+   *           HTTP authentication scheme.
+   */
+  public static String checkAuthScheme(String scheme) {
+    if (BASIC.equalsIgnoreCase(scheme)) {
+      return BASIC;
+    } else if (NEGOTIATE.equalsIgnoreCase(scheme)) {
+      return NEGOTIATE;
+    } else if (DIGEST.equalsIgnoreCase(scheme)) {
+      return DIGEST;
+    }
+    throw new IllegalArgumentException(String.format(
+        "Unsupported HTTP authentication scheme %s ."
+            + " Supported schemes are [%s, %s, %s]", scheme, BASIC, NEGOTIATE,
+        DIGEST));
+  }
+
+  /**
+   * This method checks if the specified <code>authToken</code> belongs to the
+   * specified HTTP authentication <code>scheme</code>.
+   *
+   * @param scheme HTTP authentication scheme to be checked
+   * @param auth Authentication header value which is to be compared with the
+   *          authentication scheme.
+   * @return true If the authentication header value corresponds to the
+   *         specified authentication scheme false Otherwise.
+   */
+  public static boolean matchAuthScheme(String scheme, String auth) {
+    scheme = Preconditions.checkNotNull(scheme).trim();
+    auth = Preconditions.checkNotNull(auth).trim();
+    return auth.regionMatches(true, 0, scheme, 0, scheme.length());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/CompositeAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/CompositeAuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/CompositeAuthenticationHandler.java
new file mode 100644
index 0000000..b1c73a3
--- /dev/null
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/CompositeAuthenticationHandler.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.server;
+
+import java.util.Collection;
+
+/**
+ * Interface to support multiple authentication mechanisms simultaneously.
+ *
+ */
+public interface CompositeAuthenticationHandler extends AuthenticationHandler {
+  /**
+   * This method returns the token types supported by this authentication
+   * handler.
+   *
+   * @return the token types supported by this authentication handler.
+   */
+  Collection<String> getTokenTypes();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/HttpConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/HttpConstants.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/HttpConstants.java
new file mode 100644
index 0000000..4268b6f
--- /dev/null
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/HttpConstants.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.server;
+
+/**
+ * This class defines constants used for HTTP protocol entities (such as
+ * headers, methods and their values).
+ */
+public final class HttpConstants {
+
+  /**
+   * This class defines the HTTP protocol constants. Hence it is not intended
+   * to be instantiated.
+   */
+  private HttpConstants() {
+  }
+
+  /**
+   * HTTP header used by the server endpoint during an authentication sequence.
+   */
+  public static final String WWW_AUTHENTICATE_HEADER = "WWW-Authenticate";
+
+  /**
+   * HTTP header used by the client endpoint during an authentication sequence.
+   */
+  public static final String AUTHORIZATION_HEADER = "Authorization";
+
+  /**
+   * HTTP header prefix used by the SPNEGO client/server endpoints during an
+   * authentication sequence.
+   */
+  public static final String NEGOTIATE = "Negotiate";
+
+  /**
+   * HTTP header prefix used during the Basic authentication sequence.
+   */
+  public static final String BASIC = "Basic";
+
+  /**
+   * HTTP header prefix used during the Basic authentication sequence.
+   */
+  public static final String DIGEST = "Digest";
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/LdapAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/LdapAuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/LdapAuthenticationHandler.java
new file mode 100644
index 0000000..f452317
--- /dev/null
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/LdapAuthenticationHandler.java
@@ -0,0 +1,339 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.server;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Hashtable;
+import java.util.Properties;
+
+import javax.naming.Context;
+import javax.naming.NamingException;
+import javax.naming.directory.InitialDirContext;
+import javax.naming.ldap.InitialLdapContext;
+import javax.naming.ldap.LdapContext;
+import javax.naming.ldap.StartTlsRequest;
+import javax.naming.ldap.StartTlsResponse;
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.SSLSession;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+/**
+ * The {@link LdapAuthenticationHandler} implements the BASIC authentication
+ * mechanism for HTTP using LDAP back-end.
+ *
+ * The supported configuration properties are:
+ * <ul>
+ * <li>ldap.providerurl: The url of the LDAP server. It does not have a default
+ * value.</li>
+ * <li>ldap.basedn: the base distinguished name (DN) to be used with the LDAP
+ * server. This value is appended to the provided user id for authentication
+ * purpose. It does not have a default value.</li>
+ * <li>ldap.binddomain: the LDAP bind domain value to be used with the LDAP
+ * server. This property is optional and useful only in case of Active
+ * Directory server.
+ * <li>ldap.enablestarttls: A boolean value used to define if the LDAP server
+ * supports 'StartTLS' extension.</li>
+ * </ul>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class LdapAuthenticationHandler implements AuthenticationHandler {
+  private static Logger logger = LoggerFactory
+      .getLogger(LdapAuthenticationHandler.class);
+
+  /**
+   * Constant that identifies the authentication mechanism.
+   */
+  public static final String TYPE = "ldap";
+
+  /**
+   * Constant that identifies the authentication mechanism to be used with the
+   * LDAP server.
+   */
+  public static final String SECURITY_AUTHENTICATION = "simple";
+
+  /**
+   * Constant for the configuration property that indicates the url of the LDAP
+   * server.
+   */
+  public static final String PROVIDER_URL = TYPE + ".providerurl";
+
+  /**
+   * Constant for the configuration property that indicates the base
+   * distinguished name (DN) to be used with the LDAP server. This value is
+   * appended to the provided user id for authentication purpose.
+   */
+  public static final String BASE_DN = TYPE + ".basedn";
+
+  /**
+   * Constant for the configuration property that indicates the LDAP bind
+   * domain value to be used with the LDAP server.
+   */
+  public static final String LDAP_BIND_DOMAIN = TYPE + ".binddomain";
+
+  /**
+   * Constant for the configuration property that indicates the base
+   * distinguished name (DN) to be used with the LDAP server. This value is
+   * appended to the provided user id for authentication purpose.
+   */
+  public static final String ENABLE_START_TLS = TYPE + ".enablestarttls";
+
+  private String ldapDomain;
+  private String baseDN;
+  private String providerUrl;
+  private Boolean enableStartTls;
+  private Boolean disableHostNameVerification;
+
+  /**
+   * Configure StartTLS LDAP extension for this handler.
+   *
+   * @param enableStartTls true If the StartTLS LDAP extension is to be enabled
+   *          false otherwise
+   */
+  @VisibleForTesting
+  public void setEnableStartTls(Boolean enableStartTls) {
+    this.enableStartTls = enableStartTls;
+  }
+
+  /**
+   * Configure the Host name verification for this handler. This method is
+   * introduced only for unit testing and should never be used in production.
+   *
+   * @param disableHostNameVerification true to disable host-name verification
+   *          false otherwise
+   */
+  @VisibleForTesting
+  public void setDisableHostNameVerification(
+      Boolean disableHostNameVerification) {
+    this.disableHostNameVerification = disableHostNameVerification;
+  }
+
+  @Override
+  public String getType() {
+    return TYPE;
+  }
+
+  @Override
+  public void init(Properties config) throws ServletException {
+    this.baseDN = config.getProperty(BASE_DN);
+    this.providerUrl = config.getProperty(PROVIDER_URL);
+    this.ldapDomain = config.getProperty(LDAP_BIND_DOMAIN);
+    this.enableStartTls =
+        Boolean.valueOf(config.getProperty(ENABLE_START_TLS, "false"));
+
+    Preconditions
+        .checkNotNull(this.providerUrl, "The LDAP URI can not be null");
+    Preconditions.checkArgument((this.baseDN == null)
+        ^ (this.ldapDomain == null),
+        "Either LDAP base DN or LDAP domain value needs to be specified");
+    if (this.enableStartTls) {
+      String tmp = this.providerUrl.toLowerCase();
+      Preconditions.checkArgument(!tmp.startsWith("ldaps"),
+          "Can not use ldaps and StartTLS option at the same time");
+    }
+  }
+
+  @Override
+  public void destroy() {
+  }
+
+  @Override
+  public boolean managementOperation(AuthenticationToken token,
+      HttpServletRequest request, HttpServletResponse response)
+      throws IOException, AuthenticationException {
+    return true;
+  }
+
+  @Override
+  public AuthenticationToken authenticate(HttpServletRequest request,
+      HttpServletResponse response)
+          throws IOException, AuthenticationException {
+    AuthenticationToken token = null;
+    String authorization =
+        request.getHeader(HttpConstants.AUTHORIZATION_HEADER);
+
+    if (authorization == null
+        || !AuthenticationHandlerUtil.matchAuthScheme(HttpConstants.BASIC,
+            authorization)) {
+      response.setHeader(WWW_AUTHENTICATE, HttpConstants.BASIC);
+      response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+      if (authorization == null) {
+        logger.trace("Basic auth starting");
+      } else {
+        logger.warn("'" + HttpConstants.AUTHORIZATION_HEADER
+            + "' does not start with '" + HttpConstants.BASIC + "' :  {}",
+            authorization);
+      }
+    } else {
+      authorization =
+          authorization.substring(HttpConstants.BASIC.length()).trim();
+      final Base64 base64 = new Base64(0);
+      // As per RFC7617, UTF-8 charset should be used for decoding.
+      String[] credentials = new String(base64.decode(authorization),
+          StandardCharsets.UTF_8).split(":", 2);
+      if (credentials.length == 2) {
+        token = authenticateUser(credentials[0], credentials[1]);
+        response.setStatus(HttpServletResponse.SC_OK);
+      }
+    }
+    return token;
+  }
+
+  private AuthenticationToken authenticateUser(String userName,
+      String password) throws AuthenticationException {
+    if (userName == null || userName.isEmpty()) {
+      throw new AuthenticationException("Error validating LDAP user:"
+          + " a null or blank username has been provided");
+    }
+
+    // If the domain is available in the config, then append it unless domain
+    // is already part of the username. LDAP providers like Active Directory
+    // use a fully qualified user name like foo@bar.com.
+    if (!hasDomain(userName) && ldapDomain != null) {
+      userName = userName + "@" + ldapDomain;
+    }
+
+    if (password == null || password.isEmpty() ||
+        password.getBytes(StandardCharsets.UTF_8)[0] == 0) {
+      throw new AuthenticationException("Error validating LDAP user:"
+          + " a null or blank password has been provided");
+    }
+
+    // setup the security principal
+    String bindDN;
+    if (baseDN == null) {
+      bindDN = userName;
+    } else {
+      bindDN = "uid=" + userName + "," + baseDN;
+    }
+
+    if (this.enableStartTls) {
+      authenticateWithTlsExtension(bindDN, password);
+    } else {
+      authenticateWithoutTlsExtension(bindDN, password);
+    }
+
+    return new AuthenticationToken(userName, userName, TYPE);
+  }
+
+  private void authenticateWithTlsExtension(String userDN, String password)
+      throws AuthenticationException {
+    LdapContext ctx = null;
+    Hashtable<String, Object> env = new Hashtable<String, Object>();
+    env.put(Context.INITIAL_CONTEXT_FACTORY,
+        "com.sun.jndi.ldap.LdapCtxFactory");
+    env.put(Context.PROVIDER_URL, providerUrl);
+
+    try {
+      // Create initial context
+      ctx = new InitialLdapContext(env, null);
+      // Establish TLS session
+      StartTlsResponse tls =
+          (StartTlsResponse) ctx.extendedOperation(new StartTlsRequest());
+
+      if (disableHostNameVerification) {
+        tls.setHostnameVerifier(new HostnameVerifier() {
+          @Override
+          public boolean verify(String hostname, SSLSession session) {
+            return true;
+          }
+        });
+      }
+
+      tls.negotiate();
+
+      // Initialize security credentials & perform read operation for
+      // verification.
+      ctx.addToEnvironment(Context.SECURITY_AUTHENTICATION,
+          SECURITY_AUTHENTICATION);
+      ctx.addToEnvironment(Context.SECURITY_PRINCIPAL, userDN);
+      ctx.addToEnvironment(Context.SECURITY_CREDENTIALS, password);
+      ctx.lookup(userDN);
+      logger.debug("Authentication successful for {}", userDN);
+
+    } catch (NamingException | IOException ex) {
+      throw new AuthenticationException("Error validating LDAP user", ex);
+    } finally {
+      if (ctx != null) {
+        try {
+          ctx.close();
+        } catch (NamingException e) { /* Ignore. */
+        }
+      }
+    }
+  }
+
+  private void authenticateWithoutTlsExtension(String userDN, String password)
+      throws AuthenticationException {
+    Hashtable<String, Object> env = new Hashtable<String, Object>();
+    env.put(Context.INITIAL_CONTEXT_FACTORY,
+        "com.sun.jndi.ldap.LdapCtxFactory");
+    env.put(Context.PROVIDER_URL, providerUrl);
+    env.put(Context.SECURITY_AUTHENTICATION, SECURITY_AUTHENTICATION);
+    env.put(Context.SECURITY_PRINCIPAL, userDN);
+    env.put(Context.SECURITY_CREDENTIALS, password);
+
+    try {
+      // Create initial context
+      Context ctx = new InitialDirContext(env);
+      ctx.close();
+      logger.debug("Authentication successful for {}", userDN);
+
+    } catch (NamingException e) {
+      throw new AuthenticationException("Error validating LDAP user", e);
+    }
+  }
+
+  private static boolean hasDomain(String userName) {
+    return (indexOfDomainMatch(userName) > 0);
+  }
+
+  /*
+   * Get the index separating the user name from domain name (the user's name
+   * up to the first '/' or '@').
+   *
+   * @param userName full user name.
+   *
+   * @return index of domain match or -1 if not found
+   */
+  private static int indexOfDomainMatch(String userName) {
+    if (userName == null) {
+      return -1;
+    }
+
+    int idx = userName.indexOf('/');
+    int idx2 = userName.indexOf('@');
+    int endIdx = Math.min(idx, idx2); // Use the earlier match.
+    // Unless at least one of '/' or '@' was not found, in
+    // which case, user the latter match.
+    if (endIdx == -1) {
+      endIdx = Math.max(idx, idx2);
+    }
+    return endIdx;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/MultiSchemeAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/MultiSchemeAuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/MultiSchemeAuthenticationHandler.java
new file mode 100644
index 0000000..aa49100
--- /dev/null
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/MultiSchemeAuthenticationHandler.java
@@ -0,0 +1,209 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.server;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
+
+/**
+ * The {@link MultiSchemeAuthenticationHandler} supports configuring multiple
+ * authentication mechanisms simultaneously. e.g. server can support multiple
+ * authentication mechanisms such as Kerberos (SPENGO) and LDAP. During the
+ * authentication phase, server will specify all possible authentication schemes
+ * and let client choose the appropriate scheme. Please refer to RFC-2616 and
+ * HADOOP-12082 for more details.
+ * <p>
+ * The supported configuration properties are:
+ * <ul>
+ * <li>multi-scheme-auth-handler.schemes: A comma separated list of HTTP
+ * authentication mechanisms supported by this handler. It does not have a
+ * default value. e.g. multi-scheme-auth-handler.schemes=basic,negotiate
+ * <li>multi-scheme-auth-handler.schemes.${scheme-name}.handler: The
+ * authentication handler implementation to be used for the specified
+ * authentication scheme. It does not have a default value. e.g.
+ * multi-scheme-auth-handler.schemes.negotiate.handler=kerberos
+ * </ul>
+ *
+ * It expected that for every authentication scheme specified in
+ * multi-scheme-auth-handler.schemes property, a handler needs to be configured.
+ * Note that while scheme values in 'multi-scheme-auth-handler.schemes' property
+ * are case-insensitive, the scheme value in the handler configuration property
+ * name must be lower case. i.e. property name such as
+ * multi-scheme-auth-handler.schemes.Negotiate.handler is invalid.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MultiSchemeAuthenticationHandler implements
+    CompositeAuthenticationHandler {
+  private static Logger logger = LoggerFactory
+      .getLogger(MultiSchemeAuthenticationHandler.class);
+  public static final String SCHEMES_PROPERTY =
+      "multi-scheme-auth-handler.schemes";
+  public static final String AUTH_HANDLER_PROPERTY =
+      "multi-scheme-auth-handler.schemes.%s.handler";
+  private static final Splitter STR_SPLITTER = Splitter.on(',').trimResults()
+      .omitEmptyStrings();
+
+  private final Map<String, AuthenticationHandler> schemeToAuthHandlerMapping =
+      new HashMap<>();
+  private final Collection<String> types = new HashSet<>();
+  private final String authType;
+
+  /**
+   * Constant that identifies the authentication mechanism.
+   */
+  public static final String TYPE = "multi-scheme";
+
+  public MultiSchemeAuthenticationHandler() {
+    this(TYPE);
+  }
+
+  public MultiSchemeAuthenticationHandler(String authType) {
+    this.authType = authType;
+  }
+
+  @Override
+  public String getType() {
+    return authType;
+  }
+
+  /**
+   * This method returns the token types supported by this authentication
+   * handler.
+   *
+   * @return the token types supported by this authentication handler.
+   */
+  @Override
+  public Collection<String> getTokenTypes() {
+    return types;
+  }
+
+  @Override
+  public void init(Properties config) throws ServletException {
+    // Useful for debugging purpose.
+    for (Map.Entry prop : config.entrySet()) {
+      logger.info("{} : {}", prop.getKey(), prop.getValue());
+    }
+
+    this.types.clear();
+
+    String schemesProperty =
+        Preconditions.checkNotNull(config.getProperty(SCHEMES_PROPERTY),
+            "%s system property is not specified.", SCHEMES_PROPERTY);
+    for (String scheme : STR_SPLITTER.split(schemesProperty)) {
+      scheme = AuthenticationHandlerUtil.checkAuthScheme(scheme);
+      if (schemeToAuthHandlerMapping.containsKey(scheme)) {
+        throw new IllegalArgumentException("Handler is already specified for "
+            + scheme + " authentication scheme.");
+      }
+
+      String authHandlerPropName =
+          String.format(AUTH_HANDLER_PROPERTY, scheme).toLowerCase();
+      String authHandlerName = config.getProperty(authHandlerPropName);
+      Preconditions.checkNotNull(authHandlerName,
+          "No auth handler configured for scheme %s.", scheme);
+
+      String authHandlerClassName =
+          AuthenticationHandlerUtil
+              .getAuthenticationHandlerClassName(authHandlerName);
+      AuthenticationHandler handler =
+          initializeAuthHandler(authHandlerClassName, config);
+      schemeToAuthHandlerMapping.put(scheme, handler);
+      types.add(handler.getType());
+    }
+    logger.info("Successfully initialized MultiSchemeAuthenticationHandler");
+  }
+
+  protected AuthenticationHandler initializeAuthHandler(
+      String authHandlerClassName, Properties config) throws ServletException {
+    try {
+      Preconditions.checkNotNull(authHandlerClassName);
+      logger.debug("Initializing Authentication handler of type "
+          + authHandlerClassName);
+      Class<?> klass =
+          Thread.currentThread().getContextClassLoader()
+              .loadClass(authHandlerClassName);
+      AuthenticationHandler authHandler =
+          (AuthenticationHandler) klass.newInstance();
+      authHandler.init(config);
+      logger.info("Successfully initialized Authentication handler of type "
+          + authHandlerClassName);
+      return authHandler;
+    } catch (ClassNotFoundException | InstantiationException
+        | IllegalAccessException ex) {
+      logger.error("Failed to initialize authentication handler "
+          + authHandlerClassName, ex);
+      throw new ServletException(ex);
+    }
+  }
+
+  @Override
+  public void destroy() {
+    for (AuthenticationHandler handler : schemeToAuthHandlerMapping.values()) {
+      handler.destroy();
+    }
+  }
+
+  @Override
+  public boolean managementOperation(AuthenticationToken token,
+      HttpServletRequest request, HttpServletResponse response)
+      throws IOException, AuthenticationException {
+    return true;
+  }
+
+  @Override
+  public AuthenticationToken authenticate(HttpServletRequest request,
+      HttpServletResponse response)
+          throws IOException, AuthenticationException {
+    String authorization =
+        request.getHeader(HttpConstants.AUTHORIZATION_HEADER);
+    if (authorization != null) {
+      for (String scheme : schemeToAuthHandlerMapping.keySet()) {
+        if (AuthenticationHandlerUtil.matchAuthScheme(scheme, authorization)) {
+          AuthenticationHandler handler =
+              schemeToAuthHandlerMapping.get(scheme);
+          AuthenticationToken token = handler.authenticate(request, response);
+          logger.trace("Token generated with type {}", token.getType());
+          return token;
+        }
+      }
+    }
+
+    // Handle the case when (authorization == null) or an invalid authorization
+    // header (e.g. a header value without the scheme name).
+    response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+    for (String scheme : schemeToAuthHandlerMapping.keySet()) {
+      response.addHeader(HttpConstants.WWW_AUTHENTICATE_HEADER, scheme);
+    }
+
+    return null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/package-info.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/package-info.java
new file mode 100644
index 0000000..a42f092
--- /dev/null
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/**
+ * Provides the server-side framework for authentication.
+ */
+@InterfaceAudience.LimitedPrivate({ "HBase", "HDFS", "MapReduce" })
+@InterfaceStability.Evolving
+package org.apache.hadoop.security.authentication.server;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/src/site/markdown/Configuration.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/site/markdown/Configuration.md b/hadoop-common-project/hadoop-auth/src/site/markdown/Configuration.md
index 2a1f73b..d9b275e 100644
--- a/hadoop-common-project/hadoop-auth/src/site/markdown/Configuration.md
+++ b/hadoop-common-project/hadoop-auth/src/site/markdown/Configuration.md
@@ -225,6 +225,143 @@ The AltKerberos authentication mechanism is a partially implemented derivative o
     </web-app>
 ```
 
+### LDAP Configuration
+
+**IMPORTANT**: A LDAP server must be configured and running. When TLS is enabled for communication with LDAP server (either via ldaps scheme or 'start TLS' extension), configure the public certificate of the LDAP server in the local truststore.
+
+The LDAP authentication mechanism uses HTTP Basic authentication scheme to verify user specified credentials against a configured LDAP (or Active
+Directory) server. The authentication filter must be configured with the following init parameters:
+
+*   `[PREFIX.]type`: The keyword `ldap`.
+
+*   `[PREFIX.]ldap.providerurl`: The url of the LDAP server.
+
+*   `[PREFIX.]ldap.basedn`: The base distinguished name (DN) to be used with the LDAP server. This value is appended to the provided user id for authentication purpose. This property is not useful in case of Active Directory server.
+
+*   `[PREFIX.]ldap.binddomain`: The LDAP bind domain value to be used with the LDAP server. This property is optional and useful only in case of Active Directory server (e.g. example.com).
+
+*   `[PREFIX.]ldap.enablestarttls`: A boolean value used to define if the LDAP server supports 'StartTLS' extension.
+
+**Example**:
+
+```xml
+    <web-app version="2.5" xmlns="http://java.sun.com/xml/ns/javaee">
+        ...
+
+        <filter>
+            <filter-name>authFilter</filter-name>
+            <filter-class>org.apache.hadoop.security.authentication.server.AuthenticationFilter</filter-class>
+            <init-param>
+                <param-name>type</param-name>
+                <param-value>ldap</param-value>
+            </init-param>
+            <init-param>
+                <param-name>ldap.providerurl</param-name>
+                <param-value>ldap://ldap-server-host:8920</param-value>
+            </init-param>
+            <init-param>
+                <param-name>ldap.basedn</param-name>
+                <param-value>ou=users,dc=example,dc=com</param-value>
+            </init-param>
+            <init-param>
+                <param-name>ldap.enablestarttls</param-name>
+                <param-value>true</param-value>
+            </init-param>
+         </filter>
+
+        <filter-mapping>
+            <filter-name>authFilter</filter-name>
+            <url-pattern>/ldap/*</url-pattern>
+        </filter-mapping>
+
+        ...
+    </web-app>
+```
+
+### Multi-scheme Configuration
+
+**IMPORTANT**: This configuration supports multiple authentication mechanisms (e.g. kerberos, ldap etc.) together. Please refer to the documentation for each individual scheme for configuration related details.
+
+The multi-scheme authentication mechanism supports multiple authentication mechanisms (e.g. kerberos, ldap etc.) by implementing a HTTP auth negotiation mechanism (Please refer to RFC-2616). For enabling each type of authentication mechanism (e.g. ldap) a corresponding authentication handler must be configured. Please refer to following configuration parameters:
+
+*   `[PREFIX.]type`: The keyword `multi-scheme`.
+
+*   `[PREFIX.]multi-scheme-auth-handler.schemes`: A comma separated list of HTTP authentication mechanisms supported by this handler. It is a required parameter and it does not have a default value (e.g. multi-scheme-auth-handler.schemes=basic,negotiate).
+
+*   `[PREFIX.]multi-scheme-auth-handler.schemes.<scheme-name>.handler`: The authentication handler implementation to be used for the specified authentication scheme. It does not have a default value (e.g. multi-scheme-auth-handler.schemes.negotiate.handler=kerberos). Add this handler configuration for each of the scheme configured.
+
+In addition to these parameters, please specify the init parameters for each handler configured as well.
+
+
+**Example**:
+
+```xml
+    <web-app version="2.5" xmlns="http://java.sun.com/xml/ns/javaee">
+        ...
+
+        <filter>
+            <filter-name>authFilter</filter-name>
+            <filter-class>org.apache.hadoop.security.authentication.server.AuthenticationFilter</filter-class>
+            <init-param>
+                <param-name>type</param-name>
+                <param-value>multi-scheme</param-value>
+            </init-param>
+            <init-param>
+                <param-name>multi-scheme-auth-handler.schemes</param-name>
+                <param-value>basic,negotiate</param-value>
+            </init-param>
+            <init-param>
+                <param-name>multi-scheme-auth-handler.basic.handler</param-name>
+                <param-value>ldap</param-value>
+            </init-param>
+            <init-param>
+                <param-name>multi-scheme-auth-handler.negotiate.handler</param-name>
+                <param-value>kerberos</param-value>
+            </init-param>
+            <init-param>
+                <param-name>ldap.providerurl</param-name>
+                <param-value>ldap://ldap-server-host:8920</param-value>
+            </init-param>
+            <init-param>
+                <param-name>ldap.basedn</param-name>
+                <param-value>ou=users,dc=example,dc=com</param-value>
+            </init-param>
+            <init-param>
+                <param-name>ldap.enablestarttls</param-name>
+                <param-value>true</param-value>
+            </init-param>
+            <init-param>
+                <param-name>token.validity</param-name>
+                <param-value>30</param-value>
+            </init-param>
+            <init-param>
+                <param-name>cookie.domain</param-name>
+                <param-value>.foo.com</param-value>
+            </init-param>
+            <init-param>
+                <param-name>cookie.path</param-name>
+                <param-value>/</param-value>
+            </init-param>
+            <init-param>
+                <param-name>kerberos.principal</param-name>
+                <param-value>HTTP/localhost@LOCALHOST</param-value>
+            </init-param>
+            <init-param>
+                <param-name>kerberos.keytab</param-name>
+                <param-value>/tmp/auth.keytab</param-value>
+            </init-param>
+         </filter>
+
+        <filter-mapping>
+            <filter-name>authFilter</filter-name>
+            <url-pattern>/multi-scheme/*</url-pattern>
+        </filter-mapping>
+
+        ...
+    </web-app>
+```
+
+
 ### SignerSecretProvider Configuration
 
 The SignerSecretProvider is used to provide more advanced behaviors for the secret used for signing the HTTP Cookies.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java
index 6c49d15..8ba1205 100644
--- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java
+++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java
@@ -13,15 +13,22 @@
  */
 package org.apache.hadoop.security.authentication.client;
 
+import static org.apache.hadoop.security.authentication.server.MultiSchemeAuthenticationHandler.SCHEMES_PROPERTY;
+import static org.apache.hadoop.security.authentication.server.MultiSchemeAuthenticationHandler.AUTH_HANDLER_PROPERTY;
+import static org.apache.hadoop.security.authentication.server.AuthenticationFilter.AUTH_TYPE;
+import static org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler.PRINCIPAL;
+import static org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler.KEYTAB;
+import static org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler.NAME_RULES;
+
 import org.apache.hadoop.minikdc.KerberosSecurityTestcase;
 import org.apache.hadoop.security.authentication.KerberosTestUtils;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.hadoop.security.authentication.server.MultiSchemeAuthenticationHandler;
 import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
 import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
 import org.junit.runner.RunWith;
 import org.junit.Test;
 
@@ -71,6 +78,19 @@ public class TestKerberosAuthenticator extends KerberosSecurityTestcase {
     return props;
   }
 
+  private Properties getMultiAuthHandlerConfiguration() {
+    Properties props = new Properties();
+    props.setProperty(AUTH_TYPE, MultiSchemeAuthenticationHandler.TYPE);
+    props.setProperty(SCHEMES_PROPERTY, "negotiate");
+    props.setProperty(String.format(AUTH_HANDLER_PROPERTY, "negotiate"),
+        "kerberos");
+    props.setProperty(PRINCIPAL, KerberosTestUtils.getServerPrincipal());
+    props.setProperty(KEYTAB, KerberosTestUtils.getKeytabFile());
+    props.setProperty(NAME_RULES,
+        "RULE:[1:$1@$0](.*@" + KerberosTestUtils.getRealm() + ")s/@.*//\n");
+    return props;
+  }
+
   @Test(timeout=60000)
   public void testFallbacktoPseudoAuthenticator() throws Exception {
     AuthenticatorTestCase auth = new AuthenticatorTestCase(useTomcat);
@@ -162,4 +182,53 @@ public class TestKerberosAuthenticator extends KerberosSecurityTestcase {
       }
     });
   }
+
+  @Test(timeout = 60000)
+  public void testNotAuthenticatedWithMultiAuthHandler() throws Exception {
+    AuthenticatorTestCase auth = new AuthenticatorTestCase(useTomcat);
+    AuthenticatorTestCase
+        .setAuthenticationHandlerConfig(getMultiAuthHandlerConfiguration());
+    auth.start();
+    try {
+      URL url = new URL(auth.getBaseURL());
+      HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+      conn.connect();
+      Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED,
+          conn.getResponseCode());
+      Assert.assertTrue(conn
+          .getHeaderField(KerberosAuthenticator.WWW_AUTHENTICATE) != null);
+    } finally {
+      auth.stop();
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testAuthenticationWithMultiAuthHandler() throws Exception {
+    final AuthenticatorTestCase auth = new AuthenticatorTestCase(useTomcat);
+    AuthenticatorTestCase
+        .setAuthenticationHandlerConfig(getMultiAuthHandlerConfiguration());
+    KerberosTestUtils.doAsClient(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        auth._testAuthentication(new KerberosAuthenticator(), false);
+        return null;
+      }
+    });
+  }
+
+  @Test(timeout = 60000)
+  public void testAuthenticationHttpClientPostWithMultiAuthHandler()
+      throws Exception {
+    final AuthenticatorTestCase auth = new AuthenticatorTestCase(useTomcat);
+    AuthenticatorTestCase
+        .setAuthenticationHandlerConfig(getMultiAuthHandlerConfiguration());
+    KerberosTestUtils.doAsClient(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        auth._testAuthenticationHttpClient(new KerberosAuthenticator(), true);
+        return null;
+      }
+    });
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/LdapConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/LdapConstants.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/LdapConstants.java
new file mode 100644
index 0000000..3e8615f
--- /dev/null
+++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/LdapConstants.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.server;
+
+/**
+ * This class defines the constants used by the LDAP integration tests.
+ */
+public final class LdapConstants {
+
+  /**
+   * This class defines constants to be used for LDAP integration testing.
+   * Hence this class is not expected to be instantiated.
+   */
+  private LdapConstants() {
+  }
+
+  public static final String LDAP_BASE_DN = "dc=example,dc=com";
+  public static final String LDAP_SERVER_ADDR = "localhost";
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestLdapAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestLdapAuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestLdapAuthenticationHandler.java
new file mode 100644
index 0000000..59aef5a
--- /dev/null
+++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestLdapAuthenticationHandler.java
@@ -0,0 +1,159 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.server;
+
+import java.util.Properties;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import static org.apache.hadoop.security.authentication.server.LdapAuthenticationHandler.*;
+import static org.apache.hadoop.security.authentication.server.LdapConstants.*;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.directory.server.annotations.CreateLdapServer;
+import org.apache.directory.server.annotations.CreateTransport;
+import org.apache.directory.server.core.annotations.ApplyLdifs;
+import org.apache.directory.server.core.annotations.ContextEntry;
+import org.apache.directory.server.core.annotations.CreateDS;
+import org.apache.directory.server.core.annotations.CreatePartition;
+import org.apache.directory.server.core.integ.AbstractLdapTestUnit;
+import org.apache.directory.server.core.integ.FrameworkRunner;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mockito;
+
+/**
+ * This unit test verifies the functionality of LDAP authentication handler.
+ */
+@RunWith(FrameworkRunner.class)
+@CreateLdapServer(
+    transports =
+      {
+        @CreateTransport(protocol = "LDAP", address= LDAP_SERVER_ADDR),
+      })
+@CreateDS(allowAnonAccess = true,
+          partitions = {
+            @CreatePartition(
+                name = "Test_Partition", suffix = LDAP_BASE_DN,
+                contextEntry = @ContextEntry(
+                    entryLdif = "dn: " + LDAP_BASE_DN + " \n"
+                              + "dc: example\n"
+                              + "objectClass: top\n"
+                              + "objectClass: domain\n\n"))})
+@ApplyLdifs({
+    "dn: uid=bjones," + LDAP_BASE_DN,
+    "cn: Bob Jones",
+    "sn: Jones",
+    "objectClass: inetOrgPerson",
+    "uid: bjones",
+    "userPassword: p@ssw0rd"})
+public class TestLdapAuthenticationHandler extends AbstractLdapTestUnit {
+  private LdapAuthenticationHandler handler;
+
+  @Before
+  public void setup() throws Exception {
+    handler = new LdapAuthenticationHandler();
+    try {
+      handler.init(getDefaultProperties());
+    } catch (Exception e) {
+      handler = null;
+      throw e;
+    }
+  }
+
+  protected Properties getDefaultProperties() {
+    Properties p = new Properties();
+    p.setProperty(BASE_DN, LDAP_BASE_DN);
+    p.setProperty(PROVIDER_URL, String.format("ldap://%s:%s", LDAP_SERVER_ADDR,
+        getLdapServer().getPort()));
+    return p;
+  }
+
+  @Test(timeout = 60000)
+  public void testRequestWithoutAuthorization() throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+
+    Assert.assertNull(handler.authenticate(request, response));
+    Mockito.verify(response).setHeader(WWW_AUTHENTICATE, HttpConstants.BASIC);
+    Mockito.verify(response).setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+  }
+
+  @Test(timeout = 60000)
+  public void testRequestWithInvalidAuthorization() throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+
+    final Base64 base64 = new Base64(0);
+    String credentials = "bjones:invalidpassword";
+    Mockito.when(request.getHeader(HttpConstants.AUTHORIZATION_HEADER))
+        .thenReturn(base64.encodeToString(credentials.getBytes()));
+    Assert.assertNull(handler.authenticate(request, response));
+    Mockito.verify(response).setHeader(WWW_AUTHENTICATE, HttpConstants.BASIC);
+    Mockito.verify(response).setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+  }
+
+  @Test(timeout = 60000)
+  public void testRequestWithIncompleteAuthorization() throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+
+    Mockito.when(request.getHeader(HttpConstants.AUTHORIZATION_HEADER))
+        .thenReturn(HttpConstants.BASIC);
+    Assert.assertNull(handler.authenticate(request, response));
+  }
+
+  @Test(timeout = 60000)
+  public void testRequestWithAuthorization() throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+
+    final Base64 base64 = new Base64(0);
+    String credentials = base64.encodeToString("bjones:p@ssw0rd".getBytes());
+    String authHeader = HttpConstants.BASIC + " " + credentials;
+    Mockito.when(request.getHeader(HttpConstants.AUTHORIZATION_HEADER))
+        .thenReturn(authHeader);
+    AuthenticationToken token = handler.authenticate(request, response);
+    Assert.assertNotNull(token);
+    Mockito.verify(response).setStatus(HttpServletResponse.SC_OK);
+    Assert.assertEquals(TYPE, token.getType());
+    Assert.assertEquals("bjones", token.getUserName());
+    Assert.assertEquals("bjones", token.getName());
+  }
+
+  @Test(timeout = 60000)
+  public void testRequestWithWrongCredentials() throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+
+    final Base64 base64 = new Base64(0);
+    String credentials = base64.encodeToString("bjones:foo123".getBytes());
+    String authHeader = HttpConstants.BASIC + " " + credentials;
+    Mockito.when(request.getHeader(HttpConstants.AUTHORIZATION_HEADER))
+        .thenReturn(authHeader);
+
+    try {
+      handler.authenticate(request, response);
+      Assert.fail();
+    } catch (AuthenticationException ex) {
+      // Expected
+    } catch (Exception ex) {
+      Assert.fail();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestMultiSchemeAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestMultiSchemeAuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestMultiSchemeAuthenticationHandler.java
new file mode 100644
index 0000000..8f9668a
--- /dev/null
+++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestMultiSchemeAuthenticationHandler.java
@@ -0,0 +1,189 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.server;
+
+import static org.apache.hadoop.security.authentication.server.LdapAuthenticationHandler.BASE_DN;
+import static org.apache.hadoop.security.authentication.server.LdapAuthenticationHandler.PROVIDER_URL;
+import static org.apache.hadoop.security.authentication.server.LdapAuthenticationHandler.TYPE;
+import static org.apache.hadoop.security.authentication.server.MultiSchemeAuthenticationHandler.SCHEMES_PROPERTY;
+import static org.apache.hadoop.security.authentication.server.MultiSchemeAuthenticationHandler.AUTH_HANDLER_PROPERTY;
+import static org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler.PRINCIPAL;
+import static org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler.KEYTAB;
+import static org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler.NAME_RULES;
+import static org.apache.hadoop.security.authentication.server.LdapConstants.*;
+import static org.apache.hadoop.security.authentication.server.HttpConstants.*;
+
+import java.io.File;
+import java.util.Properties;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.directory.server.annotations.CreateLdapServer;
+import org.apache.directory.server.annotations.CreateTransport;
+import org.apache.directory.server.core.annotations.ApplyLdifs;
+import org.apache.directory.server.core.annotations.ContextEntry;
+import org.apache.directory.server.core.annotations.CreateDS;
+import org.apache.directory.server.core.annotations.CreatePartition;
+import org.apache.directory.server.core.integ.AbstractLdapTestUnit;
+import org.apache.directory.server.core.integ.FrameworkRunner;
+import org.apache.hadoop.minikdc.KerberosSecurityTestcase;
+import org.apache.hadoop.security.authentication.KerberosTestUtils;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mockito;
+
+/**
+ * This unit test verifies the functionality of "multi-scheme" auth handler.
+ */
+@RunWith(FrameworkRunner.class)
+@CreateLdapServer(
+    transports =
+      {
+        @CreateTransport(protocol = "LDAP", address = LDAP_SERVER_ADDR),
+      })
+@CreateDS(allowAnonAccess = true,
+          partitions = {
+            @CreatePartition(
+              name = "Test_Partition", suffix = LDAP_BASE_DN,
+              contextEntry = @ContextEntry(
+                  entryLdif = "dn: "+ LDAP_BASE_DN+ " \n"
+                              + "dc: example\n"
+                              + "objectClass: top\n"
+                              + "objectClass: domain\n\n"))})
+@ApplyLdifs({
+    "dn: uid=bjones," + LDAP_BASE_DN,
+    "cn: Bob Jones",
+    "sn: Jones",
+    "objectClass: inetOrgPerson",
+    "uid: bjones",
+    "userPassword: p@ssw0rd"})
+public class TestMultiSchemeAuthenticationHandler
+    extends AbstractLdapTestUnit {
+  private KerberosSecurityTestcase krbTest = new KerberosSecurityTestcase();
+  private MultiSchemeAuthenticationHandler handler;
+
+  @Before
+  public void setUp()  throws Exception {
+    krbTest.startMiniKdc();
+
+    // create keytab
+    File keytabFile = new File(KerberosTestUtils.getKeytabFile());
+    String clientPrinc = KerberosTestUtils.getClientPrincipal();
+    String serverPrinc = KerberosTestUtils.getServerPrincipal();
+    clientPrinc = clientPrinc.substring(0, clientPrinc.lastIndexOf("@"));
+    serverPrinc = serverPrinc.substring(0, serverPrinc.lastIndexOf("@"));
+    krbTest.getKdc().createPrincipal(keytabFile, clientPrinc, serverPrinc);
+    // configure handler
+    handler = new MultiSchemeAuthenticationHandler();
+    try {
+      handler.init(getDefaultProperties());
+    } catch (Exception e) {
+      throw e;
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    krbTest.stopMiniKdc();
+  }
+
+  private Properties getDefaultProperties() {
+    Properties p = new Properties();
+    p.setProperty(SCHEMES_PROPERTY, BASIC + "," + NEGOTIATE);
+    p.setProperty(String.format(AUTH_HANDLER_PROPERTY, "negotiate"),
+        "kerberos");
+    p.setProperty(String.format(AUTH_HANDLER_PROPERTY, "basic"), "ldap");
+    // Kerberos related config
+    p.setProperty(PRINCIPAL, KerberosTestUtils.getServerPrincipal());
+    p.setProperty(KEYTAB, KerberosTestUtils.getKeytabFile());
+    p.setProperty(NAME_RULES,
+        "RULE:[1:$1@$0](.*@" + KerberosTestUtils.getRealm()+")s/@.*//\n");
+    // LDAP related config
+    p.setProperty(BASE_DN, LDAP_BASE_DN);
+    p.setProperty(PROVIDER_URL, String.format("ldap://%s:%s", LDAP_SERVER_ADDR,
+        getLdapServer().getPort()));
+    return p;
+  }
+
+  @Test(timeout = 60000)
+  public void testRequestWithoutAuthorization() throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+
+    Assert.assertNull(handler.authenticate(request, response));
+    Mockito.verify(response).addHeader(WWW_AUTHENTICATE_HEADER, BASIC);
+    Mockito.verify(response).addHeader(WWW_AUTHENTICATE_HEADER, NEGOTIATE);
+    Mockito.verify(response).setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+  }
+
+  @Test(timeout = 60000)
+  public void testRequestWithInvalidAuthorization() throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+
+    final Base64 base64 = new Base64(0);
+    String credentials = "bjones:invalidpassword";
+    Mockito.when(request.getHeader(AUTHORIZATION_HEADER))
+        .thenReturn(base64.encodeToString(credentials.getBytes()));
+    Assert.assertNull(handler.authenticate(request, response));
+    Mockito.verify(response).addHeader(WWW_AUTHENTICATE_HEADER, BASIC);
+    Mockito.verify(response).addHeader(WWW_AUTHENTICATE_HEADER, NEGOTIATE);
+    Mockito.verify(response).setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+  }
+
+  @Test(timeout = 60000)
+  public void testRequestWithLdapAuthorization() throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+
+    final Base64 base64 = new Base64(0);
+    String credentials = base64.encodeToString("bjones:p@ssw0rd".getBytes());
+    String authHeader = BASIC + " " + credentials;
+    Mockito.when(request.getHeader(AUTHORIZATION_HEADER))
+        .thenReturn(authHeader);
+    AuthenticationToken token = handler.authenticate(request, response);
+    Assert.assertNotNull(token);
+    Mockito.verify(response).setStatus(HttpServletResponse.SC_OK);
+    Assert.assertEquals(TYPE, token.getType());
+    Assert.assertEquals("bjones", token.getUserName());
+    Assert.assertEquals("bjones", token.getName());
+  }
+
+  @Test(timeout = 60000)
+  public void testRequestWithInvalidKerberosAuthorization() throws Exception {
+    String token = new Base64(0).encodeToString(new byte[]{0, 1, 2});
+
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+
+    Mockito.when(request.getHeader(AUTHORIZATION_HEADER)).thenReturn(
+        NEGOTIATE + token);
+
+    try {
+      handler.authenticate(request, response);
+      Assert.fail();
+    } catch (AuthenticationException ex) {
+      // Expected
+    } catch (Exception ex) {
+      Assert.fail();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java
index 112c952..f5e798e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
 import org.apache.hadoop.security.authentication.server.AuthenticationToken;
 import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
+import org.apache.hadoop.security.authentication.server.MultiSchemeAuthenticationHandler;
 import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
 import org.apache.hadoop.security.authentication.util.ZKSignerSecretProvider;
 import org.apache.hadoop.security.authorize.AuthorizationException;
@@ -38,7 +39,6 @@ import org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManage
 import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.http.NameValuePair;
 import org.apache.http.client.utils.URLEncodedUtils;
-import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,13 +51,10 @@ import javax.servlet.http.HttpServletRequestWrapper;
 import javax.servlet.http.HttpServletResponse;
 
 import java.io.IOException;
-import java.io.Writer;
 import java.nio.charset.Charset;
 import java.security.Principal;
 import java.util.Enumeration;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.Properties;
 
 /**
@@ -142,6 +139,9 @@ public class DelegationTokenAuthenticationFilter
     } else if (authType.equals(KerberosAuthenticationHandler.TYPE)) {
       props.setProperty(AUTH_TYPE,
           KerberosDelegationTokenAuthenticationHandler.class.getName());
+    } else if (authType.equals(MultiSchemeAuthenticationHandler.TYPE)) {
+      props.setProperty(AUTH_TYPE,
+          MultiSchemeDelegationTokenAuthenticationHandler.class.getName());
     }
   }
 
@@ -307,5 +307,4 @@ public class DelegationTokenAuthenticationFilter
       }
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
index 50c3eb5..c23a94f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
@@ -112,6 +112,10 @@ public abstract class DelegationTokenAuthenticationHandler
     return tokenManager;
   }
 
+  AuthenticationHandler getAuthHandler() {
+    return authHandler;
+  }
+
   @Override
   public void init(Properties config) throws ServletException {
     authHandler.init(config);
@@ -162,6 +166,24 @@ public abstract class DelegationTokenAuthenticationHandler
 
   private static final String ENTER = System.getProperty("line.separator");
 
+  /**
+   * This method checks if the given HTTP request corresponds to a management
+   * operation.
+   *
+   * @param request The HTTP request
+   * @return true if the given HTTP request corresponds to a management
+   *         operation false otherwise
+   * @throws IOException In case of I/O error.
+   */
+  protected final boolean isManagementOperation(HttpServletRequest request)
+      throws IOException {
+    String op = ServletUtils.getParameter(request,
+        KerberosDelegationTokenAuthenticator.OP_PARAM);
+    op = (op != null) ? StringUtils.toUpperCase(op) : null;
+    return DELEGATION_TOKEN_OPS.contains(op) &&
+        !request.getMethod().equals("OPTIONS");
+  }
+
   @Override
   @SuppressWarnings("unchecked")
   public boolean managementOperation(AuthenticationToken token,
@@ -171,8 +193,7 @@ public abstract class DelegationTokenAuthenticationHandler
     String op = ServletUtils.getParameter(request,
         KerberosDelegationTokenAuthenticator.OP_PARAM);
     op = (op != null) ? StringUtils.toUpperCase(op) : null;
-    if (DELEGATION_TOKEN_OPS.contains(op) &&
-        !request.getMethod().equals("OPTIONS")) {
+    if (isManagementOperation(request)) {
       KerberosDelegationTokenAuthenticator.DelegationTokenOperation dtOp =
           KerberosDelegationTokenAuthenticator.
               DelegationTokenOperation.valueOf(op);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/MultiSchemeDelegationTokenAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/MultiSchemeDelegationTokenAuthenticationHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/MultiSchemeDelegationTokenAuthenticationHandler.java
new file mode 100644
index 0000000..fc32a19
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/MultiSchemeDelegationTokenAuthenticationHandler.java
@@ -0,0 +1,182 @@
+/**
+ * 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.hadoop.security.token.delegation.web;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.server.AuthenticationHandlerUtil;
+import org.apache.hadoop.security.authentication.server.AuthenticationToken;
+import org.apache.hadoop.security.authentication.server.CompositeAuthenticationHandler;
+import org.apache.hadoop.security.authentication.server.HttpConstants;
+import org.apache.hadoop.security.authentication.server.MultiSchemeAuthenticationHandler;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
+
+/**
+ * A {@link CompositeAuthenticationHandler} that supports multiple HTTP
+ * authentication schemes along with Delegation Token functionality. e.g.
+ * server can support multiple authentication mechanisms such as Kerberos
+ * (SPENGO) and LDAP. During the authentication phase, server will specify
+ * all possible authentication schemes and let client choose the appropriate
+ * scheme. Please refer to RFC-2616 and HADOOP-12082 for more details.
+ *
+ * Internally it uses {@link MultiSchemeAuthenticationHandler} implementation.
+ * This handler also provides an option to enable delegation token management
+ * functionality for only a specified subset of authentication schemes. This is
+ * required to ensure that only schemes with strongest level of security should
+ * be used for delegation token management.
+ *
+ * <p/>
+ * In addition to the wrapped {@link AuthenticationHandler} configuration
+ * properties, this handler supports the following properties prefixed with the
+ * type of the wrapped <code>AuthenticationHandler</code>:
+ * <ul>
+ * <li>delegation-token.token-kind: the token kind for generated tokens (no
+ * default, required property).</li>
+ * <li>delegation-token.update-interval.sec: secret manager master key update
+ * interval in seconds (default 1 day).</li>
+ * <li>delegation-token.max-lifetime.sec: maximum life of a delegation token in
+ * seconds (default 7 days).</li>
+ * <li>delegation-token.renewal-interval.sec: renewal interval for delegation
+ * tokens in seconds (default 1 day).</li>
+ * <li>delegation-token.removal-scan-interval.sec: delegation tokens removal
+ * scan interval in seconds (default 1 hour).</li>
+ * <li>delegation.http.schemes: A comma separated list of HTTP authentication
+ * mechanisms (e.g. Negotiate, Basic) etc. to be allowed for delegation token
+ * management operations.</li>
+ * </ul>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MultiSchemeDelegationTokenAuthenticationHandler extends
+    DelegationTokenAuthenticationHandler implements
+    CompositeAuthenticationHandler {
+
+  public static final String DELEGATION_TOKEN_SCHEMES_PROPERTY =
+      "multi-scheme-auth-handler.delegation.schemes";
+  private static final Splitter STR_SPLITTER = Splitter.on(',').trimResults()
+      .omitEmptyStrings();
+
+  private Set<String> delegationAuthSchemes = null;
+
+  public MultiSchemeDelegationTokenAuthenticationHandler() {
+    super(new MultiSchemeAuthenticationHandler(
+        MultiSchemeAuthenticationHandler.TYPE + TYPE_POSTFIX));
+  }
+
+  @Override
+  public Collection<String> getTokenTypes() {
+    return ((CompositeAuthenticationHandler) getAuthHandler()).getTokenTypes();
+  }
+
+  @Override
+  public void init(Properties config) throws ServletException {
+    super.init(config);
+
+    // Figure out the HTTP authentication schemes configured.
+    String schemesProperty =
+        Preconditions.checkNotNull(config
+            .getProperty(MultiSchemeAuthenticationHandler.SCHEMES_PROPERTY));
+
+    // Figure out the HTTP authentication schemes configured for delegation
+    // tokens.
+    String delegationAuthSchemesProp =
+        Preconditions.checkNotNull(config
+            .getProperty(DELEGATION_TOKEN_SCHEMES_PROPERTY));
+
+    Set<String> authSchemes = new HashSet<>();
+    for (String scheme : STR_SPLITTER.split(schemesProperty)) {
+      authSchemes.add(AuthenticationHandlerUtil.checkAuthScheme(scheme));
+    }
+
+    delegationAuthSchemes = new HashSet<>();
+    for (String scheme : STR_SPLITTER.split(delegationAuthSchemesProp)) {
+      delegationAuthSchemes.add(AuthenticationHandlerUtil
+          .checkAuthScheme(scheme));
+    }
+
+    Preconditions.checkArgument(authSchemes.containsAll(delegationAuthSchemes));
+  }
+
+  /**
+   * This method is overridden to restrict HTTP authentication schemes
+   * available for delegation token management functionality. The
+   * authentication schemes to be used for delegation token management are
+   * configured using {@link DELEGATION_TOKEN_SCHEMES_PROPERTY}
+   *
+   * The basic logic here is to check if the current request is for delegation
+   * token management. If yes then check if the request contains an
+   * "Authorization" header. If it is missing, then return the HTTP 401
+   * response with WWW-Authenticate header for each scheme configured for
+   * delegation token management.
+   *
+   * It is also possible for a client to preemptively send Authorization header
+   * for a scheme not configured for delegation token management. We detect
+   * this case and return the HTTP 401 response with WWW-Authenticate header
+   * for each scheme configured for delegation token management.
+   *
+   * If a client has sent a request with "Authorization" header for a scheme
+   * configured for delegation token management, then it is forwarded to
+   * underlying {@link MultiSchemeAuthenticationHandler} for actual
+   * authentication.
+   *
+   * Finally all other requests (excluding delegation token management) are
+   * forwarded to underlying {@link MultiSchemeAuthenticationHandler} for
+   * actual authentication.
+   */
+  @Override
+  public AuthenticationToken authenticate(HttpServletRequest request,
+      HttpServletResponse response)
+          throws IOException, AuthenticationException {
+    String authorization =
+        request.getHeader(HttpConstants.AUTHORIZATION_HEADER);
+
+    if (isManagementOperation(request)) {
+      boolean schemeConfigured = false;
+      if (authorization != null) {
+        for (String scheme : delegationAuthSchemes) {
+          if (AuthenticationHandlerUtil.
+              matchAuthScheme(scheme, authorization)) {
+            schemeConfigured = true;
+            break;
+          }
+        }
+      }
+      if (!schemeConfigured) {
+        response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+        for (String scheme : delegationAuthSchemes) {
+          response.addHeader(WWW_AUTHENTICATE, scheme);
+        }
+        return null;
+      }
+    }
+
+    return super.authenticate(request, response);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bca3852/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 5826cf6..c8cec4d 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -90,6 +90,10 @@
     <guice.version>4.0</guice.version>
     <joda-time.version>2.9.4</joda-time.version>
 
+    <!-- Required for testing LDAP integration -->
+    <apacheds.version>2.0.0-M21</apacheds.version>
+    <ldap-api.version>1.0.0-M33</ldap-api.version>
+
     <!-- define the Java language version used by the compiler -->
     <javac.version>1.8</javac.version>
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org