Prechádzať zdrojové kódy

HDFS-6849. Replace HttpFS custom proxyuser handling with common implementation. (tucu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1617831 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur 10 rokov pred
rodič
commit
065d9ec5d9

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSAuthenticationFilter.java

@@ -91,4 +91,14 @@ public class HttpFSAuthenticationFilter
     return props;
   }
 
+  protected Configuration getProxyuserConfiguration(FilterConfig filterConfig) {
+    Map<String, String> proxyuserConf = HttpFSServerWebApp.get().getConfig().
+        getValByRegex("httpfs\\.proxyuser\\.");
+    Configuration conf = new Configuration(false);
+    for (Map.Entry<String, String> entry : proxyuserConf.entrySet()) {
+      conf.set(entry.getKey().substring("httpfs.".length()), entry.getValue());
+    }
+    return conf;
+  }
+
 }

+ 29 - 79
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java

@@ -30,8 +30,6 @@ import org.apache.hadoop.lib.wsrs.Param;
 import org.apache.hadoop.lib.wsrs.ParametersProvider;
 import org.apache.hadoop.lib.wsrs.ShortParam;
 import org.apache.hadoop.lib.wsrs.StringParam;
-import org.apache.hadoop.lib.wsrs.UserProvider;
-import org.slf4j.MDC;
 
 import javax.ws.rs.ext.Provider;
 import java.util.HashMap;
@@ -53,57 +51,44 @@ public class HttpFSParametersProvider extends ParametersProvider {
 
   static {
     PARAMS_DEF.put(Operation.OPEN,
-      new Class[]{DoAsParam.class, OffsetParam.class, LenParam.class});
-    PARAMS_DEF.put(Operation.GETFILESTATUS, new Class[]{DoAsParam.class});
-    PARAMS_DEF.put(Operation.LISTSTATUS,
-      new Class[]{DoAsParam.class, FilterParam.class});
-    PARAMS_DEF.put(Operation.GETHOMEDIRECTORY, new Class[]{DoAsParam.class});
-    PARAMS_DEF.put(Operation.GETCONTENTSUMMARY, new Class[]{DoAsParam.class});
-    PARAMS_DEF.put(Operation.GETFILECHECKSUM, new Class[]{DoAsParam.class});
-    PARAMS_DEF.put(Operation.GETFILEBLOCKLOCATIONS,
-      new Class[]{DoAsParam.class});
-    PARAMS_DEF.put(Operation.GETACLSTATUS, new Class[]{DoAsParam.class});
-    PARAMS_DEF.put(Operation.INSTRUMENTATION, new Class[]{DoAsParam.class});
-    PARAMS_DEF.put(Operation.APPEND,
-      new Class[]{DoAsParam.class, DataParam.class});
+        new Class[]{OffsetParam.class, LenParam.class});
+    PARAMS_DEF.put(Operation.GETFILESTATUS, new Class[]{});
+    PARAMS_DEF.put(Operation.LISTSTATUS, new Class[]{FilterParam.class});
+    PARAMS_DEF.put(Operation.GETHOMEDIRECTORY, new Class[]{});
+    PARAMS_DEF.put(Operation.GETCONTENTSUMMARY, new Class[]{});
+    PARAMS_DEF.put(Operation.GETFILECHECKSUM, new Class[]{});
+    PARAMS_DEF.put(Operation.GETFILEBLOCKLOCATIONS, new Class[]{});
+    PARAMS_DEF.put(Operation.GETACLSTATUS, new Class[]{});
+    PARAMS_DEF.put(Operation.INSTRUMENTATION, new Class[]{});
+    PARAMS_DEF.put(Operation.APPEND, new Class[]{DataParam.class});
     PARAMS_DEF.put(Operation.CONCAT, new Class[]{SourcesParam.class});
     PARAMS_DEF.put(Operation.CREATE,
-      new Class[]{DoAsParam.class, PermissionParam.class, OverwriteParam.class,
+      new Class[]{PermissionParam.class, OverwriteParam.class,
                   ReplicationParam.class, BlockSizeParam.class, DataParam.class});
-    PARAMS_DEF.put(Operation.MKDIRS,
-      new Class[]{DoAsParam.class, PermissionParam.class});
-    PARAMS_DEF.put(Operation.RENAME,
-      new Class[]{DoAsParam.class, DestinationParam.class});
+    PARAMS_DEF.put(Operation.MKDIRS, new Class[]{PermissionParam.class});
+    PARAMS_DEF.put(Operation.RENAME, new Class[]{DestinationParam.class});
     PARAMS_DEF.put(Operation.SETOWNER,
-      new Class[]{DoAsParam.class, OwnerParam.class, GroupParam.class});
-    PARAMS_DEF.put(Operation.SETPERMISSION,
-      new Class[]{DoAsParam.class, PermissionParam.class});
+        new Class[]{OwnerParam.class, GroupParam.class});
+    PARAMS_DEF.put(Operation.SETPERMISSION, new Class[]{PermissionParam.class});
     PARAMS_DEF.put(Operation.SETREPLICATION,
-      new Class[]{DoAsParam.class, ReplicationParam.class});
+        new Class[]{ReplicationParam.class});
     PARAMS_DEF.put(Operation.SETTIMES,
-      new Class[]{DoAsParam.class, ModifiedTimeParam.class,
-                  AccessTimeParam.class});
-    PARAMS_DEF.put(Operation.DELETE,
-      new Class[]{DoAsParam.class, RecursiveParam.class});
-    PARAMS_DEF.put(Operation.SETACL,
-            new Class[]{DoAsParam.class, AclPermissionParam.class});
-    PARAMS_DEF.put(Operation.REMOVEACL,
-            new Class[]{DoAsParam.class});
+        new Class[]{ModifiedTimeParam.class, AccessTimeParam.class});
+    PARAMS_DEF.put(Operation.DELETE, new Class[]{RecursiveParam.class});
+    PARAMS_DEF.put(Operation.SETACL, new Class[]{AclPermissionParam.class});
+    PARAMS_DEF.put(Operation.REMOVEACL, new Class[]{});
     PARAMS_DEF.put(Operation.MODIFYACLENTRIES,
-            new Class[]{DoAsParam.class, AclPermissionParam.class});
+        new Class[]{AclPermissionParam.class});
     PARAMS_DEF.put(Operation.REMOVEACLENTRIES,
-            new Class[]{DoAsParam.class, AclPermissionParam.class});
-    PARAMS_DEF.put(Operation.REMOVEDEFAULTACL,
-            new Class[]{DoAsParam.class});
+        new Class[]{AclPermissionParam.class});
+    PARAMS_DEF.put(Operation.REMOVEDEFAULTACL, new Class[]{});
     PARAMS_DEF.put(Operation.SETXATTR,
-      new Class[]{DoAsParam.class, XAttrNameParam.class, XAttrValueParam.class, 
+        new Class[]{XAttrNameParam.class, XAttrValueParam.class,
                   XAttrSetFlagParam.class});
-    PARAMS_DEF.put(Operation.REMOVEXATTR, 
-      new Class[]{DoAsParam.class, XAttrNameParam.class});
+    PARAMS_DEF.put(Operation.REMOVEXATTR, new Class[]{XAttrNameParam.class});
     PARAMS_DEF.put(Operation.GETXATTRS, 
-      new Class[]{DoAsParam.class, XAttrNameParam.class, XAttrEncodingParam.class});
-    PARAMS_DEF.put(Operation.LISTXATTRS,
-      new Class[]{DoAsParam.class});
+        new Class[]{XAttrNameParam.class, XAttrEncodingParam.class});
+    PARAMS_DEF.put(Operation.LISTXATTRS, new Class[]{});
   }
 
   public HttpFSParametersProvider() {
@@ -205,41 +190,6 @@ public class HttpFSParametersProvider extends ParametersProvider {
     }
   }
 
-  /**
-   * Class for do-as parameter.
-   */
-  @InterfaceAudience.Private
-  public static class DoAsParam extends StringParam {
-
-    /**
-     * Parameter name.
-     */
-    public static final String NAME = HttpFSFileSystem.DO_AS_PARAM;
-
-    /**
-     * Constructor.
-     */
-    public DoAsParam() {
-      super(NAME, null, UserProvider.getUserPattern());
-    }
-
-    /**
-     * Delegates to parent and then adds do-as user to
-     * MDC context for logging purposes.
-     *
-     *
-     * @param str parameter value.
-     *
-     * @return parsed parameter
-     */
-    @Override
-    public String parseParam(String str) {
-      String doAs = super.parseParam(str);
-      MDC.put(getName(), (doAs != null) ? doAs : "-");
-      return doAs;
-    }
-  }
-
   /**
    * Class for filter parameter.
    */
@@ -275,7 +225,7 @@ public class HttpFSParametersProvider extends ParametersProvider {
      * Constructor.
      */
     public GroupParam() {
-      super(NAME, null, UserProvider.getUserPattern());
+      super(NAME, null);
     }
 
   }
@@ -371,7 +321,7 @@ public class HttpFSParametersProvider extends ParametersProvider {
      * Constructor.
      */
     public OwnerParam() {
-      super(NAME, null, UserProvider.getUserPattern());
+      super(NAME, null);
     }
 
   }

+ 46 - 98
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java

@@ -29,7 +29,6 @@ import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.AclPermissionPa
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.BlockSizeParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.DataParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.DestinationParam;
-import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.DoAsParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.FilterParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.GroupParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.LenParam;
@@ -50,12 +49,11 @@ import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.lib.service.FileSystemAccessException;
 import org.apache.hadoop.lib.service.Groups;
 import org.apache.hadoop.lib.service.Instrumentation;
-import org.apache.hadoop.lib.service.ProxyUser;
 import org.apache.hadoop.lib.servlet.FileSystemReleaseFilter;
-import org.apache.hadoop.lib.servlet.HostnameFilter;
 import org.apache.hadoop.lib.wsrs.InputStreamEntity;
 import org.apache.hadoop.lib.wsrs.Parameters;
-import org.apache.hadoop.security.authentication.server.AuthenticationToken;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.delegation.web.HttpUserGroupInformation;
 import org.json.simple.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -79,7 +77,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
 import java.security.AccessControlException;
-import java.security.Principal;
 import java.text.MessageFormat;
 import java.util.EnumSet;
 import java.util.List;
@@ -96,49 +93,11 @@ import java.util.Map;
 public class HttpFSServer {
   private static Logger AUDIT_LOG = LoggerFactory.getLogger("httpfsaudit");
 
-  /**
-   * Resolves the effective user that will be used to request a FileSystemAccess filesystem.
-   * <p/>
-   * If the doAs-user is NULL or the same as the user, it returns the user.
-   * <p/>
-   * Otherwise it uses proxyuser rules (see {@link ProxyUser} to determine if the
-   * current user can impersonate the doAs-user.
-   * <p/>
-   * If the current user cannot impersonate the doAs-user an
-   * <code>AccessControlException</code> will be thrown.
-   *
-   * @param user principal for whom the filesystem instance is.
-   * @param doAs do-as user, if any.
-   *
-   * @return the effective user.
-   *
-   * @throws IOException thrown if an IO error occurrs.
-   * @throws AccessControlException thrown if the current user cannot impersonate
-   * the doAs-user.
-   */
-  private String getEffectiveUser(Principal user, String doAs) throws IOException {
-    String effectiveUser = user.getName();
-    if (doAs != null && !doAs.equals(user.getName())) {
-      ProxyUser proxyUser = HttpFSServerWebApp.get().get(ProxyUser.class);
-      String proxyUserName;
-      if (user instanceof AuthenticationToken) {
-        proxyUserName = ((AuthenticationToken)user).getUserName();
-      } else {
-        proxyUserName = user.getName();
-      }
-      proxyUser.validate(proxyUserName, HostnameFilter.get(), doAs);
-      effectiveUser = doAs;
-      AUDIT_LOG.info("Proxy user [{}] DoAs user [{}]", proxyUserName, doAs);
-    }
-    return effectiveUser;
-  }
-
   /**
    * Executes a {@link FileSystemAccess.FileSystemExecutor} using a filesystem for the effective
    * user.
    *
-   * @param user principal making the request.
-   * @param doAs do-as user, if any.
+   * @param ugi user making the request.
    * @param executor FileSystemExecutor to execute.
    *
    * @return FileSystemExecutor response
@@ -147,12 +106,11 @@ public class HttpFSServer {
    * @throws FileSystemAccessException thrown if a FileSystemAccess releated error occurred. Thrown
    * exceptions are handled by {@link HttpFSExceptionProvider}.
    */
-  private <T> T fsExecute(Principal user, String doAs, FileSystemAccess.FileSystemExecutor<T> executor)
+  private <T> T fsExecute(UserGroupInformation ugi, FileSystemAccess.FileSystemExecutor<T> executor)
     throws IOException, FileSystemAccessException {
-    String hadoopUser = getEffectiveUser(user, doAs);
     FileSystemAccess fsAccess = HttpFSServerWebApp.get().get(FileSystemAccess.class);
     Configuration conf = HttpFSServerWebApp.get().get(FileSystemAccess.class).getFileSystemConfiguration();
-    return fsAccess.execute(hadoopUser, conf, executor);
+    return fsAccess.execute(ugi.getShortUserName(), conf, executor);
   }
 
   /**
@@ -162,8 +120,7 @@ public class HttpFSServer {
    * If a do-as user is specified, the current user must be a valid proxyuser, otherwise an
    * <code>AccessControlException</code> will be thrown.
    *
-   * @param user principal for whom the filesystem instance is.
-   * @param doAs do-as user, if any.
+   * @param ugi principal for whom the filesystem instance is.
    *
    * @return a filesystem for the specified user or do-as user.
    *
@@ -172,8 +129,9 @@ public class HttpFSServer {
    * @throws FileSystemAccessException thrown if a FileSystemAccess releated error occurred. Thrown
    * exceptions are handled by {@link HttpFSExceptionProvider}.
    */
-  private FileSystem createFileSystem(Principal user, String doAs) throws IOException, FileSystemAccessException {
-    String hadoopUser = getEffectiveUser(user, doAs);
+  private FileSystem createFileSystem(UserGroupInformation ugi)
+      throws IOException, FileSystemAccessException {
+    String hadoopUser = ugi.getShortUserName();
     FileSystemAccess fsAccess = HttpFSServerWebApp.get().get(FileSystemAccess.class);
     Configuration conf = HttpFSServerWebApp.get().get(FileSystemAccess.class).getFileSystemConfiguration();
     FileSystem fs = fsAccess.createFileSystem(hadoopUser, conf);
@@ -192,7 +150,6 @@ public class HttpFSServer {
   /**
    * Special binding for '/' as it is not handled by the wildcard binding.
    *
-   * @param user the principal of the user making the request.
    * @param op the HttpFS operation of the request.
    * @param params the HttpFS parameters of the request.
    *
@@ -206,11 +163,10 @@ public class HttpFSServer {
    */
   @GET
   @Produces(MediaType.APPLICATION_JSON)
-  public Response getRoot(@Context Principal user,
-                          @QueryParam(OperationParam.NAME) OperationParam op,
+  public Response getRoot(@QueryParam(OperationParam.NAME) OperationParam op,
                           @Context Parameters params)
     throws IOException, FileSystemAccessException {
-    return get(user, "", op, params);
+    return get("", op, params);
   }
 
   private String makeAbsolute(String path) {
@@ -220,7 +176,6 @@ public class HttpFSServer {
   /**
    * Binding to handle GET requests, supported operations are
    *
-   * @param user the principal of the user making the request.
    * @param path the path for operation.
    * @param op the HttpFS operation of the request.
    * @param params the HttpFS parameters of the request.
@@ -236,21 +191,20 @@ public class HttpFSServer {
   @GET
   @Path("{path:.*}")
   @Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON})
-  public Response get(@Context Principal user,
-                      @PathParam("path") String path,
+  public Response get(@PathParam("path") String path,
                       @QueryParam(OperationParam.NAME) OperationParam op,
                       @Context Parameters params)
     throws IOException, FileSystemAccessException {
+    UserGroupInformation user = HttpUserGroupInformation.get();
     Response response;
     path = makeAbsolute(path);
     MDC.put(HttpFSFileSystem.OP_PARAM, op.value().name());
-    String doAs = params.get(DoAsParam.NAME, DoAsParam.class);
     switch (op.value()) {
       case OPEN: {
         //Invoking the command directly using an unmanaged FileSystem that is
         // released by the FileSystemReleaseFilter
         FSOperations.FSOpen command = new FSOperations.FSOpen(path);
-        FileSystem fs = createFileSystem(user, doAs);
+        FileSystem fs = createFileSystem(user);
         InputStream is = command.execute(fs);
         Long offset = params.get(OffsetParam.NAME, OffsetParam.class);
         Long len = params.get(LenParam.NAME, LenParam.class);
@@ -264,7 +218,7 @@ public class HttpFSServer {
       case GETFILESTATUS: {
         FSOperations.FSFileStatus command =
           new FSOperations.FSFileStatus(path);
-        Map json = fsExecute(user, doAs, command);
+        Map json = fsExecute(user, command);
         AUDIT_LOG.info("[{}]", path);
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
@@ -273,7 +227,7 @@ public class HttpFSServer {
         String filter = params.get(FilterParam.NAME, FilterParam.class);
         FSOperations.FSListStatus command = new FSOperations.FSListStatus(
           path, filter);
-        Map json = fsExecute(user, doAs, command);
+        Map json = fsExecute(user, command);
         AUDIT_LOG.info("[{}] filter [{}]", path,
                        (filter != null) ? filter : "-");
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
@@ -282,7 +236,7 @@ public class HttpFSServer {
       case GETHOMEDIRECTORY: {
         enforceRootPath(op.value(), path);
         FSOperations.FSHomeDir command = new FSOperations.FSHomeDir();
-        JSONObject json = fsExecute(user, doAs, command);
+        JSONObject json = fsExecute(user, command);
         AUDIT_LOG.info("");
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
@@ -290,7 +244,7 @@ public class HttpFSServer {
       case INSTRUMENTATION: {
         enforceRootPath(op.value(), path);
         Groups groups = HttpFSServerWebApp.get().get(Groups.class);
-        List<String> userGroups = groups.getGroups(user.getName());
+        List<String> userGroups = groups.getGroups(user.getShortUserName());
         if (!userGroups.contains(HttpFSServerWebApp.get().getAdminGroup())) {
           throw new AccessControlException(
             "User not in HttpFSServer admin group");
@@ -304,7 +258,7 @@ public class HttpFSServer {
       case GETCONTENTSUMMARY: {
         FSOperations.FSContentSummary command =
           new FSOperations.FSContentSummary(path);
-        Map json = fsExecute(user, doAs, command);
+        Map json = fsExecute(user, command);
         AUDIT_LOG.info("[{}]", path);
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
@@ -312,7 +266,7 @@ public class HttpFSServer {
       case GETFILECHECKSUM: {
         FSOperations.FSFileChecksum command =
           new FSOperations.FSFileChecksum(path);
-        Map json = fsExecute(user, doAs, command);
+        Map json = fsExecute(user, command);
         AUDIT_LOG.info("[{}]", path);
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
@@ -324,7 +278,7 @@ public class HttpFSServer {
       case GETACLSTATUS: {
         FSOperations.FSAclStatus command =
                 new FSOperations.FSAclStatus(path);
-        Map json = fsExecute(user, doAs, command);
+        Map json = fsExecute(user, command);
         AUDIT_LOG.info("ACL status for [{}]", path);
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
@@ -337,7 +291,7 @@ public class HttpFSServer {
         FSOperations.FSGetXAttrs command = new FSOperations.FSGetXAttrs(path, 
             xattrNames, encoding);
         @SuppressWarnings("rawtypes")
-        Map json = fsExecute(user, doAs, command);
+        Map json = fsExecute(user, command);
         AUDIT_LOG.info("XAttrs for [{}]", path);
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
@@ -345,7 +299,7 @@ public class HttpFSServer {
       case LISTXATTRS: {
         FSOperations.FSListXAttrs command = new FSOperations.FSListXAttrs(path);
         @SuppressWarnings("rawtypes")
-        Map json = fsExecute(user, doAs, command);
+        Map json = fsExecute(user, command);
         AUDIT_LOG.info("XAttr names for [{}]", path);
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
@@ -363,7 +317,6 @@ public class HttpFSServer {
   /**
    * Binding to handle DELETE requests.
    *
-   * @param user the principal of the user making the request.
    * @param path the path for operation.
    * @param op the HttpFS operation of the request.
    * @param params the HttpFS parameters of the request.
@@ -379,15 +332,14 @@ public class HttpFSServer {
   @DELETE
   @Path("{path:.*}")
   @Produces(MediaType.APPLICATION_JSON)
-  public Response delete(@Context Principal user,
-                      @PathParam("path") String path,
-                      @QueryParam(OperationParam.NAME) OperationParam op,
-                      @Context Parameters params)
+  public Response delete(@PathParam("path") String path,
+                         @QueryParam(OperationParam.NAME) OperationParam op,
+                         @Context Parameters params)
     throws IOException, FileSystemAccessException {
+    UserGroupInformation user = HttpUserGroupInformation.get();
     Response response;
     path = makeAbsolute(path);
     MDC.put(HttpFSFileSystem.OP_PARAM, op.value().name());
-    String doAs = params.get(DoAsParam.NAME, DoAsParam.class);
     switch (op.value()) {
       case DELETE: {
         Boolean recursive =
@@ -395,7 +347,7 @@ public class HttpFSServer {
         AUDIT_LOG.info("[{}] recursive [{}]", path, recursive);
         FSOperations.FSDelete command =
           new FSOperations.FSDelete(path, recursive);
-        JSONObject json = fsExecute(user, doAs, command);
+        JSONObject json = fsExecute(user, command);
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
       }
@@ -412,7 +364,6 @@ public class HttpFSServer {
    * Binding to handle POST requests.
    *
    * @param is the inputstream for the request payload.
-   * @param user the principal of the user making the request.
    * @param uriInfo the of the request.
    * @param path the path for operation.
    * @param op the HttpFS operation of the request.
@@ -431,18 +382,17 @@ public class HttpFSServer {
   @Consumes({"*/*"})
   @Produces({MediaType.APPLICATION_JSON})
   public Response post(InputStream is,
-                       @Context Principal user,
                        @Context UriInfo uriInfo,
                        @PathParam("path") String path,
                        @QueryParam(OperationParam.NAME) OperationParam op,
                        @Context Parameters params)
     throws IOException, FileSystemAccessException {
+    UserGroupInformation user = HttpUserGroupInformation.get();
     Response response;
     path = makeAbsolute(path);
     MDC.put(HttpFSFileSystem.OP_PARAM, op.value().name());
     switch (op.value()) {
       case APPEND: {
-        String doAs = params.get(DoAsParam.NAME, DoAsParam.class);
         Boolean hasData = params.get(DataParam.NAME, DataParam.class);
         if (!hasData) {
           response = Response.temporaryRedirect(
@@ -451,7 +401,7 @@ public class HttpFSServer {
         } else {
           FSOperations.FSAppend command =
             new FSOperations.FSAppend(is, path);
-          fsExecute(user, doAs, command);
+          fsExecute(user, command);
           AUDIT_LOG.info("[{}]", path);
           response = Response.ok().type(MediaType.APPLICATION_JSON).build();
         }
@@ -463,7 +413,7 @@ public class HttpFSServer {
 
         FSOperations.FSConcat command =
             new FSOperations.FSConcat(path, sources.split(","));
-        fsExecute(user, null, command);
+        fsExecute(user, command);
         AUDIT_LOG.info("[{}]", path);
         System.out.println("SENT RESPONSE");
         response = Response.ok().build();
@@ -498,7 +448,6 @@ public class HttpFSServer {
    * Binding to handle PUT requests.
    *
    * @param is the inputstream for the request payload.
-   * @param user the principal of the user making the request.
    * @param uriInfo the of the request.
    * @param path the path for operation.
    * @param op the HttpFS operation of the request.
@@ -517,16 +466,15 @@ public class HttpFSServer {
   @Consumes({"*/*"})
   @Produces({MediaType.APPLICATION_JSON})
   public Response put(InputStream is,
-                       @Context Principal user,
                        @Context UriInfo uriInfo,
                        @PathParam("path") String path,
                        @QueryParam(OperationParam.NAME) OperationParam op,
                        @Context Parameters params)
     throws IOException, FileSystemAccessException {
+    UserGroupInformation user = HttpUserGroupInformation.get();
     Response response;
     path = makeAbsolute(path);
     MDC.put(HttpFSFileSystem.OP_PARAM, op.value().name());
-    String doAs = params.get(DoAsParam.NAME, DoAsParam.class);
     switch (op.value()) {
       case CREATE: {
         Boolean hasData = params.get(DataParam.NAME, DataParam.class);
@@ -546,7 +494,7 @@ public class HttpFSServer {
           FSOperations.FSCreate command =
             new FSOperations.FSCreate(is, path, permission, override,
                                       replication, blockSize);
-          fsExecute(user, doAs, command);
+          fsExecute(user, command);
           AUDIT_LOG.info(
             "[{}] permission [{}] override [{}] replication [{}] blockSize [{}]",
             new Object[]{path, permission, override, replication, blockSize});
@@ -564,7 +512,7 @@ public class HttpFSServer {
 
         FSOperations.FSSetXAttr command = new FSOperations.FSSetXAttr(
             path, xattrName, xattrValue, flag);
-        fsExecute(user, doAs, command);
+        fsExecute(user, command);
         AUDIT_LOG.info("[{}] to xAttr [{}]", path, xattrName);
         response = Response.ok().build();
         break;
@@ -573,7 +521,7 @@ public class HttpFSServer {
         String xattrName = params.get(XAttrNameParam.NAME, XAttrNameParam.class);
         FSOperations.FSRemoveXAttr command = new FSOperations.FSRemoveXAttr(
             path, xattrName);
-        fsExecute(user, doAs, command);
+        fsExecute(user, command);
         AUDIT_LOG.info("[{}] removed xAttr [{}]", path, xattrName);
         response = Response.ok().build();
         break;
@@ -583,7 +531,7 @@ public class HttpFSServer {
                                        PermissionParam.class);
         FSOperations.FSMkdirs command =
           new FSOperations.FSMkdirs(path, permission);
-        JSONObject json = fsExecute(user, doAs, command);
+        JSONObject json = fsExecute(user, command);
         AUDIT_LOG.info("[{}] permission [{}]", path, permission);
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
@@ -592,7 +540,7 @@ public class HttpFSServer {
         String toPath = params.get(DestinationParam.NAME, DestinationParam.class);
         FSOperations.FSRename command =
           new FSOperations.FSRename(path, toPath);
-        JSONObject json = fsExecute(user, doAs, command);
+        JSONObject json = fsExecute(user, command);
         AUDIT_LOG.info("[{}] to [{}]", path, toPath);
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
@@ -602,7 +550,7 @@ public class HttpFSServer {
         String group = params.get(GroupParam.NAME, GroupParam.class);
         FSOperations.FSSetOwner command =
           new FSOperations.FSSetOwner(path, owner, group);
-        fsExecute(user, doAs, command);
+        fsExecute(user, command);
         AUDIT_LOG.info("[{}] to (O/G)[{}]", path, owner + ":" + group);
         response = Response.ok().build();
         break;
@@ -612,7 +560,7 @@ public class HttpFSServer {
                                       PermissionParam.class);
         FSOperations.FSSetPermission command =
           new FSOperations.FSSetPermission(path, permission);
-        fsExecute(user, doAs, command);
+        fsExecute(user, command);
         AUDIT_LOG.info("[{}] to [{}]", path, permission);
         response = Response.ok().build();
         break;
@@ -622,7 +570,7 @@ public class HttpFSServer {
                                        ReplicationParam.class);
         FSOperations.FSSetReplication command =
           new FSOperations.FSSetReplication(path, replication);
-        JSONObject json = fsExecute(user, doAs, command);
+        JSONObject json = fsExecute(user, command);
         AUDIT_LOG.info("[{}] to [{}]", path, replication);
         response = Response.ok(json).build();
         break;
@@ -634,7 +582,7 @@ public class HttpFSServer {
                                      AccessTimeParam.class);
         FSOperations.FSSetTimes command =
           new FSOperations.FSSetTimes(path, modifiedTime, accessTime);
-        fsExecute(user, doAs, command);
+        fsExecute(user, command);
         AUDIT_LOG.info("[{}] to (M/A)[{}]", path,
                        modifiedTime + ":" + accessTime);
         response = Response.ok().build();
@@ -645,7 +593,7 @@ public class HttpFSServer {
                 AclPermissionParam.class);
         FSOperations.FSSetAcl command =
                 new FSOperations.FSSetAcl(path, aclSpec);
-        fsExecute(user, doAs, command);
+        fsExecute(user, command);
         AUDIT_LOG.info("[{}] to acl [{}]", path, aclSpec);
         response = Response.ok().build();
         break;
@@ -653,7 +601,7 @@ public class HttpFSServer {
       case REMOVEACL: {
         FSOperations.FSRemoveAcl command =
                 new FSOperations.FSRemoveAcl(path);
-        fsExecute(user, doAs, command);
+        fsExecute(user, command);
         AUDIT_LOG.info("[{}] removed acl", path);
         response = Response.ok().build();
         break;
@@ -663,7 +611,7 @@ public class HttpFSServer {
                 AclPermissionParam.class);
         FSOperations.FSModifyAclEntries command =
                 new FSOperations.FSModifyAclEntries(path, aclSpec);
-        fsExecute(user, doAs, command);
+        fsExecute(user, command);
         AUDIT_LOG.info("[{}] modify acl entry with [{}]", path, aclSpec);
         response = Response.ok().build();
         break;
@@ -673,7 +621,7 @@ public class HttpFSServer {
                 AclPermissionParam.class);
         FSOperations.FSRemoveAclEntries command =
                 new FSOperations.FSRemoveAclEntries(path, aclSpec);
-        fsExecute(user, doAs, command);
+        fsExecute(user, command);
         AUDIT_LOG.info("[{}] remove acl entry [{}]", path, aclSpec);
         response = Response.ok().build();
         break;
@@ -681,7 +629,7 @@ public class HttpFSServer {
       case REMOVEDEFAULTACL: {
         FSOperations.FSRemoveDefaultAcl command =
                 new FSOperations.FSRemoveDefaultAcl(path);
-        fsExecute(user, doAs, command);
+        fsExecute(user, command);
         AUDIT_LOG.info("[{}] remove default acl", path);
         response = Response.ok().build();
         break;

+ 0 - 4
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java

@@ -24,7 +24,6 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.lib.server.ServerException;
 import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.lib.servlet.ServerWebApp;
-import org.apache.hadoop.lib.wsrs.UserProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -103,9 +102,6 @@ public class HttpFSServerWebApp extends ServerWebApp {
     LOG.info("Connects to Namenode [{}]",
              get().get(FileSystemAccess.class).getFileSystemConfiguration().
                get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
-    String userPattern = getConfig().get(UserProvider.USER_PATTERN_KEY, 
-      UserProvider.USER_PATTERN_DEFAULT);
-    UserProvider.setUserPattern(userPattern);
   }
 
   /**

+ 0 - 31
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/ProxyUser.java

@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.lib.service;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-import java.io.IOException;
-import java.security.AccessControlException;
-
-@InterfaceAudience.Private
-public interface ProxyUser {
-
-  public void validate(String proxyUser, String proxyHost, String doAsUser) throws IOException, AccessControlException;
-
-}

+ 0 - 179
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/security/ProxyUserService.java

@@ -1,179 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.lib.service.security;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.lib.lang.XException;
-import org.apache.hadoop.lib.server.BaseService;
-import org.apache.hadoop.lib.server.ServiceException;
-import org.apache.hadoop.lib.service.Groups;
-import org.apache.hadoop.lib.service.ProxyUser;
-import org.apache.hadoop.lib.util.Check;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.security.AccessControlException;
-import java.text.MessageFormat;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-@InterfaceAudience.Private
-public class ProxyUserService extends BaseService implements ProxyUser {
-  private static Logger LOG = LoggerFactory.getLogger(ProxyUserService.class);
-
-  @InterfaceAudience.Private
-  public static enum ERROR implements XException.ERROR {
-    PRXU01("Could not normalize host name [{0}], {1}"),
-    PRXU02("Missing [{0}] property");
-
-    private String template;
-
-    ERROR(String template) {
-      this.template = template;
-    }
-
-    @Override
-    public String getTemplate() {
-      return template;
-    }
-  }
-
-  private static final String PREFIX = "proxyuser";
-  private static final String GROUPS = ".groups";
-  private static final String HOSTS = ".hosts";
-
-  private Map<String, Set<String>> proxyUserHosts = new HashMap<String, Set<String>>();
-  private Map<String, Set<String>> proxyUserGroups = new HashMap<String, Set<String>>();
-
-  public ProxyUserService() {
-    super(PREFIX);
-  }
-
-  @Override
-  public Class getInterface() {
-    return ProxyUser.class;
-  }
-
-  @Override
-  public Class[] getServiceDependencies() {
-    return new Class[]{Groups.class};
-  }
-
-  @Override
-  protected void init() throws ServiceException {
-    for (Map.Entry<String, String> entry : getServiceConfig()) {
-      String key = entry.getKey();
-      if (key.endsWith(GROUPS)) {
-        String proxyUser = key.substring(0, key.lastIndexOf(GROUPS));
-        if (getServiceConfig().get(proxyUser + HOSTS) == null) {
-          throw new ServiceException(ERROR.PRXU02, getPrefixedName(proxyUser + HOSTS));
-        }
-        String value = entry.getValue().trim();
-        LOG.info("Loading proxyuser settings [{}]=[{}]", key, value);
-        Set<String> values = null;
-        if (!value.equals("*")) {
-          values = new HashSet<String>(Arrays.asList(value.split(",")));
-        }
-        proxyUserGroups.put(proxyUser, values);
-      }
-      if (key.endsWith(HOSTS)) {
-        String proxyUser = key.substring(0, key.lastIndexOf(HOSTS));
-        if (getServiceConfig().get(proxyUser + GROUPS) == null) {
-          throw new ServiceException(ERROR.PRXU02, getPrefixedName(proxyUser + GROUPS));
-        }
-        String value = entry.getValue().trim();
-        LOG.info("Loading proxyuser settings [{}]=[{}]", key, value);
-        Set<String> values = null;
-        if (!value.equals("*")) {
-          String[] hosts = value.split(",");
-          for (int i = 0; i < hosts.length; i++) {
-            String originalName = hosts[i];
-            try {
-              hosts[i] = normalizeHostname(originalName);
-            } catch (Exception ex) {
-              throw new ServiceException(ERROR.PRXU01, originalName, ex.getMessage(), ex);
-            }
-            LOG.info("  Hostname, original [{}], normalized [{}]", originalName, hosts[i]);
-          }
-          values = new HashSet<String>(Arrays.asList(hosts));
-        }
-        proxyUserHosts.put(proxyUser, values);
-      }
-    }
-  }
-
-  @Override
-  public void validate(String proxyUser, String proxyHost, String doAsUser) throws IOException,
-    AccessControlException {
-    Check.notEmpty(proxyUser, "proxyUser");
-    Check.notEmpty(proxyHost, "proxyHost");
-    Check.notEmpty(doAsUser, "doAsUser");
-    LOG.debug("Authorization check proxyuser [{}] host [{}] doAs [{}]",
-              new Object[]{proxyUser, proxyHost, doAsUser});
-    if (proxyUserHosts.containsKey(proxyUser)) {
-      proxyHost = normalizeHostname(proxyHost);
-      validateRequestorHost(proxyUser, proxyHost, proxyUserHosts.get(proxyUser));
-      validateGroup(proxyUser, doAsUser, proxyUserGroups.get(proxyUser));
-    } else {
-      throw new AccessControlException(MessageFormat.format("User [{0}] not defined as proxyuser", proxyUser));
-    }
-  }
-
-  private void validateRequestorHost(String proxyUser, String hostname, Set<String> validHosts)
-    throws IOException, AccessControlException {
-    if (validHosts != null) {
-      if (!validHosts.contains(hostname) && !validHosts.contains(normalizeHostname(hostname))) {
-        throw new AccessControlException(MessageFormat.format("Unauthorized host [{0}] for proxyuser [{1}]",
-                                                              hostname, proxyUser));
-      }
-    }
-  }
-
-  private void validateGroup(String proxyUser, String user, Set<String> validGroups) throws IOException,
-    AccessControlException {
-    if (validGroups != null) {
-      List<String> userGroups = getServer().get(Groups.class).getGroups(user);
-      for (String g : validGroups) {
-        if (userGroups.contains(g)) {
-          return;
-        }
-      }
-      throw new AccessControlException(
-        MessageFormat.format("Unauthorized proxyuser [{0}] for user [{1}], not in proxyuser groups",
-                             proxyUser, user));
-    }
-  }
-
-  private String normalizeHostname(String name) {
-    try {
-      InetAddress address = InetAddress.getByName(name);
-      return address.getCanonicalHostName();
-    } catch (IOException ex) {
-      throw new AccessControlException(MessageFormat.format("Could not resolve host [{0}], {1}", name,
-                                                            ex.getMessage()));
-    }
-  }
-
-}

+ 0 - 109
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/UserProvider.java

@@ -1,109 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.lib.wsrs;
-
-import com.sun.jersey.api.core.HttpContext;
-import com.sun.jersey.core.spi.component.ComponentContext;
-import com.sun.jersey.core.spi.component.ComponentScope;
-import com.sun.jersey.server.impl.inject.AbstractHttpContextInjectable;
-import com.sun.jersey.spi.inject.Injectable;
-import com.sun.jersey.spi.inject.InjectableProvider;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.slf4j.MDC;
-
-import javax.ws.rs.core.Context;
-import javax.ws.rs.ext.Provider;
-import java.lang.reflect.Type;
-import java.security.Principal;
-import java.text.MessageFormat;
-import java.util.regex.Pattern;
-
-@Provider
-@InterfaceAudience.Private
-public class UserProvider extends AbstractHttpContextInjectable<Principal> implements
-  InjectableProvider<Context, Type> {
-
-  public static final String USER_NAME_PARAM = "user.name";
-
-
-  public static final String USER_PATTERN_KEY 
-    = "httpfs.user.provider.user.pattern";
-
-  public static final String USER_PATTERN_DEFAULT 
-    = "^[A-Za-z_][A-Za-z0-9._-]*[$]?$";
-
-  private static Pattern userPattern = Pattern.compile(USER_PATTERN_DEFAULT);
-
-  public static void setUserPattern(String pattern) {
-    userPattern = Pattern.compile(pattern);
-  }
-
-  public static Pattern getUserPattern() {
-    return userPattern;
-  }
-
-  static class UserParam extends StringParam {
-
-    public UserParam(String user) {
-      super(USER_NAME_PARAM, user, getUserPattern());
-    }
-
-    @Override
-    public String parseParam(String str) {
-      if (str != null) {
-        int len = str.length();
-        if (len < 1) {
-          throw new IllegalArgumentException(MessageFormat.format(
-            "Parameter [{0}], it's length must be at least 1", getName()));
-        }
-      }
-      return super.parseParam(str);
-    }
-  }
-
-  @Override
-  public Principal getValue(HttpContext httpContext) {
-    Principal principal = httpContext.getRequest().getUserPrincipal();
-    if (principal == null) {
-      final String user = httpContext.getRequest().getQueryParameters().getFirst(USER_NAME_PARAM);
-      if (user != null) {
-        principal = new Principal() {
-          @Override
-          public String getName() {
-            return new UserParam(user).value();
-          }
-        };
-      }
-    }
-    if (principal != null) {
-      MDC.put("user", principal.getName());
-    }
-    return principal;
-  }
-
-  @Override
-  public ComponentScope getScope() {
-    return ComponentScope.PerRequest;
-  }
-
-  @Override
-  public Injectable getInjectable(ComponentContext componentContext, Context context, Type type) {
-    return (type.equals(Principal.class)) ? this : null;
-  }
-}

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml

@@ -34,7 +34,6 @@
       org.apache.hadoop.lib.service.instrumentation.InstrumentationService,
       org.apache.hadoop.lib.service.scheduler.SchedulerService,
       org.apache.hadoop.lib.service.security.GroupsService,
-      org.apache.hadoop.lib.service.security.ProxyUserService,
       org.apache.hadoop.lib.service.hadoop.FileSystemAccessService
     </value>
     <description>
@@ -118,6 +117,10 @@
   </property>
 
   <!-- HttpFSServer proxy user Configuration -->
+<!--
+
+  The following 2 properties within this comment are provided as an
+  example to facilitate configuring HttpFS proxyusers.
 
   <property>
     <name>httpfs.proxyuser.#USER#.hosts</name>
@@ -152,6 +155,7 @@
       in the property name.
     </description>
   </property>
+-->
 
   <!-- HttpFS Delegation Token configuration -->
 

+ 0 - 226
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/TestProxyUserService.java

@@ -1,226 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.lib.service.security;
-
-import static org.junit.Assert.assertNotNull;
-
-import java.security.AccessControlException;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.lib.server.Server;
-import org.apache.hadoop.lib.server.ServiceException;
-import org.apache.hadoop.lib.service.Groups;
-import org.apache.hadoop.lib.service.ProxyUser;
-import org.apache.hadoop.test.HTestCase;
-import org.apache.hadoop.test.TestDir;
-import org.apache.hadoop.test.TestDirHelper;
-import org.apache.hadoop.test.TestException;
-import org.apache.hadoop.util.StringUtils;
-import org.junit.Test;
-
-public class TestProxyUserService extends HTestCase {
-
-  @Test
-  @TestDir
-  public void service() throws Exception {
-    String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    Configuration conf = new Configuration(false);
-    conf.set("server.services", StringUtils.join(",", Arrays.asList(GroupsService.class.getName(),
-                                                                    ProxyUserService.class.getName())));
-    Server server = new Server("server", dir, dir, dir, dir, conf);
-    server.init();
-    ProxyUser proxyUser = server.get(ProxyUser.class);
-    assertNotNull(proxyUser);
-    server.destroy();
-  }
-
-  @Test
-  @TestException(exception = ServiceException.class, msgRegExp = "PRXU02.*")
-  @TestDir
-  public void wrongConfigGroups() throws Exception {
-    String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    Configuration conf = new Configuration(false);
-    conf.set("server.services", StringUtils.join(",", Arrays.asList(GroupsService.class.getName(),
-                                                                    ProxyUserService.class.getName())));
-    conf.set("server.proxyuser.foo.hosts", "*");
-    Server server = new Server("server", dir, dir, dir, dir, conf);
-    server.init();
-  }
-
-  @Test
-  @TestException(exception = ServiceException.class, msgRegExp = "PRXU01.*")
-  @TestDir
-  public void wrongHost() throws Exception {
-    String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    Configuration conf = new Configuration(false);
-    conf.set("server.services", StringUtils.join(",", Arrays.asList(GroupsService.class.getName(),
-                                                                    ProxyUserService.class.getName())));
-    conf.set("server.proxyuser.foo.hosts", "otherhost");
-    conf.set("server.proxyuser.foo.groups", "*");
-    Server server = new Server("server", dir, dir, dir, dir, conf);
-    server.init();
-  }
-
-  @Test
-  @TestException(exception = ServiceException.class, msgRegExp = "PRXU02.*")
-  @TestDir
-  public void wrongConfigHosts() throws Exception {
-    String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    Configuration conf = new Configuration(false);
-    conf.set("server.services", StringUtils.join(",", Arrays.asList(GroupsService.class.getName(),
-                                                                    ProxyUserService.class.getName())));
-    conf.set("server.proxyuser.foo.groups", "*");
-    Server server = new Server("server", dir, dir, dir, dir, conf);
-    server.init();
-  }
-
-  @Test
-  @TestDir
-  public void validateAnyHostAnyUser() throws Exception {
-    String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    Configuration conf = new Configuration(false);
-    conf.set("server.services", StringUtils.join(",", Arrays.asList(GroupsService.class.getName(),
-                                                                    ProxyUserService.class.getName())));
-    conf.set("server.proxyuser.foo.hosts", "*");
-    conf.set("server.proxyuser.foo.groups", "*");
-    Server server = new Server("server", dir, dir, dir, dir, conf);
-    server.init();
-    ProxyUser proxyUser = server.get(ProxyUser.class);
-    assertNotNull(proxyUser);
-    proxyUser.validate("foo", "localhost", "bar");
-    server.destroy();
-  }
-
-  @Test(expected = AccessControlException.class)
-  @TestDir
-  public void invalidProxyUser() throws Exception {
-    String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    Configuration conf = new Configuration(false);
-    conf.set("server.services", StringUtils.join(",", Arrays.asList(GroupsService.class.getName(),
-                                                                    ProxyUserService.class.getName())));
-    conf.set("server.proxyuser.foo.hosts", "*");
-    conf.set("server.proxyuser.foo.groups", "*");
-    Server server = new Server("server", dir, dir, dir, dir, conf);
-    server.init();
-    ProxyUser proxyUser = server.get(ProxyUser.class);
-    assertNotNull(proxyUser);
-    proxyUser.validate("bar", "localhost", "foo");
-    server.destroy();
-  }
-
-  @Test
-  @TestDir
-  public void validateHost() throws Exception {
-    String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    Configuration conf = new Configuration(false);
-    conf.set("server.services", StringUtils.join(",", Arrays.asList(GroupsService.class.getName(),
-                                                                    ProxyUserService.class.getName())));
-    conf.set("server.proxyuser.foo.hosts", "localhost");
-    conf.set("server.proxyuser.foo.groups", "*");
-    Server server = new Server("server", dir, dir, dir, dir, conf);
-    server.init();
-    ProxyUser proxyUser = server.get(ProxyUser.class);
-    assertNotNull(proxyUser);
-    proxyUser.validate("foo", "localhost", "bar");
-    server.destroy();
-  }
-
-  private String getGroup() throws Exception {
-    String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    Configuration conf = new Configuration(false);
-    conf.set("server.services", StringUtils.join(",", Arrays.asList(GroupsService.class.getName())));
-    Server server = new Server("server", dir, dir, dir, dir, conf);
-    server.init();
-    Groups groups = server.get(Groups.class);
-    List<String> g = groups.getGroups(System.getProperty("user.name"));
-    server.destroy();
-    return g.get(0);
-  }
-
-  @Test
-  @TestDir
-  public void validateGroup() throws Exception {
-    String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    Configuration conf = new Configuration(false);
-    conf.set("server.services", StringUtils.join(",", Arrays.asList(GroupsService.class.getName(),
-                                                                    ProxyUserService.class.getName())));
-    conf.set("server.proxyuser.foo.hosts", "*");
-    conf.set("server.proxyuser.foo.groups", getGroup());
-    Server server = new Server("server", dir, dir, dir, dir, conf);
-    server.init();
-    ProxyUser proxyUser = server.get(ProxyUser.class);
-    assertNotNull(proxyUser);
-    proxyUser.validate("foo", "localhost", System.getProperty("user.name"));
-    server.destroy();
-  }
-
-
-  @Test(expected = AccessControlException.class)
-  @TestDir
-  public void unknownHost() throws Exception {
-    String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    Configuration conf = new Configuration(false);
-    conf.set("server.services", StringUtils.join(",", Arrays.asList(GroupsService.class.getName(),
-                                                                    ProxyUserService.class.getName())));
-    conf.set("server.proxyuser.foo.hosts", "localhost");
-    conf.set("server.proxyuser.foo.groups", "*");
-    Server server = new Server("server", dir, dir, dir, dir, conf);
-    server.init();
-    ProxyUser proxyUser = server.get(ProxyUser.class);
-    assertNotNull(proxyUser);
-    proxyUser.validate("foo", "unknownhost.bar.foo", "bar");
-    server.destroy();
-  }
-
-  @Test(expected = AccessControlException.class)
-  @TestDir
-  public void invalidHost() throws Exception {
-    String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    Configuration conf = new Configuration(false);
-    conf.set("server.services", StringUtils.join(",", Arrays.asList(GroupsService.class.getName(),
-                                                                    ProxyUserService.class.getName())));
-    conf.set("server.proxyuser.foo.hosts", "localhost");
-    conf.set("server.proxyuser.foo.groups", "*");
-    Server server = new Server("server", dir, dir, dir, dir, conf);
-    server.init();
-    ProxyUser proxyUser = server.get(ProxyUser.class);
-    assertNotNull(proxyUser);
-    proxyUser.validate("foo", "www.yahoo.com", "bar");
-    server.destroy();
-  }
-
-  @Test(expected = AccessControlException.class)
-  @TestDir
-  public void invalidGroup() throws Exception {
-    String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    Configuration conf = new Configuration(false);
-    conf.set("server.services", StringUtils.join(",", Arrays.asList(GroupsService.class.getName(),
-                                                                    ProxyUserService.class.getName())));
-    conf.set("server.proxyuser.foo.hosts", "localhost");
-    conf.set("server.proxyuser.foo.groups", "nobody");
-    Server server = new Server("server", dir, dir, dir, dir, conf);
-    server.init();
-    ProxyUser proxyUser = server.get(ProxyUser.class);
-    assertNotNull(proxyUser);
-    proxyUser.validate("foo", "localhost", System.getProperty("user.name"));
-    server.destroy();
-  }
-}

+ 0 - 142
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestUserProvider.java

@@ -1,142 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.lib.wsrs;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-
-import java.security.Principal;
-
-import javax.ws.rs.core.MultivaluedMap;
-
-import org.apache.hadoop.test.TestException;
-import org.apache.hadoop.test.TestExceptionHelper;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.MethodRule;
-import org.mockito.Mockito;
-import org.slf4j.MDC;
-
-import com.sun.jersey.api.core.HttpContext;
-import com.sun.jersey.api.core.HttpRequestContext;
-import com.sun.jersey.core.spi.component.ComponentScope;
-
-public class TestUserProvider {
-
-  @Rule
-  public MethodRule exceptionHelper = new TestExceptionHelper();
-
-  @Test
-  @SuppressWarnings("unchecked")
-  public void noUser() {
-    MDC.remove("user");
-    HttpRequestContext request = Mockito.mock(HttpRequestContext.class);
-    Mockito.when(request.getUserPrincipal()).thenReturn(null);
-    MultivaluedMap map = Mockito.mock(MultivaluedMap.class);
-    Mockito.when(map.getFirst(UserProvider.USER_NAME_PARAM)).thenReturn(null);
-    Mockito.when(request.getQueryParameters()).thenReturn(map);
-    HttpContext context = Mockito.mock(HttpContext.class);
-    Mockito.when(context.getRequest()).thenReturn(request);
-    UserProvider up = new UserProvider();
-    assertNull(up.getValue(context));
-    assertNull(MDC.get("user"));
-  }
-
-  @Test
-  @SuppressWarnings("unchecked")
-  public void queryStringUser() {
-    MDC.remove("user");
-    HttpRequestContext request = Mockito.mock(HttpRequestContext.class);
-    Mockito.when(request.getUserPrincipal()).thenReturn(null);
-    MultivaluedMap map = Mockito.mock(MultivaluedMap.class);
-    Mockito.when(map.getFirst(UserProvider.USER_NAME_PARAM)).thenReturn("foo");
-    Mockito.when(request.getQueryParameters()).thenReturn(map);
-    HttpContext context = Mockito.mock(HttpContext.class);
-    Mockito.when(context.getRequest()).thenReturn(request);
-    UserProvider up = new UserProvider();
-    assertEquals(up.getValue(context).getName(), "foo");
-    assertEquals(MDC.get("user"), "foo");
-  }
-
-  @Test
-  @SuppressWarnings("unchecked")
-  public void principalUser() {
-    MDC.remove("user");
-    HttpRequestContext request = Mockito.mock(HttpRequestContext.class);
-    Mockito.when(request.getUserPrincipal()).thenReturn(new Principal() {
-      @Override
-      public String getName() {
-        return "bar";
-      }
-    });
-    HttpContext context = Mockito.mock(HttpContext.class);
-    Mockito.when(context.getRequest()).thenReturn(request);
-    UserProvider up = new UserProvider();
-    assertEquals(up.getValue(context).getName(), "bar");
-    assertEquals(MDC.get("user"), "bar");
-  }
-
-  @Test
-  public void getters() {
-    UserProvider up = new UserProvider();
-    assertEquals(up.getScope(), ComponentScope.PerRequest);
-    assertEquals(up.getInjectable(null, null, Principal.class), up);
-    assertNull(up.getInjectable(null, null, String.class));
-  }
-
-  @Test
-  @TestException(exception = IllegalArgumentException.class)
-  public void userNameEmpty() {
-    new UserProvider.UserParam("");
-  }
-
-  @Test
-  @TestException(exception = IllegalArgumentException.class)
-  public void userNameInvalidStart() {
-    new UserProvider.UserParam("1x");
-  }
-
-  @Test
-  @TestException(exception = IllegalArgumentException.class)
-  public void userNameInvalidDollarSign() {
-    new UserProvider.UserParam("1$x");
-  }
-
-  @Test
-  public void userNameMinLength() {
-    new UserProvider.UserParam("a");
-  }
-
-  @Test
-  public void userNameValidDollarSign() {
-    new UserProvider.UserParam("a$");
-  }
-
-  @Test
-  public void customUserPattern() {
-    try {
-      UserProvider.setUserPattern("1");
-      new UserProvider.UserParam("1");      
-    } finally {
-      UserProvider.setUserPattern(UserProvider.USER_PATTERN_DEFAULT);
-    }
-  }
-
-}

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -399,6 +399,9 @@ Release 2.6.0 - UNRELEASED
     HDFS-6567. Normalize the order of public final in HdfsFileStatus.
     (Tassapol Athiapinya via wheat9)
 
+    HDFS-6849. Replace HttpFS custom proxyuser handling with common 
+    implementation. (tucu)
+
   OPTIMIZATIONS
 
     HDFS-6690. Deduplicate xattr names in memory. (wang)