You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by li...@apache.org on 2016/12/02 19:14:55 UTC

hadoop git commit: HDFS-11156. Add new op GETFILEBLOCKLOCATIONS to WebHDFS REST API. Contributed by Weiwei Yang

Repository: hadoop
Updated Branches:
  refs/heads/trunk 0cfd7ad21 -> c7ff34f8d


HDFS-11156. Add new op GETFILEBLOCKLOCATIONS to WebHDFS REST API. Contributed by Weiwei Yang


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c7ff34f8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c7ff34f8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c7ff34f8

Branch: refs/heads/trunk
Commit: c7ff34f8dcca3a2024230c5383abd9299daa1b20
Parents: 0cfd7ad
Author: Mingliang Liu <li...@apache.org>
Authored: Fri Dec 2 11:10:09 2016 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Fri Dec 2 11:10:13 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  | 32 ++++++++++++
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      | 13 +++--
 .../hadoop/hdfs/web/resources/GetOpParam.java   | 12 ++++-
 .../web/resources/NamenodeWebHdfsMethods.java   | 17 +++++++
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    | 30 ++++++++++++
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java | 51 ++++++++++++++++++++
 6 files changed, 151 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ff34f8/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
index a75f4f1..12899f4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
@@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectReader;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary.Builder;
 import org.apache.hadoop.fs.FileChecksum;
@@ -588,4 +589,35 @@ class JsonUtilClient {
         lastLocatedBlock, isLastBlockComplete, null, null);
   }
 
+  /** Convert a Json map to BlockLocation. **/
+  static BlockLocation toBlockLocation(Map<String, Object> m)
+      throws IOException{
+    long length = ((Number) m.get("length")).longValue();
+    long offset = ((Number) m.get("offset")).longValue();
+    boolean corrupt = Boolean.
+        getBoolean(m.get("corrupt").toString());
+    String[] storageIds = toStringArray(getList(m, "storageIds"));
+    String[] cachedHosts = toStringArray(getList(m, "cachedHosts"));
+    String[] hosts = toStringArray(getList(m, "hosts"));
+    String[] names = toStringArray(getList(m, "names"));
+    String[] topologyPaths = toStringArray(getList(m, "topologyPaths"));
+    StorageType[] storageTypes = toStorageTypeArray(
+        getList(m, "storageTypes"));
+    return new BlockLocation(names, hosts, cachedHosts,
+        topologyPaths, storageIds, storageTypes,
+        offset, length, corrupt);
+  }
+
+  static String[] toStringArray(List<?> list) {
+    if (list == null) {
+      return null;
+    } else {
+      final String[] array = new String[list.size()];
+      int i = 0;
+      for (Object object : list) {
+        array[i++] = object.toString();
+      }
+      return array;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ff34f8/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index 23804b7..e82e9f6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -1610,13 +1610,20 @@ public class WebHdfsFileSystem extends FileSystem
     statistics.incrementReadOps(1);
     storageStatistics.incrementOpCounter(OpType.GET_FILE_BLOCK_LOCATIONS);
 
-    final HttpOpParam.Op op = GetOpParam.Op.GET_BLOCK_LOCATIONS;
+    final HttpOpParam.Op op = GetOpParam.Op.GETFILEBLOCKLOCATIONS;
     return new FsPathResponseRunner<BlockLocation[]>(op, p,
         new OffsetParam(offset), new LengthParam(length)) {
       @Override
+      @SuppressWarnings("unchecked")
       BlockLocation[] decodeResponse(Map<?,?> json) throws IOException {
-        return DFSUtilClient.locatedBlocks2Locations(
-            JsonUtilClient.toLocatedBlocks(json));
+        List<?> list = JsonUtilClient.getList(json, "BlockLocations");
+        BlockLocation[] locations = new BlockLocation[list.size()];
+        for(int i=0; i<locations.length; i++) {
+          BlockLocation bl = JsonUtilClient.
+              toBlockLocation((Map<String, Object>) list.get(i));
+          locations[i] = bl;
+        }
+        return locations;
       }
     }.run();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ff34f8/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
index 635e6d7..ccb0bb3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
@@ -33,8 +33,18 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     GETHOMEDIRECTORY(false, HttpURLConnection.HTTP_OK),
     GETDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK, true),
 
-    /** GET_BLOCK_LOCATIONS is a private unstable op. */
+    /**
+     * GET_BLOCK_LOCATIONS is a private/stable API op. It returns a
+     * {@link org.apache.hadoop.hdfs.protocol.LocatedBlocks}
+     * json object.
+     */
     GET_BLOCK_LOCATIONS(false, HttpURLConnection.HTTP_OK),
+    /**
+     * GETFILEBLOCKLOCATIONS is the public op that complies with
+     * {@link org.apache.hadoop.fs.FileSystem#getFileBlockLocations}
+     * interface.
+     */
+    GETFILEBLOCKLOCATIONS(false, HttpURLConnection.HTTP_OK),
     GETACLSTATUS(false, HttpURLConnection.HTTP_OK),
     GETXATTRS(false, HttpURLConnection.HTTP_OK),
     GETTRASHROOT(false, HttpURLConnection.HTTP_OK),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ff34f8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index 5d9b12a..107d4ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -54,6 +54,7 @@ import javax.ws.rs.core.StreamingOutput;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -975,6 +976,22 @@ public class NamenodeWebHdfsMethods {
         return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
       }
     }
+    case GETFILEBLOCKLOCATIONS:
+    {
+      final long offsetValue = offset.getValue();
+      final Long lengthValue = length.getValue();
+
+      try (final FileSystem fs = FileSystem.get(conf != null ?
+          conf : new Configuration())) {
+        BlockLocation[] locations = fs.getFileBlockLocations(
+            new org.apache.hadoop.fs.Path(fullpath),
+            offsetValue,
+            lengthValue != null? lengthValue: Long.MAX_VALUE);
+        final String js = JsonUtil.toJsonString("BlockLocations",
+            JsonUtil.toJsonArray(locations));
+        return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+      }
+    }
     case GET_BLOCK_LOCATIONS:
     {
       final long offsetValue = offset.getValue();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ff34f8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
index a0dadbd..affa861 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
@@ -436,4 +436,34 @@ public class JsonUtil {
     return MAPPER.writeValueAsString(obj);
   }
 
+  public static Object[] toJsonArray(BlockLocation[] locations)
+      throws IOException {
+    if(locations == null) {
+      return null;
+    }
+    Object[] blockLocations = new Object[locations.length];
+    for(int i=0; i<locations.length; i++) {
+      blockLocations[i] = toJsonMap(locations[i]);
+    }
+    return blockLocations;
+  }
+
+  public static Map<String, Object> toJsonMap(
+      final BlockLocation blockLocation) throws IOException {
+    if (blockLocation == null) {
+      return null;
+    }
+
+    final Map<String, Object> m = new TreeMap<String, Object>();
+    m.put("length", blockLocation.getLength());
+    m.put("offset", blockLocation.getOffset());
+    m.put("corrupt", blockLocation.isCorrupt());
+    m.put("storageTypes", toJsonArray(blockLocation.getStorageTypes()));
+    m.put("storageIds", blockLocation.getStorageIds());
+    m.put("cachedHosts", blockLocation.getCachedHosts());
+    m.put("hosts", blockLocation.getHosts());
+    m.put("names", blockLocation.getNames());
+    m.put("topologyPaths", blockLocation.getTopologyPaths());
+    return m;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ff34f8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
index 5386a45..82b708a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
@@ -37,6 +37,7 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
+import java.util.Map;
 import java.util.Random;
 
 import org.apache.commons.io.IOUtils;
@@ -88,6 +89,8 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.type.MapType;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Mockito.doReturn;
@@ -852,6 +855,46 @@ public class TestWebHDFS {
         Assert.assertTrue(storageTypes != null && storageTypes.length > 0 &&
             storageTypes[0] == StorageType.DISK);
       }
+
+      // Query webhdfs REST API to get block locations
+      InetSocketAddress addr = cluster.getNameNode().getHttpAddress();
+      URL url = new URL("http", addr.getHostString(), addr.getPort(),
+          WebHdfsFileSystem.PATH_PREFIX + "/foo?op=GETFILEBLOCKLOCATIONS");
+      LOG.info("Sending GETFILEBLOCKLOCATIONS request " + url);
+
+      String response = getResponse(url, "GET");
+      LOG.info("The output of GETFILEBLOCKLOCATIONS request " + response);
+      // Expected output from rest API
+      // { "BlockLoactions" : [{Block_Loation_Json}, ...] }
+      ObjectMapper mapper = new ObjectMapper();
+      MapType jsonType = mapper.getTypeFactory().constructMapType(
+          Map.class,
+          String.class,
+          BlockLocation[].class);
+      Map<String, BlockLocation[]> jsonMap = mapper.readValue(response,
+          jsonType);
+      BlockLocation[] array = jsonMap.get("BlockLocations");
+
+      for(int i=0; i<locations.length; i++) {
+        BlockLocation raw = locations[i];
+        BlockLocation rest = array[i];
+        Assert.assertEquals(raw.getLength(),
+            rest.getLength());
+        Assert.assertEquals(raw.getOffset(),
+            rest.getOffset());
+        Assert.assertArrayEquals(raw.getCachedHosts(),
+            rest.getCachedHosts());
+        Assert.assertArrayEquals(raw.getHosts(),
+            rest.getHosts());
+        Assert.assertArrayEquals(raw.getNames(),
+            rest.getNames());
+        Assert.assertArrayEquals(raw.getStorageIds(),
+            rest.getStorageIds());
+        Assert.assertArrayEquals(raw.getTopologyPaths(),
+            rest.getTopologyPaths());
+        Assert.assertArrayEquals(raw.getStorageTypes(),
+            rest.getStorageTypes());
+      }
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -859,6 +902,14 @@ public class TestWebHDFS {
     }
   }
 
+  private static String getResponse(URL url, String httpRequestType)
+      throws IOException {
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod(httpRequestType);
+    conn.setInstanceFollowRedirects(false);
+    return IOUtils.toString(conn.getInputStream());
+  }
+
   private WebHdfsFileSystem createWebHDFSAsTestUser(final Configuration conf,
       final URI uri, final String userName) throws Exception {
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org