Explorar o código

Merged from trunk to branch-2. HDFS-6375. Listing extended attributes with the search permission. Contributed by Charles Lamb

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1602133 13f79535-47bb-0310-9956-ffa450edef68
Uma Maheswara Rao G %!s(int64=11) %!d(string=hai) anos
pai
achega
d59ca5c4e1
Modificáronse 24 ficheiros con 385 adicións e 10 borrados
  1. 24 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
  2. 34 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  3. 5 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
  4. 5 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java
  5. 5 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java
  6. 7 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
  7. 2 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
  8. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  9. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java
  10. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  11. 17 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  12. 21 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  13. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  14. 14 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  15. 17 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  16. 24 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  17. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  18. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  19. 31 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  20. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  21. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
  22. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  23. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/xattr.proto
  24. 111 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSXAttrBaseTest.java

+ 24 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java

@@ -1168,6 +1168,30 @@ public abstract class AbstractFileSystem {
         + " doesn't support getXAttrs");
   }
 
+  /**
+   * Get all of the xattr names for a file or directory.
+   * Only the xattr names for which the logged-in user has permissions to view
+   * are returned.
+   * <p/>
+   * A regular user can only get xattr names for the "user" namespace.
+   * The super user can only get xattr names for the "user" and "trusted"
+   * namespaces.
+   * The xattr names in the "security" and "system" namespaces are only
+   * used/exposed internally by/to the FS impl.
+   * <p/>
+   * @see <a href="http://en.wikipedia.org/wiki/Extended_file_attributes">
+   * http://en.wikipedia.org/wiki/Extended_file_attributes</a>
+   *
+   * @param path Path to get extended attributes
+   * @return Map<String, byte[]> describing the XAttrs of the file or directory
+   * @throws IOException
+   */
+  public List<String> listXAttrs(Path path)
+          throws IOException {
+    throw new UnsupportedOperationException(getClass().getSimpleName()
+            + " doesn't support listXAttrs");
+  }
+
   /**
    * Remove an xattr of a file or directory.
    * The name must be prefixed with user/trusted/security/system and

+ 34 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java

@@ -2404,8 +2404,8 @@ public abstract class FileSystem extends Configured implements Closeable {
    * <p/>
    * The access permissions of an xattr in the "user" namespace are
    * defined by the file and directory permission bits.
-   * An xattr can only be set when the logged-in user has the correct permissions.
-   * If the xattr exists, it will be replaced.
+   * An xattr can only be set if the logged-in user has the correct permissions.
+   * If the xattr exists, it is replaced.
    * <p/>
    * @see <a href="http://en.wikipedia.org/wiki/Extended_file_attributes">
    * http://en.wikipedia.org/wiki/Extended_file_attributes</a>
@@ -2423,7 +2423,7 @@ public abstract class FileSystem extends Configured implements Closeable {
   }
 
   /**
-   * Get an xattr for a file or directory.
+   * Get an xattr name and value for a file or directory.
    * The name must be prefixed with user/trusted/security/system and
    * followed by ".". For example, "user.attr".
    * <p/>
@@ -2433,7 +2433,8 @@ public abstract class FileSystem extends Configured implements Closeable {
    * The xattrs of the "security" and "system" namespaces are only used/exposed 
    * internally by/to the FS impl.
    * <p/>
-   * An xattr will only be returned when the logged-in user has the correct permissions.
+   * An xattr will only be returned if the logged-in user has the
+   * correct permissions.
    * <p/>
    * @see <a href="http://en.wikipedia.org/wiki/Extended_file_attributes">
    * http://en.wikipedia.org/wiki/Extended_file_attributes</a>
@@ -2449,13 +2450,13 @@ public abstract class FileSystem extends Configured implements Closeable {
   }
 
   /**
-   * Get all of the xattrs for a file or directory.
-   * Only those xattrs for which the logged-in user has permissions to view
+   * Get all of the xattr name/value pairs for a file or directory.
+   * Only those xattrs which the logged-in user has permissions to view
    * are returned.
    * <p/>
    * A regular user can only get xattrs for the "user" namespace.
    * The super user can only get xattrs for "user" and "trusted" namespaces.
-   * The xattr of "security" and "system" namespaces are only used/exposed 
+   * The xattrs of the "security" and "system" namespaces are only used/exposed
    * internally by/to the FS impl.
    * <p/>
    * @see <a href="http://en.wikipedia.org/wiki/Extended_file_attributes">
@@ -2471,13 +2472,13 @@ public abstract class FileSystem extends Configured implements Closeable {
   }
 
   /**
-   * Get all of the xattrs for a file or directory.
-   * Only those xattrs for which the logged-in user has permissions to view
+   * Get all of the xattrs name/value pairs for a file or directory.
+   * Only those xattrs which the logged-in user has permissions to view
    * are returned.
    * <p/>
    * A regular user can only get xattrs for the "user" namespace.
    * The super user can only get xattrs for "user" and "trusted" namespaces.
-   * The xattr of "security" and "system" namespaces are only used/exposed 
+   * The xattrs of the "security" and "system" namespaces are only used/exposed
    * internally by/to the FS impl.
    * <p/>
    * @see <a href="http://en.wikipedia.org/wiki/Extended_file_attributes">
@@ -2494,6 +2495,29 @@ public abstract class FileSystem extends Configured implements Closeable {
         + " doesn't support getXAttrs");
   }
 
+  /**
+   * Get all of the xattr names for a file or directory.
+   * Only those xattr names which the logged-in user has permissions to view
+   * are returned.
+   * <p/>
+   * A regular user can only get xattr names for the "user" namespace.
+   * The super user can only get xattr names for "user" and "trusted"
+   * namespaces.
+   * The xattrs of the "security" and "system" namespaces are only
+   * used/exposed internally by/to the FS impl.
+   * <p/>
+   * @see <a href="http://en.wikipedia.org/wiki/Extended_file_attributes">
+   * http://en.wikipedia.org/wiki/Extended_file_attributes</a>
+   *
+   * @param path Path to get extended attributes
+   * @return Map<String, byte[]> describing the XAttrs of the file or directory
+   * @throws IOException
+   */
+  public List<String> listXAttrs(Path path) throws IOException {
+    throw new UnsupportedOperationException(getClass().getSimpleName()
+            + " doesn't support listXAttrs");
+  }
+
   /**
    * Remove an xattr of a file or directory.
    * The name must be prefixed with user/trusted/security/system and

+ 5 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java

@@ -572,6 +572,11 @@ public class FilterFileSystem extends FileSystem {
     return fs.getXAttrs(path, names);
   }
 
+  @Override
+  public List<String> listXAttrs(Path path) throws IOException {
+    return fs.listXAttrs(path);
+  }
+
   @Override
   public void removeXAttr(Path path, String name) throws IOException {
     fs.removeXAttr(path, name);

+ 5 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFs.java

@@ -346,6 +346,11 @@ public abstract class FilterFs extends AbstractFileSystem {
     return myFs.getXAttrs(path, names);
   }
 
+  @Override
+  public List<String> listXAttrs(Path path) throws IOException {
+    return myFs.listXAttrs(path);
+  }
+
   @Override
   public void removeXAttr(Path path, String name) throws IOException {
     myFs.removeXAttr(path, name);

+ 5 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ChRootedFileSystem.java

@@ -337,6 +337,11 @@ class ChRootedFileSystem extends FilterFileSystem {
     return super.getXAttrs(fullPath(path), names);
   }
 
+  @Override
+  public List<String> listXAttrs(Path path) throws IOException {
+    return super.listXAttrs(fullPath(path));
+  }
+
   @Override
   public void removeXAttr(Path path, String name) throws IOException {
     super.removeXAttr(fullPath(path), name);

+ 7 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java

@@ -551,6 +551,13 @@ public class ViewFileSystem extends FileSystem {
     return res.targetFileSystem.getXAttrs(res.remainingPath, names);
   }
 
+  @Override
+  public List<String> listXAttrs(Path path) throws IOException {
+    InodeTree.ResolveResult<FileSystem> res =
+      fsState.resolve(getUriPath(path), true);
+    return res.targetFileSystem.listXAttrs(res.remainingPath);
+  }
+
   @Override
   public void removeXAttr(Path path, String name) throws IOException {
     InodeTree.ResolveResult<FileSystem> res = fsState.resolve(getUriPath(path),

+ 2 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java

@@ -196,6 +196,8 @@ public class TestHarFileSystem {
     public Map<String, byte[]> getXAttrs(Path path, List<String> names)
         throws IOException;
 
+    public List<String> listXAttrs(Path path) throws IOException;
+
     public void removeXAttr(Path path, String name) throws IOException;
 
     public AclStatus getAclStatus(Path path) throws IOException;

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

@@ -358,6 +358,9 @@ Release 2.5.0 - UNRELEASED
     HDFS-6464. Support multiple xattr.name parameters for WebHDFS getXAttrs.
     (Yi Liu via umamahesh)
 
+    HDFS-6375. Listing extended attributes with the search permission.
+    (Charles Lamb via wang)
+
   BREAKDOWN OF HDFS-2006 SUBTASKS AND RELATED JIRAS
 
     HDFS-6299. Protobuf for XAttr and client-side implementation. (Yi Liu via umamahesh)

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java

@@ -437,6 +437,11 @@ public class Hdfs extends AbstractFileSystem {
       throws IOException {
     return dfs.getXAttrs(getUriPath(path), names);
   }
+
+  @Override
+  public List<String> listXAttrs(Path path) throws IOException {
+    return dfs.listXAttrs(getUriPath(path));
+  }
   
   @Override
   public void removeXAttr(Path path, String name) throws IOException {

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -91,6 +91,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.net.SocketFactory;
 
+import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -2845,6 +2846,20 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
     }
   }
   
+  public List<String> listXAttrs(String src)
+          throws IOException {
+    checkOpen();
+    try {
+      final Map<String, byte[]> xattrs =
+        XAttrHelper.buildXAttrMap(namenode.listXAttrs(src));
+      return Lists.newArrayList(xattrs.keySet());
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+                                     FileNotFoundException.class,
+                                     UnresolvedPathException.class);
+    }
+  }
+
   public void removeXAttr(String src, String name) throws IOException {
     checkOpen();
     try {

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -1912,6 +1912,23 @@ public class DistributedFileSystem extends FileSystem {
     }.resolve(this, absF);
   }
   
+  @Override
+  public List<String> listXAttrs(Path path)
+          throws IOException {
+    final Path absF = fixRelativePart(path);
+    return new FileSystemLinkResolver<List<String>>() {
+      @Override
+      public List<String> doCall(final Path p) throws IOException {
+        return dfs.listXAttrs(getPathName(p));
+      }
+      @Override
+      public List<String> next(final FileSystem fs, final Path p)
+              throws IOException, UnresolvedLinkException {
+        return fs.listXAttrs(p);
+      }
+    }.resolve(this, absF);
+  }
+
   @Override
   public void removeXAttr(Path path, final String name) throws IOException {
     Path absF = fixRelativePart(path);

+ 21 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -1300,6 +1300,27 @@ public interface ClientProtocol {
   @Idempotent
   public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs) 
       throws IOException;
+
+  /**
+   * List the xattrs names for a file or directory.
+   * Only the xattr names for which the logged in user has the permissions to
+   * access will be returned.
+   * <p/>
+   * A regular user only can get xattr names from the "user" namespace.
+   * A super user can get xattr names of the "user" and "trusted" namespace.
+   * XAttr names of the "security" and "system" namespaces are only used/exposed
+   * internally by the file system impl.
+   * <p/>
+   * @see <a href="http://en.wikipedia.org/wiki/Extended_file_attributes">
+   * http://en.wikipedia.org/wiki/Extended_file_attributes</a>
+   * @param src file or directory
+   * @param xAttrs xAttrs to get
+   * @return List<XAttr> <code>XAttr</code> list
+   * @throws IOException
+   */
+  @Idempotent
+  public List<XAttr> listXAttrs(String src)
+      throws IOException;
   
   /**
    * Remove xattr of a file or directory.Value in xAttr parameter is ignored.

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -174,8 +174,11 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
@@ -1297,6 +1300,16 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public ListXAttrsResponseProto listXAttrs(RpcController controller,
+    ListXAttrsRequestProto req) throws ServiceException {
+    try {
+      return PBHelper.convertListXAttrsResponse(server.listXAttrs(req.getSrc()));
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
   
   @Override
   public RemoveXAttrResponseProto removeXAttr(RpcController controller,

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -144,6 +144,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTim
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
@@ -1305,6 +1306,19 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
   }
   
+  @Override
+  public List<XAttr> listXAttrs(String src)
+      throws IOException {
+    ListXAttrsRequestProto.Builder builder = ListXAttrsRequestProto.newBuilder();
+    builder.setSrc(src);
+    ListXAttrsRequestProto req = builder.build();
+    try {
+      return PBHelper.convert(rpcProxy.listXAttrs(null, req));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
   @Override
   public void removeXAttr(String src, XAttr xAttr) throws IOException {
     RemoveXAttrRequestProto req = RemoveXAttrRequestProto

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -152,6 +153,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrProto.XAttrNamespaceProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrSetFlagProto;
@@ -2178,6 +2180,21 @@ public class PBHelper {
     return builder.build();
   }
 
+  public static List<XAttr> convert(ListXAttrsResponseProto a) {
+    final List<XAttrProto> xAttrs = a.getXAttrsList();
+    return convertXAttrs(xAttrs);
+  }
+
+  public static ListXAttrsResponseProto convertListXAttrsResponse(
+    List<XAttr> names) {
+    ListXAttrsResponseProto.Builder builder =
+      ListXAttrsResponseProto.newBuilder();
+    if (names != null) {
+      builder.addAllXAttrs(convertXAttrProto(names));
+    }
+    return builder.build();
+  }
+
   public static ShortCircuitShmSlotProto convert(SlotId slotId) {
     return ShortCircuitShmSlotProto.newBuilder().
         setShmId(convert(slotId.getShmId())).

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -148,6 +148,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
@@ -8025,6 +8026,29 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       readUnlock();
     }
   }
+
+  List<XAttr> listXAttrs(String src) throws IOException {
+    nnConf.checkXAttrsConfigFlag();
+    final FSPermissionChecker pc = getPermissionChecker();
+    checkOperation(OperationCategory.READ);
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      if (isPermissionEnabled) {
+        /* To access xattr names, you need EXECUTE in the owning directory. */
+        checkParentAccess(pc, src, FsAction.EXECUTE);
+      }
+      final List<XAttr> all = dir.getXAttrs(src);
+      final List<XAttr> filteredAll = XAttrPermissionFilter.
+        filterXAttrsForApi(pc, all);
+      return filteredAll;
+    } catch (AccessControlException e) {
+      logAuditEvent(false, "listXAttrs", src);
+      throw e;
+    } finally {
+      readUnlock();
+    }
+  }
   
   void removeXAttr(String src, XAttr xAttr) throws IOException {
     nnConf.checkXAttrsConfigFlag();

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -1411,6 +1411,11 @@ class NameNodeRpcServer implements NamenodeProtocols {
       throws IOException {
     return namesystem.getXAttrs(src, xAttrs);
   }
+
+  @Override
+  public List<XAttr> listXAttrs(String src) throws IOException {
+    return namesystem.listXAttrs(src);
+  }
   
   @Override
   public void removeXAttr(String src, XAttr xAttr) throws IOException {

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

@@ -870,6 +870,11 @@ public class NamenodeWebHdfsMethods {
           xattrEncoding.getEncoding());
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
+    case LISTXATTRS: {
+      final List<XAttr> xAttrs = np.listXAttrs(fullpath);
+      final String js = JsonUtil.toJsonString(xAttrs);
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
     default:
       throw new UnsupportedOperationException(op + " is not supported");
     }

+ 31 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
 import org.mortbay.util.ajax.JSON;
 
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 import java.io.ByteArrayInputStream;
@@ -703,6 +704,18 @@ public class JsonUtil {
     return JSON.toString(finalMap);
   }
   
+  public static String toJsonString(final List<XAttr> xAttrs)
+      throws IOException {
+    final List<String> names = Lists.newArrayListWithCapacity(xAttrs.size());
+    for (XAttr xAttr : xAttrs) {
+      names.add(XAttrHelper.getPrefixName(xAttr));
+    }
+    String ret = JSON.toString(names);
+    final Map<String, Object> finalMap = new TreeMap<String, Object>();
+    finalMap.put("XAttrNames", ret);
+    return JSON.toString(finalMap);
+  }
+  
   public static byte[] getXAttr(final Map<?, ?> json, final String name) 
       throws IOException {
     if (json == null) {
@@ -726,6 +739,24 @@ public class JsonUtil {
     return toXAttrMap((Object[])json.get("XAttrs"));
   }
   
+  public static List<String> toXAttrNames(final Map<?, ?> json)
+      throws IOException {
+    if (json == null) {
+      return null;
+    }
+
+    final String namesInJson = (String) json.get("XAttrNames");
+    final Object[] xattrs = (Object[]) JSON.parse(namesInJson);
+    final List<String> names = Lists.newArrayListWithCapacity(json.keySet()
+        .size());
+
+    for (int i = 0; i < xattrs.length; i++) {
+      names.add((String) (xattrs[i]));
+    }
+    return names;
+  }
+  
+  
   private static Map<String, byte[]> toXAttrMap(final Object[] objects) 
       throws IOException {
     if (objects == null) {

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -889,6 +889,17 @@ public class WebHdfsFileSystem extends FileSystem
     }.run();
   }
   
+  @Override
+  public List<String> listXAttrs(Path p) throws IOException {
+    final HttpOpParam.Op op = GetOpParam.Op.LISTXATTRS;
+    return new FsPathResponseRunner<List<String>>(op, p) {
+      @Override
+      List<String> decodeResponse(Map<?, ?> json) throws IOException {
+        return JsonUtil.toXAttrNames(json);
+      }
+    }.run();
+  }
+
   @Override
   public void removeXAttr(Path p, String name) throws IOException {
     statistics.incrementWriteOps(1);

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

@@ -37,6 +37,7 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     GET_BLOCK_LOCATIONS(false, HttpURLConnection.HTTP_OK),
     GETACLSTATUS(false, HttpURLConnection.HTTP_OK),
     GETXATTRS(false, HttpURLConnection.HTTP_OK),
+    LISTXATTRS(false, HttpURLConnection.HTTP_OK),
 
     NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED);
 

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -764,6 +764,8 @@ service ClientNamenodeProtocol {
       returns(SetXAttrResponseProto);
   rpc getXAttrs(GetXAttrsRequestProto)
       returns(GetXAttrsResponseProto);
+  rpc listXAttrs(ListXAttrsRequestProto)
+      returns(ListXAttrsResponseProto);
   rpc removeXAttr(RemoveXAttrRequestProto)
       returns(RemoveXAttrResponseProto);
 }

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/xattr.proto

@@ -62,6 +62,14 @@ message GetXAttrsResponseProto {
   repeated XAttrProto xAttrs = 1;
 }
 
+message ListXAttrsRequestProto {
+  required string src = 1;
+}
+
+message ListXAttrsResponseProto {
+  repeated XAttrProto xAttrs = 1;
+}
+
 message RemoveXAttrRequestProto {
   required string src        = 1;
   optional XAttrProto xAttr  = 2;

+ 111 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSXAttrBaseTest.java

@@ -17,7 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.FileNotFoundException;
+import java.security.PrivilegedExceptionAction;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
@@ -32,8 +35,11 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -381,6 +387,111 @@ public class FSXAttrBaseTest {
     
     fs.removeXAttr(path, name3);
   }
+
+  /**
+   * Test the listXAttrs api.
+   * listXAttrs on a path that doesn't exist.
+   * listXAttrs on a path with no XAttrs
+   * Check basic functionality.
+   * Check that read access to parent dir is not enough to get xattr names
+   * Check that write access to the parent dir is not enough to get names
+   * Check that execute/scan access to the parent dir is sufficient to get
+   *  xattr names.
+   */
+  @Test(timeout = 120000)
+  public void testListXAttrs() throws Exception {
+    final UserGroupInformation user = UserGroupInformation.
+      createUserForTesting("user", new String[] {"mygroup"});
+
+    /* listXAttrs in a path that doesn't exist. */
+    try {
+      fs.listXAttrs(path);
+      fail("expected FileNotFoundException");
+    } catch (FileNotFoundException e) {
+      GenericTestUtils.assertExceptionContains("cannot find", e);
+    }
+
+    FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short) 0750));
+
+    /* listXAttrs on a path with no XAttrs.*/
+    final List<String> noXAttrs = fs.listXAttrs(path);
+    assertTrue("XAttrs were found?", noXAttrs.size() == 0);
+
+    fs.setXAttr(path, name1, value1, EnumSet.of(XAttrSetFlag.CREATE));
+    fs.setXAttr(path, name2, value2, EnumSet.of(XAttrSetFlag.CREATE));
+
+    /** Check basic functionality. */
+    final List<String> xattrNames = fs.listXAttrs(path);
+    assertTrue(xattrNames.contains(name1));
+    assertTrue(xattrNames.contains(name2));
+    assertTrue(xattrNames.size() == 2);
+
+    /* Check that read access to parent dir is not enough to get xattr names. */
+    fs.setPermission(path, new FsPermission((short) 0704));
+    final Path childDir = new Path(path, "child" + pathCount);
+    FileSystem.mkdirs(fs, childDir, FsPermission.createImmutable((short) 0700));
+    fs.setXAttr(childDir, name1, "1234".getBytes());
+    try {
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            final FileSystem userFs = dfsCluster.getFileSystem();
+            userFs.listXAttrs(childDir);
+            return null;
+          }
+        });
+      fail("expected IOException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Permission denied", e);
+    }
+
+    /*
+     * Check that write access to the parent dir is not enough to get names.
+     */
+    fs.setPermission(path, new FsPermission((short) 0702));
+    try {
+      user.doAs(new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            final FileSystem userFs = dfsCluster.getFileSystem();
+            userFs.listXAttrs(childDir);
+            return null;
+          }
+        });
+      fail("expected IOException");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Permission denied", e);
+    }
+
+    /*
+     * Check that execute/scan access to the parent dir is sufficient to get
+     * xattr names.
+     */
+    fs.setPermission(path, new FsPermission((short) 0701));
+    user.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public Object run() throws Exception {
+          final FileSystem userFs = dfsCluster.getFileSystem();
+          userFs.listXAttrs(childDir);
+          return null;
+        }
+      });
+
+    /*
+     * Test that xattrs in the "trusted" namespace are filtered correctly.
+     */
+    fs.setXAttr(childDir, "trusted.myxattr", "1234".getBytes());
+    user.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public Object run() throws Exception {
+          final FileSystem userFs = dfsCluster.getFileSystem();
+          assertTrue(userFs.listXAttrs(childDir).size() == 1);
+          return null;
+        }
+      });
+
+    assertTrue(fs.listXAttrs(childDir).size() == 2);
+  }
   
   /**
    * Steps: