You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by cm...@apache.org on 2014/08/20 01:50:25 UTC

svn commit: r1619012 [2/35] - in /hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project: hadoop-hdfs-httpfs/ hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/ hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/ hadoop-...

Propchange: hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs:r1606534
  Merged /hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs:r1588992-1596568
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs:r1588388-1619000

Modified: hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml?rev=1619012&r1=1619011&r2=1619012&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml (original)
+++ hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml Tue Aug 19 23:49:39 2014
@@ -108,22 +108,10 @@
           <artifactId>commons-httpclient</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>tomcat</groupId>
-          <artifactId>jasper-compiler</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>tomcat</groupId>
-          <artifactId>jasper-runtime</artifactId>
-        </exclusion>
-        <exclusion>
           <groupId>javax.servlet</groupId>
           <artifactId>servlet-api</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>javax.servlet</groupId>
-          <artifactId>jsp-api</artifactId>
-        </exclusion>
-        <exclusion>
           <groupId>javax.servlet.jsp</groupId>
           <artifactId>jsp-api</artifactId>
         </exclusion>
@@ -137,10 +125,6 @@
         </exclusion>
         <exclusion>
           <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jsp-api-2.1</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
           <artifactId>servlet-api-2.5</artifactId>
         </exclusion>
         <exclusion>
@@ -151,10 +135,6 @@
           <groupId>org.eclipse.jdt</groupId>
           <artifactId>core</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>commons-el</groupId>
-          <artifactId>commons-el</artifactId>
-        </exclusion>
       </exclusions>
     </dependency>
     <dependency>
@@ -171,26 +151,10 @@
           <artifactId>commons-httpclient</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>tomcat</groupId>
-          <artifactId>jasper-compiler</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>tomcat</groupId>
-          <artifactId>jasper-runtime</artifactId>
-        </exclusion>
-        <exclusion>
           <groupId>javax.servlet</groupId>
           <artifactId>servlet-api</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>javax.servlet</groupId>
-          <artifactId>jsp-api</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>javax.servlet.jsp</groupId>
-          <artifactId>jsp-api</artifactId>
-        </exclusion>
-        <exclusion>
           <groupId>org.mortbay.jetty</groupId>
           <artifactId>jetty</artifactId>
         </exclusion>
@@ -200,10 +164,6 @@
         </exclusion>
         <exclusion>
           <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jsp-api-2.1</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
           <artifactId>servlet-api-2.5</artifactId>
         </exclusion>
         <exclusion>
@@ -214,10 +174,6 @@
           <groupId>org.eclipse.jdt</groupId>
           <artifactId>core</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>commons-el</groupId>
-          <artifactId>commons-el</artifactId>
-        </exclusion>
       </exclusions>
     </dependency>
     <dependency>

Modified: hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java?rev=1619012&r1=1619011&r2=1619012&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java (original)
+++ hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java Tue Aug 19 23:49:39 2014
@@ -18,6 +18,7 @@
 package org.apache.hadoop.fs.http.client;
 
 import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
@@ -31,19 +32,32 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PositionedReadable;
 import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.fs.XAttrCodec;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.lib.wsrs.EnumSetParam;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
-import org.apache.hadoop.security.authentication.client.Authenticator;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
+import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
@@ -55,7 +69,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.HttpURLConnection;
-import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
@@ -63,7 +76,6 @@ import java.security.PrivilegedException
 import java.text.MessageFormat;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.concurrent.Callable;
 
 /**
  * HttpFSServer implementation of the FileSystemAccess FileSystem.
@@ -86,6 +98,7 @@ public class HttpFSFileSystem extends Fi
   public static final String REPLICATION_PARAM = "replication";
   public static final String BLOCKSIZE_PARAM = "blocksize";
   public static final String PERMISSION_PARAM = "permission";
+  public static final String ACLSPEC_PARAM = "aclspec";
   public static final String DESTINATION_PARAM = "destination";
   public static final String RECURSIVE_PARAM = "recursive";
   public static final String SOURCES_PARAM = "sources";
@@ -93,8 +106,13 @@ public class HttpFSFileSystem extends Fi
   public static final String GROUP_PARAM = "group";
   public static final String MODIFICATION_TIME_PARAM = "modificationtime";
   public static final String ACCESS_TIME_PARAM = "accesstime";
+  public static final String XATTR_NAME_PARAM = "xattr.name";
+  public static final String XATTR_VALUE_PARAM = "xattr.value";
+  public static final String XATTR_SET_FLAG_PARAM = "flag";
+  public static final String XATTR_ENCODING_PARAM = "encoding";
 
   public static final Short DEFAULT_PERMISSION = 0755;
+  public static final String ACLSPEC_DEFAULT = "";
 
   public static final String RENAME_JSON = "boolean";
 
@@ -138,6 +156,10 @@ public class HttpFSFileSystem extends Fi
   public static final String MODIFICATION_TIME_JSON = "modificationTime";
   public static final String BLOCK_SIZE_JSON = "blockSize";
   public static final String REPLICATION_JSON = "replication";
+  public static final String XATTRS_JSON = "XAttrs";
+  public static final String XATTR_NAME_JSON = "name";
+  public static final String XATTR_VALUE_JSON = "value";
+  public static final String XATTRNAMES_JSON = "XAttrNames";
 
   public static final String FILE_CHECKSUM_JSON = "FileChecksum";
   public static final String CHECKSUM_ALGORITHM_JSON = "algorithm";
@@ -152,6 +174,11 @@ public class HttpFSFileSystem extends Fi
   public static final String CONTENT_SUMMARY_SPACE_CONSUMED_JSON = "spaceConsumed";
   public static final String CONTENT_SUMMARY_SPACE_QUOTA_JSON = "spaceQuota";
 
+  public static final String ACL_STATUS_JSON = "AclStatus";
+  public static final String ACL_STICKY_BIT_JSON = "stickyBit";
+  public static final String ACL_ENTRIES_JSON = "entries";
+  public static final String ACL_BIT_JSON = "aclBit";
+
   public static final String ERROR_JSON = "RemoteException";
   public static final String ERROR_EXCEPTION_JSON = "exception";
   public static final String ERROR_CLASSNAME_JSON = "javaClassName";
@@ -169,11 +196,14 @@ public class HttpFSFileSystem extends Fi
     OPEN(HTTP_GET), GETFILESTATUS(HTTP_GET), LISTSTATUS(HTTP_GET),
     GETHOMEDIRECTORY(HTTP_GET), GETCONTENTSUMMARY(HTTP_GET),
     GETFILECHECKSUM(HTTP_GET),  GETFILEBLOCKLOCATIONS(HTTP_GET),
-    INSTRUMENTATION(HTTP_GET),
+    INSTRUMENTATION(HTTP_GET), GETACLSTATUS(HTTP_GET),
     APPEND(HTTP_POST), CONCAT(HTTP_POST),
     CREATE(HTTP_PUT), MKDIRS(HTTP_PUT), RENAME(HTTP_PUT), SETOWNER(HTTP_PUT),
     SETPERMISSION(HTTP_PUT), SETREPLICATION(HTTP_PUT), SETTIMES(HTTP_PUT),
-    DELETE(HTTP_DELETE);
+    MODIFYACLENTRIES(HTTP_PUT), REMOVEACLENTRIES(HTTP_PUT),
+    REMOVEDEFAULTACL(HTTP_PUT), REMOVEACL(HTTP_PUT), SETACL(HTTP_PUT),
+    DELETE(HTTP_DELETE), SETXATTR(HTTP_PUT), GETXATTRS(HTTP_GET),
+    REMOVEXATTR(HTTP_PUT), LISTXATTRS(HTTP_GET);
 
     private String httpMethod;
 
@@ -187,34 +217,15 @@ public class HttpFSFileSystem extends Fi
 
   }
 
-
-  private AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
+  private DelegationTokenAuthenticatedURL authURL;
+  private DelegationTokenAuthenticatedURL.Token authToken =
+      new DelegationTokenAuthenticatedURL.Token();
   private URI uri;
-  private InetSocketAddress httpFSAddr;
   private Path workingDir;
   private UserGroupInformation realUser;
   private String doAs;
-  private Token<?> delegationToken;
 
-  //This method enables handling UGI doAs with SPNEGO, we have to
-  //fallback to the realuser who logged in with Kerberos credentials
-  private <T> T doAsRealUserIfNecessary(final Callable<T> callable)
-    throws IOException {
-    try {
-      if (realUser.getShortUserName().equals(doAs)) {
-        return callable.call();
-      } else {
-        return realUser.doAs(new PrivilegedExceptionAction<T>() {
-          @Override
-          public T run() throws Exception {
-            return callable.call();
-          }
-        });
-      }
-    } catch (Exception ex) {
-      throw new IOException(ex.toString(), ex);
-    }
-  }
+
 
   /**
    * Convenience method that creates a <code>HttpURLConnection</code> for the
@@ -236,20 +247,51 @@ public class HttpFSFileSystem extends Fi
   private HttpURLConnection getConnection(final String method,
       Map<String, String> params, Path path, boolean makeQualified)
       throws IOException {
-    if (!realUser.getShortUserName().equals(doAs)) {
-      params.put(DO_AS_PARAM, doAs);
-    }
-    HttpFSKerberosAuthenticator.injectDelegationToken(params, delegationToken);
+    return getConnection(method, params, null, path, makeQualified);
+  }
+
+  /**
+   * Convenience method that creates a <code>HttpURLConnection</code> for the
+   * HttpFSServer file system operations.
+   * <p/>
+   * This methods performs and injects any needed authentication credentials
+   * via the {@link #getConnection(URL, String)} method
+   *
+   * @param method the HTTP method.
+   * @param params the query string parameters.
+   * @param multiValuedParams multi valued parameters of the query string
+   * @param path the file path
+   * @param makeQualified if the path should be 'makeQualified'
+   *
+   * @return HttpURLConnection a <code>HttpURLConnection</code> for the
+   *         HttpFSServer server, authenticated and ready to use for the
+   *         specified path and file system operation.
+   *
+   * @throws IOException thrown if an IO error occurrs.
+   */
+  private HttpURLConnection getConnection(final String method,
+      Map<String, String> params, Map<String, List<String>> multiValuedParams,
+      Path path, boolean makeQualified) throws IOException {
     if (makeQualified) {
       path = makeQualified(path);
     }
-    final URL url = HttpFSUtils.createURL(path, params);
-    return doAsRealUserIfNecessary(new Callable<HttpURLConnection>() {
-      @Override
-      public HttpURLConnection call() throws Exception {
-        return getConnection(url, method);
+    final URL url = HttpFSUtils.createURL(path, params, multiValuedParams);
+    try {
+      return UserGroupInformation.getCurrentUser().doAs(
+          new PrivilegedExceptionAction<HttpURLConnection>() {
+            @Override
+            public HttpURLConnection run() throws Exception {
+              return getConnection(url, method);
+            }
+          }
+      );
+    } catch (Exception ex) {
+      if (ex instanceof IOException) {
+        throw (IOException) ex;
+      } else {
+        throw new IOException(ex);
       }
-    });
+    }
   }
 
   /**
@@ -266,12 +308,8 @@ public class HttpFSFileSystem extends Fi
    * @throws IOException thrown if an IO error occurrs.
    */
   private HttpURLConnection getConnection(URL url, String method) throws IOException {
-    Class<? extends Authenticator> klass =
-      getConf().getClass("httpfs.authenticator.class",
-                         HttpFSKerberosAuthenticator.class, Authenticator.class);
-    Authenticator authenticator = ReflectionUtils.newInstance(klass, getConf());
     try {
-      HttpURLConnection conn = new AuthenticatedURL(authenticator).openConnection(url, authToken);
+      HttpURLConnection conn = authURL.openConnection(url, authToken);
       conn.setRequestMethod(method);
       if (method.equals(HTTP_POST) || method.equals(HTTP_PUT)) {
         conn.setDoOutput(true);
@@ -302,10 +340,17 @@ public class HttpFSFileSystem extends Fi
     super.initialize(name, conf);
     try {
       uri = new URI(name.getScheme() + "://" + name.getAuthority());
-      httpFSAddr = NetUtils.createSocketAddr(getCanonicalUri().toString());
     } catch (URISyntaxException ex) {
       throw new IOException(ex);
     }
+
+    Class<? extends DelegationTokenAuthenticator> klass =
+        getConf().getClass("httpfs.authenticator.class",
+            KerberosDelegationTokenAuthenticator.class,
+            DelegationTokenAuthenticator.class);
+    DelegationTokenAuthenticator authenticator =
+        ReflectionUtils.newInstance(klass, getConf());
+    authURL = new DelegationTokenAuthenticatedURL(authenticator);
   }
 
   @Override
@@ -574,7 +619,6 @@ public class HttpFSFileSystem extends Fi
    *
    * @deprecated Use delete(Path, boolean) instead
    */
-  @SuppressWarnings({"deprecation"})
   @Deprecated
   @Override
   public boolean delete(Path f) throws IOException {
@@ -798,6 +842,105 @@ public class HttpFSFileSystem extends Fi
     return (Boolean) json.get(SET_REPLICATION_JSON);
   }
 
+  /**
+   * Modify the ACL entries for a file.
+   *
+   * @param path Path to modify
+   * @param aclSpec List<AclEntry> describing modifications
+   * @throws IOException
+   */
+  @Override
+  public void modifyAclEntries(Path path, List<AclEntry> aclSpec)
+          throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.MODIFYACLENTRIES.toString());
+    params.put(ACLSPEC_PARAM, AclEntry.aclSpecToString(aclSpec));
+    HttpURLConnection conn = getConnection(
+            Operation.MODIFYACLENTRIES.getMethod(), params, path, true);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+  }
+
+  /**
+   * Remove the specified ACL entries from a file
+   * @param path Path to modify
+   * @param aclSpec List<AclEntry> describing entries to remove
+   * @throws IOException
+   */
+  @Override
+  public void removeAclEntries(Path path, List<AclEntry> aclSpec)
+          throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.REMOVEACLENTRIES.toString());
+    params.put(ACLSPEC_PARAM, AclEntry.aclSpecToString(aclSpec));
+    HttpURLConnection conn = getConnection(
+            Operation.REMOVEACLENTRIES.getMethod(), params, path, true);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+  }
+
+  /**
+   * Removes the default ACL for the given file
+   * @param path Path from which to remove the default ACL.
+   * @throws IOException
+   */
+  @Override
+  public void removeDefaultAcl(Path path) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.REMOVEDEFAULTACL.toString());
+    HttpURLConnection conn = getConnection(
+            Operation.REMOVEDEFAULTACL.getMethod(), params, path, true);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+  }
+
+  /**
+   * Remove all ACLs from a file
+   * @param path Path from which to remove all ACLs
+   * @throws IOException
+   */
+  @Override
+  public void removeAcl(Path path) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.REMOVEACL.toString());
+    HttpURLConnection conn = getConnection(Operation.REMOVEACL.getMethod(),
+            params, path, true);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+  }
+
+  /**
+   * Set the ACLs for the given file
+   * @param path Path to modify
+   * @param aclSpec List<AclEntry> describing modifications, must include
+   *                entries for user, group, and others for compatibility
+   *                with permission bits.
+   * @throws IOException
+   */
+  @Override
+  public void setAcl(Path path, List<AclEntry> aclSpec) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.SETACL.toString());
+    params.put(ACLSPEC_PARAM, AclEntry.aclSpecToString(aclSpec));
+    HttpURLConnection conn = getConnection(Operation.SETACL.getMethod(),
+                                           params, path, true);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+  }
+
+  /**
+   * Get the ACL information for a given file
+   * @param path Path to acquire ACL info for
+   * @return the ACL information in JSON format
+   * @throws IOException
+   */
+  @Override
+  public AclStatus getAclStatus(Path path) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.GETACLSTATUS.toString());
+    HttpURLConnection conn = getConnection(Operation.GETACLSTATUS.getMethod(),
+            params, path, true);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
+    json = (JSONObject) json.get(ACL_STATUS_JSON);
+    return createAclStatus(json);
+  }
+
   private FileStatus createFileStatus(Path parent, JSONObject json) {
     String pathSuffix = (String) json.get(PATH_SUFFIX_JSON);
     Path path = (pathSuffix.equals("")) ? parent : new Path(parent, pathSuffix);
@@ -830,6 +973,23 @@ public class HttpFSFileSystem extends Fi
     return fileStatus;
   }
 
+  /**
+   * Convert the given JSON object into an AclStatus
+   * @param json Input JSON representing the ACLs
+   * @return Resulting AclStatus
+   */
+  private AclStatus createAclStatus(JSONObject json) {
+    AclStatus.Builder aclStatusBuilder = new AclStatus.Builder()
+            .owner((String) json.get(OWNER_JSON))
+            .group((String) json.get(GROUP_JSON))
+            .stickyBit((Boolean) json.get(ACL_STICKY_BIT_JSON));
+    JSONArray entries = (JSONArray) json.get(ACL_ENTRIES_JSON);
+    for ( Object e : entries ) {
+      aclStatusBuilder.addEntry(AclEntry.parseAclEntry(e.toString(), true));
+    }
+    return aclStatusBuilder.build();
+  }
+
   @Override
   public ContentSummary getContentSummary(Path f) throws IOException {
     Map<String, String> params = new HashMap<String, String>();
@@ -889,38 +1049,165 @@ public class HttpFSFileSystem extends Fi
   @Override
   public Token<?> getDelegationToken(final String renewer)
     throws IOException {
-    return doAsRealUserIfNecessary(new Callable<Token<?>>() {
-      @Override
-      public Token<?> call() throws Exception {
-        return HttpFSKerberosAuthenticator.
-          getDelegationToken(uri, httpFSAddr, authToken, renewer);
+    try {
+      return UserGroupInformation.getCurrentUser().doAs(
+          new PrivilegedExceptionAction<Token<?>>() {
+            @Override
+            public Token<?> run() throws Exception {
+              return authURL.getDelegationToken(uri.toURL(), authToken,
+                  renewer);
+            }
+          }
+      );
+    } catch (Exception ex) {
+      if (ex instanceof IOException) {
+        throw (IOException) ex;
+      } else {
+        throw new IOException(ex);
       }
-    });
+    }
   }
 
   public long renewDelegationToken(final Token<?> token) throws IOException {
-    return doAsRealUserIfNecessary(new Callable<Long>() {
-      @Override
-      public Long call() throws Exception {
-        return HttpFSKerberosAuthenticator.
-          renewDelegationToken(uri,  authToken, token);
+    try {
+      return UserGroupInformation.getCurrentUser().doAs(
+          new PrivilegedExceptionAction<Long>() {
+            @Override
+            public Long run() throws Exception {
+              return authURL.renewDelegationToken(uri.toURL(), authToken);
+            }
+          }
+      );
+    } catch (Exception ex) {
+      if (ex instanceof IOException) {
+        throw (IOException) ex;
+      } else {
+        throw new IOException(ex);
       }
-    });
+    }
   }
 
   public void cancelDelegationToken(final Token<?> token) throws IOException {
-    HttpFSKerberosAuthenticator.
-      cancelDelegationToken(uri, authToken, token);
+    authURL.cancelDelegationToken(uri.toURL(), authToken);
   }
 
   @Override
   public Token<?> getRenewToken() {
-    return delegationToken;
+    return null; //TODO : for renewer
   }
 
   @Override
+  @SuppressWarnings("unchecked")
   public <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
-    delegationToken = token;
+    //TODO : for renewer
   }
 
+  @Override
+  public void setXAttr(Path f, String name, byte[] value,
+      EnumSet<XAttrSetFlag> flag) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.SETXATTR.toString());
+    params.put(XATTR_NAME_PARAM, name);
+    if (value != null) {
+      params.put(XATTR_VALUE_PARAM, 
+          XAttrCodec.encodeValue(value, XAttrCodec.HEX));
+    }
+    params.put(XATTR_SET_FLAG_PARAM, EnumSetParam.toString(flag));
+    HttpURLConnection conn = getConnection(Operation.SETXATTR.getMethod(),
+        params, f, true);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+  }
+
+  @Override
+  public byte[] getXAttr(Path f, String name) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.GETXATTRS.toString());
+    params.put(XATTR_NAME_PARAM, name);
+    HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
+        params, f, true);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
+    Map<String, byte[]> xAttrs = createXAttrMap(
+        (JSONArray) json.get(XATTRS_JSON));
+    return xAttrs != null ? xAttrs.get(name) : null;
+  }
+
+  /** Convert xAttrs json to xAttrs map */
+  private Map<String, byte[]> createXAttrMap(JSONArray jsonArray) 
+      throws IOException {
+    Map<String, byte[]> xAttrs = Maps.newHashMap();
+    for (Object obj : jsonArray) {
+      JSONObject jsonObj = (JSONObject) obj;
+      final String name = (String)jsonObj.get(XATTR_NAME_JSON);
+      final byte[] value = XAttrCodec.decodeValue(
+          (String)jsonObj.get(XATTR_VALUE_JSON));
+      xAttrs.put(name, value);
+    }
+
+    return xAttrs;
+  }
+
+  /** Convert xAttr names json to names list */
+  private List<String> createXAttrNames(String xattrNamesStr) throws IOException {
+    JSONParser parser = new JSONParser();
+    JSONArray jsonArray;
+    try {
+      jsonArray = (JSONArray)parser.parse(xattrNamesStr);
+      List<String> names = Lists.newArrayListWithCapacity(jsonArray.size());
+      for (Object name : jsonArray) {
+        names.add((String) name);
+      }
+      return names;
+    } catch (ParseException e) {
+      throw new IOException("JSON parser error, " + e.getMessage(), e);
+    }
+  }
+
+  @Override
+  public Map<String, byte[]> getXAttrs(Path f) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.GETXATTRS.toString());
+    HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
+        params, f, true);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
+    return createXAttrMap((JSONArray) json.get(XATTRS_JSON));
+  }
+
+  @Override
+  public Map<String, byte[]> getXAttrs(Path f, List<String> names)
+      throws IOException {
+    Preconditions.checkArgument(names != null && !names.isEmpty(), 
+        "XAttr names cannot be null or empty.");
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.GETXATTRS.toString());
+    Map<String, List<String>> multiValuedParams = Maps.newHashMap();
+    multiValuedParams.put(XATTR_NAME_PARAM, names);
+    HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
+        params, multiValuedParams, f, true);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
+    return createXAttrMap((JSONArray) json.get(XATTRS_JSON));
+  }
+
+  @Override
+  public List<String> listXAttrs(Path f) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.LISTXATTRS.toString());
+    HttpURLConnection conn = getConnection(Operation.LISTXATTRS.getMethod(),
+        params, f, true);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
+    return createXAttrNames((String) json.get(XATTRNAMES_JSON));
+  }
+
+  @Override
+  public void removeXAttr(Path f, String name) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.REMOVEXATTR.toString());
+    params.put(XATTR_NAME_PARAM, name);
+    HttpURLConnection conn = getConnection(Operation.REMOVEXATTR.getMethod(),
+        params, f, true);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+  }
 }

Modified: hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java?rev=1619012&r1=1619011&r2=1619012&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java (original)
+++ hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java Tue Aug 19 23:49:39 2014
@@ -31,6 +31,7 @@ import java.net.URI;
 import java.net.URL;
 import java.net.URLEncoder;
 import java.text.MessageFormat;
+import java.util.List;
 import java.util.Map;
 
 /**
@@ -59,6 +60,24 @@ public class HttpFSUtils {
    */
   static URL createURL(Path path, Map<String, String> params)
     throws IOException {
+    return createURL(path, params, null);
+  }
+
+  /**
+   * Convenience method that creates an HTTP <code>URL</code> for the
+   * HttpFSServer file system operations.
+   * <p/>
+   *
+   * @param path the file path.
+   * @param params the query string parameters.
+   * @param multiValuedParams multi valued parameters of the query string
+   *
+   * @return URL a <code>URL</code> for the HttpFSServer server,
+   *
+   * @throws IOException thrown if an IO error occurs.
+   */
+  static URL createURL(Path path, Map<String, String> params, Map<String, 
+      List<String>> multiValuedParams) throws IOException {
     URI uri = path.toUri();
     String realScheme;
     if (uri.getScheme().equalsIgnoreCase(HttpFSFileSystem.SCHEME)) {
@@ -81,6 +100,18 @@ public class HttpFSUtils {
         append(URLEncoder.encode(entry.getValue(), "UTF8"));
       separator = "&";
     }
+    if (multiValuedParams != null) {
+      for (Map.Entry<String, List<String>> multiValuedEntry : 
+        multiValuedParams.entrySet()) {
+        String name = URLEncoder.encode(multiValuedEntry.getKey(), "UTF8");
+        List<String> values = multiValuedEntry.getValue();
+        for (String value : values) {
+          sb.append(separator).append(name).append("=").
+            append(URLEncoder.encode(value, "UTF8"));
+          separator = "&";
+        }
+      }
+    }
     return new URL(sb.toString());
   }
 
@@ -96,7 +127,7 @@ public class HttpFSUtils {
    * @throws IOException thrown if the current status code does not match the
    * expected one.
    */
-  @SuppressWarnings({"unchecked", "deprecation"})
+  @SuppressWarnings({"unchecked"})
   static void validateResponse(HttpURLConnection conn, int expected)
     throws IOException {
     int status = conn.getResponseCode();

Modified: hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java?rev=1619012&r1=1619011&r2=1619012&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java (original)
+++ hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java Tue Aug 19 23:49:39 2014
@@ -25,8 +25,13 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.GlobFilter;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.XAttrCodec;
+import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.AclException;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.json.simple.JSONArray;
@@ -35,8 +40,11 @@ import org.json.simple.JSONObject;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.EnumSet;
 import java.util.LinkedHashMap;
+import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 
 /**
  * FileSystem operation executors used by {@link HttpFSServer}.
@@ -44,34 +52,170 @@ import java.util.Map;
 @InterfaceAudience.Private
 public class FSOperations {
 
-  @SuppressWarnings({"unchecked", "deprecation"})
-  private static Map fileStatusToJSONRaw(FileStatus status, boolean emptyPathSuffix) {
-    Map json = new LinkedHashMap();
-    json.put(HttpFSFileSystem.PATH_SUFFIX_JSON, (emptyPathSuffix) ? "" : status.getPath().getName());
-    json.put(HttpFSFileSystem.TYPE_JSON, HttpFSFileSystem.FILE_TYPE.getType(status).toString());
-    json.put(HttpFSFileSystem.LENGTH_JSON, status.getLen());
-    json.put(HttpFSFileSystem.OWNER_JSON, status.getOwner());
-    json.put(HttpFSFileSystem.GROUP_JSON, status.getGroup());
-    json.put(HttpFSFileSystem.PERMISSION_JSON, HttpFSFileSystem.permissionToString(status.getPermission()));
-    json.put(HttpFSFileSystem.ACCESS_TIME_JSON, status.getAccessTime());
-    json.put(HttpFSFileSystem.MODIFICATION_TIME_JSON, status.getModificationTime());
-    json.put(HttpFSFileSystem.BLOCK_SIZE_JSON, status.getBlockSize());
-    json.put(HttpFSFileSystem.REPLICATION_JSON, status.getReplication());
-    return json;
+  /**
+   * This class is used to group a FileStatus and an AclStatus together.
+   * It's needed for the GETFILESTATUS and LISTSTATUS calls, which take
+   * most info from the FileStatus and a wee bit from the AclStatus.
+   */
+  private static class StatusPair {
+    private FileStatus fileStatus;
+    private AclStatus aclStatus;
+
+    /**
+     * Simple constructor
+     * @param fileStatus Existing FileStatus object
+     * @param aclStatus Existing AclStatus object
+     */
+    public StatusPair(FileStatus fileStatus, AclStatus aclStatus) {
+      this.fileStatus = fileStatus;
+      this.aclStatus = aclStatus;
+    }
+
+    /**
+     * Create one StatusPair by performing the underlying calls to
+     * fs.getFileStatus and fs.getAclStatus
+     * @param fs The FileSystem where 'path' lives
+     * @param path The file/directory to query
+     * @throws IOException
+     */
+    public StatusPair(FileSystem fs, Path path) throws IOException {
+      fileStatus = fs.getFileStatus(path);
+      aclStatus = null;
+      try {
+        aclStatus = fs.getAclStatus(path);
+      } catch (AclException e) {
+        /*
+         * The cause is almost certainly an "ACLS aren't enabled"
+         * exception, so leave aclStatus at null and carry on.
+         */
+      } catch (UnsupportedOperationException e) {
+        /* Ditto above - this is the case for a local file system */
+      }
+    }
+
+    /**
+     * Return a Map suitable for conversion into JSON format
+     * @return The JSONish Map
+     */
+    public Map<String,Object> toJson() {
+      Map<String,Object> json = new LinkedHashMap<String,Object>();
+      json.put(HttpFSFileSystem.FILE_STATUS_JSON, toJsonInner(true));
+      return json;
+    }
+
+    /**
+     * Return in inner part of the JSON for the status - used by both the
+     * GETFILESTATUS and LISTSTATUS calls.
+     * @param emptyPathSuffix Whether or not to include PATH_SUFFIX_JSON
+     * @return The JSONish Map
+     */
+    public Map<String,Object> toJsonInner(boolean emptyPathSuffix) {
+      Map<String,Object> json = new LinkedHashMap<String,Object>();
+      json.put(HttpFSFileSystem.PATH_SUFFIX_JSON,
+              (emptyPathSuffix) ? "" : fileStatus.getPath().getName());
+      json.put(HttpFSFileSystem.TYPE_JSON,
+              HttpFSFileSystem.FILE_TYPE.getType(fileStatus).toString());
+      json.put(HttpFSFileSystem.LENGTH_JSON, fileStatus.getLen());
+      json.put(HttpFSFileSystem.OWNER_JSON, fileStatus.getOwner());
+      json.put(HttpFSFileSystem.GROUP_JSON, fileStatus.getGroup());
+      json.put(HttpFSFileSystem.PERMISSION_JSON,
+              HttpFSFileSystem.permissionToString(fileStatus.getPermission()));
+      json.put(HttpFSFileSystem.ACCESS_TIME_JSON, fileStatus.getAccessTime());
+      json.put(HttpFSFileSystem.MODIFICATION_TIME_JSON,
+              fileStatus.getModificationTime());
+      json.put(HttpFSFileSystem.BLOCK_SIZE_JSON, fileStatus.getBlockSize());
+      json.put(HttpFSFileSystem.REPLICATION_JSON, fileStatus.getReplication());
+      if ( (aclStatus != null) && !(aclStatus.getEntries().isEmpty()) ) {
+        json.put(HttpFSFileSystem.ACL_BIT_JSON,true);
+      }
+      return json;
+    }
   }
 
   /**
-   * Converts a FileSystemAccess <code>FileStatus</code> object into a JSON
-   * object.
+   * Simple class used to contain and operate upon a list of StatusPair
+   * objects.  Used by LISTSTATUS.
+   */
+  private static class StatusPairs {
+    private StatusPair[] statusPairs;
+
+    /**
+     * Construct a list of StatusPair objects
+     * @param fs The FileSystem where 'path' lives
+     * @param path The directory to query
+     * @param filter A possible filter for entries in the directory
+     * @throws IOException
+     */
+    public StatusPairs(FileSystem fs, Path path, PathFilter filter)
+            throws IOException {
+      /* Grab all the file statuses at once in an array */
+      FileStatus[] fileStatuses = fs.listStatus(path, filter);
+
+      /* We'll have an array of StatusPairs of the same length */
+      AclStatus aclStatus = null;
+      statusPairs = new StatusPair[fileStatuses.length];
+
+      /*
+       * For each FileStatus, attempt to acquire an AclStatus.  If the
+       * getAclStatus throws an exception, we assume that ACLs are turned
+       * off entirely and abandon the attempt.
+       */
+      boolean useAcls = true;   // Assume ACLs work until proven otherwise
+      for (int i = 0; i < fileStatuses.length; i++) {
+        if (useAcls) {
+          try {
+            aclStatus = fs.getAclStatus(fileStatuses[i].getPath());
+          } catch (AclException e) {
+            /* Almost certainly due to an "ACLs not enabled" exception */
+            aclStatus = null;
+            useAcls = false;
+          } catch (UnsupportedOperationException e) {
+            /* Ditto above - this is the case for a local file system */
+            aclStatus = null;
+            useAcls = false;
+          }
+        }
+        statusPairs[i] = new StatusPair(fileStatuses[i], aclStatus);
+      }
+    }
+
+    /**
+     * Return a Map suitable for conversion into JSON.
+     * @return A JSONish Map
+     */
+    @SuppressWarnings({"unchecked"})
+    public Map<String,Object> toJson() {
+      Map<String,Object> json = new LinkedHashMap<String,Object>();
+      Map<String,Object> inner = new LinkedHashMap<String,Object>();
+      JSONArray statuses = new JSONArray();
+      for (StatusPair s : statusPairs) {
+        statuses.add(s.toJsonInner(false));
+      }
+      inner.put(HttpFSFileSystem.FILE_STATUS_JSON, statuses);
+      json.put(HttpFSFileSystem.FILE_STATUSES_JSON, inner);
+      return json;
+    }
+  }
+
+  /** Converts an <code>AclStatus</code> object into a JSON object.
    *
-   * @param status FileSystemAccess file status.
+   * @param aclStatus AclStatus object
    *
-   * @return The JSON representation of the file status.
+   * @return The JSON representation of the ACLs for the file
    */
-  @SuppressWarnings({"unchecked", "deprecation"})
-  private static Map fileStatusToJSON(FileStatus status) {
-    Map json = new LinkedHashMap();
-    json.put(HttpFSFileSystem.FILE_STATUS_JSON, fileStatusToJSONRaw(status, true));
+  @SuppressWarnings({"unchecked"})
+  private static Map<String,Object> aclStatusToJSON(AclStatus aclStatus) {
+    Map<String,Object> json = new LinkedHashMap<String,Object>();
+    Map<String,Object> inner = new LinkedHashMap<String,Object>();
+    JSONArray entriesArray = new JSONArray();
+    inner.put(HttpFSFileSystem.OWNER_JSON, aclStatus.getOwner());
+    inner.put(HttpFSFileSystem.GROUP_JSON, aclStatus.getGroup());
+    inner.put(HttpFSFileSystem.ACL_STICKY_BIT_JSON, aclStatus.isStickyBit());
+    for ( AclEntry e : aclStatus.getEntries() ) {
+      entriesArray.add(e.toString());
+    }
+    inner.put(HttpFSFileSystem.ACL_ENTRIES_JSON, entriesArray);
+    json.put(HttpFSFileSystem.ACL_STATUS_JSON, inner);
     return json;
   }
 
@@ -96,6 +240,50 @@ public class FSOperations {
   }
 
   /**
+   * Converts xAttrs to a JSON object.
+   *
+   * @param xAttrs file xAttrs.
+   * @param encoding format of xattr values.
+   *
+   * @return The JSON representation of the xAttrs.
+   * @throws IOException 
+   */
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  private static Map xAttrsToJSON(Map<String, byte[]> xAttrs, 
+      XAttrCodec encoding) throws IOException {
+    Map jsonMap = new LinkedHashMap();
+    JSONArray jsonArray = new JSONArray();
+    if (xAttrs != null) {
+      for (Entry<String, byte[]> e : xAttrs.entrySet()) {
+        Map json = new LinkedHashMap();
+        json.put(HttpFSFileSystem.XATTR_NAME_JSON, e.getKey());
+        if (e.getValue() != null) {
+          json.put(HttpFSFileSystem.XATTR_VALUE_JSON, 
+              XAttrCodec.encodeValue(e.getValue(), encoding));
+        }
+        jsonArray.add(json);
+      }
+    }
+    jsonMap.put(HttpFSFileSystem.XATTRS_JSON, jsonArray);
+    return jsonMap;
+  }
+
+  /**
+   * Converts xAttr names to a JSON object.
+   *
+   * @param names file xAttr names.
+   *
+   * @return The JSON representation of the xAttr names.
+   * @throws IOException 
+   */
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  private static Map xAttrNamesToJSON(List<String> names) throws IOException {
+    Map jsonMap = new LinkedHashMap();
+    jsonMap.put(HttpFSFileSystem.XATTRNAMES_JSON, JSONArray.toJSONString(names));
+    return jsonMap;
+  }
+
+  /**
    * Converts a <code>ContentSummary</code> object into a JSON array
    * object.
    *
@@ -118,30 +306,6 @@ public class FSOperations {
   }
 
   /**
-   * Converts a FileSystemAccess <code>FileStatus</code> array into a JSON array
-   * object.
-   *
-   * @param status FileSystemAccess file status array.
-   * <code>SCHEME://HOST:PORT</code> in the file status.
-   *
-   * @return The JSON representation of the file status array.
-   */
-  @SuppressWarnings("unchecked")
-  private static Map fileStatusToJSON(FileStatus[] status) {
-    JSONArray json = new JSONArray();
-    if (status != null) {
-      for (FileStatus s : status) {
-        json.add(fileStatusToJSONRaw(s, false));
-      }
-    }
-    Map response = new LinkedHashMap();
-    Map temp = new LinkedHashMap();
-    temp.put(HttpFSFileSystem.FILE_STATUS_JSON, json);
-    response.put(HttpFSFileSystem.FILE_STATUSES_JSON, temp);
-    return response;
-  }
-
-  /**
    * Converts an object into a Json Map with with one key-value entry.
    * <p/>
    * It assumes the given value is either a JSON primitive type or a
@@ -418,18 +582,19 @@ public class FSOperations {
     }
 
     /**
-     * Executes the filesystem operation.
+     * Executes the filesystem getFileStatus operation and returns the
+     * result in a JSONish Map.
      *
      * @param fs filesystem instance to use.
      *
      * @return a Map object (JSON friendly) with the file status.
      *
-     * @throws IOException thrown if an IO error occured.
+     * @throws IOException thrown if an IO error occurred.
      */
     @Override
     public Map execute(FileSystem fs) throws IOException {
-      FileStatus status = fs.getFileStatus(path);
-      return fileStatusToJSON(status);
+      StatusPair sp = new StatusPair(fs, path);
+      return sp.toJson();
     }
 
   }
@@ -482,19 +647,20 @@ public class FSOperations {
     }
 
     /**
-     * Executes the filesystem operation.
+     * Returns data for a JSON Map containing the information for
+     * the set of files in 'path' that match 'filter'.
      *
      * @param fs filesystem instance to use.
      *
      * @return a Map with the file status of the directory
-     *         contents.
+     *         contents that match the filter
      *
-     * @throws IOException thrown if an IO error occured.
+     * @throws IOException thrown if an IO error occurred.
      */
     @Override
     public Map execute(FileSystem fs) throws IOException {
-      FileStatus[] status = fs.listStatus(path, filter);
-      return fileStatusToJSON(status);
+      StatusPairs sp = new StatusPairs(fs, path, filter);
+      return sp.toJson();
     }
 
     @Override
@@ -691,6 +857,218 @@ public class FSOperations {
   }
 
   /**
+   * Executor that sets the acl for a file in a FileSystem
+   */
+  @InterfaceAudience.Private
+  public static class FSSetAcl implements FileSystemAccess.FileSystemExecutor<Void> {
+
+    private Path path;
+    private List<AclEntry> aclEntries;
+
+    /**
+     * Creates a set-acl executor.
+     *
+     * @param path path to set the acl.
+     * @param aclSpec acl to set.
+     */
+    public FSSetAcl(String path, String aclSpec) {
+      this.path = new Path(path);
+      this.aclEntries = AclEntry.parseAclSpec(aclSpec, true);
+    }
+
+    /**
+     * Executes the filesystem operation.
+     *
+     * @param fs filesystem instance to use.
+     *
+     * @return void.
+     *
+     * @throws IOException thrown if an IO error occurred.
+     */
+    @Override
+    public Void execute(FileSystem fs) throws IOException {
+      fs.setAcl(path, aclEntries);
+      return null;
+    }
+
+  }
+
+  /**
+   * Executor that removes all acls from a file in a FileSystem
+   */
+  @InterfaceAudience.Private
+  public static class FSRemoveAcl implements FileSystemAccess.FileSystemExecutor<Void> {
+
+    private Path path;
+
+    /**
+     * Creates a remove-acl executor.
+     *
+     * @param path path from which to remove the acl.
+     */
+    public FSRemoveAcl(String path) {
+      this.path = new Path(path);
+    }
+
+    /**
+     * Executes the filesystem operation.
+     *
+     * @param fs filesystem instance to use.
+     *
+     * @return void.
+     *
+     * @throws IOException thrown if an IO error occurred.
+     */
+    @Override
+    public Void execute(FileSystem fs) throws IOException {
+      fs.removeAcl(path);
+      return null;
+    }
+
+  }
+
+  /**
+   * Executor that modifies acl entries for a file in a FileSystem
+   */
+  @InterfaceAudience.Private
+  public static class FSModifyAclEntries implements FileSystemAccess.FileSystemExecutor<Void> {
+
+    private Path path;
+    private List<AclEntry> aclEntries;
+
+    /**
+     * Creates a modify-acl executor.
+     *
+     * @param path path to set the acl.
+     * @param aclSpec acl to set.
+     */
+    public FSModifyAclEntries(String path, String aclSpec) {
+      this.path = new Path(path);
+      this.aclEntries = AclEntry.parseAclSpec(aclSpec, true);
+    }
+
+    /**
+     * Executes the filesystem operation.
+     *
+     * @param fs filesystem instance to use.
+     *
+     * @return void.
+     *
+     * @throws IOException thrown if an IO error occurred.
+     */
+    @Override
+    public Void execute(FileSystem fs) throws IOException {
+      fs.modifyAclEntries(path, aclEntries);
+      return null;
+    }
+
+  }
+
+  /**
+   * Executor that removes acl entries from a file in a FileSystem
+   */
+  @InterfaceAudience.Private
+  public static class FSRemoveAclEntries implements FileSystemAccess.FileSystemExecutor<Void> {
+
+    private Path path;
+    private List<AclEntry> aclEntries;
+
+    /**
+     * Creates a remove acl entry executor.
+     *
+     * @param path path to set the acl.
+     * @param aclSpec acl parts to remove.
+     */
+    public FSRemoveAclEntries(String path, String aclSpec) {
+      this.path = new Path(path);
+      this.aclEntries = AclEntry.parseAclSpec(aclSpec, true);
+    }
+
+    /**
+     * Executes the filesystem operation.
+     *
+     * @param fs filesystem instance to use.
+     *
+     * @return void.
+     *
+     * @throws IOException thrown if an IO error occurred.
+     */
+    @Override
+    public Void execute(FileSystem fs) throws IOException {
+      fs.removeAclEntries(path, aclEntries);
+      return null;
+    }
+
+  }
+
+  /**
+   * Executor that removes the default acl from a directory in a FileSystem
+   */
+  @InterfaceAudience.Private
+  public static class FSRemoveDefaultAcl implements FileSystemAccess.FileSystemExecutor<Void> {
+
+    private Path path;
+
+    /**
+     * Creates an executor for removing the default acl.
+     *
+     * @param path path to set the acl.
+     */
+    public FSRemoveDefaultAcl(String path) {
+      this.path = new Path(path);
+    }
+
+    /**
+     * Executes the filesystem operation.
+     *
+     * @param fs filesystem instance to use.
+     *
+     * @return void.
+     *
+     * @throws IOException thrown if an IO error occurred.
+     */
+    @Override
+    public Void execute(FileSystem fs) throws IOException {
+      fs.removeDefaultAcl(path);
+      return null;
+    }
+
+  }
+
+  /**
+   * Executor that gets the ACL information for a given file.
+   */
+  @InterfaceAudience.Private
+  public static class FSAclStatus implements FileSystemAccess.FileSystemExecutor<Map> {
+    private Path path;
+
+    /**
+     * Creates an executor for getting the ACLs for a file.
+     *
+     * @param path the path to retrieve the ACLs.
+     */
+    public FSAclStatus(String path) {
+      this.path = new Path(path);
+    }
+
+    /**
+     * Executes the filesystem operation.
+     *
+     * @param fs filesystem instance to use.
+     *
+     * @return a Map object (JSON friendly) with the file status.
+     *
+     * @throws IOException thrown if an IO error occurred.
+     */
+    @Override
+    public Map execute(FileSystem fs) throws IOException {
+      AclStatus status = fs.getAclStatus(path);
+      return aclStatusToJSON(status);
+    }
+
+  }
+
+  /**
    * Executor that performs a set-replication FileSystemAccess files system operation.
    */
   @InterfaceAudience.Private
@@ -769,4 +1147,132 @@ public class FSOperations {
 
   }
 
+  /**
+   * Executor that performs a setxattr FileSystemAccess files system operation.
+   */
+  @InterfaceAudience.Private
+  public static class FSSetXAttr implements 
+      FileSystemAccess.FileSystemExecutor<Void> {
+
+    private Path path;
+    private String name;
+    private byte[] value;
+    private EnumSet<XAttrSetFlag> flag;
+
+    public FSSetXAttr(String path, String name, String encodedValue, 
+        EnumSet<XAttrSetFlag> flag) throws IOException {
+      this.path = new Path(path);
+      this.name = name;
+      this.value = XAttrCodec.decodeValue(encodedValue);
+      this.flag = flag;
+    }
+
+    @Override
+    public Void execute(FileSystem fs) throws IOException {
+      fs.setXAttr(path, name, value, flag);
+      return null;
+    }
+  }
+
+  /**
+   * Executor that performs a removexattr FileSystemAccess files system 
+   * operation.
+   */
+  @InterfaceAudience.Private
+  public static class FSRemoveXAttr implements 
+      FileSystemAccess.FileSystemExecutor<Void> {
+
+    private Path path;
+    private String name;
+
+    public FSRemoveXAttr(String path, String name) {
+      this.path = new Path(path);
+      this.name = name;
+    }
+
+    @Override
+    public Void execute(FileSystem fs) throws IOException {
+      fs.removeXAttr(path, name);
+      return null;
+    }
+  }
+
+  /**
+   * Executor that performs listing xattrs FileSystemAccess files system 
+   * operation.
+   */
+  @SuppressWarnings("rawtypes")
+  @InterfaceAudience.Private
+  public static class FSListXAttrs implements 
+      FileSystemAccess.FileSystemExecutor<Map> {
+    private Path path;
+
+    /**
+     * Creates listing xattrs executor.
+     *
+     * @param path the path to retrieve the xattrs.
+     */
+    public FSListXAttrs(String path) {
+      this.path = new Path(path);
+    }
+
+    /**
+     * Executes the filesystem operation.
+     *
+     * @param fs filesystem instance to use.
+     *
+     * @return Map a map object (JSON friendly) with the xattr names.
+     *
+     * @throws IOException thrown if an IO error occured.
+     */
+    @Override
+    public Map execute(FileSystem fs) throws IOException {
+      List<String> names = fs.listXAttrs(path);
+      return xAttrNamesToJSON(names);
+    }
+  }
+
+  /**
+   * Executor that performs getting xattrs FileSystemAccess files system 
+   * operation.
+   */
+  @SuppressWarnings("rawtypes")
+  @InterfaceAudience.Private
+  public static class FSGetXAttrs implements 
+      FileSystemAccess.FileSystemExecutor<Map> {
+    private Path path;
+    private List<String> names;
+    private XAttrCodec encoding;
+
+    /**
+     * Creates getting xattrs executor.
+     *
+     * @param path the path to retrieve the xattrs.
+     */
+    public FSGetXAttrs(String path, List<String> names, XAttrCodec encoding) {
+      this.path = new Path(path);
+      this.names = names;
+      this.encoding = encoding;
+    }
+
+    /**
+     * Executes the filesystem operation.
+     *
+     * @param fs filesystem instance to use.
+     *
+     * @return Map a map object (JSON friendly) with the xattrs.
+     *
+     * @throws IOException thrown if an IO error occured.
+     */
+    @Override
+    public Map execute(FileSystem fs) throws IOException {
+      Map<String, byte[]> xattrs = null;
+      if (names != null && !names.isEmpty()) {
+        xattrs = fs.getXAttrs(path, names);
+      } else {
+        xattrs = fs.getXAttrs(path);
+      }
+      return xAttrsToJSON(xattrs, encoding);
+    }
+  }
 }

Modified: hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java?rev=1619012&r1=1619011&r2=1619012&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java (original)
+++ hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java Tue Aug 19 23:49:39 2014
@@ -18,27 +18,32 @@
 package org.apache.hadoop.fs.http.server;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.XAttrCodec;
+import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
 import org.apache.hadoop.fs.http.client.HttpFSFileSystem.Operation;
 import org.apache.hadoop.lib.wsrs.BooleanParam;
 import org.apache.hadoop.lib.wsrs.EnumParam;
+import org.apache.hadoop.lib.wsrs.EnumSetParam;
 import org.apache.hadoop.lib.wsrs.LongParam;
 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;
 import java.util.Map;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT;
 
 /**
  * HttpFS ParametersProvider.
  */
 @Provider
 @InterfaceAudience.Private
+@SuppressWarnings("unchecked")
 public class HttpFSParametersProvider extends ParametersProvider {
 
   private static final Map<Enum, Class<Param<?>>[]> PARAMS_DEF =
@@ -46,37 +51,44 @@ public class HttpFSParametersProvider ex
 
   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.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});
+        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[]{AclPermissionParam.class});
+    PARAMS_DEF.put(Operation.REMOVEACLENTRIES,
+        new Class[]{AclPermissionParam.class});
+    PARAMS_DEF.put(Operation.REMOVEDEFAULTACL, new Class[]{});
+    PARAMS_DEF.put(Operation.SETXATTR,
+        new Class[]{XAttrNameParam.class, XAttrValueParam.class,
+                  XAttrSetFlagParam.class});
+    PARAMS_DEF.put(Operation.REMOVEXATTR, new Class[]{XAttrNameParam.class});
+    PARAMS_DEF.put(Operation.GETXATTRS, 
+        new Class[]{XAttrNameParam.class, XAttrEncodingParam.class});
+    PARAMS_DEF.put(Operation.LISTXATTRS, new Class[]{});
   }
 
   public HttpFSParametersProvider() {
@@ -179,41 +191,6 @@ public class HttpFSParametersProvider ex
   }
 
   /**
-   * 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.
    */
   @InterfaceAudience.Private
@@ -248,7 +225,7 @@ public class HttpFSParametersProvider ex
      * Constructor.
      */
     public GroupParam() {
-      super(NAME, null, UserProvider.getUserPattern());
+      super(NAME, null);
     }
 
   }
@@ -344,7 +321,7 @@ public class HttpFSParametersProvider ex
      * Constructor.
      */
     public OwnerParam() {
-      super(NAME, null, UserProvider.getUserPattern());
+      super(NAME, null);
     }
 
   }
@@ -371,6 +348,26 @@ public class HttpFSParametersProvider ex
   }
 
   /**
+   * Class for AclPermission parameter.
+   */
+  @InterfaceAudience.Private
+  public static class AclPermissionParam extends StringParam {
+
+    /**
+     * Parameter name.
+     */
+    public static final String NAME = HttpFSFileSystem.ACLSPEC_PARAM;
+
+    /**
+     * Constructor.
+     */
+    public AclPermissionParam() {
+      super(NAME, HttpFSFileSystem.ACLSPEC_DEFAULT,
+              Pattern.compile(DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT));
+    }
+  }
+
+  /**
    * Class for replication parameter.
    */
   @InterfaceAudience.Private
@@ -426,4 +423,79 @@ public class HttpFSParametersProvider ex
       super(NAME, null);
     }
   }
+  
+  /**
+   * Class for xattr parameter.
+   */
+  @InterfaceAudience.Private
+  public static class XAttrNameParam extends StringParam {
+    public static final String XATTR_NAME_REGX = 
+        "^(user\\.|trusted\\.|system\\.|security\\.).+";
+    /**
+     * Parameter name.
+     */
+    public static final String NAME = HttpFSFileSystem.XATTR_NAME_PARAM;
+    private static final Pattern pattern = Pattern.compile(XATTR_NAME_REGX);
+
+    /**
+     * Constructor.
+     */
+    public XAttrNameParam() {
+      super(NAME, null, pattern);
+    }
+  }
+
+  /**
+   * Class for xattr parameter.
+   */
+  @InterfaceAudience.Private
+  public static class XAttrValueParam extends StringParam {
+    /**
+     * Parameter name.
+     */
+    public static final String NAME = HttpFSFileSystem.XATTR_VALUE_PARAM;
+
+    /**
+     * Constructor.
+     */
+    public XAttrValueParam() {
+      super(NAME, null);
+    }
+  }
+
+  /**
+   * Class for xattr parameter.
+   */
+  @InterfaceAudience.Private
+  public static class XAttrSetFlagParam extends EnumSetParam<XAttrSetFlag> {
+    /**
+     * Parameter name.
+     */
+    public static final String NAME = HttpFSFileSystem.XATTR_SET_FLAG_PARAM;
+
+    /**
+     * Constructor.
+     */
+    public XAttrSetFlagParam() {
+      super(NAME, XAttrSetFlag.class, null);
+    }
+  }
+
+  /**
+   * Class for xattr parameter.
+   */
+  @InterfaceAudience.Private
+  public static class XAttrEncodingParam extends EnumParam<XAttrCodec> {
+    /**
+     * Parameter name.
+     */
+    public static final String NAME = HttpFSFileSystem.XATTR_ENCODING_PARAM;
+
+    /**
+     * Constructor.
+     */
+    public XAttrEncodingParam() {
+      super(NAME, XAttrCodec.class, null);
+    }
+  }
 }

Modified: hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java?rev=1619012&r1=1619011&r2=1619012&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java (original)
+++ hadoop/common/branches/HADOOP-10388/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java Tue Aug 19 23:49:39 2014
@@ -21,12 +21,14 @@ package org.apache.hadoop.fs.http.server
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.XAttrCodec;
+import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.AccessTimeParam;
+import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.AclPermissionParam;
 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;
@@ -39,16 +41,19 @@ import org.apache.hadoop.fs.http.server.
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.RecursiveParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.ReplicationParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.SourcesParam;
+import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrEncodingParam;
+import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrNameParam;
+import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrSetFlagParam;
+import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrValueParam;
 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;
@@ -72,8 +77,8 @@ 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;
 import java.util.Map;
 
@@ -89,48 +94,10 @@ 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
@@ -139,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);
   }
 
   /**
@@ -154,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.
    *
@@ -164,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);
@@ -184,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.
    *
@@ -198,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) {
@@ -212,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.
@@ -228,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);
@@ -256,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;
@@ -265,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();
@@ -274,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;
@@ -282,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");
@@ -296,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;
@@ -304,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;
@@ -313,6 +275,35 @@ public class HttpFSServer {
         response = Response.status(Response.Status.BAD_REQUEST).build();
         break;
       }
+      case GETACLSTATUS: {
+        FSOperations.FSAclStatus command =
+                new FSOperations.FSAclStatus(path);
+        Map json = fsExecute(user, command);
+        AUDIT_LOG.info("ACL status for [{}]", path);
+        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+        break;
+      }
+      case GETXATTRS: {
+        List<String> xattrNames = params.getValues(XAttrNameParam.NAME, 
+            XAttrNameParam.class);
+        XAttrCodec encoding = params.get(XAttrEncodingParam.NAME, 
+            XAttrEncodingParam.class);
+        FSOperations.FSGetXAttrs command = new FSOperations.FSGetXAttrs(path, 
+            xattrNames, encoding);
+        @SuppressWarnings("rawtypes")
+        Map json = fsExecute(user, command);
+        AUDIT_LOG.info("XAttrs for [{}]", path);
+        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+        break;
+      }
+      case LISTXATTRS: {
+        FSOperations.FSListXAttrs command = new FSOperations.FSListXAttrs(path);
+        @SuppressWarnings("rawtypes")
+        Map json = fsExecute(user, command);
+        AUDIT_LOG.info("XAttr names for [{}]", path);
+        response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+        break;
+      }
       default: {
         throw new IOException(
           MessageFormat.format("Invalid HTTP GET operation [{0}]",
@@ -326,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.
@@ -342,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 =
@@ -358,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;
       }
@@ -375,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.
@@ -394,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(
@@ -414,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();
         }
@@ -426,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();
@@ -461,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.
@@ -480,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);
@@ -509,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});
@@ -517,12 +502,36 @@ public class HttpFSServer {
         }
         break;
       }
+      case SETXATTR: {
+        String xattrName = params.get(XAttrNameParam.NAME, 
+            XAttrNameParam.class);
+        String xattrValue = params.get(XAttrValueParam.NAME, 
+            XAttrValueParam.class);
+        EnumSet<XAttrSetFlag> flag = params.get(XAttrSetFlagParam.NAME, 
+            XAttrSetFlagParam.class);
+
+        FSOperations.FSSetXAttr command = new FSOperations.FSSetXAttr(
+            path, xattrName, xattrValue, flag);
+        fsExecute(user, command);
+        AUDIT_LOG.info("[{}] to xAttr [{}]", path, xattrName);
+        response = Response.ok().build();
+        break;
+      }
+      case REMOVEXATTR: {
+        String xattrName = params.get(XAttrNameParam.NAME, XAttrNameParam.class);
+        FSOperations.FSRemoveXAttr command = new FSOperations.FSRemoveXAttr(
+            path, xattrName);
+        fsExecute(user, command);
+        AUDIT_LOG.info("[{}] removed xAttr [{}]", path, xattrName);
+        response = Response.ok().build();
+        break;
+      }
       case MKDIRS: {
         Short permission = params.get(PermissionParam.NAME,
                                        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;
@@ -531,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;
@@ -541,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;
@@ -551,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;
@@ -561,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;
@@ -573,12 +582,58 @@ 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();
         break;
       }
+      case SETACL: {
+        String aclSpec = params.get(AclPermissionParam.NAME,
+                AclPermissionParam.class);
+        FSOperations.FSSetAcl command =
+                new FSOperations.FSSetAcl(path, aclSpec);
+        fsExecute(user, command);
+        AUDIT_LOG.info("[{}] to acl [{}]", path, aclSpec);
+        response = Response.ok().build();
+        break;
+      }
+      case REMOVEACL: {
+        FSOperations.FSRemoveAcl command =
+                new FSOperations.FSRemoveAcl(path);
+        fsExecute(user, command);
+        AUDIT_LOG.info("[{}] removed acl", path);
+        response = Response.ok().build();
+        break;
+      }
+      case MODIFYACLENTRIES: {
+        String aclSpec = params.get(AclPermissionParam.NAME,
+                AclPermissionParam.class);
+        FSOperations.FSModifyAclEntries command =
+                new FSOperations.FSModifyAclEntries(path, aclSpec);
+        fsExecute(user, command);
+        AUDIT_LOG.info("[{}] modify acl entry with [{}]", path, aclSpec);
+        response = Response.ok().build();
+        break;
+      }
+      case REMOVEACLENTRIES: {
+        String aclSpec = params.get(AclPermissionParam.NAME,
+                AclPermissionParam.class);
+        FSOperations.FSRemoveAclEntries command =
+                new FSOperations.FSRemoveAclEntries(path, aclSpec);
+        fsExecute(user, command);
+        AUDIT_LOG.info("[{}] remove acl entry [{}]", path, aclSpec);
+        response = Response.ok().build();
+        break;
+      }
+      case REMOVEDEFAULTACL: {
+        FSOperations.FSRemoveDefaultAcl command =
+                new FSOperations.FSRemoveDefaultAcl(path);
+        fsExecute(user, command);
+        AUDIT_LOG.info("[{}] remove default acl", path);
+        response = Response.ok().build();
+        break;
+      }
       default: {
         throw new IOException(
           MessageFormat.format("Invalid HTTP PUT operation [{0}]",