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 xg...@apache.org on 2016/12/09 22:25:52 UTC

[15/50] [abbrv] hadoop git commit: Revert "HDFS-11156. Add new op GETFILEBLOCKLOCATIONS to WebHDFS REST API. Contributed by Weiwei Yang"

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

This reverts commit c7ff34f8dcca3a2024230c5383abd9299daa1b20.


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

Branch: refs/heads/YARN-5734
Commit: 08a7253bc0eb6c9155457feecb9c5cdc17c3a814
Parents: b2a3d6c
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Dec 5 23:08:49 2016 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Mon Dec 5 23:09:35 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, 4 insertions(+), 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/08a7253b/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 12899f4..a75f4f1 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,7 +22,6 @@ 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;
@@ -589,35 +588,4 @@ 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/08a7253b/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 e82e9f6..23804b7 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,20 +1610,13 @@ public class WebHdfsFileSystem extends FileSystem
     statistics.incrementReadOps(1);
     storageStatistics.incrementOpCounter(OpType.GET_FILE_BLOCK_LOCATIONS);
 
-    final HttpOpParam.Op op = GetOpParam.Op.GETFILEBLOCKLOCATIONS;
+    final HttpOpParam.Op op = GetOpParam.Op.GET_BLOCK_LOCATIONS;
     return new FsPathResponseRunner<BlockLocation[]>(op, p,
         new OffsetParam(offset), new LengthParam(length)) {
       @Override
-      @SuppressWarnings("unchecked")
       BlockLocation[] decodeResponse(Map<?,?> json) throws IOException {
-        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;
+        return DFSUtilClient.locatedBlocks2Locations(
+            JsonUtilClient.toLocatedBlocks(json));
       }
     }.run();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08a7253b/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 ccb0bb3..635e6d7 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,18 +33,8 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     GETHOMEDIRECTORY(false, HttpURLConnection.HTTP_OK),
     GETDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK, true),
 
-    /**
-     * GET_BLOCK_LOCATIONS is a private/stable API op. It returns a
-     * {@link org.apache.hadoop.hdfs.protocol.LocatedBlocks}
-     * json object.
-     */
+    /** GET_BLOCK_LOCATIONS is a private unstable op. */
     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/08a7253b/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 107d4ed..5d9b12a 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,7 +54,6 @@ 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;
@@ -976,22 +975,6 @@ 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/08a7253b/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 affa861..a0dadbd 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,34 +436,4 @@ 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/08a7253b/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 82b708a..5386a45 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,7 +37,6 @@ 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;
@@ -89,8 +88,6 @@ 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;
@@ -855,46 +852,6 @@ 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();
@@ -902,14 +859,6 @@ 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