浏览代码

HDFS-2539. Support doAs and GETHOMEDIRECTORY in webhdfs.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1200731 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 13 年之前
父节点
当前提交
09a156fcce
共有 14 个文件被更改,包括 499 次插入94 次删除
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 46 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
  3. 91 49
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  4. 73 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/AuthFilter.java
  5. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/ParamFilter.java
  6. 26 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  7. 41 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DoAsParam.java
  8. 20 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExceptionHandler.java
  9. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
  10. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java
  11. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserProvider.java
  12. 78 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java
  13. 26 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
  14. 90 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/WebHdfsTestUtil.java

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

@@ -442,6 +442,9 @@ Release 0.23.0 - 2011-11-01
     HDFS-2385. Support renew and cancel delegation tokens in webhdfs.
     HDFS-2385. Support renew and cancel delegation tokens in webhdfs.
     (szetszwo)
     (szetszwo)
 
 
+    HDFS-2539. Support doAs and GETHOMEDIRECTORY in webhdfs.
+    (szetszwo)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-1875. MiniDFSCluster hard-codes dfs.datanode.address to localhost
     HDFS-1875. MiniDFSCluster hard-codes dfs.datanode.address to localhost

+ 46 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java

@@ -56,6 +56,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeHttpServer;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeHttpServer;
 import org.apache.hadoop.hdfs.web.resources.DelegationParam;
 import org.apache.hadoop.hdfs.web.resources.DelegationParam;
+import org.apache.hadoop.hdfs.web.resources.DoAsParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.http.HtmlQuoting;
 import org.apache.hadoop.http.HtmlQuoting;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -64,6 +65,8 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.hadoop.security.authentication.util.KerberosName;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
 
 
@@ -535,9 +538,10 @@ public class JspHelper {
       final boolean tryUgiParameter) throws IOException {
       final boolean tryUgiParameter) throws IOException {
     final UserGroupInformation ugi;
     final UserGroupInformation ugi;
     final String usernameFromQuery = getUsernameFromQuery(request, tryUgiParameter);
     final String usernameFromQuery = getUsernameFromQuery(request, tryUgiParameter);
+    final String doAsUserFromQuery = request.getParameter(DoAsParam.NAME);
 
 
     if(UserGroupInformation.isSecurityEnabled()) {
     if(UserGroupInformation.isSecurityEnabled()) {
-      final String user = request.getRemoteUser();
+      final String remoteUser = request.getRemoteUser();
       String tokenString = request.getParameter(DELEGATION_PARAMETER_NAME);
       String tokenString = request.getParameter(DELEGATION_PARAMETER_NAME);
       if (tokenString != null) {
       if (tokenString != null) {
         Token<DelegationTokenIdentifier> token = 
         Token<DelegationTokenIdentifier> token = 
@@ -561,26 +565,36 @@ public class JspHelper {
           }
           }
         }
         }
         ugi = id.getUser();
         ugi = id.getUser();
-        checkUsername(ugi.getShortUserName(), usernameFromQuery);
-        checkUsername(ugi.getShortUserName(), user);
+        if (ugi.getRealUser() == null) {
+          //non-proxy case
+          checkUsername(ugi.getShortUserName(), usernameFromQuery);
+          checkUsername(null, doAsUserFromQuery);
+        } else {
+          //proxy case
+          checkUsername(ugi.getRealUser().getShortUserName(), usernameFromQuery);
+          checkUsername(ugi.getShortUserName(), doAsUserFromQuery);
+          ProxyUsers.authorize(ugi, request.getRemoteAddr(), conf);
+        }
         ugi.addToken(token);
         ugi.addToken(token);
         ugi.setAuthenticationMethod(AuthenticationMethod.TOKEN);
         ugi.setAuthenticationMethod(AuthenticationMethod.TOKEN);
       } else {
       } else {
-        if(user == null) {
+        if(remoteUser == null) {
           throw new IOException("Security enabled but user not " +
           throw new IOException("Security enabled but user not " +
                                 "authenticated by filter");
                                 "authenticated by filter");
         }
         }
-        ugi = UserGroupInformation.createRemoteUser(user);
-        checkUsername(ugi.getShortUserName(), usernameFromQuery);
+        final UserGroupInformation realUgi = UserGroupInformation.createRemoteUser(remoteUser);
+        checkUsername(realUgi.getShortUserName(), usernameFromQuery);
         // This is not necessarily true, could have been auth'ed by user-facing
         // This is not necessarily true, could have been auth'ed by user-facing
         // filter
         // filter
-        ugi.setAuthenticationMethod(secureAuthMethod);
+        realUgi.setAuthenticationMethod(secureAuthMethod);
+        ugi = initUGI(realUgi, doAsUserFromQuery, request, true, conf);
       }
       }
     } else { // Security's not on, pull from url
     } else { // Security's not on, pull from url
-      ugi = usernameFromQuery == null?
+      final UserGroupInformation realUgi = usernameFromQuery == null?
           getDefaultWebUser(conf) // not specified in request
           getDefaultWebUser(conf) // not specified in request
           : UserGroupInformation.createRemoteUser(usernameFromQuery);
           : UserGroupInformation.createRemoteUser(usernameFromQuery);
-      ugi.setAuthenticationMethod(AuthenticationMethod.SIMPLE);
+      realUgi.setAuthenticationMethod(AuthenticationMethod.SIMPLE);
+      ugi = initUGI(realUgi, doAsUserFromQuery, request, false, conf);
     }
     }
     
     
     if(LOG.isDebugEnabled())
     if(LOG.isDebugEnabled())
@@ -588,12 +602,34 @@ public class JspHelper {
     return ugi;
     return ugi;
   }
   }
 
 
+  private static UserGroupInformation initUGI(final UserGroupInformation realUgi,
+      final String doAsUserFromQuery, final HttpServletRequest request,
+      final boolean isSecurityEnabled, final Configuration conf
+      ) throws AuthorizationException {
+    final UserGroupInformation ugi;
+    if (doAsUserFromQuery == null) {
+      //non-proxy case
+      ugi = realUgi;
+    } else {
+      //proxy case
+      ugi = UserGroupInformation.createProxyUser(doAsUserFromQuery, realUgi);
+      ugi.setAuthenticationMethod(
+          isSecurityEnabled? AuthenticationMethod.PROXY: AuthenticationMethod.SIMPLE);
+      ProxyUsers.authorize(ugi, request.getRemoteAddr(), conf);
+    }
+    return ugi;
+  }
+
   /**
   /**
    * Expected user name should be a short name.
    * Expected user name should be a short name.
    */
    */
   private static void checkUsername(final String expected, final String name
   private static void checkUsername(final String expected, final String name
       ) throws IOException {
       ) throws IOException {
-    if (name == null) {
+    if (expected == null && name != null) {
+      throw new IOException("Usernames not matched: expecting null but name="
+          + name);
+    }
+    if (name == null) { //name is optional, null is okay
       return;
       return;
     }
     }
     KerberosName u = new KerberosName(name);
     KerberosName u = new KerberosName(name);

+ 91 - 49
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -69,6 +69,7 @@ import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
 import org.apache.hadoop.hdfs.web.resources.DelegationParam;
 import org.apache.hadoop.hdfs.web.resources.DelegationParam;
 import org.apache.hadoop.hdfs.web.resources.DeleteOpParam;
 import org.apache.hadoop.hdfs.web.resources.DeleteOpParam;
 import org.apache.hadoop.hdfs.web.resources.DestinationParam;
 import org.apache.hadoop.hdfs.web.resources.DestinationParam;
+import org.apache.hadoop.hdfs.web.resources.DoAsParam;
 import org.apache.hadoop.hdfs.web.resources.GetOpParam;
 import org.apache.hadoop.hdfs.web.resources.GetOpParam;
 import org.apache.hadoop.hdfs.web.resources.GroupParam;
 import org.apache.hadoop.hdfs.web.resources.GroupParam;
 import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
 import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
@@ -116,6 +117,21 @@ public class NamenodeWebHdfsMethods {
   private @Context HttpServletRequest request;
   private @Context HttpServletRequest request;
   private @Context HttpServletResponse response;
   private @Context HttpServletResponse response;
 
 
+  private void init(final UserGroupInformation ugi,
+      final DelegationParam delegation,
+      final UserParam username, final DoAsParam doAsUser,
+      final UriFsPathParam path, final HttpOpParam<?> op,
+      final Param<?, ?>... parameters) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("HTTP " + op.getValue().getType() + ": " + op + ", " + path
+          + ", ugi=" + ugi + ", " + username + ", " + doAsUser
+          + Param.toSortedString(", ", parameters));
+    }
+
+    //clear content type
+    response.setContentType(null);
+  }
+
   private static DatanodeInfo chooseDatanode(final NameNode namenode,
   private static DatanodeInfo chooseDatanode(final NameNode namenode,
       final String path, final HttpOpParam.Op op, final long openOffset,
       final String path, final HttpOpParam.Op op, final long openOffset,
       Configuration conf) throws IOException {
       Configuration conf) throws IOException {
@@ -161,6 +177,7 @@ public class NamenodeWebHdfsMethods {
 
 
   private URI redirectURI(final NameNode namenode,
   private URI redirectURI(final NameNode namenode,
       final UserGroupInformation ugi, final DelegationParam delegation,
       final UserGroupInformation ugi, final DelegationParam delegation,
+      final UserParam username, final DoAsParam doAsUser,
       final String path, final HttpOpParam.Op op, final long openOffset,
       final String path, final HttpOpParam.Op op, final long openOffset,
       final Param<?, ?>... parameters) throws URISyntaxException, IOException {
       final Param<?, ?>... parameters) throws URISyntaxException, IOException {
     final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
     final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
@@ -169,7 +186,7 @@ public class NamenodeWebHdfsMethods {
     final String delegationQuery;
     final String delegationQuery;
     if (!UserGroupInformation.isSecurityEnabled()) {
     if (!UserGroupInformation.isSecurityEnabled()) {
       //security disabled
       //security disabled
-      delegationQuery = "";
+      delegationQuery = Param.toSortedString("&", doAsUser, username);
     } else if (delegation.getValue() != null) {
     } else if (delegation.getValue() != null) {
       //client has provided a token
       //client has provided a token
       delegationQuery = "&" + delegation;
       delegationQuery = "&" + delegation;
@@ -179,8 +196,7 @@ public class NamenodeWebHdfsMethods {
           namenode, ugi, request.getUserPrincipal().getName());
           namenode, ugi, request.getUserPrincipal().getName());
       delegationQuery = "&" + new DelegationParam(t.encodeToUrlString());
       delegationQuery = "&" + new DelegationParam(t.encodeToUrlString());
     }
     }
-    final String query = op.toQueryString()
-        + '&' + new UserParam(ugi) + delegationQuery
+    final String query = op.toQueryString() + delegationQuery
         + Param.toSortedString("&", parameters);
         + Param.toSortedString("&", parameters);
     final String uripath = WebHdfsFileSystem.PATH_PREFIX + path;
     final String uripath = WebHdfsFileSystem.PATH_PREFIX + path;
 
 
@@ -201,6 +217,10 @@ public class NamenodeWebHdfsMethods {
       @Context final UserGroupInformation ugi,
       @Context final UserGroupInformation ugi,
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
           final DelegationParam delegation,
           final DelegationParam delegation,
+      @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
+          final UserParam username,
+      @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
+          final DoAsParam doAsUser,
       @QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT)
       @QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT)
           final PutOpParam op,
           final PutOpParam op,
       @QueryParam(DestinationParam.NAME) @DefaultValue(DestinationParam.DEFAULT)
       @QueryParam(DestinationParam.NAME) @DefaultValue(DestinationParam.DEFAULT)
@@ -225,12 +245,13 @@ public class NamenodeWebHdfsMethods {
           final AccessTimeParam accessTime,
           final AccessTimeParam accessTime,
       @QueryParam(RenameOptionSetParam.NAME) @DefaultValue(RenameOptionSetParam.DEFAULT)
       @QueryParam(RenameOptionSetParam.NAME) @DefaultValue(RenameOptionSetParam.DEFAULT)
           final RenameOptionSetParam renameOptions,
           final RenameOptionSetParam renameOptions,
-      @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT) 
+      @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT)
           final TokenArgumentParam delegationTokenArgument
           final TokenArgumentParam delegationTokenArgument
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
-    return put(ugi, delegation, ROOT, op, destination, owner, group,
-        permission, overwrite, bufferSize, replication, blockSize,
-        modificationTime, accessTime, renameOptions, delegationTokenArgument);
+    return put(ugi, delegation, username, doAsUser, ROOT, op, destination,
+        owner, group, permission, overwrite, bufferSize, replication,
+        blockSize, modificationTime, accessTime, renameOptions,
+        delegationTokenArgument);
   }
   }
 
 
   /** Handle HTTP PUT request. */
   /** Handle HTTP PUT request. */
@@ -242,6 +263,10 @@ public class NamenodeWebHdfsMethods {
       @Context final UserGroupInformation ugi,
       @Context final UserGroupInformation ugi,
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
           final DelegationParam delegation,
           final DelegationParam delegation,
+      @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
+          final UserParam username,
+      @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
+          final DoAsParam doAsUser,
       @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
       @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
       @QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT)
       @QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT)
           final PutOpParam op,
           final PutOpParam op,
@@ -267,19 +292,13 @@ public class NamenodeWebHdfsMethods {
           final AccessTimeParam accessTime,
           final AccessTimeParam accessTime,
       @QueryParam(RenameOptionSetParam.NAME) @DefaultValue(RenameOptionSetParam.DEFAULT)
       @QueryParam(RenameOptionSetParam.NAME) @DefaultValue(RenameOptionSetParam.DEFAULT)
           final RenameOptionSetParam renameOptions,
           final RenameOptionSetParam renameOptions,
-      @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT) 
+      @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT)
           final TokenArgumentParam delegationTokenArgument
           final TokenArgumentParam delegationTokenArgument
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
 
 
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(op + ": " + path + ", ugi=" + ugi
-          + Param.toSortedString(", ", destination, owner, group, permission,
-              overwrite, bufferSize, replication, blockSize,
-              modificationTime, accessTime, renameOptions));
-    }
-
-    //clear content type
-    response.setContentType(null);
+    init(ugi, delegation, username, doAsUser, path, op, destination, owner,
+        group, permission, overwrite, bufferSize, replication, blockSize,
+        modificationTime, accessTime, renameOptions, delegationTokenArgument);
 
 
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
       @Override
@@ -295,8 +314,8 @@ public class NamenodeWebHdfsMethods {
     switch(op.getValue()) {
     switch(op.getValue()) {
     case CREATE:
     case CREATE:
     {
     {
-      final URI uri = redirectURI(namenode, ugi, delegation, fullpath,
-          op.getValue(), -1L,
+      final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser,
+          fullpath, op.getValue(), -1L,
           permission, overwrite, bufferSize, replication, blockSize);
           permission, overwrite, bufferSize, replication, blockSize);
       return Response.temporaryRedirect(uri).build();
       return Response.temporaryRedirect(uri).build();
     } 
     } 
@@ -379,12 +398,16 @@ public class NamenodeWebHdfsMethods {
       @Context final UserGroupInformation ugi,
       @Context final UserGroupInformation ugi,
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
           final DelegationParam delegation,
           final DelegationParam delegation,
+      @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
+          final UserParam username,
+      @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
+          final DoAsParam doAsUser,
       @QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT)
       @QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT)
           final PostOpParam op,
           final PostOpParam op,
       @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
       @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
           final BufferSizeParam bufferSize
           final BufferSizeParam bufferSize
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
-    return post(ugi, delegation, ROOT, op, bufferSize);
+    return post(ugi, delegation, username, doAsUser, ROOT, op, bufferSize);
   }
   }
 
 
   /** Handle HTTP POST request. */
   /** Handle HTTP POST request. */
@@ -396,6 +419,10 @@ public class NamenodeWebHdfsMethods {
       @Context final UserGroupInformation ugi,
       @Context final UserGroupInformation ugi,
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
           final DelegationParam delegation,
           final DelegationParam delegation,
+      @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
+          final UserParam username,
+      @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
+          final DoAsParam doAsUser,
       @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
       @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
       @QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT)
       @QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT)
           final PostOpParam op,
           final PostOpParam op,
@@ -403,13 +430,7 @@ public class NamenodeWebHdfsMethods {
           final BufferSizeParam bufferSize
           final BufferSizeParam bufferSize
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
 
 
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(op + ": " + path + ", ugi=" + ugi
-          + Param.toSortedString(", ", bufferSize));
-    }
-
-    //clear content type
-    response.setContentType(null);
+    init(ugi, delegation, username, doAsUser, path, op, bufferSize);
 
 
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
       @Override
@@ -423,8 +444,8 @@ public class NamenodeWebHdfsMethods {
     switch(op.getValue()) {
     switch(op.getValue()) {
     case APPEND:
     case APPEND:
     {
     {
-      final URI uri = redirectURI(namenode, ugi, delegation, fullpath,
-          op.getValue(), -1L, bufferSize);
+      final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser,
+          fullpath, op.getValue(), -1L, bufferSize);
       return Response.temporaryRedirect(uri).build();
       return Response.temporaryRedirect(uri).build();
     }
     }
     default:
     default:
@@ -446,6 +467,10 @@ public class NamenodeWebHdfsMethods {
       @Context final UserGroupInformation ugi,
       @Context final UserGroupInformation ugi,
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
           final DelegationParam delegation,
           final DelegationParam delegation,
+      @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
+          final UserParam username,
+      @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
+          final DoAsParam doAsUser,
       @QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT)
       @QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT)
           final GetOpParam op,
           final GetOpParam op,
       @QueryParam(OffsetParam.NAME) @DefaultValue(OffsetParam.DEFAULT)
       @QueryParam(OffsetParam.NAME) @DefaultValue(OffsetParam.DEFAULT)
@@ -457,7 +482,8 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
       @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
           final BufferSizeParam bufferSize
           final BufferSizeParam bufferSize
       ) throws IOException, URISyntaxException, InterruptedException {
       ) throws IOException, URISyntaxException, InterruptedException {
-    return get(ugi, delegation, ROOT, op, offset, length, renewer, bufferSize);
+    return get(ugi, delegation, username, doAsUser, ROOT, op,
+        offset, length, renewer, bufferSize);
   }
   }
 
 
   /** Handle HTTP GET request. */
   /** Handle HTTP GET request. */
@@ -468,6 +494,10 @@ public class NamenodeWebHdfsMethods {
       @Context final UserGroupInformation ugi,
       @Context final UserGroupInformation ugi,
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
       @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
           final DelegationParam delegation,
           final DelegationParam delegation,
+      @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
+          final UserParam username,
+      @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
+          final DoAsParam doAsUser,
       @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
       @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
       @QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT)
       @QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT)
           final GetOpParam op,
           final GetOpParam op,
@@ -481,13 +511,8 @@ public class NamenodeWebHdfsMethods {
           final BufferSizeParam bufferSize
           final BufferSizeParam bufferSize
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
 
 
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(op + ": " + path + ", ugi=" + ugi
-          + Param.toSortedString(", ", offset, length, renewer, bufferSize));
-    }
-
-    //clear content type
-    response.setContentType(null);
+    init(ugi, delegation, username, doAsUser, path, op,
+        offset, length, renewer, bufferSize);
 
 
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
       @Override
@@ -502,8 +527,8 @@ public class NamenodeWebHdfsMethods {
     switch(op.getValue()) {
     switch(op.getValue()) {
     case OPEN:
     case OPEN:
     {
     {
-      final URI uri = redirectURI(namenode, ugi, delegation, fullpath,
-          op.getValue(), offset.getValue(), offset, length, bufferSize);
+      final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser,
+          fullpath, op.getValue(), offset.getValue(), offset, length, bufferSize);
       return Response.temporaryRedirect(uri).build();
       return Response.temporaryRedirect(uri).build();
     }
     }
     case GET_BLOCK_LOCATIONS:
     case GET_BLOCK_LOCATIONS:
@@ -538,17 +563,28 @@ public class NamenodeWebHdfsMethods {
     }
     }
     case GETFILECHECKSUM:
     case GETFILECHECKSUM:
     {
     {
-      final URI uri = redirectURI(namenode, ugi, delegation, fullpath,
-          op.getValue(), -1L);
+      final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser,
+          fullpath, op.getValue(), -1L);
       return Response.temporaryRedirect(uri).build();
       return Response.temporaryRedirect(uri).build();
     }
     }
     case GETDELEGATIONTOKEN:
     case GETDELEGATIONTOKEN:
     {
     {
+      if (delegation.getValue() != null) {
+        throw new IllegalArgumentException(delegation.getName()
+            + " parameter is not null.");
+      }
       final Token<? extends TokenIdentifier> token = generateDelegationToken(
       final Token<? extends TokenIdentifier> token = generateDelegationToken(
           namenode, ugi, renewer.getValue());
           namenode, ugi, renewer.getValue());
       final String js = JsonUtil.toJsonString(token);
       final String js = JsonUtil.toJsonString(token);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     }
+    case GETHOMEDIRECTORY:
+    {
+      final String js = JsonUtil.toJsonString(
+          org.apache.hadoop.fs.Path.class.getSimpleName(),
+          WebHdfsFileSystem.getHomeDirectoryString(ugi));
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
     default:
     default:
       throw new UnsupportedOperationException(op + " is not supported");
       throw new UnsupportedOperationException(op + " is not supported");
     }    
     }    
@@ -610,12 +646,18 @@ public class NamenodeWebHdfsMethods {
   @Produces(MediaType.APPLICATION_JSON)
   @Produces(MediaType.APPLICATION_JSON)
   public Response deleteRoot(
   public Response deleteRoot(
       @Context final UserGroupInformation ugi,
       @Context final UserGroupInformation ugi,
+      @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
+          final DelegationParam delegation,
+      @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
+          final UserParam username,
+      @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
+          final DoAsParam doAsUser,
       @QueryParam(DeleteOpParam.NAME) @DefaultValue(DeleteOpParam.DEFAULT)
       @QueryParam(DeleteOpParam.NAME) @DefaultValue(DeleteOpParam.DEFAULT)
           final DeleteOpParam op,
           final DeleteOpParam op,
       @QueryParam(RecursiveParam.NAME) @DefaultValue(RecursiveParam.DEFAULT)
       @QueryParam(RecursiveParam.NAME) @DefaultValue(RecursiveParam.DEFAULT)
           final RecursiveParam recursive
           final RecursiveParam recursive
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
-    return delete(ugi, ROOT, op, recursive);
+    return delete(ugi, delegation, username, doAsUser, ROOT, op, recursive);
   }
   }
 
 
   /** Handle HTTP DELETE request. */
   /** Handle HTTP DELETE request. */
@@ -624,6 +666,12 @@ public class NamenodeWebHdfsMethods {
   @Produces(MediaType.APPLICATION_JSON)
   @Produces(MediaType.APPLICATION_JSON)
   public Response delete(
   public Response delete(
       @Context final UserGroupInformation ugi,
       @Context final UserGroupInformation ugi,
+      @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
+          final DelegationParam delegation,
+      @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
+          final UserParam username,
+      @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
+          final DoAsParam doAsUser,
       @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
       @PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
       @QueryParam(DeleteOpParam.NAME) @DefaultValue(DeleteOpParam.DEFAULT)
       @QueryParam(DeleteOpParam.NAME) @DefaultValue(DeleteOpParam.DEFAULT)
           final DeleteOpParam op,
           final DeleteOpParam op,
@@ -631,13 +679,7 @@ public class NamenodeWebHdfsMethods {
           final RecursiveParam recursive
           final RecursiveParam recursive
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
 
 
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(op + ": " + path + ", ugi=" + ugi
-          + Param.toSortedString(", ", recursive));
-    }
-
-    //clear content type
-    response.setContentType(null);
+    init(ugi, delegation, username, doAsUser, path, op, recursive);
 
 
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
       @Override

+ 73 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/AuthFilter.java

@@ -18,6 +18,12 @@
 package org.apache.hadoop.hdfs.web;
 package org.apache.hadoop.hdfs.web;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 import java.util.Properties;
 
 
 import javax.servlet.FilterChain;
 import javax.servlet.FilterChain;
@@ -26,6 +32,7 @@ import javax.servlet.ServletException;
 import javax.servlet.ServletRequest;
 import javax.servlet.ServletRequest;
 import javax.servlet.ServletResponse;
 import javax.servlet.ServletResponse;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
 
 
 import org.apache.hadoop.hdfs.web.resources.DelegationParam;
 import org.apache.hadoop.hdfs.web.resources.DelegationParam;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -67,15 +74,77 @@ public class AuthFilter extends AuthenticationFilter {
   @Override
   @Override
   public void doFilter(ServletRequest request, ServletResponse response,
   public void doFilter(ServletRequest request, ServletResponse response,
       FilterChain filterChain) throws IOException, ServletException {
       FilterChain filterChain) throws IOException, ServletException {
-    HttpServletRequest httpRequest = (HttpServletRequest) request;
-    String tokenString = httpRequest
-        .getParameter(DelegationParam.NAME);
+    final HttpServletRequest httpRequest = toLowerCase((HttpServletRequest)request);
+    final String tokenString = httpRequest.getParameter(DelegationParam.NAME);
     if (tokenString != null) {
     if (tokenString != null) {
       //Token is present in the url, therefore token will be used for
       //Token is present in the url, therefore token will be used for
       //authentication, bypass kerberos authentication.
       //authentication, bypass kerberos authentication.
       filterChain.doFilter(httpRequest, response);
       filterChain.doFilter(httpRequest, response);
       return;
       return;
     }
     }
-    super.doFilter(request, response, filterChain);
+    super.doFilter(httpRequest, response, filterChain);
+  }
+
+  private static HttpServletRequest toLowerCase(final HttpServletRequest request) {
+    @SuppressWarnings("unchecked")
+    final Map<String, String[]> original = (Map<String, String[]>)request.getParameterMap();
+    if (!ParamFilter.containsUpperCase(original.keySet())) {
+      return request;
+    }
+
+    final Map<String, List<String>> m = new HashMap<String, List<String>>();
+    for(Map.Entry<String, String[]> entry : original.entrySet()) {
+      final String key = entry.getKey().toLowerCase();
+      List<String> strings = m.get(key);
+      if (strings == null) {
+        strings = new ArrayList<String>();
+        m.put(key, strings);
+      }
+      for(String v : entry.getValue()) {
+        strings.add(v);
+      }
+    }
+
+    return new HttpServletRequestWrapper(request) {
+      private Map<String, String[]> parameters = null;
+
+      @Override
+      public Map<String, String[]> getParameterMap() {
+        if (parameters == null) {
+          parameters = new HashMap<String, String[]>();
+          for(Map.Entry<String, List<String>> entry : m.entrySet()) {
+            final List<String> a = entry.getValue();
+            parameters.put(entry.getKey(), a.toArray(new String[a.size()]));
+          }
+        }
+       return parameters;
+      }
+
+      @Override
+      public String getParameter(String name) {
+        final List<String> a = m.get(name);
+        return a == null? null: a.get(0);
+      }
+      
+      @Override
+      public String[] getParameterValues(String name) {
+        return getParameterMap().get(name);
+      }
+
+      @Override
+      public Enumeration<String> getParameterNames() {
+        final Iterator<String> i = m.keySet().iterator();
+        return new Enumeration<String>() {
+          @Override
+          public boolean hasMoreElements() {
+            return i.hasNext();
+          }
+          @Override
+          public String nextElement() {
+            return i.next();
+          }
+        };
+      }
+    };
   }
   }
 }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/ParamFilter.java

@@ -59,7 +59,7 @@ public class ParamFilter implements ResourceFilter {
   }
   }
 
 
   /** Do the strings contain upper case letters? */
   /** Do the strings contain upper case letters? */
-  private static boolean containsUpperCase(final Iterable<String> strings) {
+  static boolean containsUpperCase(final Iterable<String> strings) {
     for(String s : strings) {
     for(String s : strings) {
       for(int i = 0; i < s.length(); i++) {
       for(int i = 0; i < s.length(); i++) {
         if (Character.isUpperCase(s.charAt(i))) {
         if (Character.isUpperCase(s.charAt(i))) {

+ 26 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -92,6 +92,8 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenRenewer;
 import org.apache.hadoop.security.token.TokenRenewer;
@@ -201,9 +203,14 @@ public class WebHdfsFileSystem extends FileSystem
     }
     }
   }
   }
 
 
+  /** @return the home directory. */
+  public static String getHomeDirectoryString(final UserGroupInformation ugi) {
+    return "/user/" + ugi.getShortUserName();
+  }
+
   @Override
   @Override
   public Path getHomeDirectory() {
   public Path getHomeDirectory() {
-    return makeQualified(new Path("/user/" + ugi.getShortUserName()));
+    return makeQualified(new Path(getHomeDirectoryString(ugi)));
   }
   }
 
 
   @Override
   @Override
@@ -225,7 +232,7 @@ public class WebHdfsFileSystem extends FileSystem
     return f.isAbsolute()? f: new Path(workingDir, f);
     return f.isAbsolute()? f: new Path(workingDir, f);
   }
   }
 
 
-  private static Map<?, ?> jsonParse(final InputStream in) throws IOException {
+  static Map<?, ?> jsonParse(final InputStream in) throws IOException {
     if (in == null) {
     if (in == null) {
       throw new IOException("The input stream is null.");
       throw new IOException("The input stream is null.");
     }
     }
@@ -251,13 +258,16 @@ public class WebHdfsFileSystem extends FileSystem
 
 
       final RemoteException re = JsonUtil.toRemoteException(m);
       final RemoteException re = JsonUtil.toRemoteException(m);
       throw re.unwrapRemoteException(AccessControlException.class,
       throw re.unwrapRemoteException(AccessControlException.class,
-          DSQuotaExceededException.class,
+          InvalidToken.class,
+          AuthenticationException.class,
+          AuthorizationException.class,
           FileAlreadyExistsException.class,
           FileAlreadyExistsException.class,
           FileNotFoundException.class,
           FileNotFoundException.class,
           ParentNotDirectoryException.class,
           ParentNotDirectoryException.class,
+          UnresolvedPathException.class,
           SafeModeException.class,
           SafeModeException.class,
-          NSQuotaExceededException.class,
-          UnresolvedPathException.class);
+          DSQuotaExceededException.class,
+          NSQuotaExceededException.class);
     }
     }
     return null;
     return null;
   }
   }
@@ -352,7 +362,7 @@ public class WebHdfsFileSystem extends FileSystem
   /**
   /**
    * Two-step Create/Append:
    * Two-step Create/Append:
    * Step 1) Submit a Http request with neither auto-redirect nor data. 
    * Step 1) Submit a Http request with neither auto-redirect nor data. 
-   * Step 2) Submit Http PUT with the URL from the Location header with data.
+   * Step 2) Submit another Http request with the URL from the Location header with data.
    * 
    * 
    * The reason of having two-step create/append is for preventing clients to
    * The reason of having two-step create/append is for preventing clients to
    * send out the data before the redirect. This issue is addressed by the
    * send out the data before the redirect. This issue is addressed by the
@@ -362,7 +372,7 @@ public class WebHdfsFileSystem extends FileSystem
    * 100-continue". The two-step create/append is a temporary workaround for
    * 100-continue". The two-step create/append is a temporary workaround for
    * the software library bugs.
    * the software library bugs.
    */
    */
-  private static HttpURLConnection twoStepWrite(HttpURLConnection conn,
+  static HttpURLConnection twoStepWrite(HttpURLConnection conn,
       final HttpOpParam.Op op) throws IOException {
       final HttpOpParam.Op op) throws IOException {
     //Step 1) Submit a Http request with neither auto-redirect nor data. 
     //Step 1) Submit a Http request with neither auto-redirect nor data. 
     conn.setInstanceFollowRedirects(false);
     conn.setInstanceFollowRedirects(false);
@@ -372,7 +382,7 @@ public class WebHdfsFileSystem extends FileSystem
     final String redirect = conn.getHeaderField("Location");
     final String redirect = conn.getHeaderField("Location");
     conn.disconnect();
     conn.disconnect();
 
 
-    //Step 2) Submit Http PUT with the URL from the Location header with data.
+    //Step 2) Submit another Http request with the URL from the Location header with data.
     conn = (HttpURLConnection)new URL(redirect).openConnection();
     conn = (HttpURLConnection)new URL(redirect).openConnection();
     conn.setRequestMethod(op.getType().toString());
     conn.setRequestMethod(op.getType().toString());
     return conn;
     return conn;
@@ -507,7 +517,7 @@ public class WebHdfsFileSystem extends FileSystem
         DFSConfigKeys.DFS_REPLICATION_DEFAULT);
         DFSConfigKeys.DFS_REPLICATION_DEFAULT);
   }
   }
 
 
-  private FSDataOutputStream write(final HttpOpParam.Op op,
+  FSDataOutputStream write(final HttpOpParam.Op op,
       final HttpURLConnection conn, final int bufferSize) throws IOException {
       final HttpURLConnection conn, final int bufferSize) throws IOException {
     return new FSDataOutputStream(new BufferedOutputStream(
     return new FSDataOutputStream(new BufferedOutputStream(
         conn.getOutputStream(), bufferSize), statistics) {
         conn.getOutputStream(), bufferSize), statistics) {
@@ -516,7 +526,11 @@ public class WebHdfsFileSystem extends FileSystem
         try {
         try {
           super.close();
           super.close();
         } finally {
         } finally {
-          validateResponse(op, conn);
+          try {
+            validateResponse(op, conn);
+          } finally {
+            conn.disconnect();
+          }
         }
         }
       }
       }
     };
     };
@@ -630,7 +644,7 @@ public class WebHdfsFileSystem extends FileSystem
   }
   }
 
 
   static class OffsetUrlInputStream extends ByteRangeInputStream {
   static class OffsetUrlInputStream extends ByteRangeInputStream {
-    OffsetUrlInputStream(URLOpener o, URLOpener r) {
+    OffsetUrlInputStream(OffsetUrlOpener o, OffsetUrlOpener r) {
       super(o, r);
       super(o, r);
     }
     }
     
     
@@ -673,7 +687,7 @@ public class WebHdfsFileSystem extends FileSystem
     final HttpOpParam.Op op = GetOpParam.Op.GETDELEGATIONTOKEN;
     final HttpOpParam.Op op = GetOpParam.Op.GETDELEGATIONTOKEN;
     final Map<?, ?> m = run(op, null, new RenewerParam(renewer));
     final Map<?, ?> m = run(op, null, new RenewerParam(renewer));
     final Token<DelegationTokenIdentifier> token = JsonUtil.toDelegationToken(m); 
     final Token<DelegationTokenIdentifier> token = JsonUtil.toDelegationToken(m); 
-    token.setService(new Text(getCanonicalServiceName()));
+    SecurityUtil.setTokenService(token, nnAddr);
     return token;
     return token;
   }
   }
 
 

+ 41 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DoAsParam.java

@@ -0,0 +1,41 @@
+/**
+ * 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.hdfs.web.resources;
+
+/** DoAs parameter for proxy user. */
+public class DoAsParam extends StringParam {
+  /** Parameter name. */
+  public static final String NAME = "doas";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain DOMAIN = new Domain(NAME, null);
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public DoAsParam(final String str) {
+    super(DOMAIN, str == null || str.equals(DEFAULT)? null: str);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 20 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ExceptionHandler.java

@@ -30,14 +30,25 @@ import javax.ws.rs.ext.Provider;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.web.JsonUtil;
 import org.apache.hadoop.hdfs.web.JsonUtil;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.security.authorize.AuthorizationException;
 
 
 import com.sun.jersey.api.ParamException;
 import com.sun.jersey.api.ParamException;
+import com.sun.jersey.api.container.ContainerException;
 
 
 /** Handle exceptions. */
 /** Handle exceptions. */
 @Provider
 @Provider
 public class ExceptionHandler implements ExceptionMapper<Exception> {
 public class ExceptionHandler implements ExceptionMapper<Exception> {
   public static final Log LOG = LogFactory.getLog(ExceptionHandler.class);
   public static final Log LOG = LogFactory.getLog(ExceptionHandler.class);
 
 
+  private static Exception toCause(Exception e) {
+    final Throwable t = e.getCause();
+    if (t != null && t instanceof Exception) {
+      e = (Exception)e.getCause();
+    }
+    return e;
+  }
+
   private @Context HttpServletResponse response;
   private @Context HttpServletResponse response;
 
 
   @Override
   @Override
@@ -55,12 +66,20 @@ public class ExceptionHandler implements ExceptionMapper<Exception> {
       e = new IllegalArgumentException("Invalid value for webhdfs parameter \""
       e = new IllegalArgumentException("Invalid value for webhdfs parameter \""
           + paramexception.getParameterName() + "\": "
           + paramexception.getParameterName() + "\": "
           + e.getCause().getMessage(), e);
           + e.getCause().getMessage(), e);
-    } 
+    }
+    if (e instanceof ContainerException) {
+      e = toCause(e);
+    }
+    if (e instanceof RemoteException) {
+      e = ((RemoteException)e).unwrapRemoteException();
+    }
 
 
     //Map response status
     //Map response status
     final Response.Status s;
     final Response.Status s;
     if (e instanceof SecurityException) {
     if (e instanceof SecurityException) {
       s = Response.Status.UNAUTHORIZED;
       s = Response.Status.UNAUTHORIZED;
+    } else if (e instanceof AuthorizationException) {
+      s = Response.Status.UNAUTHORIZED;
     } else if (e instanceof FileNotFoundException) {
     } else if (e instanceof FileNotFoundException) {
       s = Response.Status.NOT_FOUND;
       s = Response.Status.NOT_FOUND;
     } else if (e instanceof IOException) {
     } else if (e instanceof IOException) {

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java

@@ -30,6 +30,7 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     GETCONTENTSUMMARY(HttpURLConnection.HTTP_OK),
     GETCONTENTSUMMARY(HttpURLConnection.HTTP_OK),
     GETFILECHECKSUM(HttpURLConnection.HTTP_OK),
     GETFILECHECKSUM(HttpURLConnection.HTTP_OK),
 
 
+    GETHOMEDIRECTORY(HttpURLConnection.HTTP_OK),
     GETDELEGATIONTOKEN(HttpURLConnection.HTTP_OK),
     GETDELEGATIONTOKEN(HttpURLConnection.HTTP_OK),
 
 
     /** GET_BLOCK_LOCATIONS is a private unstable op. */
     /** GET_BLOCK_LOCATIONS is a private unstable op. */

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java

@@ -58,7 +58,7 @@ public abstract class HttpOpParam<E extends Enum<E> & HttpOpParam.Op>
     public static TemporaryRedirectOp valueOf(final Op op) {
     public static TemporaryRedirectOp valueOf(final Op op) {
       if (op == CREATE.op) {
       if (op == CREATE.op) {
         return CREATE;
         return CREATE;
-      } if (op == APPEND.op) {
+      } else if (op == APPEND.op) {
         return APPEND;
         return APPEND;
       }
       }
       throw new IllegalArgumentException(op + " not found.");
       throw new IllegalArgumentException(op + " not found.");

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserProvider.java

@@ -53,7 +53,8 @@ public class UserProvider
       return JspHelper.getUGI(servletcontext, request, conf,
       return JspHelper.getUGI(servletcontext, request, conf,
           AuthenticationMethod.KERBEROS, false);
           AuthenticationMethod.KERBEROS, false);
     } catch (IOException e) {
     } catch (IOException e) {
-      throw new SecurityException("Failed to obtain user group information.", e);
+      throw new SecurityException(
+          "Failed to obtain user group information: " + e, e);
     }
     }
   }
   }
 
 

+ 78 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java

@@ -23,29 +23,46 @@ package org.apache.hadoop.hdfs.security;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.IOException;
+import java.net.HttpURLConnection;
 import java.net.InetAddress;
 import java.net.InetAddress;
 import java.net.NetworkInterface;
 import java.net.NetworkInterface;
+import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.Enumeration;
+import java.util.Map;
 
 
-import junit.framework.Assert;
+import javax.servlet.http.HttpServletResponse;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
+import org.apache.hadoop.hdfs.web.resources.DoAsParam;
+import org.apache.hadoop.hdfs.web.resources.ExceptionHandler;
+import org.apache.hadoop.hdfs.web.resources.GetOpParam;
+import org.apache.hadoop.hdfs.web.resources.PutOpParam;
 import org.apache.hadoop.security.TestDoAsEffectiveUser;
 import org.apache.hadoop.security.TestDoAsEffectiveUser;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
+import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
@@ -89,6 +106,7 @@ public class TestDelegationTokenForProxyUser {
   @Before
   @Before
   public void setUp() throws Exception {
   public void setUp() throws Exception {
     config = new HdfsConfiguration();
     config = new HdfsConfiguration();
+    config.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     config.setLong(
     config.setLong(
         DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, 10000);
         DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, 10000);
     config.setLong(
     config.setLong(
@@ -137,4 +155,63 @@ public class TestDelegationTokenForProxyUser {
     }
     }
   }
   }
   
   
+  @Test
+  public void testWebHdfsDoAs() throws Exception {
+    WebHdfsTestUtil.LOG.info("START: testWebHdfsDoAs()");
+    ((Log4JLogger)NamenodeWebHdfsMethods.LOG).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)ExceptionHandler.LOG).getLogger().setLevel(Level.ALL);
+    final UserGroupInformation ugi = UserGroupInformation.createRemoteUser(REAL_USER);
+    WebHdfsTestUtil.LOG.info("ugi.getShortUserName()=" + ugi.getShortUserName());
+    final WebHdfsFileSystem webhdfs = WebHdfsTestUtil.getWebHdfsFileSystemAs(ugi, config);
+    
+    final Path root = new Path("/");
+    cluster.getFileSystem().setPermission(root, new FsPermission((short)0777));
+
+    {
+      //test GETHOMEDIRECTORY with doAs
+      final URL url = WebHdfsTestUtil.toUrl(webhdfs,
+          GetOpParam.Op.GETHOMEDIRECTORY,  root, new DoAsParam(PROXY_USER));
+      final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+      final Map<?, ?> m = WebHdfsTestUtil.connectAndGetJson(conn, HttpServletResponse.SC_OK);
+      conn.disconnect();
+  
+      final Object responsePath = m.get(Path.class.getSimpleName());
+      WebHdfsTestUtil.LOG.info("responsePath=" + responsePath);
+      Assert.assertEquals("/user/" + PROXY_USER, responsePath);
+    }
+
+    {
+      //test GETHOMEDIRECTORY with DOas
+      final URL url = WebHdfsTestUtil.toUrl(webhdfs,
+          GetOpParam.Op.GETHOMEDIRECTORY,  root, new DoAsParam(PROXY_USER) {
+            @Override
+            public String getName() {
+              return "DOas";
+            }
+      });
+      final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+      final Map<?, ?> m = WebHdfsTestUtil.connectAndGetJson(conn, HttpServletResponse.SC_OK);
+      conn.disconnect();
+  
+      final Object responsePath = m.get(Path.class.getSimpleName());
+      WebHdfsTestUtil.LOG.info("responsePath=" + responsePath);
+      Assert.assertEquals("/user/" + PROXY_USER, responsePath);
+    }
+
+    {
+      //test create file with doAs
+      final Path f = new Path("/testWebHdfsDoAs/a.txt");
+      final PutOpParam.Op op = PutOpParam.Op.CREATE;
+      final URL url = WebHdfsTestUtil.toUrl(webhdfs, op,  f, new DoAsParam(PROXY_USER));
+      HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+      conn = WebHdfsTestUtil.twoStepWrite(conn, op);
+      final FSDataOutputStream out = WebHdfsTestUtil.write(webhdfs, op, conn, 4096);
+      out.write("Hello, webhdfs user!".getBytes());
+      out.close();
+  
+      final FileStatus status = webhdfs.getFileStatus(f);
+      WebHdfsTestUtil.LOG.info("status.getOwner()=" + status.getOwner());
+      Assert.assertEquals(PROXY_USER, status.getOwner());
+    }
+  }
 }
 }

+ 26 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java

@@ -23,9 +23,8 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
 import java.net.HttpURLConnection;
 import java.net.HttpURLConnection;
-import java.net.URI;
 import java.net.URL;
 import java.net.URL;
-import java.security.PrivilegedExceptionAction;
+import java.util.Map;
 
 
 import javax.servlet.http.HttpServletResponse;
 import javax.servlet.http.HttpServletResponse;
 
 
@@ -34,12 +33,12 @@ import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemContractBaseTest;
 import org.apache.hadoop.fs.FileSystemContractBaseTest;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.web.resources.DoAsParam;
 import org.apache.hadoop.hdfs.web.resources.GetOpParam;
 import org.apache.hadoop.hdfs.web.resources.GetOpParam;
 import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
 import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
 import org.apache.hadoop.hdfs.web.resources.PutOpParam;
 import org.apache.hadoop.hdfs.web.resources.PutOpParam;
@@ -51,6 +50,8 @@ public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest {
   private static final Configuration conf = new Configuration();
   private static final Configuration conf = new Configuration();
   private static final MiniDFSCluster cluster;
   private static final MiniDFSCluster cluster;
   private String defaultWorkingDirectory;
   private String defaultWorkingDirectory;
+  
+  private UserGroupInformation ugi;
 
 
   static {
   static {
     conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
@@ -68,20 +69,11 @@ public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest {
 
 
   @Override
   @Override
   protected void setUp() throws Exception {
   protected void setUp() throws Exception {
-    final String uri = WebHdfsFileSystem.SCHEME  + "://"
-        + conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
-
     //get file system as a non-superuser
     //get file system as a non-superuser
     final UserGroupInformation current = UserGroupInformation.getCurrentUser();
     final UserGroupInformation current = UserGroupInformation.getCurrentUser();
-    final UserGroupInformation ugi = UserGroupInformation.createUserForTesting(
+    ugi = UserGroupInformation.createUserForTesting(
         current.getShortUserName() + "x", new String[]{"user"});
         current.getShortUserName() + "x", new String[]{"user"});
-    fs = ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
-      @Override
-      public FileSystem run() throws Exception {
-        return FileSystem.get(new URI(uri), conf);
-      }
-    });
-
+    fs = WebHdfsTestUtil.getWebHdfsFileSystemAs(ugi, conf);
     defaultWorkingDirectory = fs.getWorkingDirectory().toUri().getPath();
     defaultWorkingDirectory = fs.getWorkingDirectory().toUri().getPath();
   }
   }
 
 
@@ -263,9 +255,29 @@ public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest {
 
 
   public void testResponseCode() throws IOException {
   public void testResponseCode() throws IOException {
     final WebHdfsFileSystem webhdfs = (WebHdfsFileSystem)fs;
     final WebHdfsFileSystem webhdfs = (WebHdfsFileSystem)fs;
+    final Path root = new Path("/");
     final Path dir = new Path("/test/testUrl");
     final Path dir = new Path("/test/testUrl");
     assertTrue(webhdfs.mkdirs(dir));
     assertTrue(webhdfs.mkdirs(dir));
 
 
+    {//test GETHOMEDIRECTORY
+      final URL url = webhdfs.toUrl(GetOpParam.Op.GETHOMEDIRECTORY, root);
+      final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+      final Map<?, ?> m = WebHdfsTestUtil.connectAndGetJson(
+          conn, HttpServletResponse.SC_OK);
+      assertEquals(WebHdfsFileSystem.getHomeDirectoryString(ugi),
+          m.get(Path.class.getSimpleName()));
+      conn.disconnect();
+    }
+
+    {//test GETHOMEDIRECTORY with unauthorized doAs
+      final URL url = webhdfs.toUrl(GetOpParam.Op.GETHOMEDIRECTORY, root,
+          new DoAsParam(ugi.getShortUserName() + "proxy"));
+      final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+      conn.connect();
+      assertEquals(HttpServletResponse.SC_UNAUTHORIZED, conn.getResponseCode());
+      conn.disconnect();
+    }
+
     {//test set owner with empty parameters
     {//test set owner with empty parameters
       final URL url = webhdfs.toUrl(PutOpParam.Op.SETOWNER, dir);
       final URL url = webhdfs.toUrl(PutOpParam.Op.SETOWNER, dir);
       final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
       final HttpURLConnection conn = (HttpURLConnection) url.openConnection();

+ 90 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/WebHdfsTestUtil.java

@@ -0,0 +1,90 @@
+/**
+ * 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.hdfs.web;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.security.PrivilegedExceptionAction;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
+import org.apache.hadoop.hdfs.web.resources.Param;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Assert;
+
+public class WebHdfsTestUtil {
+  public static final Log LOG = LogFactory.getLog(WebHdfsTestUtil.class);
+
+  public static WebHdfsFileSystem getWebHdfsFileSystem(final Configuration conf
+      ) throws IOException, URISyntaxException {
+    final String uri = WebHdfsFileSystem.SCHEME  + "://"
+        + conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+    return (WebHdfsFileSystem)FileSystem.get(new URI(uri), conf);
+  }
+
+  public static WebHdfsFileSystem getWebHdfsFileSystemAs(
+      final UserGroupInformation ugi, final Configuration conf
+      ) throws IOException, URISyntaxException, InterruptedException {
+    return ugi.doAs(new PrivilegedExceptionAction<WebHdfsFileSystem>() {
+      @Override
+      public WebHdfsFileSystem run() throws Exception {
+        return getWebHdfsFileSystem(conf);
+      }
+    });
+  }
+
+  public static URL toUrl(final WebHdfsFileSystem webhdfs,
+      final HttpOpParam.Op op, final Path fspath,
+      final Param<?,?>... parameters) throws IOException {
+    final URL url = webhdfs.toUrl(op, fspath, parameters);
+    WebHdfsTestUtil.LOG.info("url=" + url);
+    return url;
+  }
+
+  public static Map<?, ?> connectAndGetJson(final HttpURLConnection conn,
+      final int expectedResponseCode) throws IOException {
+    conn.connect();
+    Assert.assertEquals(expectedResponseCode, conn.getResponseCode());
+    return WebHdfsFileSystem.jsonParse(conn.getInputStream());
+  }
+  
+  public static HttpURLConnection twoStepWrite(HttpURLConnection conn,
+      final HttpOpParam.Op op) throws IOException {
+    conn.setRequestMethod(op.getType().toString());
+    conn = WebHdfsFileSystem.twoStepWrite(conn, op);
+    conn.setDoOutput(true);
+    conn.connect();
+    return conn;
+  }
+
+  public static FSDataOutputStream write(final WebHdfsFileSystem webhdfs,
+      final HttpOpParam.Op op, final HttpURLConnection conn,
+      final int bufferSize) throws IOException {
+    return webhdfs.write(op, conn, bufferSize);
+  }
+}