浏览代码

HDFS-6259. Support extended attributes via WebHDFS. Contributed by Yi Liu.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2006@1596104 13f79535-47bb-0310-9956-ffa450edef68
Yi Liu 11 年之前
父节点
当前提交
8367907467
共有 14 个文件被更改,包括 593 次插入12 次删除
  1. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-2006.txt
  2. 68 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  3. 124 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  4. 64 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  5. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
  6. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java
  7. 56 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrEncodingParam.java
  8. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrNameParam.java
  9. 53 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrSetFlagParam.java
  10. 45 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrValueParam.java
  11. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSXAttrBaseTest.java
  12. 48 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
  13. 36 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSXAttr.java
  14. 40 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-2006.txt

@@ -38,6 +38,8 @@ HDFS-2006 (Unreleased)
     HDFS-6412. Interface audience and stability annotations missing from
     HDFS-6412. Interface audience and stability annotations missing from
     several new classes related to xattrs. (wang)
     several new classes related to xattrs. (wang)
 
 
+    HDFS-6259. Support extended attributes via WebHDFS. (yliu)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-6346. Optimize OP_SET_XATTRS by persisting single Xattr entry per setXattr/removeXattr api call
     HDFS-6346. Optimize OP_SET_XATTRS by persisting single Xattr entry per setXattr/removeXattr api call

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

@@ -28,6 +28,7 @@ import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.EnumSet;
+import java.util.List;
 
 
 import javax.servlet.ServletContext;
 import javax.servlet.ServletContext;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequest;
@@ -53,8 +54,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -101,6 +104,10 @@ import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
 import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam;
 import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam;
 import org.apache.hadoop.hdfs.web.resources.UriFsPathParam;
 import org.apache.hadoop.hdfs.web.resources.UriFsPathParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
+import org.apache.hadoop.hdfs.web.resources.XAttrEncodingParam;
+import org.apache.hadoop.hdfs.web.resources.XAttrNameParam;
+import org.apache.hadoop.hdfs.web.resources.XAttrSetFlagParam;
+import org.apache.hadoop.hdfs.web.resources.XAttrValueParam;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NetworkTopology.InvalidTopologyException;
 import org.apache.hadoop.net.NetworkTopology.InvalidTopologyException;
@@ -341,12 +348,19 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT)
       @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT)
           final TokenArgumentParam delegationTokenArgument,
           final TokenArgumentParam delegationTokenArgument,
       @QueryParam(AclPermissionParam.NAME) @DefaultValue(AclPermissionParam.DEFAULT) 
       @QueryParam(AclPermissionParam.NAME) @DefaultValue(AclPermissionParam.DEFAULT) 
-          final AclPermissionParam aclPermission
+          final AclPermissionParam aclPermission,
+      @QueryParam(XAttrNameParam.NAME) @DefaultValue(XAttrNameParam.DEFAULT)
+          final XAttrNameParam xattrName,
+      @QueryParam(XAttrValueParam.NAME) @DefaultValue(XAttrValueParam.DEFAULT)
+          final XAttrValueParam xattrValue,
+      @QueryParam(XAttrSetFlagParam.NAME) @DefaultValue(XAttrSetFlagParam.DEFAULT)
+          final XAttrSetFlagParam xattrSetFlag
           )throws IOException, InterruptedException {
           )throws IOException, InterruptedException {
     return put(ugi, delegation, username, doAsUser, ROOT, op, destination,
     return put(ugi, delegation, username, doAsUser, ROOT, op, destination,
         owner, group, permission, overwrite, bufferSize, replication,
         owner, group, permission, overwrite, bufferSize, replication,
         blockSize, modificationTime, accessTime, renameOptions, createParent,
         blockSize, modificationTime, accessTime, renameOptions, createParent,
-        delegationTokenArgument,aclPermission);
+        delegationTokenArgument,aclPermission, xattrName, xattrValue,
+        xattrSetFlag);
   }
   }
 
 
   /** Handle HTTP PUT request. */
   /** Handle HTTP PUT request. */
@@ -392,12 +406,19 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT)
       @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT)
           final TokenArgumentParam delegationTokenArgument,
           final TokenArgumentParam delegationTokenArgument,
       @QueryParam(AclPermissionParam.NAME) @DefaultValue(AclPermissionParam.DEFAULT) 
       @QueryParam(AclPermissionParam.NAME) @DefaultValue(AclPermissionParam.DEFAULT) 
-          final AclPermissionParam aclPermission
+          final AclPermissionParam aclPermission,
+      @QueryParam(XAttrNameParam.NAME) @DefaultValue(XAttrNameParam.DEFAULT)
+          final XAttrNameParam xattrName,
+      @QueryParam(XAttrValueParam.NAME) @DefaultValue(XAttrValueParam.DEFAULT)
+          final XAttrValueParam xattrValue,
+      @QueryParam(XAttrSetFlagParam.NAME) @DefaultValue(XAttrSetFlagParam.DEFAULT)
+          final XAttrSetFlagParam xattrSetFlag
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
 
 
     init(ugi, delegation, username, doAsUser, path, op, destination, owner,
     init(ugi, delegation, username, doAsUser, path, op, destination, owner,
         group, permission, overwrite, bufferSize, replication, blockSize,
         group, permission, overwrite, bufferSize, replication, blockSize,
-        modificationTime, accessTime, renameOptions, delegationTokenArgument,aclPermission);
+        modificationTime, accessTime, renameOptions, delegationTokenArgument,aclPermission, 
+        xattrName, xattrValue, xattrSetFlag);
 
 
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
       @Override
@@ -407,7 +428,8 @@ public class NamenodeWebHdfsMethods {
               path.getAbsolutePath(), op, destination, owner, group,
               path.getAbsolutePath(), op, destination, owner, group,
               permission, overwrite, bufferSize, replication, blockSize,
               permission, overwrite, bufferSize, replication, blockSize,
               modificationTime, accessTime, renameOptions, createParent,
               modificationTime, accessTime, renameOptions, createParent,
-              delegationTokenArgument,aclPermission);
+              delegationTokenArgument,aclPermission, xattrName, 
+              xattrValue, xattrSetFlag);
         } finally {
         } finally {
           reset();
           reset();
         }
         }
@@ -435,7 +457,10 @@ public class NamenodeWebHdfsMethods {
       final RenameOptionSetParam renameOptions,
       final RenameOptionSetParam renameOptions,
       final CreateParentParam createParent,
       final CreateParentParam createParent,
       final TokenArgumentParam delegationTokenArgument,
       final TokenArgumentParam delegationTokenArgument,
-      final AclPermissionParam aclPermission
+      final AclPermissionParam aclPermission,
+      final XAttrNameParam xattrName,
+      final XAttrValueParam xattrValue,
+      final XAttrSetFlagParam xattrSetFlag
       ) throws IOException, URISyntaxException {
       ) throws IOException, URISyntaxException {
 
 
     final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
     final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
@@ -535,6 +560,15 @@ public class NamenodeWebHdfsMethods {
       np.setAcl(fullpath, aclPermission.getAclPermission(true));
       np.setAcl(fullpath, aclPermission.getAclPermission(true));
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
       return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
     }
     }
+    case SETXATTR: {
+      np.setXAttr(fullpath, XAttrHelper.buildXAttr(xattrName.getXAttrName(), 
+          xattrValue.getXAttrValue()), xattrSetFlag.getFlag());
+      return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
+    }
+    case REMOVEXATTR: {
+      np.removeXAttr(fullpath, XAttrHelper.buildXAttr(xattrName.getXAttrName()));
+      return Response.ok().type(MediaType.APPLICATION_OCTET_STREAM).build();
+    }
     default:
     default:
       throw new UnsupportedOperationException(op + " is not supported");
       throw new UnsupportedOperationException(op + " is not supported");
     }
     }
@@ -650,10 +684,14 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(RenewerParam.NAME) @DefaultValue(RenewerParam.DEFAULT)
       @QueryParam(RenewerParam.NAME) @DefaultValue(RenewerParam.DEFAULT)
           final RenewerParam renewer,
           final RenewerParam renewer,
       @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
       @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
-          final BufferSizeParam bufferSize
+          final BufferSizeParam bufferSize,
+      @QueryParam(XAttrNameParam.NAME) @DefaultValue(XAttrNameParam.DEFAULT)
+          final XAttrNameParam xattrName,
+      @QueryParam(XAttrEncodingParam.NAME) @DefaultValue(XAttrEncodingParam.DEFAULT)
+          final XAttrEncodingParam xattrEncoding
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
     return get(ugi, delegation, username, doAsUser, ROOT, op,
     return get(ugi, delegation, username, doAsUser, ROOT, op,
-        offset, length, renewer, bufferSize);
+        offset, length, renewer, bufferSize, xattrName, xattrEncoding);
   }
   }
 
 
   /** Handle HTTP GET request. */
   /** Handle HTTP GET request. */
@@ -678,18 +716,23 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(RenewerParam.NAME) @DefaultValue(RenewerParam.DEFAULT)
       @QueryParam(RenewerParam.NAME) @DefaultValue(RenewerParam.DEFAULT)
           final RenewerParam renewer,
           final RenewerParam renewer,
       @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
       @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
-          final BufferSizeParam bufferSize
+          final BufferSizeParam bufferSize,
+      @QueryParam(XAttrNameParam.NAME) @DefaultValue(XAttrNameParam.DEFAULT)
+          final XAttrNameParam xattrName,
+      @QueryParam(XAttrEncodingParam.NAME) @DefaultValue(XAttrEncodingParam.DEFAULT)
+          final XAttrEncodingParam xattrEncoding
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
 
 
     init(ugi, delegation, username, doAsUser, path, op,
     init(ugi, delegation, username, doAsUser, path, op,
-        offset, length, renewer, bufferSize);
+        offset, length, renewer, bufferSize, xattrName, xattrEncoding);
 
 
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
       @Override
       public Response run() throws IOException, URISyntaxException {
       public Response run() throws IOException, URISyntaxException {
         try {
         try {
           return get(ugi, delegation, username, doAsUser,
           return get(ugi, delegation, username, doAsUser,
-              path.getAbsolutePath(), op, offset, length, renewer, bufferSize);
+              path.getAbsolutePath(), op, offset, length, renewer, bufferSize,
+              xattrName, xattrEncoding);
         } finally {
         } finally {
           reset();
           reset();
         }
         }
@@ -707,7 +750,9 @@ public class NamenodeWebHdfsMethods {
       final OffsetParam offset,
       final OffsetParam offset,
       final LengthParam length,
       final LengthParam length,
       final RenewerParam renewer,
       final RenewerParam renewer,
-      final BufferSizeParam bufferSize
+      final BufferSizeParam bufferSize,
+      final XAttrNameParam xattrName,
+      final XAttrEncodingParam xattrEncoding
       ) throws IOException, URISyntaxException {
       ) throws IOException, URISyntaxException {
     final NameNode namenode = (NameNode)context.getAttribute("name.node");
     final NameNode namenode = (NameNode)context.getAttribute("name.node");
     final NamenodeProtocols np = getRPCServer(namenode);
     final NamenodeProtocols np = getRPCServer(namenode);
@@ -782,6 +827,17 @@ public class NamenodeWebHdfsMethods {
       final String js = JsonUtil.toJsonString(status);
       final String js = JsonUtil.toJsonString(status);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     }
+    case GETXATTR: {
+      XAttr xAttr = XAttrHelper.getFirstXAttr(np.getXAttrs(fullpath, 
+          XAttrHelper.buildXAttrAsList(xattrName.getXAttrName())));
+      final String js = JsonUtil.toJsonString(xAttr, xattrEncoding.getEncoding());
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
+    case GETXATTRS: {
+      List<XAttr> xAttrs = np.getXAttrs(fullpath, null);
+      final String js = JsonUtil.toJsonString(xAttrs, xattrEncoding.getEncoding());
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
     default:
     default:
       throw new UnsupportedOperationException(op + " is not supported");
       throw new UnsupportedOperationException(op + " is not supported");
     }
     }

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

@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -34,6 +35,8 @@ import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.mortbay.util.ajax.JSON;
 import org.mortbay.util.ajax.JSON;
 
 
+import com.google.common.collect.Maps;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.IOException;
@@ -661,4 +664,125 @@ public class JsonUtil {
     aclStatusBuilder.addEntries(aclEntryList);
     aclStatusBuilder.addEntries(aclEntryList);
     return aclStatusBuilder.build();
     return aclStatusBuilder.build();
   }
   }
+  
+  public static String toJsonString(final XAttr xAttr, 
+      final XAttrCodec encoding) throws IOException {
+    if (xAttr == null) {
+      return "{}";
+    }
+    final Map<String, Object> m = new TreeMap<String, Object>();
+    m.put("name", XAttrHelper.getPrefixName(xAttr));
+    m.put("value", xAttr.getValue() != null ? 
+        XAttrCodec.encodeValue(xAttr.getValue(), encoding) : null);
+    final Map<String, Map<String, Object>> finalMap =
+        new TreeMap<String, Map<String, Object>>();
+    finalMap.put(XAttr.class.getSimpleName(), m);
+    return JSON.toString(finalMap);
+  }
+  
+  private static Map<String, Object> toJsonMap(final XAttr xAttr,
+      final XAttrCodec encoding) throws IOException {
+    if (xAttr == null) {
+      return null;
+    }
+ 
+    final Map<String, Object> m = new TreeMap<String, Object>();
+    m.put("name", XAttrHelper.getPrefixName(xAttr));
+    m.put("value", xAttr.getValue() != null ? 
+        XAttrCodec.encodeValue(xAttr.getValue(), encoding) : null);
+    return m;
+  }
+  
+  private static Object[] toJsonArray(final List<XAttr> array,
+      final XAttrCodec encoding) throws IOException {
+    if (array == null) {
+      return null;
+    } else if (array.size() == 0) {
+      return EMPTY_OBJECT_ARRAY;
+    } else {
+      final Object[] a = new Object[array.size()];
+      for(int i = 0; i < array.size(); i++) {
+        a[i] = toJsonMap(array.get(i), encoding);
+      }
+      return a;
+    }
+  }
+  
+  public static String toJsonString(final List<XAttr> xAttrs, 
+      final XAttrCodec encoding) throws IOException {
+    final Map<String, Object> finalMap = new TreeMap<String, Object>();
+    finalMap.put("XAttrs", toJsonArray(xAttrs, encoding));
+    return JSON.toString(finalMap);
+  }
+  
+  public static XAttr toXAttr(final Map<?, ?> json) throws IOException {
+    if (json == null) {
+      return null;
+    }
+    
+    Map<?, ?> m = (Map<?, ?>) json.get(XAttr.class.getSimpleName());
+    if (m == null) {
+      return null;
+    }
+    String name = (String) m.get("name");
+    String value = (String) m.get("value");
+    return XAttrHelper.buildXAttr(name, decodeXAttrValue(value));
+  }
+  
+  public static Map<String, byte[]> toXAttrs(final Map<?, ?> json) 
+      throws IOException {
+    if (json == null) {
+      return null;
+    }
+    
+    return toXAttrMap((Object[])json.get("XAttrs"));
+  }
+  
+  public static Map<String, byte[]> toXAttrs(final Map<?, ?> json, 
+      List<String> names) throws IOException {
+    if (json == null || names == null) {
+      return null;
+    }
+    if (names.isEmpty()) {
+      return Maps.newHashMap();
+    }
+    Map<String, byte[]> xAttrs = toXAttrs(json);
+    if (xAttrs == null || xAttrs.isEmpty()) {
+      return xAttrs;
+    }
+    
+    Map<String, byte[]> result = Maps.newHashMap();
+    for (String name : names) {
+      if (xAttrs.containsKey(name)) {
+        result.put(name, xAttrs.get(name));
+      }
+    }
+    return result;
+  }
+  
+  private static Map<String, byte[]> toXAttrMap(final Object[] objects) 
+      throws IOException {
+    if (objects == null) {
+      return null;
+    } else if (objects.length == 0) {
+      return Maps.newHashMap();
+    } else {
+      final Map<String, byte[]> xAttrs = Maps.newHashMap();
+      for(int i = 0; i < objects.length; i++) {
+        Map<?, ?> m = (Map<?, ?>) objects[i];
+        String name = (String) m.get("name");
+        String value = (String) m.get("value");
+        xAttrs.put(name, decodeXAttrValue(value));
+      }
+      return xAttrs;
+    }
+  }
+  
+  private static byte[] decodeXAttrValue(String value) throws IOException {
+    if (value != null) {
+      return XAttrCodec.decodeValue(value);
+    } else {
+      return new byte[0];
+    }
+  }
 }
 }

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

@@ -30,6 +30,7 @@ import java.net.URI;
 import java.net.URL;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.StringTokenizer;
 import java.util.StringTokenizer;
@@ -49,6 +50,9 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttr;
+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.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -813,6 +817,66 @@ public class WebHdfsFileSystem extends FileSystem
         new RenameOptionSetParam(options)
         new RenameOptionSetParam(options)
     ).run();
     ).run();
   }
   }
+  
+  @Override
+  public void setXAttr(Path p, String name, byte[] value, 
+      EnumSet<XAttrSetFlag> flag) throws IOException {
+    statistics.incrementWriteOps(1);
+    final HttpOpParam.Op op = PutOpParam.Op.SETXATTR;
+    if (value != null) {
+      new FsPathRunner(op, p, new XAttrNameParam(name), new XAttrValueParam(
+          XAttrCodec.encodeValue(value, XAttrCodec.HEX)), 
+          new XAttrSetFlagParam(flag)).run();
+    } else {
+      new FsPathRunner(op, p, new XAttrNameParam(name), 
+          new XAttrSetFlagParam(flag)).run();
+    }
+  }
+  
+  @Override
+  public byte[] getXAttr(Path p, String name) throws IOException {
+    final HttpOpParam.Op op = GetOpParam.Op.GETXATTR;
+    return new FsPathResponseRunner<byte[]>(op, p, new XAttrNameParam(name), 
+        new XAttrEncodingParam(XAttrCodec.HEX)) {
+      @Override
+      byte[] decodeResponse(Map<?, ?> json) throws IOException {
+        XAttr xAttr = JsonUtil.toXAttr(json);
+        return xAttr != null ? xAttr.getValue() : null;
+      }
+    }.run();
+  }
+  
+  @Override
+  public Map<String, byte[]> getXAttrs(Path p) throws IOException {
+    final HttpOpParam.Op op = GetOpParam.Op.GETXATTRS;
+    return new FsPathResponseRunner<Map<String, byte[]>>(op, p, 
+        new XAttrEncodingParam(XAttrCodec.HEX)) {
+      @Override
+      Map<String, byte[]> decodeResponse(Map<?, ?> json) throws IOException {
+        return JsonUtil.toXAttrs(json);
+      }
+    }.run();
+  }
+  
+  @Override
+  public Map<String, byte[]> getXAttrs(Path p, final List<String> names) 
+      throws IOException {
+    final HttpOpParam.Op op = GetOpParam.Op.GETXATTRS;
+    return new FsPathResponseRunner<Map<String, byte[]>>(op, p, 
+        new XAttrEncodingParam(XAttrCodec.HEX)) {
+      @Override
+      Map<String, byte[]> decodeResponse(Map<?, ?> json) throws IOException {
+        return JsonUtil.toXAttrs(json, names);
+      }
+    }.run();
+  }
+  
+  @Override
+  public void removeXAttr(Path p, String name) throws IOException {
+    statistics.incrementWriteOps(1);
+    final HttpOpParam.Op op = PutOpParam.Op.REMOVEXATTR;
+    new FsPathRunner(op, p, new XAttrNameParam(name)).run();
+  }
 
 
   @Override
   @Override
   public void setOwner(final Path p, final String owner, final String group
   public void setOwner(final Path p, final String owner, final String group

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

@@ -36,6 +36,8 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     /** GET_BLOCK_LOCATIONS is a private unstable op. */
     /** GET_BLOCK_LOCATIONS is a private unstable op. */
     GET_BLOCK_LOCATIONS(false, HttpURLConnection.HTTP_OK),
     GET_BLOCK_LOCATIONS(false, HttpURLConnection.HTTP_OK),
     GETACLSTATUS(false, HttpURLConnection.HTTP_OK),
     GETACLSTATUS(false, HttpURLConnection.HTTP_OK),
+    GETXATTR(false, HttpURLConnection.HTTP_OK),
+    GETXATTRS(false, HttpURLConnection.HTTP_OK),
 
 
     NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED);
     NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED);
 
 

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java

@@ -42,6 +42,8 @@ public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
     REMOVEDEFAULTACL(false, HttpURLConnection.HTTP_OK),
     REMOVEDEFAULTACL(false, HttpURLConnection.HTTP_OK),
     REMOVEACL(false, HttpURLConnection.HTTP_OK),
     REMOVEACL(false, HttpURLConnection.HTTP_OK),
     SETACL(false, HttpURLConnection.HTTP_OK),
     SETACL(false, HttpURLConnection.HTTP_OK),
+    SETXATTR(false, HttpURLConnection.HTTP_OK),
+    REMOVEXATTR(false, HttpURLConnection.HTTP_OK),
     
     
     NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED);
     NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED);
 
 

+ 56 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrEncodingParam.java

@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+import org.apache.hadoop.fs.XAttrCodec;
+
+public class XAttrEncodingParam extends EnumParam<XAttrCodec> {
+  /** Parameter name. */
+  public static final String NAME = "encoding";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+  
+  private static final Domain<XAttrCodec> DOMAIN = 
+      new Domain<XAttrCodec>(NAME, XAttrCodec.class);
+  
+  public XAttrEncodingParam(final XAttrCodec encoding) {
+    super(DOMAIN, encoding);
+  }
+  
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public XAttrEncodingParam(final String str) {
+    super(DOMAIN, str != null && !str.isEmpty() ? DOMAIN.parse(str) : null);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+  
+  @Override
+  public String getValueString() {
+    return value.toString();
+  }
+  
+  public XAttrCodec getEncoding() {
+    return getValue();
+  }
+}

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrNameParam.java

@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+import java.util.regex.Pattern;
+
+public class XAttrNameParam extends StringParam {
+  /** Parameter name. **/
+  public static final String NAME = "xattr.name";
+  /** Default parameter value. **/
+  public static final String DEFAULT = "";
+  
+  private static Domain DOMAIN = new Domain(NAME, 
+      Pattern.compile("^(user\\.|trusted\\.|system\\.|security\\.).+"));
+  
+  public XAttrNameParam(final String str) {
+    super(DOMAIN, str == null || str.equals(DEFAULT) ? null : str);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+  
+  public String getXAttrName() {
+    final String v = getValue();
+    return v;
+  }
+}

+ 53 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrSetFlagParam.java

@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+import java.util.EnumSet;
+
+import org.apache.hadoop.fs.XAttrSetFlag;
+
+public class XAttrSetFlagParam extends EnumSetParam<XAttrSetFlag> {
+  /** Parameter name. */
+  public static final String NAME = "flag";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain<XAttrSetFlag> DOMAIN = new Domain<XAttrSetFlag>(
+      NAME, XAttrSetFlag.class);
+
+  public XAttrSetFlagParam(final EnumSet<XAttrSetFlag> flag) {
+    super(DOMAIN, flag);
+  }
+  
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public XAttrSetFlagParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+  
+  public EnumSet<XAttrSetFlag> getFlag() {
+    return getValue();
+  }
+}

+ 45 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/XAttrValueParam.java

@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.XAttrCodec;
+
+public class XAttrValueParam extends StringParam {
+  /** Parameter name. **/
+  public static final String NAME = "xattr.value";
+  /** Default parameter value. **/
+  public static final String DEFAULT = "";
+  
+  private static Domain DOMAIN = new Domain(NAME, null);
+  
+  public XAttrValueParam(final String str) {
+    super(DOMAIN, str == null || str.equals(DEFAULT) ? null : str);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+  
+  public byte[] getXAttrValue() throws IOException {
+    final String v = getValue();
+    return XAttrCodec.decodeValue(v);
+  }
+}

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

@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.AfterClass;
@@ -231,6 +232,8 @@ public class FSXAttrBaseTest {
       Assert.fail("Setting xattr with null name should fail.");
       Assert.fail("Setting xattr with null name should fail.");
     } catch (NullPointerException e) {
     } catch (NullPointerException e) {
       GenericTestUtils.assertExceptionContains("XAttr name cannot be null", e);
       GenericTestUtils.assertExceptionContains("XAttr name cannot be null", e);
+    } catch (RemoteException e) {
+      GenericTestUtils.assertExceptionContains("XAttr name cannot be null", e);
     }
     }
     
     
     // Set xattr with empty name: "user."
     // Set xattr with empty name: "user."
@@ -240,6 +243,9 @@ public class FSXAttrBaseTest {
       Assert.fail("Setting xattr with empty name should fail.");
       Assert.fail("Setting xattr with empty name should fail.");
     } catch (HadoopIllegalArgumentException e) {
     } catch (HadoopIllegalArgumentException e) {
       GenericTestUtils.assertExceptionContains("XAttr name cannot be empty", e);
       GenericTestUtils.assertExceptionContains("XAttr name cannot be empty", e);
+    } catch (IllegalArgumentException e) {
+      GenericTestUtils.assertExceptionContains("Invalid value: \"user.\" does " + 
+          "not belong to the domain ^(user\\.|trusted\\.|system\\.|security\\.).+", e);
     }
     }
     
     
     // Set xattr with invalid name: "a1"
     // Set xattr with invalid name: "a1"
@@ -250,6 +256,9 @@ public class FSXAttrBaseTest {
           "name prefix should fail.");
           "name prefix should fail.");
     } catch (HadoopIllegalArgumentException e) {
     } catch (HadoopIllegalArgumentException e) {
       GenericTestUtils.assertExceptionContains("XAttr name must be prefixed", e);
       GenericTestUtils.assertExceptionContains("XAttr name must be prefixed", e);
+    } catch (IllegalArgumentException e) {
+      GenericTestUtils.assertExceptionContains("Invalid value: \"a1\" does " + 
+          "not belong to the domain ^(user\\.|trusted\\.|system\\.|security\\.).+", e);
     }
     }
     
     
     // Set xattr without XAttrSetFlag
     // Set xattr without XAttrSetFlag

+ 48 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java

@@ -22,16 +22,22 @@ import static org.apache.hadoop.fs.permission.AclEntryType.*;
 import static org.apache.hadoop.fs.permission.FsAction.*;
 import static org.apache.hadoop.fs.permission.FsAction.*;
 import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.*;
 import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.*;
 
 
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
+import java.util.Map.Entry;
 
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrCodec;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
@@ -186,6 +192,48 @@ public class TestJsonUtil {
         JsonUtil.toJsonString(aclStatusBuilder.build()));
         JsonUtil.toJsonString(aclStatusBuilder.build()));
 
 
   }
   }
+  
+  @Test
+  public void testToJsonFromXAttrs() throws IOException {
+    String jsonString = 
+        "{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
+        "{\"name\":\"user.a2\",\"value\":\"0x313131\"}]}";
+    XAttr xAttr1 = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.USER).
+        setName("a1").setValue(XAttrCodec.decodeValue("0x313233")).build();
+    XAttr xAttr2 = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.USER).
+        setName("a2").setValue(XAttrCodec.decodeValue("0x313131")).build();
+    List<XAttr> xAttrs = Lists.newArrayList();
+    xAttrs.add(xAttr1);
+    xAttrs.add(xAttr2);
+    
+    Assert.assertEquals(jsonString, JsonUtil.toJsonString(xAttrs, 
+        XAttrCodec.HEX));
+  }
+  
+  @Test
+  public void testToXAttrMap() throws IOException {
+    String jsonString = 
+        "{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
+        "{\"name\":\"user.a2\",\"value\":\"0x313131\"}]}";
+    Map<?, ?> json = (Map<?, ?>)JSON.parse(jsonString);
+    XAttr xAttr1 = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.USER).
+        setName("a1").setValue(XAttrCodec.decodeValue("0x313233")).build();
+    XAttr xAttr2 = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.USER).
+        setName("a2").setValue(XAttrCodec.decodeValue("0x313131")).build();
+    List<XAttr> xAttrs = Lists.newArrayList();
+    xAttrs.add(xAttr1);
+    xAttrs.add(xAttr2);
+    Map<String, byte[]> xAttrMap = XAttrHelper.buildXAttrMap(xAttrs);
+    Map<String, byte[]> parsedXAttrMap = JsonUtil.toXAttrs(json);
+    
+    Assert.assertEquals(xAttrMap.size(), parsedXAttrMap.size());
+    Iterator<Entry<String, byte[]>> iter = xAttrMap.entrySet().iterator();
+    while(iter.hasNext()) {
+      Entry<String, byte[]> entry = iter.next();
+      Assert.assertArrayEquals(entry.getValue(), 
+          parsedXAttrMap.get(entry.getKey()));
+    }
+  }
 
 
   private void checkDecodeFailure(Map<String, Object> map) {
   private void checkDecodeFailure(Map<String, Object> map) {
     try {
     try {

+ 36 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSXAttr.java

@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web;
+
+import org.apache.hadoop.hdfs.server.namenode.FSXAttrBaseTest;
+
+/**
+ * Tests XAttr APIs via WebHDFS.
+ */
+public class TestWebHDFSXAttr extends FSXAttrBaseTest {
+  /**
+   * Overridden to provide a WebHdfsFileSystem wrapper for the super-user.
+   *
+   * @return WebHdfsFileSystem for super-user
+   * @throws Exception if creation fails
+   */
+  @Override
+  protected WebHdfsFileSystem createFileSystem() throws Exception {
+    return WebHdfsTestUtil.getWebHdfsFileSystem(conf, WebHdfsFileSystem.SCHEME);
+  }
+}

+ 40 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.web.resources;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertNull;
 
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.List;
@@ -30,6 +31,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+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.AclEntry;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -348,6 +351,43 @@ public class TestParam {
     }
     }
   }
   }
  
  
+  @Test
+  public void testXAttrNameParam() {
+    final XAttrNameParam p = new XAttrNameParam("user.a1");
+    Assert.assertEquals(p.getXAttrName(), "user.a1");
+    try {
+      new XAttrNameParam("a1");
+      Assert.fail();
+    } catch (IllegalArgumentException e) {
+      LOG.info("EXPECTED: " + e);
+    }
+  }
+  
+  @Test
+  public void testXAttrValueParam() throws IOException {
+    final XAttrValueParam p = new XAttrValueParam("0x313233");
+    Assert.assertArrayEquals(p.getXAttrValue(), 
+        XAttrCodec.decodeValue("0x313233"));
+  }
+  
+  @Test
+  public void testXAttrEncodingParam() {
+    final XAttrEncodingParam p = new XAttrEncodingParam(XAttrCodec.BASE64);
+    Assert.assertEquals(p.getEncoding(), XAttrCodec.BASE64);
+    final XAttrEncodingParam p1 = new XAttrEncodingParam(p.getValueString());
+    Assert.assertEquals(p1.getEncoding(), XAttrCodec.BASE64);
+  }
+  
+  @Test
+  public void testXAttrSetFlagParam() {
+    EnumSet<XAttrSetFlag> flag = EnumSet.of(
+        XAttrSetFlag.CREATE, XAttrSetFlag.REPLACE);
+    final XAttrSetFlagParam p = new XAttrSetFlagParam(flag);
+    Assert.assertEquals(p.getFlag(), flag);
+    final XAttrSetFlagParam p1 = new XAttrSetFlagParam(p.getValueString());
+    Assert.assertEquals(p1.getFlag(), flag);
+  }
+  
   @Test
   @Test
   public void testRenameOptionSetParam() {
   public void testRenameOptionSetParam() {
     final RenameOptionSetParam p = new RenameOptionSetParam(
     final RenameOptionSetParam p = new RenameOptionSetParam(