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 ji...@apache.org on 2014/08/12 21:25:58 UTC

svn commit: r1617566 - in /hadoop/common/branches/HDFS-6584/hadoop-common-project: hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/ hadoop-common/ hadoop-common/src/main/java/ hadoop-common/src/main/java/org/apache/hadoop/sec...

Author: jing9
Date: Tue Aug 12 19:25:57 2014
New Revision: 1617566

URL: http://svn.apache.org/r1617566
Log:
Merging r1617377 through r1617565 from trunk.

Added:
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/HttpUserGroupInformation.java
      - copied unchanged from r1617565, hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/HttpUserGroupInformation.java
Modified:
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/CHANGES.txt   (contents, props changed)
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/   (props changed)
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java?rev=1617566&r1=1617565&r2=1617566&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java Tue Aug 12 19:25:57 2014
@@ -127,6 +127,7 @@ public class AuthenticationFilter implem
   public static final String SIGNATURE_PROVIDER_ATTRIBUTE =
       "org.apache.hadoop.security.authentication.util.SignerSecretProvider";
 
+  private Properties config;
   private Signer signer;
   private SignerSecretProvider secretProvider;
   private AuthenticationHandler authHandler;
@@ -150,7 +151,7 @@ public class AuthenticationFilter implem
   public void init(FilterConfig filterConfig) throws ServletException {
     String configPrefix = filterConfig.getInitParameter(CONFIG_PREFIX);
     configPrefix = (configPrefix != null) ? configPrefix + "." : "";
-    Properties config = getConfiguration(configPrefix, filterConfig);
+    config = getConfiguration(configPrefix, filterConfig);
     String authHandlerName = config.getProperty(AUTH_TYPE, null);
     String authHandlerClassName;
     if (authHandlerName == null) {
@@ -225,6 +226,17 @@ public class AuthenticationFilter implem
   }
 
   /**
+   * Returns the configuration properties of the {@link AuthenticationFilter}
+   * without the prefix. The returned properties are the same that the
+   * {@link #getConfiguration(String, FilterConfig)} method returned.
+   *
+   * @return the configuration properties.
+   */
+  protected Properties getConfiguration() {
+    return config;
+  }
+
+  /**
    * Returns the authentication handler being used.
    *
    * @return the authentication handler being used.
@@ -457,7 +469,7 @@ public class AuthenticationFilter implem
             createAuthCookie(httpResponse, signedToken, getCookieDomain(),
                     getCookiePath(), token.getExpires(), isHttps);
           }
-          filterChain.doFilter(httpRequest, httpResponse);
+          doFilter(filterChain, httpRequest, httpResponse);
         }
       } else {
         unauthorizedResponse = false;
@@ -482,6 +494,15 @@ public class AuthenticationFilter implem
   }
 
   /**
+   * Delegates call to the servlet filter chain. Sub-classes my override this
+   * method to perform pre and post tasks.
+   */
+  protected void doFilter(FilterChain filterChain, HttpServletRequest request,
+      HttpServletResponse response) throws IOException, ServletException {
+    filterChain.doFilter(request, response);
+  }
+
+  /**
    * Creates the Hadoop authentication HTTP cookie.
    *
    * @param token authentication token for the cookie.

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1617566&r1=1617565&r2=1617566&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/CHANGES.txt Tue Aug 12 19:25:57 2014
@@ -496,7 +496,14 @@ Release 2.6.0 - UNRELEASED
     HADOOP-10791. AuthenticationFilter should support externalizing the 
     secret for signing and provide rotation support. (rkanter via tucu)
 
-    HADOOP-10771. Refactor HTTP delegation support out of httpfs to common, PART 1. (tucu)
+    HADOOP-10771. Refactor HTTP delegation support out of httpfs to common. 
+    (tucu)
+
+    HADOOP-10835. Implement HTTP proxyuser support in HTTP authentication 
+    client/server libraries. (tucu)
+
+    HADOOP-10820. Throw an exception in GenericOptionsParser when passed
+    an empty Path. (Alex Holmes and Zhihai Xu via wang)
 
   OPTIMIZATIONS
 

Propchange: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/CHANGES.txt
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt:r1617377-1617565

Propchange: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java:r1617377-1617565

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java?rev=1617566&r1=1617565&r2=1617566&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java Tue Aug 12 19:25:57 2014
@@ -34,6 +34,7 @@ import java.io.IOException;
 import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
 import java.net.URL;
+import java.net.URLEncoder;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -61,6 +62,13 @@ import java.util.Map;
 public class DelegationTokenAuthenticatedURL extends AuthenticatedURL {
 
   /**
+   * Constant used in URL's query string to perform a proxy user request, the
+   * value of the <code>DO_AS</code> parameter is the user the request will be
+   * done on behalf of.
+   */
+  static final String DO_AS = "doAs";
+
+  /**
    * Client side authentication token that handles Delegation Tokens.
    */
   @InterfaceAudience.Public
@@ -247,6 +255,11 @@ public class DelegationTokenAuthenticate
       }
     }
 
+    // proxyuser
+    if (doAs != null) {
+      extraParams.put(DO_AS, URLEncoder.encode(doAs, "UTF-8"));
+    }
+
     url = augmentURL(url, extraParams);
     return super.openConnection(url, token);
   }

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java?rev=1617566&r1=1617565&r2=1617566&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java Tue Aug 12 19:25:57 2014
@@ -17,17 +17,39 @@
  */
 package org.apache.hadoop.security.token.delegation.web;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.SaslRpcServer;
+import org.apache.hadoop.security.UserGroupInformation;
 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.PseudoAuthenticationHandler;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+import org.apache.http.NameValuePair;
+import org.apache.http.client.utils.URLEncodedUtils;
+import org.codehaus.jackson.map.ObjectMapper;
 
+import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
 import javax.servlet.ServletContext;
 import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+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;
 
 /**
@@ -49,6 +71,10 @@ import java.util.Properties;
 public class DelegationTokenAuthenticationFilter
     extends AuthenticationFilter {
 
+  private static final String APPLICATION_JSON_MIME = "application/json";
+  private static final String ERROR_EXCEPTION_JSON = "exception";
+  private static final String ERROR_MESSAGE_JSON = "message";
+
   /**
    * Sets an external <code>DelegationTokenSecretManager</code> instance to
    * manage creation and verification of Delegation Tokens.
@@ -60,6 +86,14 @@ public class DelegationTokenAuthenticati
   public static final String DELEGATION_TOKEN_SECRET_MANAGER_ATTR =
       "hadoop.http.delegation-token-secret-manager";
 
+  private static final Charset UTF8_CHARSET = Charset.forName("UTF-8");
+
+  private static final ThreadLocal<UserGroupInformation> UGI_TL =
+      new ThreadLocal<UserGroupInformation>();
+  public static final String PROXYUSER_PREFIX = "proxyuser";
+
+  private SaslRpcServer.AuthMethod handlerAuthMethod;
+
   /**
    * It delegates to
    * {@link AuthenticationFilter#getConfiguration(String, FilterConfig)} and
@@ -86,17 +120,155 @@ public class DelegationTokenAuthenticati
     return props;
   }
 
+  /**
+   * Returns the proxyuser configuration. All returned properties must start
+   * with <code>proxyuser.</code>'
+   * <p/>
+   * Subclasses may override this method if the proxyuser configuration is 
+   * read from other place than the filter init parameters.
+   *
+   * @param filterConfig filter configuration object
+   * @return the proxyuser configuration properties.
+   * @throws ServletException thrown if the configuration could not be created.
+   */
+  protected Configuration getProxyuserConfiguration(FilterConfig filterConfig)
+      throws ServletException {
+    // this filter class gets the configuration from the filter configs, we are
+    // creating an empty configuration and injecting the proxyuser settings in
+    // it. In the initialization of the filter, the returned configuration is
+    // passed to the ProxyUsers which only looks for 'proxyusers.' properties.
+    Configuration conf = new Configuration(false);
+    Enumeration<?> names = filterConfig.getInitParameterNames();
+    while (names.hasMoreElements()) {
+      String name = (String) names.nextElement();
+      if (name.startsWith(PROXYUSER_PREFIX + ".")) {
+        String value = filterConfig.getInitParameter(name);
+        conf.set(name, value);
+      }
+    }
+    return conf;
+  }
+
+
   @Override
   public void init(FilterConfig filterConfig) throws ServletException {
     super.init(filterConfig);
+    AuthenticationHandler handler = getAuthenticationHandler();
     AbstractDelegationTokenSecretManager dtSecretManager =
         (AbstractDelegationTokenSecretManager) filterConfig.getServletContext().
             getAttribute(DELEGATION_TOKEN_SECRET_MANAGER_ATTR);
-    if (dtSecretManager != null && getAuthenticationHandler()
+    if (dtSecretManager != null && handler
         instanceof DelegationTokenAuthenticationHandler) {
-      DelegationTokenAuthenticationHandler handler =
+      DelegationTokenAuthenticationHandler dtHandler =
           (DelegationTokenAuthenticationHandler) getAuthenticationHandler();
-      handler.setExternalDelegationTokenSecretManager(dtSecretManager);
+      dtHandler.setExternalDelegationTokenSecretManager(dtSecretManager);
     }
+    if (handler instanceof PseudoAuthenticationHandler ||
+        handler instanceof PseudoDelegationTokenAuthenticationHandler) {
+      setHandlerAuthMethod(SaslRpcServer.AuthMethod.SIMPLE);
+    }
+    if (handler instanceof KerberosAuthenticationHandler ||
+        handler instanceof KerberosDelegationTokenAuthenticationHandler) {
+      setHandlerAuthMethod(SaslRpcServer.AuthMethod.KERBEROS);
+    }
+
+    // proxyuser configuration
+    Configuration conf = getProxyuserConfiguration(filterConfig);
+    ProxyUsers.refreshSuperUserGroupsConfiguration(conf, PROXYUSER_PREFIX);
   }
+
+  protected void setHandlerAuthMethod(SaslRpcServer.AuthMethod authMethod) {
+    this.handlerAuthMethod = authMethod;
+  }
+
+  @VisibleForTesting
+  static String getDoAs(HttpServletRequest request) {
+    List<NameValuePair> list = URLEncodedUtils.parse(request.getQueryString(),
+        UTF8_CHARSET);
+    if (list != null) {
+      for (NameValuePair nv : list) {
+        if (DelegationTokenAuthenticatedURL.DO_AS.equals(nv.getName())) {
+          return nv.getValue();
+        }
+      }
+    }
+    return null;
+  }
+
+  static UserGroupInformation getHttpUserGroupInformationInContext() {
+    return UGI_TL.get();
+  }
+
+  @Override
+  protected void doFilter(FilterChain filterChain, HttpServletRequest request,
+      HttpServletResponse response) throws IOException, ServletException {
+    boolean requestCompleted = false;
+    UserGroupInformation ugi = null;
+    AuthenticationToken authToken = (AuthenticationToken)
+        request.getUserPrincipal();
+    if (authToken != null && authToken != AuthenticationToken.ANONYMOUS) {
+      // if the request was authenticated because of a delegation token,
+      // then we ignore proxyuser (this is the same as the RPC behavior).
+      ugi = (UserGroupInformation) request.getAttribute(
+          DelegationTokenAuthenticationHandler.DELEGATION_TOKEN_UGI_ATTRIBUTE);
+      if (ugi == null) {
+        String realUser = request.getUserPrincipal().getName();
+        ugi = UserGroupInformation.createRemoteUser(realUser,
+            handlerAuthMethod);
+        String doAsUser = getDoAs(request);
+        if (doAsUser != null) {
+          ugi = UserGroupInformation.createProxyUser(doAsUser, ugi);
+          try {
+            ProxyUsers.authorize(ugi, request.getRemoteHost());
+          } catch (AuthorizationException ex) {
+            String msg = String.format(
+                "User '%s' from host '%s' not allowed to impersonate user '%s'",
+                realUser, request.getRemoteHost(), doAsUser);
+            response.setStatus(HttpServletResponse.SC_FORBIDDEN);
+            response.setContentType(APPLICATION_JSON_MIME);
+            Map<String, String> json = new HashMap<String, String>();
+            json.put(ERROR_EXCEPTION_JSON,
+                AuthorizationException.class.getName());
+            json.put(ERROR_MESSAGE_JSON, msg);
+            Writer writer = response.getWriter();
+            ObjectMapper jsonMapper = new ObjectMapper();
+            jsonMapper.writeValue(writer, json);
+            requestCompleted = true;
+          }
+        }
+      }
+      UGI_TL.set(ugi);
+    }
+    if (!requestCompleted) {
+      final UserGroupInformation ugiF = ugi;
+      try {
+        request = new HttpServletRequestWrapper(request) {
+
+          @Override
+          public String getAuthType() {
+            return (ugiF != null) ? handlerAuthMethod.toString() : null;
+          }
+
+          @Override
+          public String getRemoteUser() {
+            return (ugiF != null) ? ugiF.getShortUserName() : null;
+          }
+
+          @Override
+          public Principal getUserPrincipal() {
+            return (ugiF != null) ? new Principal() {
+              @Override
+              public String getName() {
+                return ugiF.getUserName();
+              }
+            } : null;
+          }
+        };
+        super.doFilter(filterChain, request, response);
+      } finally {
+        UGI_TL.remove();
+      }
+    }
+  }
+
 }

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java?rev=1617566&r1=1617565&r2=1617566&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java Tue Aug 12 19:25:57 2014
@@ -92,6 +92,9 @@ public abstract class DelegationTokenAut
 
   private static final Set<String> DELEGATION_TOKEN_OPS = new HashSet<String>();
 
+  static final String DELEGATION_TOKEN_UGI_ATTRIBUTE =
+      "hadoop.security.delegation-token.ugi";
+
   static {
     DELEGATION_TOKEN_OPS.add(KerberosDelegationTokenAuthenticator.
         DelegationTokenOperation.GETDELEGATIONTOKEN.toString());
@@ -342,6 +345,7 @@ public abstract class DelegationTokenAut
         token = new AuthenticationToken(shortName, ugi.getUserName(),
             getType());
         token.setExpires(0);
+        request.setAttribute(DELEGATION_TOKEN_UGI_ATTRIBUTE, ugi);
       } catch (Throwable ex) {
         throw new AuthenticationException("Could not verify DelegationToken, " +
             ex.toString(), ex);

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java?rev=1617566&r1=1617565&r2=1617566&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java Tue Aug 12 19:25:57 2014
@@ -378,9 +378,15 @@ public class GenericOptionsParser {
     if (files == null) 
       return null;
     String[] fileArr = files.split(",");
+    if (fileArr.length == 0) {
+      throw new IllegalArgumentException("File name can't be empty string");
+    }
     String[] finalArr = new String[fileArr.length];
     for (int i =0; i < fileArr.length; i++) {
       String tmp = fileArr[i];
+      if (tmp.isEmpty()) {
+        throw new IllegalArgumentException("File name can't be empty string");
+      }
       String finalPath;
       URI pathURI;
       try {

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java?rev=1617566&r1=1617565&r2=1617566&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java Tue Aug 12 19:25:57 2014
@@ -72,6 +72,10 @@ import java.util.UUID;
 import java.util.concurrent.Callable;
 
 public class TestWebDelegationToken {
+  private static final String OK_USER = "ok-user";
+  private static final String FAIL_USER = "fail-user";
+  private static final String FOO_USER = "foo";
+  
   private Server jetty;
 
   public static class DummyAuthenticationHandler
@@ -330,13 +334,13 @@ public class TestWebDelegationToken {
           new DelegationTokenAuthenticatedURL();
 
       try {
-        aUrl.getDelegationToken(nonAuthURL, token, "foo");
+        aUrl.getDelegationToken(nonAuthURL, token, FOO_USER);
         Assert.fail();
       } catch (Exception ex) {
         Assert.assertTrue(ex.getMessage().contains("401"));
       }
 
-      aUrl.getDelegationToken(authURL, token, "foo");
+      aUrl.getDelegationToken(authURL, token, FOO_USER);
       Assert.assertNotNull(token.getDelegationToken());
       Assert.assertEquals(new Text("token-kind"),
           token.getDelegationToken().getKind());
@@ -350,7 +354,7 @@ public class TestWebDelegationToken {
         Assert.assertTrue(ex.getMessage().contains("401"));
       }
 
-      aUrl.getDelegationToken(authURL, token, "foo");
+      aUrl.getDelegationToken(authURL, token, FOO_USER);
 
       try {
         aUrl.renewDelegationToken(authURL2, token);
@@ -359,15 +363,15 @@ public class TestWebDelegationToken {
         Assert.assertTrue(ex.getMessage().contains("403"));
       }
 
-      aUrl.getDelegationToken(authURL, token, "foo");
+      aUrl.getDelegationToken(authURL, token, FOO_USER);
 
       aUrl.cancelDelegationToken(authURL, token);
 
-      aUrl.getDelegationToken(authURL, token, "foo");
+      aUrl.getDelegationToken(authURL, token, FOO_USER);
 
       aUrl.cancelDelegationToken(nonAuthURL, token);
 
-      aUrl.getDelegationToken(authURL, token, "foo");
+      aUrl.getDelegationToken(authURL, token, FOO_USER);
 
       try {
         aUrl.renewDelegationToken(nonAuthURL, token);
@@ -416,7 +420,7 @@ public class TestWebDelegationToken {
       DelegationTokenAuthenticatedURL aUrl =
           new DelegationTokenAuthenticatedURL();
 
-      aUrl.getDelegationToken(authURL, token, "foo");
+      aUrl.getDelegationToken(authURL, token, FOO_USER);
       Assert.assertNotNull(token.getDelegationToken());
       Assert.assertEquals(new Text("fooKind"),
           token.getDelegationToken().getKind());
@@ -488,7 +492,7 @@ public class TestWebDelegationToken {
       jetty.start();
       final URL url = new URL(getJettyURL() + "/foo/bar");
 
-      UserGroupInformation ugi = UserGroupInformation.createRemoteUser("foo");
+      UserGroupInformation ugi = UserGroupInformation.createRemoteUser(FOO_USER);
       ugi.doAs(new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
@@ -501,10 +505,10 @@ public class TestWebDelegationToken {
               conn.getResponseCode());
           List<String> ret = IOUtils.readLines(conn.getInputStream());
           Assert.assertEquals(1, ret.size());
-          Assert.assertEquals("foo", ret.get(0));
+          Assert.assertEquals(FOO_USER, ret.get(0));
 
           try {
-            aUrl.getDelegationToken(url, token, "foo");
+            aUrl.getDelegationToken(url, token, FOO_USER);
             Assert.fail();
           } catch (AuthenticationException ex) {
             Assert.assertTrue(ex.getMessage().contains(
@@ -531,6 +535,16 @@ public class TestWebDelegationToken {
           "token-kind");
       return conf;
     }
+
+    @Override
+    protected org.apache.hadoop.conf.Configuration getProxyuserConfiguration(
+        FilterConfig filterConfig) throws ServletException {
+      org.apache.hadoop.conf.Configuration conf =
+          new org.apache.hadoop.conf.Configuration(false);
+      conf.set("proxyuser.foo.users", OK_USER);
+      conf.set("proxyuser.foo.hosts", "localhost");
+      return conf;
+    }
   }
 
   @Test
@@ -547,7 +561,7 @@ public class TestWebDelegationToken {
       jetty.start();
       final URL url = new URL(getJettyURL() + "/foo/bar");
 
-      UserGroupInformation ugi = UserGroupInformation.createRemoteUser("foo");
+      UserGroupInformation ugi = UserGroupInformation.createRemoteUser(FOO_USER);
       ugi.doAs(new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
@@ -560,9 +574,9 @@ public class TestWebDelegationToken {
               conn.getResponseCode());
           List<String> ret = IOUtils.readLines(conn.getInputStream());
           Assert.assertEquals(1, ret.size());
-          Assert.assertEquals("foo", ret.get(0));
+          Assert.assertEquals(FOO_USER, ret.get(0));
 
-          aUrl.getDelegationToken(url, token, "foo");
+          aUrl.getDelegationToken(url, token, FOO_USER);
           Assert.assertNotNull(token.getDelegationToken());
           Assert.assertEquals(new Text("token-kind"),
               token.getDelegationToken().getKind());
@@ -684,7 +698,7 @@ public class TestWebDelegationToken {
       final URL url = new URL(getJettyURL() + "/foo/bar");
 
       try {
-        aUrl.getDelegationToken(url, token, "foo");
+        aUrl.getDelegationToken(url, token, FOO_USER);
         Assert.fail();
       } catch (AuthenticationException ex) {
         Assert.assertTrue(ex.getMessage().contains("GSSException"));
@@ -700,7 +714,7 @@ public class TestWebDelegationToken {
               aUrl.renewDelegationToken(url, token);
               Assert.assertNotNull(token.getDelegationToken());
 
-              aUrl.getDelegationToken(url, token, "foo");
+              aUrl.getDelegationToken(url, token, FOO_USER);
               Assert.assertNotNull(token.getDelegationToken());
 
               try {
@@ -710,7 +724,7 @@ public class TestWebDelegationToken {
                 Assert.assertTrue(ex.getMessage().contains("403"));
               }
 
-              aUrl.getDelegationToken(url, token, "foo");
+              aUrl.getDelegationToken(url, token, FOO_USER);
 
               aUrl.cancelDelegationToken(url, token);
               Assert.assertNull(token.getDelegationToken());
@@ -724,4 +738,132 @@ public class TestWebDelegationToken {
     }
   }
 
+  @Test
+  public void testProxyUser() throws Exception {
+    final Server jetty = createJettyServer();
+    Context context = new Context();
+    context.setContextPath("/foo");
+    jetty.setHandler(context);
+    context.addFilter(new FilterHolder(PseudoDTAFilter.class), "/*", 0);
+    context.addServlet(new ServletHolder(UserServlet.class), "/bar");
+
+    try {
+      jetty.start();
+      final URL url = new URL(getJettyURL() + "/foo/bar");
+
+      UserGroupInformation ugi = UserGroupInformation.createRemoteUser(FOO_USER);
+      ugi.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          DelegationTokenAuthenticatedURL.Token token =
+              new DelegationTokenAuthenticatedURL.Token();
+          DelegationTokenAuthenticatedURL aUrl =
+              new DelegationTokenAuthenticatedURL();
+
+          // proxyuser using authentication handler authentication
+          HttpURLConnection conn = aUrl.openConnection(url, token, OK_USER);
+          Assert.assertEquals(HttpURLConnection.HTTP_OK,
+              conn.getResponseCode());
+          List<String> ret = IOUtils.readLines(conn.getInputStream());
+          Assert.assertEquals(1, ret.size());
+          Assert.assertEquals(OK_USER, ret.get(0));
+
+          // unauthorized proxy user using authentication handler authentication
+          conn = aUrl.openConnection(url, token, FAIL_USER);
+          Assert.assertEquals(HttpURLConnection.HTTP_FORBIDDEN,
+              conn.getResponseCode());
+
+          // proxy using delegation token authentication
+          aUrl.getDelegationToken(url, token, FOO_USER);
+
+          UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+          ugi.addToken(token.getDelegationToken());
+          token = new DelegationTokenAuthenticatedURL.Token();
+
+          // requests using delegation token as auth do not honor doAs
+          conn = aUrl.openConnection(url, token, OK_USER);
+          Assert.assertEquals(HttpURLConnection.HTTP_OK,
+              conn.getResponseCode());
+          ret = IOUtils.readLines(conn.getInputStream());
+          Assert.assertEquals(1, ret.size());
+          Assert.assertEquals(FOO_USER, ret.get(0));
+
+          return null;
+        }
+      });
+    } finally {
+      jetty.stop();
+    }
+  }
+
+
+  public static class UGIServlet extends HttpServlet {
+
+    @Override
+    protected void doGet(HttpServletRequest req, HttpServletResponse resp)
+        throws ServletException, IOException {
+      UserGroupInformation ugi = HttpUserGroupInformation.get();
+      if (ugi != null) {
+        String ret = "remoteuser=" + req.getRemoteUser() + ":ugi=" +
+            ugi.getShortUserName();
+        if (ugi.getAuthenticationMethod() ==
+            UserGroupInformation.AuthenticationMethod.PROXY) {
+          ret = "realugi=" + ugi.getRealUser().getShortUserName() + ":" + ret;
+        }
+        resp.setStatus(HttpServletResponse.SC_OK);
+        resp.getWriter().write(ret);
+      } else {
+        resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+      }
+    }
+  }
+
+  @Test
+  public void testHttpUGI() throws Exception {
+    final Server jetty = createJettyServer();
+    Context context = new Context();
+    context.setContextPath("/foo");
+    jetty.setHandler(context);
+    context.addFilter(new FilterHolder(PseudoDTAFilter.class), "/*", 0);
+    context.addServlet(new ServletHolder(UGIServlet.class), "/bar");
+
+    try {
+      jetty.start();
+      final URL url = new URL(getJettyURL() + "/foo/bar");
+
+      UserGroupInformation ugi = UserGroupInformation.createRemoteUser(FOO_USER);
+      ugi.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          DelegationTokenAuthenticatedURL.Token token =
+              new DelegationTokenAuthenticatedURL.Token();
+          DelegationTokenAuthenticatedURL aUrl =
+              new DelegationTokenAuthenticatedURL();
+
+          // user foo
+          HttpURLConnection conn = aUrl.openConnection(url, token);
+          Assert.assertEquals(HttpURLConnection.HTTP_OK,
+              conn.getResponseCode());
+          List<String> ret = IOUtils.readLines(conn.getInputStream());
+          Assert.assertEquals(1, ret.size());
+          Assert.assertEquals("remoteuser=" + FOO_USER+ ":ugi=" + FOO_USER, 
+              ret.get(0));
+
+          // user ok-user via proxyuser foo
+          conn = aUrl.openConnection(url, token, OK_USER);
+          Assert.assertEquals(HttpURLConnection.HTTP_OK,
+              conn.getResponseCode());
+          ret = IOUtils.readLines(conn.getInputStream());
+          Assert.assertEquals(1, ret.size());
+          Assert.assertEquals("realugi=" + FOO_USER +":remoteuser=" + OK_USER + 
+                  ":ugi=" + OK_USER, ret.get(0));
+
+          return null;
+        }
+      });
+    } finally {
+      jetty.stop();
+    }
+  }
+
 }

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java?rev=1617566&r1=1617565&r2=1617566&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java Tue Aug 12 19:25:57 2014
@@ -21,11 +21,14 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 import java.util.Map;
 
 import junit.framework.TestCase;
 
+import org.apache.commons.math3.util.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -34,12 +37,14 @@ import org.apache.hadoop.security.Creden
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.junit.Assert;
 
 import com.google.common.collect.Maps;
+import static org.junit.Assert.fail;
 
 public class TestGenericOptionsParser extends TestCase {
   File testDir;
@@ -93,6 +98,67 @@ public class TestGenericOptionsParser ex
   }
 
   /**
+   * Test the case where the libjars, files and archives arguments
+   * contains an empty token, which should create an IllegalArgumentException.
+   */
+  public void testEmptyFilenames() throws Exception {
+    List<Pair<String, String>> argsAndConfNames = new ArrayList<Pair<String, String>>();
+    argsAndConfNames.add(new Pair<String, String>("-libjars", "tmpjars"));
+    argsAndConfNames.add(new Pair<String, String>("-files", "tmpfiles"));
+    argsAndConfNames.add(new Pair<String, String>("-archives", "tmparchives"));
+    for (Pair<String, String> argAndConfName : argsAndConfNames) {
+      String arg = argAndConfName.getFirst();
+      String configName = argAndConfName.getSecond();
+
+      File tmpFileOne = new File(testDir, "tmpfile1");
+      Path tmpPathOne = new Path(tmpFileOne.toString());
+      File tmpFileTwo = new File(testDir, "tmpfile2");
+      Path tmpPathTwo = new Path(tmpFileTwo.toString());
+      localFs.create(tmpPathOne);
+      localFs.create(tmpPathTwo);
+      String[] args = new String[2];
+      args[0] = arg;
+      // create an empty path in between two valid files,
+      // which prior to HADOOP-10820 used to result in the
+      // working directory being added to "tmpjars" (or equivalent)
+      args[1] = String.format("%s,,%s",
+          tmpFileOne.toURI().toString(), tmpFileTwo.toURI().toString());
+      try {
+        new GenericOptionsParser(conf, args);
+        fail("Expected exception for empty filename");
+      } catch (IllegalArgumentException e) {
+        // expect to receive an IllegalArgumentException
+        GenericTestUtils.assertExceptionContains("File name can't be"
+            + " empty string", e);
+      }
+
+      // test zero file list length - it should create an exception
+      args[1] = ",,";
+      try {
+        new GenericOptionsParser(conf, args);
+        fail("Expected exception for zero file list length");
+      } catch (IllegalArgumentException e) {
+        // expect to receive an IllegalArgumentException
+        GenericTestUtils.assertExceptionContains("File name can't be"
+            + " empty string", e);
+      }
+
+      // test filename with space character
+      // it should create exception from parser in URI class
+      // due to URI syntax error
+      args[1] = String.format("%s, ,%s",
+          tmpFileOne.toURI().toString(), tmpFileTwo.toURI().toString());
+      try {
+        new GenericOptionsParser(conf, args);
+        fail("Expected exception for filename with space character");
+      } catch (IllegalArgumentException e) {
+        // expect to receive an IllegalArgumentException
+        GenericTestUtils.assertExceptionContains("URISyntaxException", e);
+      }
+    }
+  }
+
+  /**
    * Test that options passed to the constructor are used.
    */
   @SuppressWarnings("static-access")