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 at...@apache.org on 2011/08/20 00:31:09 UTC

svn commit: r1159804 [2/3] - in /hadoop/common/trunk: ./ hadoop-alfredo/ hadoop-alfredo/src/ hadoop-alfredo/src/examples/ hadoop-alfredo/src/examples/src/ hadoop-alfredo/src/examples/src/main/ hadoop-alfredo/src/examples/src/main/java/ hadoop-alfredo/s...

Added: hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/AuthenticationHandler.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/AuthenticationHandler.java?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/AuthenticationHandler.java (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/AuthenticationHandler.java Fri Aug 19 22:31:06 2011
@@ -0,0 +1,89 @@
+/**
+ * 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.alfredo.server;
+
+import org.apache.hadoop.alfredo.client.AuthenticationException;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * Interface for server authentication mechanisms.
+ * <p/>
+ * The {@link AuthenticationFilter} manages the lifecycle of the authentication handler.
+ * <p/>
+ * Implementations must be thread-safe as one instance is initialized and used for all requests.
+ */
+public interface AuthenticationHandler {
+
+  /**
+   * Returns the authentication type of the authentication handler.
+   * <p/>
+   * This should be a name that uniquely identifies the authentication type.
+   * For example 'simple' or 'kerberos'.
+   *
+   * @return the authentication type of the authentication handler.
+   */
+  public String getType();
+
+  /**
+   * Initializes the authentication handler instance.
+   * <p/>
+   * This method is invoked by the {@link AuthenticationFilter#init} method.
+   *
+   * @param config configuration properties to initialize the handler.
+   *
+   * @throws ServletException thrown if the handler could not be initialized.
+   */
+  public void init(Properties config) throws ServletException;
+
+  /**
+   * Destroys the authentication handler instance.
+   * <p/>
+   * This method is invoked by the {@link AuthenticationFilter#destroy} method.
+   */
+  public void destroy();
+
+  /**
+   * Performs an authentication step for the given HTTP client request.
+   * <p/>
+   * This method is invoked by the {@link AuthenticationFilter} only if the HTTP client request is
+   * not yet authenticated.
+   * <p/>
+   * Depending upon the authentication mechanism being implemented, a particular HTTP client may
+   * end up making a sequence of invocations before authentication is successfully established (this is
+   * the case of Kerberos SPNEGO).
+   * <p/>
+   * This method must return an {@link AuthenticationToken} only if the the HTTP client request has
+   * been successfully and fully authenticated.
+   * <p/>
+   * If the HTTP client request has not been completely authenticated, this method must take over
+   * the corresponding HTTP response and it must return <code>null</code>.
+   *
+   * @param request the HTTP client request.
+   * @param response the HTTP client response.
+   *
+   * @return an {@link AuthenticationToken} if the HTTP client request has been authenticated,
+   *         <code>null</code> otherwise (in this case it must take care of the response).
+   *
+   * @throws IOException thrown if an IO error occurred.
+   * @throws AuthenticationException thrown if an Authentication error occurred.
+   */
+  public AuthenticationToken authenticate(HttpServletRequest request, HttpServletResponse response)
+    throws IOException, AuthenticationException;
+
+}

Added: hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/AuthenticationToken.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/AuthenticationToken.java?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/AuthenticationToken.java (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/AuthenticationToken.java Fri Aug 19 22:31:06 2011
@@ -0,0 +1,226 @@
+/**
+ * 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.alfredo.server;
+
+import org.apache.hadoop.alfredo.client.AuthenticationException;
+
+import java.security.Principal;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.StringTokenizer;
+
+/**
+ * The {@link AuthenticationToken} contains information about an authenticated HTTP client and doubles
+ * as the {@link Principal} to be returned by authenticated {@link HttpServletRequest}s
+ * <p/>
+ * The token can be serialized/deserialized to and from a string as it is sent and received in HTTP client
+ * responses and requests as a HTTP cookie (this is done by the {@link AuthenticationFilter}).
+ */
+public class AuthenticationToken implements Principal {
+
+  /**
+   * Constant that identifies an anonymous request.
+   */
+  public static final AuthenticationToken ANONYMOUS = new AuthenticationToken();
+
+  private static final String ATTR_SEPARATOR = "&";
+  private static final String USER_NAME = "u";
+  private static final String PRINCIPAL = "p";
+  private static final String EXPIRES = "e";
+  private static final String TYPE = "t";
+
+  private final static Set<String> ATTRIBUTES =
+    new HashSet<String>(Arrays.asList(USER_NAME, PRINCIPAL, EXPIRES, TYPE));
+
+  private String userName;
+  private String principal;
+  private String type;
+  private long expires;
+  private String token;
+
+  private AuthenticationToken() {
+    userName = null;
+    principal = null;
+    type = null;
+    expires = -1;
+    token = "ANONYMOUS";
+    generateToken();
+  }
+
+  private static final String ILLEGAL_ARG_MSG = " is NULL, empty or contains a '" + ATTR_SEPARATOR + "'";
+
+  /**
+   * Creates an authentication token.
+   *
+   * @param userName user name.
+   * @param principal principal (commonly matches the user name, with Kerberos is the full/long principal
+   * name while the userName is the short name).
+   * @param type the authentication mechanism name.
+   * (<code>System.currentTimeMillis() + validityPeriod</code>).
+   */
+  public AuthenticationToken(String userName, String principal, String type) {
+    checkForIllegalArgument(userName, "userName");
+    checkForIllegalArgument(principal, "principal");
+    checkForIllegalArgument(type, "type");
+    this.userName = userName;
+    this.principal = principal;
+    this.type = type;
+    this.expires = -1;
+  }
+  
+  /**
+   * Check if the provided value is invalid. Throw an error if it is invalid, NOP otherwise.
+   * 
+   * @param value the value to check.
+   * @param name the parameter name to use in an error message if the value is invalid.
+   */
+  private static void checkForIllegalArgument(String value, String name) {
+    if (value == null || value.length() == 0 || value.contains(ATTR_SEPARATOR)) {
+      throw new IllegalArgumentException(name + ILLEGAL_ARG_MSG);
+    }
+  }
+
+  /**
+   * Sets the expiration of the token.
+   *
+   * @param expires expiration time of the token in milliseconds since the epoch.
+   */
+  public void setExpires(long expires) {
+    if (this != AuthenticationToken.ANONYMOUS) {
+      this.expires = expires;
+      generateToken();
+    }
+  }
+
+  /**
+   * Generates the token.
+   */
+  private void generateToken() {
+    StringBuffer sb = new StringBuffer();
+    sb.append(USER_NAME).append("=").append(userName).append(ATTR_SEPARATOR);
+    sb.append(PRINCIPAL).append("=").append(principal).append(ATTR_SEPARATOR);
+    sb.append(TYPE).append("=").append(type).append(ATTR_SEPARATOR);
+    sb.append(EXPIRES).append("=").append(expires);
+    token = sb.toString();
+  }
+
+  /**
+   * Returns the user name.
+   *
+   * @return the user name.
+   */
+  public String getUserName() {
+    return userName;
+  }
+
+  /**
+   * Returns the principal name (this method name comes from the JDK {@link Principal} interface).
+   *
+   * @return the principal name.
+   */
+  @Override
+  public String getName() {
+    return principal;
+  }
+
+  /**
+   * Returns the authentication mechanism of the token.
+   *
+   * @return the authentication mechanism of the token.
+   */
+  public String getType() {
+    return type;
+  }
+
+  /**
+   * Returns the expiration time of the token.
+   *
+   * @return the expiration time of the token, in milliseconds since Epoc.
+   */
+  public long getExpires() {
+    return expires;
+  }
+
+  /**
+   * Returns if the token has expired.
+   *
+   * @return if the token has expired.
+   */
+  public boolean isExpired() {
+    return expires != -1 && System.currentTimeMillis() > expires;
+  }
+
+  /**
+   * Returns the string representation of the token.
+   * <p/>
+   * This string representation is parseable by the {@link #parse} method.
+   *
+   * @return the string representation of the token.
+   */
+  @Override
+  public String toString() {
+    return token;
+  }
+
+  /**
+   * Parses a string into an authentication token.
+   *
+   * @param tokenStr string representation of a token.
+   *
+   * @return the parsed authentication token.
+   *
+   * @throws AuthenticationException thrown if the string representation could not be parsed into
+   * an authentication token.
+   */
+  public static AuthenticationToken parse(String tokenStr) throws AuthenticationException {
+    Map<String, String> map = split(tokenStr);
+    if (!map.keySet().equals(ATTRIBUTES)) {
+      throw new AuthenticationException("Invalid token string, missing attributes");
+    }
+    long expires = Long.parseLong(map.get(EXPIRES));
+    AuthenticationToken token = new AuthenticationToken(map.get(USER_NAME), map.get(PRINCIPAL), map.get(TYPE));
+    token.setExpires(expires);
+    return token;
+  }
+
+  /**
+   * Splits the string representation of a token into attributes pairs.
+   *
+   * @param tokenStr string representation of a token.
+   *
+   * @return a map with the attribute pairs of the token.
+   *
+   * @throws AuthenticationException thrown if the string representation of the token could not be broken into
+   * attribute pairs.
+   */
+  private static Map<String, String> split(String tokenStr) throws AuthenticationException {
+    Map<String, String> map = new HashMap<String, String>();
+    StringTokenizer st = new StringTokenizer(tokenStr, ATTR_SEPARATOR);
+    while (st.hasMoreTokens()) {
+      String part = st.nextToken();
+      int separator = part.indexOf('=');
+      if (separator == -1) {
+        throw new AuthenticationException("Invalid authentication token");
+      }
+      String key = part.substring(0, separator);
+      String value = part.substring(separator + 1);
+      map.put(key, value);
+    }
+    return map;
+  }
+
+}

Added: hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/KerberosAuthenticationHandler.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/KerberosAuthenticationHandler.java?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/KerberosAuthenticationHandler.java (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/KerberosAuthenticationHandler.java Fri Aug 19 22:31:06 2011
@@ -0,0 +1,310 @@
+/**
+ * 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.alfredo.server;
+
+import org.apache.hadoop.alfredo.client.AuthenticationException;
+import org.apache.hadoop.alfredo.client.KerberosAuthenticator;
+import com.sun.security.auth.module.Krb5LoginModule;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.alfredo.util.KerberosName;
+import org.ietf.jgss.GSSContext;
+import org.ietf.jgss.GSSCredential;
+import org.ietf.jgss.GSSManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.File;
+import java.io.IOException;
+import java.security.Principal;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * The {@link KerberosAuthenticationHandler} implements the Kerberos SPNEGO authentication mechanism for HTTP.
+ * <p/>
+ * The supported configuration properties are:
+ * <ul>
+ * <li>kerberos.principal: the Kerberos principal to used by the server. As stated by the Kerberos SPNEGO
+ * specification, it should be <code>HTTP/${HOSTNAME}@{REALM}</code>. The realm can be omitted from the
+ * principal as the JDK GSS libraries will use the realm name of the configured default realm.
+ * It does not have a default value.</li>
+ * <li>kerberos.keytab: the keytab file containing the credentials for the Kerberos principal.
+ * It does not have a default value.</li>
+ * </ul>
+ */
+public class KerberosAuthenticationHandler implements AuthenticationHandler {
+  private static Logger LOG = LoggerFactory.getLogger(KerberosAuthenticationHandler.class);
+
+  /**
+   * Kerberos context configuration for the JDK GSS library.
+   */
+  private static class KerberosConfiguration extends Configuration {
+    private String keytab;
+    private String principal;
+
+    public KerberosConfiguration(String keytab, String principal) {
+      this.keytab = keytab;
+      this.principal = principal;
+    }
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+      Map<String, String> options = new HashMap<String, String>();
+      options.put("keyTab", keytab);
+      options.put("principal", principal);
+      options.put("useKeyTab", "true");
+      options.put("storeKey", "true");
+      options.put("doNotPrompt", "true");
+      options.put("useTicketCache", "true");
+      options.put("renewTGT", "true");
+      options.put("refreshKrb5Config", "true");
+      options.put("isInitiator", "false");
+      String ticketCache = System.getenv("KRB5CCNAME");
+      if (ticketCache != null) {
+        options.put("ticketCache", ticketCache);
+      }
+      if (LOG.isDebugEnabled()) {
+        options.put("debug", "true");
+      }
+
+      return new AppConfigurationEntry[]{
+        new AppConfigurationEntry(Krb5LoginModule.class.getName(),
+                                  AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+                                  options),};
+    }
+  }
+
+  /**
+   * Constant that identifies the authentication mechanism.
+   */
+  public static final String TYPE = "kerberos";
+
+  /**
+   * Constant for the configuration property that indicates the kerberos principal.
+   */
+  public static final String PRINCIPAL = TYPE + ".principal";
+
+  /**
+   * Constant for the configuration property that indicates the keytab file path.
+   */
+  public static final String KEYTAB = TYPE + ".keytab";
+
+  /**
+   * Constant for the configuration property that indicates the Kerberos name
+   * rules for the Kerberos principals.
+   */
+  public static final String NAME_RULES = TYPE + ".name.rules";
+
+  private String principal;
+  private String keytab;
+  private GSSManager gssManager;
+  private LoginContext loginContext;
+
+  /**
+   * Initializes the authentication handler instance.
+   * <p/>
+   * It creates a Kerberos context using the principal and keytab specified in the configuration.
+   * <p/>
+   * This method is invoked by the {@link AuthenticationFilter#init} method.
+   *
+   * @param config configuration properties to initialize the handler.
+   *
+   * @throws ServletException thrown if the handler could not be initialized.
+   */
+  @Override
+  public void init(Properties config) throws ServletException {
+    try {
+      principal = config.getProperty(PRINCIPAL, principal);
+      if (principal == null || principal.trim().length() == 0) {
+        throw new ServletException("Principal not defined in configuration");
+      }
+      keytab = config.getProperty(KEYTAB, keytab);
+      if (keytab == null || keytab.trim().length() == 0) {
+        throw new ServletException("Keytab not defined in configuration");
+      }
+      if (!new File(keytab).exists()) {
+        throw new ServletException("Keytab does not exist: " + keytab);
+      }
+
+      String nameRules = config.getProperty(NAME_RULES, "DEFAULT");
+      KerberosName.setRules(nameRules);
+
+      Set<Principal> principals = new HashSet<Principal>();
+      principals.add(new KerberosPrincipal(principal));
+      Subject subject = new Subject(false, principals, new HashSet<Object>(), new HashSet<Object>());
+
+      KerberosConfiguration kerberosConfiguration = new KerberosConfiguration(keytab, principal);
+
+      loginContext = new LoginContext("", subject, null, kerberosConfiguration);
+      loginContext.login();
+
+      Subject serverSubject = loginContext.getSubject();
+      try {
+        gssManager = Subject.doAs(serverSubject, new PrivilegedExceptionAction<GSSManager>() {
+
+          @Override
+          public GSSManager run() throws Exception {
+            return GSSManager.getInstance();
+          }
+        });
+      } catch (PrivilegedActionException ex) {
+        throw ex.getException();
+      }
+      LOG.info("Initialized, principal [{}] from keytab [{}]", principal, keytab);
+    } catch (Exception ex) {
+      throw new ServletException(ex);
+    }
+  }
+
+  /**
+   * Releases any resources initialized by the authentication handler.
+   * <p/>
+   * It destroys the Kerberos context.
+   */
+  @Override
+  public void destroy() {
+    try {
+      if (loginContext != null) {
+        loginContext.logout();
+        loginContext = null;
+      }
+    } catch (LoginException ex) {
+      LOG.warn(ex.getMessage(), ex);
+    }
+  }
+
+  /**
+   * Returns the authentication type of the authentication handler, 'kerberos'.
+   * <p/>
+   *
+   * @return the authentication type of the authentication handler, 'kerberos'.
+   */
+  @Override
+  public String getType() {
+    return TYPE;
+  }
+
+  /**
+   * Returns the Kerberos principal used by the authentication handler.
+   *
+   * @return the Kerberos principal used by the authentication handler.
+   */
+  protected String getPrincipal() {
+    return principal;
+  }
+
+  /**
+   * Returns the keytab used by the authentication handler.
+   *
+   * @return the keytab used by the authentication handler.
+   */
+  protected String getKeytab() {
+    return keytab;
+  }
+
+  /**
+   * It enforces the the Kerberos SPNEGO authentication sequence returning an {@link AuthenticationToken} only
+   * after the Kerberos SPNEGO sequence has completed successfully.
+   * <p/>
+   *
+   * @param request the HTTP client request.
+   * @param response the HTTP client response.
+   *
+   * @return an authentication token if the Kerberos SPNEGO sequence is complete and valid,
+   *         <code>null</code> if it is in progress (in this case the handler handles the response to the client).
+   *
+   * @throws IOException thrown if an IO error occurred.
+   * @throws AuthenticationException thrown if Kerberos SPNEGO sequence failed.
+   */
+  @Override
+  public AuthenticationToken authenticate(HttpServletRequest request, final HttpServletResponse response)
+    throws IOException, AuthenticationException {
+    AuthenticationToken token = null;
+    String authorization = request.getHeader(KerberosAuthenticator.AUTHORIZATION);
+
+    if (authorization == null || !authorization.startsWith(KerberosAuthenticator.NEGOTIATE)) {
+      response.setHeader(KerberosAuthenticator.WWW_AUTHENTICATE, KerberosAuthenticator.NEGOTIATE);
+      response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+      if (authorization == null) {
+        LOG.trace("SPNEGO starting");
+      } else {
+        LOG.warn("'" + KerberosAuthenticator.AUTHORIZATION + "' does not start with '" +
+            KerberosAuthenticator.NEGOTIATE + "' :  {}", authorization);
+      }
+    } else {
+      authorization = authorization.substring(KerberosAuthenticator.NEGOTIATE.length()).trim();
+      final Base64 base64 = new Base64(0);
+      final byte[] clientToken = base64.decode(authorization);
+      Subject serverSubject = loginContext.getSubject();
+      try {
+        token = Subject.doAs(serverSubject, new PrivilegedExceptionAction<AuthenticationToken>() {
+
+          @Override
+          public AuthenticationToken run() throws Exception {
+            AuthenticationToken token = null;
+            GSSContext gssContext = null;
+            try {
+              gssContext = gssManager.createContext((GSSCredential) null);
+              byte[] serverToken = gssContext.acceptSecContext(clientToken, 0, clientToken.length);
+              if (serverToken != null && serverToken.length > 0) {
+                String authenticate = base64.encodeToString(serverToken);
+                response.setHeader(KerberosAuthenticator.WWW_AUTHENTICATE,
+                                   KerberosAuthenticator.NEGOTIATE + " " + authenticate);
+              }
+              if (!gssContext.isEstablished()) {
+                response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+                LOG.trace("SPNEGO in progress");
+              } else {
+                String clientPrincipal = gssContext.getSrcName().toString();
+                KerberosName kerberosName = new KerberosName(clientPrincipal);
+                String userName = kerberosName.getShortName();
+                token = new AuthenticationToken(userName, clientPrincipal, TYPE);
+                response.setStatus(HttpServletResponse.SC_OK);
+                LOG.trace("SPNEGO completed for principal [{}]", clientPrincipal);
+              }
+            } finally {
+              if (gssContext != null) {
+                gssContext.dispose();
+              }
+            }
+            return token;
+          }
+        });
+      } catch (PrivilegedActionException ex) {
+        if (ex.getException() instanceof IOException) {
+          throw (IOException) ex.getException();
+        }
+        else {
+          throw new AuthenticationException(ex.getException());
+        }
+      }
+    }
+    return token;
+  }
+
+}

Added: hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/PseudoAuthenticationHandler.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/PseudoAuthenticationHandler.java?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/PseudoAuthenticationHandler.java (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/server/PseudoAuthenticationHandler.java Fri Aug 19 22:31:06 2011
@@ -0,0 +1,134 @@
+/**
+ * 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.alfredo.server;
+
+import org.apache.hadoop.alfredo.client.AuthenticationException;
+import org.apache.hadoop.alfredo.client.PseudoAuthenticator;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * The <code>PseudoAuthenticationHandler</code> provides a pseudo authentication mechanism that accepts
+ * the user name specified as a query string parameter.
+ * <p/>
+ * This mimics the model of Hadoop Simple authentication which trust the 'user.name' property provided in
+ * the configuration object.
+ * <p/>
+ * This handler can be configured to support anonymous users.
+ * <p/>
+ * The only supported configuration property is:
+ * <ul>
+ * <li>simple.anonymous.allowed: <code>true|false</code>, default value is <code>false</code></li>
+ * </ul>
+ */
+public class PseudoAuthenticationHandler implements AuthenticationHandler {
+
+  /**
+   * Constant that identifies the authentication mechanism.
+   */
+  public static final String TYPE = "simple";
+
+  /**
+   * Constant for the configuration property that indicates if anonymous users are allowed.
+   */
+  public static final String ANONYMOUS_ALLOWED = TYPE + ".anonymous.allowed";
+
+  private boolean acceptAnonymous;
+
+  /**
+   * Initializes the authentication handler instance.
+   * <p/>
+   * This method is invoked by the {@link AuthenticationFilter#init} method.
+   *
+   * @param config configuration properties to initialize the handler.
+   *
+   * @throws ServletException thrown if the handler could not be initialized.
+   */
+  @Override
+  public void init(Properties config) throws ServletException {
+    acceptAnonymous = Boolean.parseBoolean(config.getProperty(ANONYMOUS_ALLOWED, "false"));
+  }
+
+  /**
+   * Returns if the handler is configured to support anonymous users.
+   *
+   * @return if the handler is configured to support anonymous users.
+   */
+  protected boolean getAcceptAnonymous() {
+    return acceptAnonymous;
+  }
+
+  /**
+   * Releases any resources initialized by the authentication handler.
+   * <p/>
+   * This implementation does a NOP.
+   */
+  @Override
+  public void destroy() {
+  }
+
+  /**
+   * Returns the authentication type of the authentication handler, 'simple'.
+   * <p/>
+   *
+   * @return the authentication type of the authentication handler, 'simple'.
+   */
+  @Override
+  public String getType() {
+    return TYPE;
+  }
+
+  /**
+   * Authenticates an HTTP client request.
+   * <p/>
+   * It extracts the {@link PseudoAuthenticator#USER_NAME} parameter from the query string and creates
+   * an {@link AuthenticationToken} with it.
+   * <p/>
+   * If the HTTP client request does not contain the {@link PseudoAuthenticator#USER_NAME} parameter and
+   * the handler is configured to allow anonymous users it returns the {@link AuthenticationToken#ANONYMOUS}
+   * token.
+   * <p/>
+   * If the HTTP client request does not contain the {@link PseudoAuthenticator#USER_NAME} parameter and
+   * the handler is configured to disallow anonymous users it throws an {@link AuthenticationException}.
+   *
+   * @param request the HTTP client request.
+   * @param response the HTTP client response.
+   *
+   * @return an authentication token if the HTTP client request is accepted and credentials are valid.
+   *
+   * @throws IOException thrown if an IO error occurred.
+   * @throws AuthenticationException thrown if HTTP client request was not accepted as an authentication request.
+   */
+  @Override
+  public AuthenticationToken authenticate(HttpServletRequest request, HttpServletResponse response)
+    throws IOException, AuthenticationException {
+    AuthenticationToken token;
+    String userName = request.getParameter(PseudoAuthenticator.USER_NAME);
+    if (userName == null) {
+      if (getAcceptAnonymous()) {
+        token = AuthenticationToken.ANONYMOUS;
+      } else {
+        throw new AuthenticationException("Anonymous requests are disallowed");
+      }
+    } else {
+      token = new AuthenticationToken(userName, userName, TYPE);
+    }
+    return token;
+  }
+
+}

Added: hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/util/KerberosName.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/util/KerberosName.java?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/util/KerberosName.java (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/util/KerberosName.java Fri Aug 19 22:31:06 2011
@@ -0,0 +1,395 @@
+package org.apache.hadoop.alfredo.util;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+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.
+ */
+@SuppressWarnings("all")
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Evolving
+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) {
+        defaultRealm="";
+    }
+  }
+
+  /**
+   * 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;
+  }
+
+  @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());
+  }
+
+  /**
+   * Set the rules.
+   * @param ruleString the rules string.
+   */
+  public static void setRules(String ruleString) {
+    rules = parseRules(ruleString);
+  }
+
+  static void printRules() throws IOException {
+    int i = 0;
+    for(Rule r: rules) {
+      System.out.println(++i + " " + r);
+    }
+  }
+
+}

Added: hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/util/Signer.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/util/Signer.java?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/util/Signer.java (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/util/Signer.java Fri Aug 19 22:31:06 2011
@@ -0,0 +1,100 @@
+/**
+ * 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.alfredo.util;
+
+import org.apache.commons.codec.binary.Base64;
+
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+
+/**
+ * Signs strings and verifies signed strings using a SHA digest.
+ */
+public class Signer {
+  private static final String SIGNATURE = "&s=";
+
+  private byte[] secret;
+
+  /**
+   * Creates a Signer instance using the specified secret.
+   *
+   * @param secret secret to use for creating the digest.
+   */
+  public Signer(byte[] secret) {
+    if (secret == null) {
+      throw new IllegalArgumentException("secret cannot be NULL");
+    }
+    this.secret = secret.clone();
+  }
+
+  /**
+   * Returns a signed string.
+   * <p/>
+   * The signature '&s=SIGNATURE' is appended at the end of the string.
+   *
+   * @param str string to sign.
+   *
+   * @return the signed string.
+   */
+  public String sign(String str) {
+    if (str == null || str.length() == 0) {
+      throw new IllegalArgumentException("NULL or empty string to sign");
+    }
+    String signature = computeSignature(str);
+    return str + SIGNATURE + signature;
+  }
+
+  /**
+   * Verifies a signed string and extracts the original string.
+   *
+   * @param signedStr the signed string to verify and extract.
+   *
+   * @return the extracted original string.
+   *
+   * @throws SignerException thrown if the given string is not a signed string or if the signature is invalid.
+   */
+  public String verifyAndExtract(String signedStr) throws SignerException {
+    int index = signedStr.lastIndexOf(SIGNATURE);
+    if (index == -1) {
+      throw new SignerException("Invalid signed text: " + signedStr);
+    }
+    String originalSignature = signedStr.substring(index + SIGNATURE.length());
+    String rawValue = signedStr.substring(0, index);
+    String currentSignature = computeSignature(rawValue);
+    if (!originalSignature.equals(currentSignature)) {
+      throw new SignerException("Invalid signature");
+    }
+    return rawValue;
+  }
+
+  /**
+   * Returns then signature of a string.
+   *
+   * @param str string to sign.
+   *
+   * @return the signature for the string.
+   */
+  protected String computeSignature(String str) {
+    try {
+      MessageDigest md = MessageDigest.getInstance("SHA");
+      md.update(str.getBytes());
+      md.update(secret);
+      byte[] digest = md.digest();
+      return new Base64(0).encodeToString(digest);
+    } catch (NoSuchAlgorithmException ex) {
+      throw new RuntimeException("It should not happen, " + ex.getMessage(), ex);
+    }
+  }
+
+}

Added: hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/util/SignerException.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/util/SignerException.java?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/util/SignerException.java (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/main/java/org/apache/hadoop/alfredo/util/SignerException.java Fri Aug 19 22:31:06 2011
@@ -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.alfredo.util;
+
+/**
+ * Exception thrown by {@link Signer} when a string signature is invalid.
+ */
+public class SignerException extends Exception {
+  
+  static final long serialVersionUID = 0;
+
+  /**
+   * Creates an exception instance.
+   *
+   * @param msg message for the exception.
+   */
+  public SignerException(String msg) {
+    super(msg);
+  }
+}

Added: hadoop/common/trunk/hadoop-alfredo/src/site/apt/BuildingIt.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/site/apt/BuildingIt.apt.vm?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/site/apt/BuildingIt.apt.vm (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/site/apt/BuildingIt.apt.vm Fri Aug 19 22:31:06 2011
@@ -0,0 +1,75 @@
+~~ 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.
+
+  ---
+  Hadoop Alfredo, Java HTTP SPNEGO ${project.version} - Building It
+  ---
+  ---
+  ${maven.build.timestamp}
+
+Hadoop Alfredo, Java HTTP SPNEGO ${project.version} - Building It
+
+  \[ {{{./index.html}Go Back}} \]
+
+* Requirements
+
+  * Java 6+
+
+  * Maven 3+
+
+  * Kerberos KDC (for running Kerberos test cases)
+
+* Building
+
+  Use Maven goals: clean, test, compile, package, install
+
+  Available profiles: docs, testKerberos
+
+* Testing
+
+  By default Kerberos testcases are not run.
+
+  The requirements to run Kerberos testcases are a running KDC, a keytab
+  file with a client principal and a kerberos principal.
+
+  To run Kerberos tescases use the <<<testKerberos>>> Maven profile:
+
++---+
+$ mvn test -PtestKerberos
++---+
+
+  The following Maven <<<-D>>> options can be used to change the default
+  values:
+
+  * <<<alfredo.test.kerberos.realm>>>: default value <<LOCALHOST>>
+
+  * <<<alfredo.test.kerberos.client.principal>>>: default value <<client>>
+
+  * <<<alfredo.test.kerberos.server.principal>>>: default value
+    <<HTTP/localhost>> (it must start 'HTTP/')
+
+  * <<<alfredo.test.kerberos.keytab.file>>>: default value
+    <<${HOME}/${USER}.keytab>>
+
+** Generating Documentation
+
+  To create the documentation use the <<<docs>>> Maven profile:
+
++---+
+$ mvn package -Pdocs
++---+
+
+  The generated documentation is available at
+  <<<hadoop-alfredo/target/site/>>>.
+
+  \[ {{{./index.html}Go Back}} \]
+

Added: hadoop/common/trunk/hadoop-alfredo/src/site/apt/Configuration.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/site/apt/Configuration.apt.vm?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/site/apt/Configuration.apt.vm (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/site/apt/Configuration.apt.vm Fri Aug 19 22:31:06 2011
@@ -0,0 +1,181 @@
+~~ 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.
+
+  ---
+  Hadoop Alfredo, Java HTTP SPNEGO ${project.version} - Server Side
+  Configuration
+  ---
+  ---
+  ${maven.build.timestamp}
+
+Hadoop Alfredo, Java HTTP SPNEGO ${project.version} - Server Side
+Configuration
+
+  \[ {{{./index.html}Go Back}} \]
+
+* Server Side Configuration Setup
+
+  The {{{./apidocs/org/apache/hadoop/alfredo/server/AuthenticationFilter.html}
+  AuthenticationFilter filter}} is Alfredo's server side component.
+
+  This filter must be configured in front of all the web application resources
+  that required authenticated requests. For example:
+
+  The Alfredo and dependent JAR files must be in the web application classpath
+  (commonly the <<<WEB-INF/lib>>> directory).
+
+  Alfredo uses SLF4J-API for logging. Alfredo Maven POM dependencies define the
+  SLF4J API dependency but it does not define the dependency on a concrete
+  logging implementation, this must be addded explicitly to the web
+  application. For example, if the web applicationan uses Log4j, the
+  SLF4J-LOG4J12 and LOG4J jar files must be part part of the web application
+  classpath as well as the Log4j configuration file.
+
+** Common Configuration parameters
+
+  * <<<config.prefix>>>: If specified, all other configuration parameter names
+    must start with the prefix. The default value is no prefix.
+
+  * <<<[PREFIX.]type>>>: the authentication type keyword (<<<simple>>> or
+    <<<kerberos>>>) or a
+    {{{./apidocs/org/apache/hadoop/alfredo/server/AuthenticationHandler.html}
+    Authentication handler implementation}}.
+
+  * <<<[PREFIX.]signature.secret>>>: The secret to SHA-sign the generated
+    authentication tokens. If a secret is not provided a random secret is
+    generated at start up time. If using multiple web application instances
+    behind a load-balancer a secret must be set for the application to work
+    properly.
+
+  * <<<[PREFIX.]token.validity>>>: The validity -in seconds- of the generated
+    authentication token. The default value is <<<3600>>> seconds.
+
+  * <<<[PREFIX.]cookie.domain>>>: domain to use for the HTTP cookie that stores
+    the authentication token.
+
+  * <<<[PREFIX.]cookie.path>>>: path to use for the HTTP cookie that stores the
+    authentication token.
+
+** Kerberos Configuration
+
+  <<IMPORTANT>>: A KDC must be configured and running.
+
+  To use Kerberos SPNEGO as the authentication mechanism, the authentication
+  filter must be configured with the following init parameters:
+
+    * <<<[PREFIX.]type>>>: the keyword <<<kerberos>>>.
+
+    * <<<[PREFIX.]kerberos.principal>>>: The web-application Kerberos principal
+      name. The Kerberos principal name must start with <<<HTTP/...>>>. For
+      example: <<<...@LOCALHOST>>>.  There is no default value.
+
+    * <<<[PREFIX.]kerberos.keytab>>>: The path to the keytab file containing
+      the credentials for the kerberos principal. For example:
+      <<</Users/tucu/alfredo.keytab>>>. There is no default value.
+
+  <<Example>>:
+
++---+
+<web-app version="2.5" xmlns="http://java.sun.com/xml/ns/javaee">
+    ...
+
+    <filter>
+        <filter-name>kerberosFilter</filter-name>
+        <filter-class>org.apache.hadoop.alfredo.server.AuthenticationFilter</filter-class>
+        <init-param>
+            <param-name>type</param-name>
+            <param-value>kerberos</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/alfredo.keytab</param-value>
+        </init-param>
+    </filter>
+
+    <filter-mapping>
+        <filter-name>kerberosFilter</filter-name>
+        <url-pattern>/kerberos/*</url-pattern>
+    </filter-mapping>
+
+    ...
+</web-app>
++---+
+
+** Pseudo/Simple Configuration
+
+  To use Pseudo/Simple as the authentication mechanism (trusting the value of
+  the query string parameter 'user.name'), the authentication filter must be
+  configured with the following init parameters:
+
+    * <<<[PREFIX.]type>>>: the keyword <<<simple>>>.
+
+    * <<<[PREFIX.]simple.anonymous.allowed>>>: is a boolean parameter that
+      indicates if anonymous requests are allowed or not. The default value is
+      <<<false>>>.
+
+  <<Example>>:
+
++---+
+<web-app version="2.5" xmlns="http://java.sun.com/xml/ns/javaee">
+    ...
+
+    <filter>
+        <filter-name>simpleFilter</filter-name>
+        <filter-class>org.apache.hadoop.alfredo.server.AuthenticationFilter</filter-class>
+        <init-param>
+            <param-name>type</param-name>
+            <param-value>simple</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>simple.anonymous.allowed</param-name>
+            <param-value>false</param-value>
+        </init-param>
+    </filter>
+
+    <filter-mapping>
+        <filter-name>simpleFilter</filter-name>
+        <url-pattern>/simple/*</url-pattern>
+    </filter-mapping>
+
+    ...
+</web-app>
++---+
+
+  \[ {{{./index.html}Go Back}} \]

Added: hadoop/common/trunk/hadoop-alfredo/src/site/apt/Examples.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/site/apt/Examples.apt.vm?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/site/apt/Examples.apt.vm (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/site/apt/Examples.apt.vm Fri Aug 19 22:31:06 2011
@@ -0,0 +1,137 @@
+~~ 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.
+
+  ---
+  Hadoop Alfredo, Java HTTP SPNEGO ${project.version} - Examples
+  ---
+  ---
+  ${maven.build.timestamp}
+
+Hadoop Alfredo, Java HTTP SPNEGO ${project.version} - Examples
+
+  \[ {{{./index.html}Go Back}} \]
+
+* Accessing a Alfredo protected URL Using a browser
+
+  <<IMPORTANT:>> The browser must support HTTP Kerberos SPNEGO. For example,
+  Firefox or Internet Explorer.
+
+  For Firefox access the low level configuration page by loading the
+  <<<about:config>>> page. Then go to the
+  <<<network.negotiate-auth.trusted-uris>>> preference and add the hostname or
+  the domain of the web server that is HTTP Kerberos SPNEGO protected (if using
+  multiple domains and hostname use comma to separate them).
+  
+* Accessing a Alfredo protected URL Using <<<curl>>>
+
+  <<IMPORTANT:>> The <<<curl>>> version must support GSS, run <<<curl -V>>>.
+
++---+
+$ curl -V
+curl 7.19.7 (universal-apple-darwin10.0) libcurl/7.19.7 OpenSSL/0.9.8l zlib/1.2.3
+Protocols: tftp ftp telnet dict ldap http file https ftps
+Features: GSS-Negotiate IPv6 Largefile NTLM SSL libz
++---+
+
+  Login to the KDC using <<kinit>> and then use <<<curl>>> to fetch protected
+  URL:
+
++---+
+$ kinit
+Please enter the password for tucu@LOCALHOST:
+$ curl --negotiate -u foo -b ~/cookiejar.txt -c ~/cookiejar.txt http://localhost:8080/alfredo-examples/kerberos/who
+Enter host password for user 'tucu':
+
+Hello Alfredo!
++---+
+
+  * The <<<--negotiate>>> option enables SPNEGO in <<<curl>>>.
+
+  * The <<<-u foo>>> option is required but the user ignored (the principal
+    that has been kinit-ed is used).
+
+  * The <<<-b>>> and <<<-c>>> are use to store and send HTTP Cookies.
+
+* Using the Java Client
+
+  Use the <<<AuthenticatedURL>>> class to obtain an authenticated HTTP
+  connection:
+
++---+
+...
+URL url = new URL("http://localhost:8080/alfredo/kerberos/who");
+AuthenticatedURL.Token token = new AuthenticatedURL.Token();
+...
+HttpURLConnection conn = new AuthenticatedURL(url, token).openConnection();
+...
+conn = new AuthenticatedURL(url, token).openConnection();
+...
++---+
+
+* Building and Running the Examples
+
+  Download Alfredo's source code, the examples are in the
+  <<<src/main/examples>>> directory.
+
+** Server Example:
+
+  Edit the <<<src/main/examples/src/main/webapp/WEB-INF/web.xml>>> and set the
+  right configuration init parameters for the <<<AuthenticationFilter>>>
+  definition configured for Kerberos (the right Kerberos principal and keytab
+  file must be specified). Refer to the {{{./Configuration.html}Configuration
+  document}} for details.
+
+  Create the web application WAR file by running the <<<mvn package>>> command.
+
+  Deploy the WAR file in a servlet container. For example, if using Tomcat,
+  copy the WAR file to Tomcat's <<<webapps/>>> directory.
+
+  Start the servlet container.
+
+** Accessing the server using <<<curl>>>
+
+  Try accessing protected resources using <<<curl>>>. The protected resources
+  are:
+
++---+
+$ kinit
+Please enter the password for tucu@LOCALHOST:
+
+$ curl http://localhost:8080/alfredo-examples/anonymous/who
+
+$ curl http://localhost:8080/alfredo-examples/simple/who?user.name=foo
+
+$ curl --negotiate -u foo -b ~/cookiejar.txt -c ~/cookiejar.txt http://localhost:8080/alfredo-examples/kerberos/who
++---+
+
+** Accessing the server using the Java client example
+
++---+
+$ kinit
+Please enter the password for tucu@LOCALHOST:
+
+$ cd examples
+
+$ mvn exec:java -Durl=http://localhost:8080/alfredo-examples/kerberos/who
+
+....
+
+Token value: "u=tucu,p=tucu@LOCALHOST,t=kerberos,e=1295305313146,s=sVZ1mpSnC5TKhZQE3QLN5p2DWBo="
+Status code: 200 OK
+
+You are: user[tucu] principal[tucu@LOCALHOST]
+
+....
+
++---+
+
+  \[ {{{./index.html}Go Back}} \]

Added: hadoop/common/trunk/hadoop-alfredo/src/site/apt/index.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/site/apt/index.apt.vm?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/site/apt/index.apt.vm (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/site/apt/index.apt.vm Fri Aug 19 22:31:06 2011
@@ -0,0 +1,53 @@
+~~ 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.
+
+  ---
+  Hadoop Alfredo, Java HTTP SPNEGO ${project.version}
+  ---
+  ---
+  ${maven.build.timestamp}
+
+Hadoop Alfredo, Java HTTP SPNEGO ${project.version}
+
+  Hadoop Alfredo is a Java library consisting of a client and a server
+  components to enable Kerberos SPNEGO authentication for HTTP.
+
+  Alfredo also supports additional authentication mechanisms on the client
+  and the server side via 2 simple interfaces.
+
+* License
+
+  Alfredo is distributed under {{{http://www.apache.org/licenses/}Apache
+  License 2.0}}.
+
+* How Does Alfredo Works?
+
+  Alfredo enforces authentication on protected resources, once authentiation
+  has been established it sets a signed HTTP Cookie that contains an
+  authentication token with the user name, user principal, authentication type
+  and expiration time.
+
+  Subsequent HTTP client requests presenting the signed HTTP Cookie have access
+  to the protected resources until the HTTP Cookie expires.
+
+* User Documentation
+
+  * {{{./Examples.html}Examples}}
+
+  * {{{./Configuration.html}Configuration}}
+
+  * {{{./BuildingIt.html}Building It}}
+
+  * {{{./apidocs/index.html}JavaDocs}}
+
+  * {{{./dependencies.html}Dependencies}}
+

Added: hadoop/common/trunk/hadoop-alfredo/src/site/site.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/site/site.xml?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/site/site.xml (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/site/site.xml Fri Aug 19 22:31:06 2011
@@ -0,0 +1,34 @@
+<!--
+ 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.
+-->
+<project name="Hadoop Alfredo">
+
+  <version position="right"/>
+
+  <bannerLeft>
+    <name>&nbsp;</name>
+  </bannerLeft>
+
+  <skin>
+    <groupId>org.apache.maven.skins</groupId>
+    <artifactId>maven-stylus-skin</artifactId>
+    <version>1.1</version>
+  </skin>
+
+  <body>
+    <links>
+      <item name="Apache Hadoop" href="http://hadoop.apache.org/"/>
+    </links>
+  </body>
+
+</project>

Added: hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/KerberosTestUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/KerberosTestUtils.java?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/KerberosTestUtils.java (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/KerberosTestUtils.java Fri Aug 19 22:31:06 2011
@@ -0,0 +1,129 @@
+/**
+ * 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.alfredo;
+
+import com.sun.security.auth.module.Krb5LoginModule;
+
+import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginContext;
+import java.io.File;
+import java.security.Principal;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+
+/**
+ * Test helper class for Java Kerberos setup.
+ */
+public class KerberosTestUtils {
+  private static final String PREFIX = "alfredo.test.";
+
+  public static final String REALM = PREFIX + "kerberos.realm";
+
+  public static final String CLIENT_PRINCIPAL = PREFIX + "kerberos.client.principal";
+
+  public static final String SERVER_PRINCIPAL = PREFIX + "kerberos.server.principal";
+
+  public static final String KEYTAB_FILE = PREFIX + "kerberos.keytab.file";
+
+  public static String getRealm() {
+    return System.getProperty(REALM, "LOCALHOST");
+  }
+
+  public static String getClientPrincipal() {
+    return System.getProperty(CLIENT_PRINCIPAL, "client") + "@" + getRealm();
+  }
+
+  public static String getServerPrincipal() {
+    return System.getProperty(SERVER_PRINCIPAL, "HTTP/localhost") + "@" + getRealm();
+  }
+
+  public static String getKeytabFile() {
+    String keytabFile =
+      new File(System.getProperty("user.home"), System.getProperty("user.name") + ".keytab").toString();
+    return System.getProperty(KEYTAB_FILE, keytabFile);
+  }
+
+  private static class KerberosConfiguration extends Configuration {
+    private String principal;
+
+    public KerberosConfiguration(String principal) {
+      this.principal = principal;
+    }
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+      Map<String, String> options = new HashMap<String, String>();
+      options.put("keyTab", KerberosTestUtils.getKeytabFile());
+      options.put("principal", principal);
+      options.put("useKeyTab", "true");
+      options.put("storeKey", "true");
+      options.put("doNotPrompt", "true");
+      options.put("useTicketCache", "true");
+      options.put("renewTGT", "true");
+      options.put("refreshKrb5Config", "true");
+      options.put("isInitiator", "true");
+      String ticketCache = System.getenv("KRB5CCNAME");
+      if (ticketCache != null) {
+        options.put("ticketCache", ticketCache);
+      }
+      options.put("debug", "true");
+
+      return new AppConfigurationEntry[]{
+        new AppConfigurationEntry(Krb5LoginModule.class.getName(),
+                                  AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+                                  options),};
+    }
+  }
+
+  public static <T> T doAs(String principal, final Callable<T> callable) throws Exception {
+    LoginContext loginContext = null;
+    try {
+      Set<Principal> principals = new HashSet<Principal>();
+      principals.add(new KerberosPrincipal(KerberosTestUtils.getClientPrincipal()));
+      Subject subject = new Subject(false, principals, new HashSet<Object>(), new HashSet<Object>());
+      loginContext = new LoginContext("", subject, null, new KerberosConfiguration(principal));
+      loginContext.login();
+      subject = loginContext.getSubject();
+      return Subject.doAs(subject, new PrivilegedExceptionAction<T>() {
+        @Override
+        public T run() throws Exception {
+          return callable.call();
+        }
+      });
+    } catch (PrivilegedActionException ex) {
+      throw ex.getException();
+    } finally {
+      if (loginContext != null) {
+        loginContext.logout();
+      }
+    }
+  }
+
+  public static <T> T doAsClient(Callable<T> callable) throws Exception {
+    return doAs(getClientPrincipal(), callable);
+  }
+
+  public static <T> T doAsServer(Callable<T> callable) throws Exception {
+    return doAs(getServerPrincipal(), callable);
+  }
+
+}

Added: hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/client/AuthenticatorTestCase.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/client/AuthenticatorTestCase.java?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/client/AuthenticatorTestCase.java (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/client/AuthenticatorTestCase.java Fri Aug 19 22:31:06 2011
@@ -0,0 +1,152 @@
+/**
+ * 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.alfredo.client;
+
+import org.apache.hadoop.alfredo.server.AuthenticationFilter;
+import junit.framework.TestCase;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.servlet.Context;
+import org.mortbay.jetty.servlet.FilterHolder;
+import org.mortbay.jetty.servlet.ServletHolder;
+
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.net.HttpURLConnection;
+import java.net.ServerSocket;
+import java.net.URL;
+import java.util.Properties;
+
+public abstract class AuthenticatorTestCase extends TestCase {
+  private Server server;
+  private String host = null;
+  private int port = -1;
+  Context context;
+
+  private static Properties authenticatorConfig;
+
+  protected static void setAuthenticationHandlerConfig(Properties config) {
+    authenticatorConfig = config;
+  }
+
+  public static class TestFilter extends AuthenticationFilter {
+
+    @Override
+    protected Properties getConfiguration(String configPrefix, FilterConfig filterConfig) throws ServletException {
+      return authenticatorConfig;
+    }
+  }
+
+  public static class TestServlet extends HttpServlet {
+
+    @Override
+    protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
+      resp.setStatus(HttpServletResponse.SC_OK);
+    }
+
+    @Override
+    protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
+      InputStream is = req.getInputStream();
+      OutputStream os = resp.getOutputStream();
+      int c = is.read();
+      while (c > -1) {
+        os.write(c);
+        c = is.read();
+      }
+      is.close();
+      os.close();
+      resp.setStatus(HttpServletResponse.SC_OK);
+    }
+  }
+
+  protected void start() throws Exception {
+    server = new Server(0);
+    context = new Context();
+    context.setContextPath("/foo");
+    server.setHandler(context);
+    context.addFilter(new FilterHolder(TestFilter.class), "/*", 0);
+    context.addServlet(new ServletHolder(TestServlet.class), "/bar");
+    host = "localhost";
+    ServerSocket ss = new ServerSocket(0);
+    port = ss.getLocalPort();
+    ss.close();
+    server.getConnectors()[0].setHost(host);
+    server.getConnectors()[0].setPort(port);
+    server.start();
+    System.out.println("Running embedded servlet container at: http://" + host + ":" + port);
+  }
+
+  protected void stop() throws Exception {
+    try {
+      server.stop();
+    } catch (Exception e) {
+    }
+
+    try {
+      server.destroy();
+    } catch (Exception e) {
+    }
+  }
+
+  protected String getBaseURL() {
+    return "http://" + host + ":" + port + "/foo/bar";
+  }
+
+  private String POST = "test";
+
+  protected void _testAuthentication(Authenticator authenticator, boolean doPost) throws Exception {
+    start();
+    try {
+      URL url = new URL(getBaseURL());
+      AuthenticatedURL.Token token = new AuthenticatedURL.Token();
+      AuthenticatedURL aUrl = new AuthenticatedURL(authenticator);
+      HttpURLConnection conn = aUrl.openConnection(url, token);
+      String tokenStr = token.toString();
+      if (doPost) {
+        conn.setRequestMethod("POST");
+        conn.setDoOutput(true);
+      }
+      conn.connect();
+      if (doPost) {
+        Writer writer = new OutputStreamWriter(conn.getOutputStream());
+        writer.write(POST);
+        writer.close();
+      }
+      assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+      if (doPost) {
+        BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
+        String echo = reader.readLine();
+        assertEquals(POST, echo);
+        assertNull(reader.readLine());
+      }
+      aUrl = new AuthenticatedURL();
+      conn = aUrl.openConnection(url, token);
+      conn.connect();
+      assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+      assertEquals(tokenStr, token.toString());
+    } finally {
+      stop();
+    }
+  }
+
+}

Added: hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/client/TestAuthenticatedURL.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/client/TestAuthenticatedURL.java?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/client/TestAuthenticatedURL.java (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/client/TestAuthenticatedURL.java Fri Aug 19 22:31:06 2011
@@ -0,0 +1,113 @@
+/**
+ * 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.alfredo.client;
+
+import junit.framework.TestCase;
+import org.mockito.Mockito;
+
+import java.net.HttpURLConnection;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class TestAuthenticatedURL extends TestCase {
+
+  public void testToken() throws Exception {
+    AuthenticatedURL.Token token = new AuthenticatedURL.Token();
+    assertFalse(token.isSet());
+    token = new AuthenticatedURL.Token("foo");
+    assertTrue(token.isSet());
+    assertEquals("foo", token.toString());
+
+    AuthenticatedURL.Token token1 = new AuthenticatedURL.Token();
+    AuthenticatedURL.Token token2 = new AuthenticatedURL.Token();
+    assertEquals(token1.hashCode(), token2.hashCode());
+    assertTrue(token1.equals(token2));
+
+    token1 = new AuthenticatedURL.Token();
+    token2 = new AuthenticatedURL.Token("foo");
+    assertNotSame(token1.hashCode(), token2.hashCode());
+    assertFalse(token1.equals(token2));
+
+    token1 = new AuthenticatedURL.Token("foo");
+    token2 = new AuthenticatedURL.Token();
+    assertNotSame(token1.hashCode(), token2.hashCode());
+    assertFalse(token1.equals(token2));
+
+    token1 = new AuthenticatedURL.Token("foo");
+    token2 = new AuthenticatedURL.Token("foo");
+    assertEquals(token1.hashCode(), token2.hashCode());
+    assertTrue(token1.equals(token2));
+
+    token1 = new AuthenticatedURL.Token("bar");
+    token2 = new AuthenticatedURL.Token("foo");
+    assertNotSame(token1.hashCode(), token2.hashCode());
+    assertFalse(token1.equals(token2));
+
+    token1 = new AuthenticatedURL.Token("foo");
+    token2 = new AuthenticatedURL.Token("bar");
+    assertNotSame(token1.hashCode(), token2.hashCode());
+    assertFalse(token1.equals(token2));
+  }
+
+  public void testInjectToken() throws Exception {
+    HttpURLConnection conn = Mockito.mock(HttpURLConnection.class);
+    AuthenticatedURL.Token token = new AuthenticatedURL.Token();
+    token.set("foo");
+    AuthenticatedURL.injectToken(conn, token);
+    Mockito.verify(conn).addRequestProperty(Mockito.eq("Cookie"), Mockito.anyString());
+  }
+
+  public void testExtractTokenOK() throws Exception {
+    HttpURLConnection conn = Mockito.mock(HttpURLConnection.class);
+
+    Mockito.when(conn.getResponseCode()).thenReturn(HttpURLConnection.HTTP_OK);
+
+    String tokenStr = "foo";
+    Map<String, List<String>> headers = new HashMap<String, List<String>>();
+    List<String> cookies = new ArrayList<String>();
+    cookies.add(AuthenticatedURL.AUTH_COOKIE + "=" + tokenStr);
+    headers.put("Set-Cookie", cookies);
+    Mockito.when(conn.getHeaderFields()).thenReturn(headers);
+
+    AuthenticatedURL.Token token = new AuthenticatedURL.Token();
+    AuthenticatedURL.extractToken(conn, token);
+
+    assertEquals(tokenStr, token.toString());
+  }
+
+  public void testExtractTokenFail() throws Exception {
+    HttpURLConnection conn = Mockito.mock(HttpURLConnection.class);
+
+    Mockito.when(conn.getResponseCode()).thenReturn(HttpURLConnection.HTTP_UNAUTHORIZED);
+
+    String tokenStr = "foo";
+    Map<String, List<String>> headers = new HashMap<String, List<String>>();
+    List<String> cookies = new ArrayList<String>();
+    cookies.add(AuthenticatedURL.AUTH_COOKIE + "=" + tokenStr);
+    headers.put("Set-Cookie", cookies);
+    Mockito.when(conn.getHeaderFields()).thenReturn(headers);
+
+    try {
+      AuthenticatedURL.extractToken(conn, new AuthenticatedURL.Token());
+      fail();
+    } catch (AuthenticationException ex) {
+      // Expected
+    } catch (Exception ex) {
+      fail();
+    }
+  }
+
+}

Added: hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/client/TestKerberosAuthenticator.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/client/TestKerberosAuthenticator.java?rev=1159804&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/client/TestKerberosAuthenticator.java (added)
+++ hadoop/common/trunk/hadoop-alfredo/src/test/java/org/apache/hadoop/alfredo/client/TestKerberosAuthenticator.java Fri Aug 19 22:31:06 2011
@@ -0,0 +1,83 @@
+/**
+ * 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.alfredo.client;
+
+import org.apache.hadoop.alfredo.KerberosTestUtils;
+import org.apache.hadoop.alfredo.server.AuthenticationFilter;
+import org.apache.hadoop.alfredo.server.PseudoAuthenticationHandler;
+import org.apache.hadoop.alfredo.server.KerberosAuthenticationHandler;
+
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.Properties;
+import java.util.concurrent.Callable;
+
+public class TestKerberosAuthenticator extends AuthenticatorTestCase {
+
+  private Properties getAuthenticationHandlerConfiguration() {
+    Properties props = new Properties();
+    props.setProperty(AuthenticationFilter.AUTH_TYPE, "kerberos");
+    props.setProperty(KerberosAuthenticationHandler.PRINCIPAL, KerberosTestUtils.getServerPrincipal());
+    props.setProperty(KerberosAuthenticationHandler.KEYTAB, KerberosTestUtils.getKeytabFile());
+    props.setProperty(KerberosAuthenticationHandler.NAME_RULES,
+                      "RULE:[1:$1@$0](.*@" + KerberosTestUtils.getRealm()+")s/@.*//\n");
+    return props;
+  }
+
+  public void testFallbacktoPseudoAuthenticator() throws Exception {
+    Properties props = new Properties();
+    props.setProperty(AuthenticationFilter.AUTH_TYPE, "simple");
+    props.setProperty(PseudoAuthenticationHandler.ANONYMOUS_ALLOWED, "false");
+    setAuthenticationHandlerConfig(props);
+    _testAuthentication(new KerberosAuthenticator(), false);
+  }
+
+  public void testNotAuthenticated() throws Exception {
+    setAuthenticationHandlerConfig(getAuthenticationHandlerConfiguration());
+    start();
+    try {
+      URL url = new URL(getBaseURL());
+      HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+      conn.connect();
+      assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode());
+      assertTrue(conn.getHeaderField(KerberosAuthenticator.WWW_AUTHENTICATE) != null);
+    } finally {
+      stop();
+    }
+  }
+
+
+  public void testAuthentication() throws Exception {
+    setAuthenticationHandlerConfig(getAuthenticationHandlerConfiguration());
+    KerberosTestUtils.doAsClient(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        _testAuthentication(new KerberosAuthenticator(), false);
+        return null;
+      }
+    });
+  }
+
+  public void testAuthenticationPost() throws Exception {
+    setAuthenticationHandlerConfig(getAuthenticationHandlerConfiguration());
+    KerberosTestUtils.doAsClient(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        _testAuthentication(new KerberosAuthenticator(), true);
+        return null;
+      }
+    });
+  }
+
+}