Explorar el Código

Merge from trunk to branch-2. HDFS-6464. Support multiple xattr.name parameters for WebHDFS getXAttrs. Contributed by Yi Liu

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1602125 13f79535-47bb-0310-9956-ffa450edef68
Uma Maheswara Rao G hace 11 años
padre
commit
5677e994e3

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

@@ -355,6 +355,9 @@ Release 2.5.0 - UNRELEASED
     HDFS-6503. Fix typo of DFSAdmin restoreFailedStorage.
     (Zesheng Wu via wheat9)
 
+    HDFS-6464. Support multiple xattr.name parameters for WebHDFS getXAttrs.
+    (Yi Liu via umamahesh)
+
   BREAKDOWN OF HDFS-2006 SUBTASKS AND RELATED JIRAS
 
     HDFS-6299. Protobuf for XAttr and client-side implementation. (Yi Liu via umamahesh)

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

@@ -152,7 +152,7 @@ public class XAttrHelper {
   public static List<XAttr> buildXAttrs(List<String> names) {
     if (names == null || names.isEmpty()) {
       throw new HadoopIllegalArgumentException("XAttr names can not be " +
-      		"null or empty.");
+          "null or empty.");
     }
     
     List<XAttr> xAttrs = Lists.newArrayListWithCapacity(names.size());

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

@@ -121,6 +121,7 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Charsets;
+import com.google.common.collect.Lists;
 import com.sun.jersey.spi.container.ResourceFilters;
 
 /** Web-hdfs NameNode implementation. */
@@ -712,12 +713,12 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
           final BufferSizeParam bufferSize,
       @QueryParam(XAttrNameParam.NAME) @DefaultValue(XAttrNameParam.DEFAULT) 
-          final XAttrNameParam xattrName,
+          final List<XAttrNameParam> xattrNames,
       @QueryParam(XAttrEncodingParam.NAME) @DefaultValue(XAttrEncodingParam.DEFAULT) 
           final XAttrEncodingParam xattrEncoding
       ) throws IOException, InterruptedException {
     return get(ugi, delegation, username, doAsUser, ROOT, op, offset, length,
-        renewer, bufferSize, xattrName, xattrEncoding);
+        renewer, bufferSize, xattrNames, xattrEncoding);
   }
 
   /** Handle HTTP GET request. */
@@ -744,13 +745,13 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
           final BufferSizeParam bufferSize,
       @QueryParam(XAttrNameParam.NAME) @DefaultValue(XAttrNameParam.DEFAULT) 
-          final XAttrNameParam xattrName,
+          final List<XAttrNameParam> xattrNames,
       @QueryParam(XAttrEncodingParam.NAME) @DefaultValue(XAttrEncodingParam.DEFAULT) 
           final XAttrEncodingParam xattrEncoding
       ) throws IOException, InterruptedException {
 
     init(ugi, delegation, username, doAsUser, path, op, offset, length,
-        renewer, bufferSize, xattrName, xattrEncoding);
+        renewer, bufferSize, xattrEncoding);
 
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
@@ -758,7 +759,7 @@ public class NamenodeWebHdfsMethods {
         try {
           return get(ugi, delegation, username, doAsUser,
               path.getAbsolutePath(), op, offset, length, renewer, bufferSize,
-              xattrName, xattrEncoding);
+              xattrNames, xattrEncoding);
         } finally {
           reset();
         }
@@ -777,7 +778,7 @@ public class NamenodeWebHdfsMethods {
       final LengthParam length,
       final RenewerParam renewer,
       final BufferSizeParam bufferSize,
-      final XAttrNameParam xattrName,
+      final List<XAttrNameParam> xattrNames,
       final XAttrEncodingParam xattrEncoding
       ) throws IOException, URISyntaxException {
     final NameNode namenode = (NameNode)context.getAttribute("name.node");
@@ -853,15 +854,18 @@ public class NamenodeWebHdfsMethods {
       final String js = JsonUtil.toJsonString(status);
       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);
+      List<String> names = null;
+      if (xattrNames != null) {
+        names = Lists.newArrayListWithCapacity(xattrNames.size());
+        for (XAttrNameParam xattrName : xattrNames) {
+          if (xattrName.getXAttrName() != null) {
+            names.add(xattrName.getXAttrName());
+          }
+        }
+      }
+      List<XAttr> xAttrs = np.getXAttrs(fullpath, (names != null && 
+          !names.isEmpty()) ? XAttrHelper.buildXAttrs(names) : null);
       final String js = JsonUtil.toJsonString(xAttrs,
           xattrEncoding.getEncoding());
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();

+ 7 - 44
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -668,21 +668,6 @@ public class JsonUtil {
     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) {
@@ -718,18 +703,18 @@ public class JsonUtil {
     return JSON.toString(finalMap);
   }
   
-  public static XAttr toXAttr(final Map<?, ?> json) throws IOException {
+  public static byte[] getXAttr(final Map<?, ?> json, final String name) 
+      throws IOException {
     if (json == null) {
       return null;
     }
     
-    Map<?, ?> m = (Map<?, ?>) json.get(XAttr.class.getSimpleName());
-    if (m == null) {
-      return null;
+    Map<String, byte[]> xAttrs = toXAttrs(json);
+    if (xAttrs != null) {
+      return xAttrs.get(name);
     }
-    String name = (String) m.get("name");
-    String value = (String) m.get("value");
-    return XAttrHelper.buildXAttr(name, decodeXAttrValue(value));
+    
+    return null;
   }
   
   public static Map<String, byte[]> toXAttrs(final Map<?, ?> json) 
@@ -741,28 +726,6 @@ public class JsonUtil {
     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) {

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

@@ -50,7 +50,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Options;
 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;
@@ -81,6 +80,7 @@ import org.mortbay.util.ajax.JSON;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
 /** A FileSystem for HDFS over the web. */
@@ -601,6 +601,13 @@ public class WebHdfsFileSystem extends FileSystem
       this.parameters = parameters;
     }
     
+    AbstractFsPathRunner(final HttpOpParam.Op op, Param<?,?>[] parameters,
+        final Path fspath) {
+      super(op, false);
+      this.fspath = fspath;
+      this.parameters = parameters;
+    }
+    
     @Override
     protected URL getUrl() throws IOException {
       return toUrl(op, fspath, parameters);
@@ -630,6 +637,11 @@ public class WebHdfsFileSystem extends FileSystem
       super(op, fspath, parameters);
     }
     
+    FsPathResponseRunner(final HttpOpParam.Op op, Param<?,?>[] parameters,
+        final Path fspath) {
+      super(op, parameters, fspath);
+    }
+    
     @Override
     final T getResponse(HttpURLConnection conn) throws IOException {
       try {
@@ -834,14 +846,13 @@ public class WebHdfsFileSystem extends FileSystem
   }
   
   @Override
-  public byte[] getXAttr(Path p, String name) throws IOException {
-    final HttpOpParam.Op op = GetOpParam.Op.GETXATTR;
+  public byte[] getXAttr(Path p, final String name) throws IOException {
+    final HttpOpParam.Op op = GetOpParam.Op.GETXATTRS;
     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;
+        return JsonUtil.getXAttr(json, name);
       }
     }.run();
   }
@@ -861,12 +872,19 @@ public class WebHdfsFileSystem extends FileSystem
   @Override
   public Map<String, byte[]> getXAttrs(Path p, final List<String> names) 
       throws IOException {
+    Preconditions.checkArgument(names != null && !names.isEmpty(), 
+        "XAttr names cannot be null or empty.");
+    Param<?,?>[] parameters = new Param<?,?>[names.size() + 1];
+    for (int i = 0; i < parameters.length - 1; i++) {
+      parameters[i] = new XAttrNameParam(names.get(i));
+    }
+    parameters[parameters.length - 1] = new XAttrEncodingParam(XAttrCodec.HEX);
+    
     final HttpOpParam.Op op = GetOpParam.Op.GETXATTRS;
-    return new FsPathResponseRunner<Map<String, byte[]>>(op, p, 
-        new XAttrEncodingParam(XAttrCodec.HEX)) {
+    return new FsPathResponseRunner<Map<String, byte[]>>(op, parameters, p) {
       @Override
       Map<String, byte[]> decodeResponse(Map<?, ?> json) throws IOException {
-        return JsonUtil.toXAttrs(json, names);
+        return JsonUtil.toXAttrs(json);
       }
     }.run();
   }

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

@@ -36,7 +36,6 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     /** GET_BLOCK_LOCATIONS is a private unstable op. */
     GET_BLOCK_LOCATIONS(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);

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

@@ -234,6 +234,18 @@ public class TestJsonUtil {
           parsedXAttrMap.get(entry.getKey()));
     }
   }
+  
+  @Test
+  public void testGetXAttrFromJson() throws IOException {
+    String jsonString = 
+        "{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
+        "{\"name\":\"user.a2\",\"value\":\"0x313131\"}]}";
+    Map<?, ?> json = (Map<?, ?>) JSON.parse(jsonString);
+    
+    // Get xattr: user.a2
+    byte[] value = JsonUtil.getXAttr(json, "user.a2");
+    Assert.assertArrayEquals(XAttrCodec.decodeValue("0x313131"), value);
+  }
 
   private void checkDecodeFailure(Map<String, Object> map) {
     try {