Quellcode durchsuchen

HDFS-8512. WebHDFS : GETFILESTATUS should return LocatedBlock with storage type info. Contributed by Xiaoyu Yao.

(cherry picked from commit e3d673901b396cf5bbede5ed6f607ce68301ec0a)
Xiaoyu Yao vor 9 Jahren
Ursprung
Commit
4f1f9f7eff

+ 20 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java

@@ -268,6 +268,23 @@ class JsonUtilClient {
     }
   }
 
+  /** Convert an Object[] to a StorageType[]. */
+  static StorageType[] toStorageTypeArray(final List<?> objects)
+      throws IOException {
+    if (objects == null) {
+      return null;
+    } else if (objects.isEmpty()) {
+      return StorageType.EMPTY_ARRAY;
+    } else {
+      final StorageType[] array = new StorageType[objects.size()];
+      int i = 0;
+      for (Object object : objects) {
+        array[i++] = StorageType.parseStorageType(object.toString());
+      }
+      return array;
+    }
+  }
+
   /** Convert a Json map to LocatedBlock. */
   static LocatedBlock toLocatedBlock(final Map<?, ?> m) throws IOException {
     if (m == null) {
@@ -282,8 +299,10 @@ class JsonUtilClient {
     final DatanodeInfo[] cachedLocations = toDatanodeInfoArray(
         getList(m, "cachedLocations"));
 
+    final StorageType[] storageTypes = toStorageTypeArray(
+        getList(m, "storageTypes"));
     final LocatedBlock locatedblock = new LocatedBlock(b, locations,
-        null, null, startOffset, isCorrupt, cachedLocations);
+        null, storageTypes, startOffset, isCorrupt, cachedLocations);
     locatedblock.setBlockToken(toBlockToken((Map<?, ?>)m.get("blockToken")));
     return locatedblock;
   }

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

@@ -820,6 +820,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9438. TestPipelinesFailover assumes Linux ifconfig.
     (John Zhuge via Yongjun Zhang)
 
+    HDFS-8512. WebHDFS : GETFILESTATUS should return LocatedBlock with storage
+    type info. (xyao)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

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

@@ -190,6 +190,21 @@ public class JsonUtil {
     }
   }
 
+  /** Convert a StorageType[] to a Json array. */
+  private static Object[] toJsonArray(final StorageType[] array) {
+    if (array == null) {
+      return null;
+    } else if (array.length == 0) {
+      return EMPTY_OBJECT_ARRAY;
+    } else {
+      final Object[] a = new Object[array.length];
+      for(int i = 0; i < array.length; i++) {
+        a[i] = array[i];
+      }
+      return a;
+    }
+  }
+
   /** Convert a LocatedBlock to a Json map. */
   private static Map<String, Object> toJsonMap(final LocatedBlock locatedblock
       ) throws IOException {
@@ -202,6 +217,7 @@ public class JsonUtil {
     m.put("isCorrupt", locatedblock.isCorrupt());
     m.put("startOffset", locatedblock.getStartOffset());
     m.put("block", toJsonMap(locatedblock.getBlock()));
+    m.put("storageTypes", toJsonArray(locatedblock.getStorageTypes()));
     m.put("locations", toJsonArray(locatedblock.getLocations()));
     m.put("cachedLocations", toJsonArray(locatedblock.getCachedLocations()));
     return m;

+ 31 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java

@@ -37,6 +37,7 @@ import org.apache.commons.io.IOUtils;
 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.CommonConfigurationKeys;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -702,6 +703,36 @@ public class TestWebHDFS {
     }
   }
 
+  @Test
+  public void testWebHdfsGetBlockLocationsWithStorageType() throws Exception{
+    MiniDFSCluster cluster = null;
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    final int OFFSET = 42;
+    final int LENGTH = 512;
+    final Path PATH = new Path("/foo");
+    byte[] CONTENTS = new byte[1024];
+    RANDOM.nextBytes(CONTENTS);
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      final WebHdfsFileSystem fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+          WebHdfsConstants.WEBHDFS_SCHEME);
+      try (OutputStream os = fs.create(PATH)) {
+        os.write(CONTENTS);
+      }
+      BlockLocation[] locations = fs.getFileBlockLocations(PATH, OFFSET,
+          LENGTH);
+      for (BlockLocation location: locations) {
+        StorageType[] storageTypes = location.getStorageTypes();
+        Assert.assertTrue(storageTypes != null && storageTypes.length > 0 &&
+            storageTypes[0] == StorageType.DISK);
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
   private WebHdfsFileSystem createWebHDFSAsTestUser(final Configuration conf,
       final URI uri, final String userName) throws Exception {