浏览代码

Merge branch 'trunk' into HDFS-7240

 Conflicts:
	hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
	hadoop-hdfs-project/hadoop-hdfs/pom.xml
	hadoop-tools/pom.xml
Anu Engineer 7 年之前
父节点
当前提交
7abc4af9e0
共有 100 个文件被更改,包括 6025 次插入280 次删除
  1. 27 0
      hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
  2. 5 9
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
  3. 10 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
  4. 6 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  5. 4 9
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
  6. 46 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InvalidPathHandleException.java
  7. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
  8. 2 9
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
  9. 8 9
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java
  10. 12 17
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
  11. 18 12
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java
  12. 0 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/BlockingThreadPoolExecutorService.java
  13. 0 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SemaphoredDelegatingExecutor.java
  14. 36 0
      hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
  15. 1 1
      hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
  16. 360 0
      hadoop-common-project/hadoop-common/src/site/markdown/release/3.0.0/CHANGES.3.0.0.md
  17. 139 0
      hadoop-common-project/hadoop-common/src/site/markdown/release/3.0.0/RELEASENOTES.3.0.0.md
  18. 2 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  19. 4 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
  20. 2 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
  21. 3 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodec.java
  22. 2 6
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/MiniKMS.java
  23. 3 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  24. 3 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
  25. 7 9
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsPathHandle.java
  26. 83 20
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
  27. 89 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java
  28. 37 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  29. 15 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
  30. 11 0
      hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.0.0.xml
  31. 6 1
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  32. 29 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  33. 37 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java
  34. 134 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
  35. 174 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
  36. 28 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  37. 222 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
  38. 103 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
  39. 153 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
  40. 14 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  41. 110 39
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  42. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
  43. 41 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  44. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  45. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java
  46. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStats.java
  47. 12 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
  48. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
  49. 109 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LocatedBlockBuilder.java
  50. 540 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
  51. 32 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/StorageTypeStats.java
  52. 33 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockAlias.java
  53. 85 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java
  54. 66 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
  55. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
  56. 113 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
  57. 178 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
  58. 274 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java
  59. 490 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
  60. 27 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/package-info.java
  61. 19 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
  62. 31 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  63. 18 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  64. 122 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
  65. 350 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
  66. 136 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
  67. 19 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
  68. 25 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java
  69. 39 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
  70. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockChecksumReconstructor.java
  71. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
  72. 32 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
  73. 58 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  74. 23 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
  75. 13 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
  76. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java
  77. 718 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
  78. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java
  79. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java
  80. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java
  81. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java
  82. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/StateStoreMetrics.java
  83. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java
  84. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java
  85. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java
  86. 71 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java
  87. 82 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterPermissionChecker.java
  88. 45 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MountTableStoreImpl.java
  89. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.java
  90. 68 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java
  91. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java
  92. 61 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MountTablePBImpl.java
  93. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
  94. 8 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  95. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageCompression.java
  96. 12 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  97. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
  98. 9 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  99. 21 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  100. 8 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java

+ 27 - 0
hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml

@@ -211,10 +211,37 @@
         <include>**/*</include>
       </includes>
     </fileSet>
+    <!-- Copy dependecies from hadoop-yarn-server-timelineservice as well -->
+    <fileSet>
+      <directory>hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/target/lib</directory>
+      <outputDirectory>share/hadoop/${hadoop.component}/timelineservice/lib</outputDirectory>
+    </fileSet>
   </fileSets>
   <moduleSets>
+    <moduleSet>
+      <includes>
+        <include>org.apache.hadoop:hadoop-yarn-server-timelineservice</include>
+        <include>org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase</include>
+      </includes>
+      <binaries>
+        <outputDirectory>share/hadoop/${hadoop.component}/timelineservice</outputDirectory>
+        <includeDependencies>false</includeDependencies>
+        <unpack>false</unpack>
+      </binaries>
+    </moduleSet>
+    <moduleSet>
+      <includes>
+        <include>org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase-tests</include>
+      </includes>
+      <binaries>
+        <outputDirectory>share/hadoop/${hadoop.component}/timelineservice/test</outputDirectory>
+        <includeDependencies>false</includeDependencies>
+        <unpack>false</unpack>
+      </binaries>
+    </moduleSet>
     <moduleSet>
       <excludes>
+        <exclude>org.apache.hadoop:hadoop-yarn-server-timelineservice*</exclude>
         <exclude>org.apache.hadoop:hadoop-yarn-ui</exclude>
       </excludes>
       <binaries>

+ 5 - 9
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java

@@ -2121,17 +2121,13 @@ public class FileContext {
               content.getPath().getName())), deleteSource, overwrite);
         }
       } else {
-        InputStream in=null;
-        OutputStream out = null;
-        try {
-          in = open(qSrc);
-          EnumSet<CreateFlag> createFlag = overwrite ? EnumSet.of(
-              CreateFlag.CREATE, CreateFlag.OVERWRITE) : 
-                EnumSet.of(CreateFlag.CREATE);
-          out = create(qDst, createFlag);
+        EnumSet<CreateFlag> createFlag = overwrite ? EnumSet.of(
+            CreateFlag.CREATE, CreateFlag.OVERWRITE) :
+            EnumSet.of(CreateFlag.CREATE);
+        InputStream in = open(qSrc);
+        try (OutputStream out = create(qDst, createFlag)) {
           IOUtils.copyBytes(in, out, conf, true);
         } finally {
-          IOUtils.closeStream(out);
           IOUtils.closeStream(in);
         }
       }

+ 10 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java

@@ -444,18 +444,18 @@ public class FileStatus implements Writable, Comparable<Object>,
     StringBuilder sb = new StringBuilder();
     sb.append(getClass().getSimpleName()); 
     sb.append("{");
-    sb.append("path=" + path);
-    sb.append("; isDirectory=" + isdir);
+    sb.append("path=" + getPath());
+    sb.append("; isDirectory=" + isDirectory());
     if(!isDirectory()){
-      sb.append("; length=" + length);
-      sb.append("; replication=" + block_replication);
-      sb.append("; blocksize=" + blocksize);
+      sb.append("; length=" + getLen());
+      sb.append("; replication=" + getReplication());
+      sb.append("; blocksize=" + getBlockSize());
     }
-    sb.append("; modification_time=" + modification_time);
-    sb.append("; access_time=" + access_time);
-    sb.append("; owner=" + owner);
-    sb.append("; group=" + group);
-    sb.append("; permission=" + permission);
+    sb.append("; modification_time=" + getModificationTime());
+    sb.append("; access_time=" + getAccessTime());
+    sb.append("; owner=" + getOwner());
+    sb.append("; group=" + getGroup());
+    sb.append("; permission=" + getPermission());
     sb.append("; isSymlink=" + isSymlink());
     if(isSymlink()) {
       try {

+ 6 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java

@@ -957,6 +957,8 @@ public abstract class FileSystem extends Configured implements Closeable {
    * resource directly and verify that the resource referenced
    * satisfies constraints specified at its construciton.
    * @param fd PathHandle object returned by the FS authority.
+   * @throws InvalidPathHandleException If {@link PathHandle} constraints are
+   *                                    not satisfied
    * @throws IOException IO failure
    * @throws UnsupportedOperationException If {@link #open(PathHandle, int)}
    *                                       not overridden by subclass
@@ -973,6 +975,8 @@ public abstract class FileSystem extends Configured implements Closeable {
    * satisfies constraints specified at its construciton.
    * @param fd PathHandle object returned by the FS authority.
    * @param bufferSize the size of the buffer to use
+   * @throws InvalidPathHandleException If {@link PathHandle} constraints are
+   *                                    not satisfied
    * @throws IOException IO failure
    * @throws UnsupportedOperationException If not overridden by subclass
    */
@@ -994,6 +998,8 @@ public abstract class FileSystem extends Configured implements Closeable {
    *         the specified constraints.
    */
   public final PathHandle getPathHandle(FileStatus stat, HandleOpt... opt) {
+    // method is final with a default so clients calling getPathHandle(stat)
+    // get the same semantics for all FileSystem implementations
     if (null == opt || 0 == opt.length) {
       return createPathHandle(stat, HandleOpt.path());
     }

+ 4 - 9
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java

@@ -1349,15 +1349,10 @@ public class FileUtil {
 
     // Write the manifest to output JAR file
     File classPathJar = File.createTempFile("classpath-", ".jar", workingDir);
-    FileOutputStream fos = null;
-    BufferedOutputStream bos = null;
-    JarOutputStream jos = null;
-    try {
-      fos = new FileOutputStream(classPathJar);
-      bos = new BufferedOutputStream(fos);
-      jos = new JarOutputStream(bos, jarManifest);
-    } finally {
-      IOUtils.cleanupWithLogger(LOG, jos, bos, fos);
+    try (FileOutputStream fos = new FileOutputStream(classPathJar);
+         BufferedOutputStream bos = new BufferedOutputStream(fos)) {
+      JarOutputStream jos = new JarOutputStream(bos, jarManifest);
+      jos.close();
     }
     String[] jarCp = {classPathJar.getCanonicalPath(),
                         unexpandedWildcardClasspath.toString()};

+ 46 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InvalidPathHandleException.java

@@ -0,0 +1,46 @@
+/**
+ * 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.fs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when the constraints enoded in a {@link PathHandle} do not hold.
+ * For example, if a handle were created with the default
+ * {@link Options.HandleOpt#path()} constraints, a call to
+ * {@link FileSystem#open(PathHandle)} would succeed if the file were
+ * modified, but if a different file was at that location then it would throw
+ * this exception.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class InvalidPathHandleException extends IOException {
+  private static final long serialVersionUID = 0xcd8ac329L;
+
+  public InvalidPathHandleException(String str) {
+    super(str);
+  }
+
+  public InvalidPathHandleException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+}

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java

@@ -37,7 +37,8 @@ public enum StorageType {
   RAM_DISK(true),
   SSD(false),
   DISK(false),
-  ARCHIVE(false);
+  ARCHIVE(false),
+  PROVIDED(false);
 
   private final boolean isTransient;
 

+ 2 - 9
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.fs.shell;
 
 import java.io.FileInputStream;
 import java.io.IOException;
-import java.io.InputStream;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.Iterator;
@@ -464,10 +463,8 @@ class CopyCommands {
         dst.fs.create(dst.path, false).close();
       }
 
-      InputStream is = null;
-      FSDataOutputStream fos = dst.fs.append(dst.path);
-
-      try {
+      FileInputStream is = null;
+      try (FSDataOutputStream fos = dst.fs.append(dst.path)) {
         if (readStdin) {
           if (args.size() == 0) {
             IOUtils.copyBytes(System.in, fos, DEFAULT_IO_LENGTH);
@@ -488,10 +485,6 @@ class CopyCommands {
         if (is != null) {
           IOUtils.closeStream(is);
         }
-
-        if (fos != null) {
-          IOUtils.closeStream(fos);
-        }
       }
     }
   }

+ 8 - 9
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java

@@ -986,23 +986,22 @@ public class MapFile {
     Configuration conf = new Configuration();
     FileSystem fs = FileSystem.getLocal(conf);
     MapFile.Reader reader = null;
-    MapFile.Writer writer = null;
     try {
       reader = new MapFile.Reader(fs, in, conf);
-      writer =
-        new MapFile.Writer(conf, fs, out,
-            reader.getKeyClass().asSubclass(WritableComparable.class),
-            reader.getValueClass());
-
       WritableComparable<?> key = ReflectionUtils.newInstance(
           reader.getKeyClass().asSubclass(WritableComparable.class), conf);
       Writable value = ReflectionUtils.newInstance(reader.getValueClass()
         .asSubclass(Writable.class), conf);
 
-      while (reader.next(key, value))               // copy all entries
-        writer.append(key, value);
+      try (MapFile.Writer writer = new MapFile.Writer(conf, fs, out,
+            reader.getKeyClass().asSubclass(WritableComparable.class),
+            reader.getValueClass())) {
+        while (reader.next(key, value)) {             // copy all entries
+          writer.append(key, value);
+        }
+      }
     } finally {
-      IOUtils.cleanupWithLogger(LOG, writer, reader);
+      IOUtils.cleanupWithLogger(LOG, reader);
     }
   }
 }

+ 12 - 17
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java

@@ -956,28 +956,23 @@ public class NativeIO {
     if (nativeLoaded && Shell.WINDOWS) {
       copyFileUnbuffered0(src.getAbsolutePath(), dst.getAbsolutePath());
     } else {
-      FileInputStream fis = null;
-      FileOutputStream fos = null;
+      FileInputStream fis = new FileInputStream(src);
       FileChannel input = null;
-      FileChannel output = null;
       try {
-        fis = new FileInputStream(src);
-        fos = new FileOutputStream(dst);
         input = fis.getChannel();
-        output = fos.getChannel();
-        long remaining = input.size();
-        long position = 0;
-        long transferred = 0;
-        while (remaining > 0) {
-          transferred = input.transferTo(position, remaining, output);
-          remaining -= transferred;
-          position += transferred;
+        try (FileOutputStream fos = new FileOutputStream(dst);
+             FileChannel output = fos.getChannel()) {
+          long remaining = input.size();
+          long position = 0;
+          long transferred = 0;
+          while (remaining > 0) {
+            transferred = input.transferTo(position, remaining, output);
+            remaining -= transferred;
+            position += transferred;
+          }
         }
       } finally {
-        IOUtils.cleanupWithLogger(LOG, output);
-        IOUtils.cleanupWithLogger(LOG, fos);
-        IOUtils.cleanupWithLogger(LOG, input);
-        IOUtils.cleanupWithLogger(LOG, fis);
+        IOUtils.cleanupWithLogger(LOG, input, fis);
       }
     }
   }

+ 18 - 12
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java

@@ -81,6 +81,11 @@ public class KDiag extends Configured implements Tool, Closeable {
    * variable. This is what kinit will use by default: {@value}
    */
   public static final String KRB5_CCNAME = "KRB5CCNAME";
+  /**
+   * Location of main kerberos configuration file as passed down via an
+   * environment variable.
+   */
+  public static final String KRB5_CONFIG = "KRB5_CONFIG";
   public static final String JAVA_SECURITY_KRB5_CONF
     = "java.security.krb5.conf";
   public static final String JAVA_SECURITY_KRB5_REALM
@@ -321,14 +326,15 @@ public class KDiag extends Configured implements Tool, Closeable {
 
     title("Environment Variables");
     for (String env : new String[]{
-      HADOOP_JAAS_DEBUG,
-      KRB5_CCNAME,
-      HADOOP_USER_NAME,
-      HADOOP_PROXY_USER,
-      HADOOP_TOKEN_FILE_LOCATION,
-      "HADOOP_SECURE_LOG",
-      "HADOOP_OPTS",
-      "HADOOP_CLIENT_OPTS",
+        HADOOP_JAAS_DEBUG,
+        KRB5_CCNAME,
+        KRB5_CONFIG,
+        HADOOP_USER_NAME,
+        HADOOP_PROXY_USER,
+        HADOOP_TOKEN_FILE_LOCATION,
+        "HADOOP_SECURE_LOG",
+        "HADOOP_OPTS",
+        "HADOOP_CLIENT_OPTS",
     }) {
       printEnv(env);
     }
@@ -562,14 +568,14 @@ public class KDiag extends Configured implements Tool, Closeable {
         krbPath = jvmKrbPath;
       }
 
-      String krb5name = System.getenv(KRB5_CCNAME);
+      String krb5name = System.getenv(KRB5_CONFIG);
       if (krb5name != null) {
         println("Setting kerberos path from environment variable %s: \"%s\"",
-          KRB5_CCNAME, krb5name);
+            KRB5_CONFIG, krb5name);
         krbPath = krb5name;
         if (jvmKrbPath != null) {
           println("Warning - both %s and %s were set - %s takes priority",
-            JAVA_SECURITY_KRB5_CONF, KRB5_CCNAME, KRB5_CCNAME);
+              JAVA_SECURITY_KRB5_CONF, KRB5_CONFIG, KRB5_CONFIG);
         }
       }
 
@@ -919,7 +925,7 @@ public class KDiag extends Configured implements Tool, Closeable {
   private void dump(File file) throws IOException {
     try (FileInputStream in = new FileInputStream(file)) {
       for (String line : IOUtils.readLines(in)) {
-        println(line);
+        println("%s", line);
       }
     }
   }

+ 0 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/BlockingThreadPoolExecutorService.java


+ 0 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SemaphoredDelegatingExecutor.java


+ 36 - 0
hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md

@@ -419,6 +419,42 @@ contains tags such as Hostname as additional information along with metrics.
 | `FileIoErrorRateNumOps` | The number of file io error operations within an interval time of metric |
 | `FileIoErrorRateAvgTime` | It measures the mean time in milliseconds from the start of an operation to hitting a failure |
 
+RouterRPCMetrics
+----------------
+RouterRPCMetrics shows the statistics of the Router component in Router-based federation.
+
+| Name | Description |
+|:---- |:---- |
+| `ProcessingOp` | Number of operations the Router processed internally |
+| `ProxyOp` | Number of operations the Router proxied to a Namenode |
+| `ProxyOpFailureStandby` | Number of operations to fail to reach NN |
+| `ProxyOpFailureCommunicate` | Number of operations to hit a standby NN |
+| `ProxyOpNotImplemented` | Number of operations not implemented |
+| `RouterFailureStateStore` | Number of failed requests due to State Store unavailable |
+| `RouterFailureReadOnly` | Number of failed requests due to read only mount point |
+| `RouterFailureLocked` | Number of failed requests due to locked path |
+| `RouterFailureSafemode` | Number of failed requests due to safe mode |
+| `ProcessingNumOps` | Number of operations the Router processed internally within an interval time of metric |
+| `ProcessingAvgTime` | Average time for the Router to process operations in nanoseconds |
+| `ProxyNumOps` | Number of times of that the Router to proxy operations to the Namenodes within an interval time of metric |
+| `ProxyAvgTime` | Average time for the Router to proxy operations to the Namenodes in nanoseconds |
+
+StateStoreMetrics
+-----------------
+StateStoreMetrics shows the statistics of the State Store component in Router-based federation.
+
+| Name | Description |
+|:---- |:---- |
+| `ReadsNumOps` | Number of GET transactions for State Store within an interval time of metric |
+| `ReadsAvgTime` | Average time of GET transactions for State Store in milliseconds |
+| `WritesNumOps` | Number of PUT transactions for State Store within an interval time of metric |
+| `WritesAvgTime` | Average time of PUT transactions for State Store in milliseconds |
+| `RemovesNumOps` | Number of REMOVE transactions for State Store within an interval time of metric |
+| `RemovesAvgTime` | Average time of REMOVE transactions for State Store in milliseconds |
+| `FailuresNumOps` | Number of failed transactions for State Store within an interval time of metric |
+| `FailuresAvgTime` | Average time of failed transactions for State Store in milliseconds |
+| `Cache`*BaseRecord*`Size` | Number of store records to cache in State Store |
+
 yarn context
 ============
 

+ 1 - 1
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md

@@ -790,7 +790,7 @@ Implementaions without a compliant call MUST throw `UnsupportedOperationExceptio
       (FS.Directories', FS.Files', FS.Symlinks')
       p' in FS.Files' where:
         FS.Files'[p'] = fd
-    if not exists(FS', p') : raise FileNotFoundException
+    if not exists(FS', p') : raise InvalidPathHandleException
 
 The implementation MUST resolve the referent of the `PathHandle` following
 the constraints specified at its creation by `getPathHandle(FileStatus)`.

+ 360 - 0
hadoop-common-project/hadoop-common/src/site/markdown/release/3.0.0/CHANGES.3.0.0.md

@@ -0,0 +1,360 @@
+
+<!---
+# 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.
+-->
+# Apache Hadoop Changelog
+
+## Release 3.0.0 - 2017-12-08
+
+### INCOMPATIBLE CHANGES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-6623](https://issues.apache.org/jira/browse/YARN-6623) | Add support to turn off launching privileged containers in the container-executor |  Blocker | nodemanager | Varun Vasudev | Varun Vasudev |
+| [HADOOP-14816](https://issues.apache.org/jira/browse/HADOOP-14816) | Update Dockerfile to use Xenial |  Major | build, test | Allen Wittenauer | Allen Wittenauer |
+| [HADOOP-14957](https://issues.apache.org/jira/browse/HADOOP-14957) | ReconfigurationTaskStatus is exposing guava Optional in its public api |  Major | common | Haibo Chen | Xiao Chen |
+| [MAPREDUCE-6983](https://issues.apache.org/jira/browse/MAPREDUCE-6983) | Moving logging APIs over to slf4j in hadoop-mapreduce-client-core |  Major | . | Jinjiang Ling | Jinjiang Ling |
+| [HDFS-12682](https://issues.apache.org/jira/browse/HDFS-12682) | ECAdmin -listPolicies will always show SystemErasureCodingPolicies state as DISABLED |  Blocker | erasure-coding | Xiao Chen | Xiao Chen |
+
+
+### NEW FEATURES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-1492](https://issues.apache.org/jira/browse/YARN-1492) | truly shared cache for jars (jobjar/libjar) |  Major | . | Sangjin Lee | Chris Trezzo |
+| [HDFS-10467](https://issues.apache.org/jira/browse/HDFS-10467) | Router-based HDFS federation |  Major | fs | Íñigo Goiri | Íñigo Goiri |
+| [YARN-5734](https://issues.apache.org/jira/browse/YARN-5734) | OrgQueue for easy CapacityScheduler queue configuration management |  Major | . | Min Shen | Min Shen |
+| [MAPREDUCE-5951](https://issues.apache.org/jira/browse/MAPREDUCE-5951) | Add support for the YARN Shared Cache |  Major | . | Chris Trezzo | Chris Trezzo |
+| [YARN-6871](https://issues.apache.org/jira/browse/YARN-6871) | Add additional deSelects params in RMWebServices#getAppReport |  Major | resourcemanager, router | Giovanni Matteo Fumarola | Tanuj Nayak |
+| [HADOOP-14840](https://issues.apache.org/jira/browse/HADOOP-14840) | Tool to estimate resource requirements of an application pipeline based on prior executions |  Major | tools | Subru Krishnan | Rui Li |
+| [YARN-3813](https://issues.apache.org/jira/browse/YARN-3813) | Support Application timeout feature in YARN. |  Major | scheduler | nijel | Rohith Sharma K S |
+
+
+### IMPROVEMENTS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-7045](https://issues.apache.org/jira/browse/YARN-7045) | Remove FSLeafQueue#addAppSchedulable |  Major | fairscheduler | Yufei Gu | Sen Zhao |
+| [YARN-7240](https://issues.apache.org/jira/browse/YARN-7240) | Add more states and transitions to stabilize the NM Container state machine |  Major | . | Arun Suresh | kartheek muthyala |
+| [HADOOP-14909](https://issues.apache.org/jira/browse/HADOOP-14909) | Fix the word of "erasure encoding" in the top page |  Trivial | documentation | Takanobu Asanuma | Takanobu Asanuma |
+| [HADOOP-14095](https://issues.apache.org/jira/browse/HADOOP-14095) | Document caveats about the default JavaKeyStoreProvider in KMS |  Major | documentation, kms | Xiao Chen | Xiao Chen |
+| [HADOOP-14928](https://issues.apache.org/jira/browse/HADOOP-14928) | Update site release notes for 3.0.0 GA |  Major | site | Andrew Wang | Andrew Wang |
+| [HDFS-12420](https://issues.apache.org/jira/browse/HDFS-12420) | Add an option to disallow 'namenode format -force' |  Major | . | Ajay Kumar | Ajay Kumar |
+| [HADOOP-14521](https://issues.apache.org/jira/browse/HADOOP-14521) | KMS client needs retry logic |  Major | . | Rushabh S Shah | Rushabh S Shah |
+| [YARN-2162](https://issues.apache.org/jira/browse/YARN-2162) | add ability in Fair Scheduler to optionally configure maxResources in terms of percentage |  Major | fairscheduler, scheduler | Ashwin Shankar | Yufei Gu |
+| [YARN-7207](https://issues.apache.org/jira/browse/YARN-7207) | Cache the RM proxy server address |  Major | RM | Yufei Gu | Yufei Gu |
+| [HADOOP-14939](https://issues.apache.org/jira/browse/HADOOP-14939) | Update project release notes with HDFS-10467 for 3.0.0 |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-12573](https://issues.apache.org/jira/browse/HDFS-12573) | Divide the total block metrics into replica and ec |  Major | erasure-coding, metrics, namenode | Takanobu Asanuma | Takanobu Asanuma |
+| [HDFS-12553](https://issues.apache.org/jira/browse/HDFS-12553) | Add nameServiceId to QJournalProtocol |  Major | qjm | Bharat Viswanadham | Bharat Viswanadham |
+| [HDFS-12603](https://issues.apache.org/jira/browse/HDFS-12603) | Enable async edit logging by default |  Major | namenode | Andrew Wang | Andrew Wang |
+| [HDFS-12642](https://issues.apache.org/jira/browse/HDFS-12642) | Log block and datanode details in BlockRecoveryWorker |  Major | datanode | Xiao Chen | Xiao Chen |
+| [HADOOP-14938](https://issues.apache.org/jira/browse/HADOOP-14938) | Configuration.updatingResource map should be initialized lazily |  Major | . | Misha Dmitriev | Misha Dmitriev |
+| [HDFS-12613](https://issues.apache.org/jira/browse/HDFS-12613) | Native EC coder should implement release() as idempotent function. |  Major | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [MAPREDUCE-6972](https://issues.apache.org/jira/browse/MAPREDUCE-6972) | Enable try-with-resources for RecordReader |  Major | . | Zoltan Haindrich | Zoltan Haindrich |
+| [HADOOP-14880](https://issues.apache.org/jira/browse/HADOOP-14880) | [KMS] Document&test missing KMS client side configs |  Minor | . | Wei-Chiu Chuang | Gabor Bota |
+| [HDFS-12619](https://issues.apache.org/jira/browse/HDFS-12619) | Do not catch and throw unchecked exceptions if IBRs fail to process |  Minor | namenode | Wei-Chiu Chuang | Wei-Chiu Chuang |
+| [HADOOP-14771](https://issues.apache.org/jira/browse/HADOOP-14771) | hadoop-client does not include hadoop-yarn-client |  Critical | common | Haibo Chen | Ajay Kumar |
+| [YARN-7359](https://issues.apache.org/jira/browse/YARN-7359) | TestAppManager.testQueueSubmitWithNoPermission() should be scheduler agnostic |  Minor | . | Haibo Chen | Haibo Chen |
+| [HDFS-12448](https://issues.apache.org/jira/browse/HDFS-12448) | Make sure user defined erasure coding policy ID will not overflow |  Major | erasure-coding | SammiChen | Huafeng Wang |
+| [HADOOP-14944](https://issues.apache.org/jira/browse/HADOOP-14944) | Add JvmMetrics to KMS |  Major | kms | Xiao Chen | Xiao Chen |
+| [YARN-7261](https://issues.apache.org/jira/browse/YARN-7261) | Add debug message for better download latency monitoring |  Major | nodemanager | Yufei Gu | Yufei Gu |
+| [YARN-7357](https://issues.apache.org/jira/browse/YARN-7357) | Several methods in TestZKRMStateStore.TestZKRMStateStoreTester.TestZKRMStateStoreInternal should have @Override annotations |  Trivial | resourcemanager | Daniel Templeton | Sen Zhao |
+| [YARN-4163](https://issues.apache.org/jira/browse/YARN-4163) | Audit getQueueInfo and getApplications calls |  Major | . | Chang Li | Chang Li |
+| [HADOOP-9657](https://issues.apache.org/jira/browse/HADOOP-9657) | NetUtils.wrapException to have special handling for 0.0.0.0 addresses and :0 ports |  Minor | net | Steve Loughran | Varun Saxena |
+| [YARN-7389](https://issues.apache.org/jira/browse/YARN-7389) | Make TestResourceManager Scheduler agnostic |  Major | test | Robert Kanter | Robert Kanter |
+| [HDFS-12544](https://issues.apache.org/jira/browse/HDFS-12544) | SnapshotDiff - support diff generation on any snapshot root descendant directory |  Major | hdfs | Manoj Govindassamy | Manoj Govindassamy |
+| [YARN-7358](https://issues.apache.org/jira/browse/YARN-7358) | TestZKConfigurationStore and TestLeveldbConfigurationStore should explicitly set capacity scheduler |  Minor | resourcemanager | Haibo Chen | Haibo Chen |
+| [YARN-7320](https://issues.apache.org/jira/browse/YARN-7320) | Duplicate LiteralByteStrings in SystemCredentialsForAppsProto.credentialsForApp\_ |  Major | . | Misha Dmitriev | Misha Dmitriev |
+| [YARN-7262](https://issues.apache.org/jira/browse/YARN-7262) | Add a hierarchy into the ZKRMStateStore for delegation token znodes to prevent jute buffer overflow |  Major | . | Robert Kanter | Robert Kanter |
+| [YARN-7397](https://issues.apache.org/jira/browse/YARN-7397) | Reduce lock contention in FairScheduler#getAppWeight() |  Major | fairscheduler | Daniel Templeton | Daniel Templeton |
+| [HADOOP-14992](https://issues.apache.org/jira/browse/HADOOP-14992) | Upgrade Avro patch version |  Major | build | Chris Douglas | Bharat Viswanadham |
+| [YARN-5326](https://issues.apache.org/jira/browse/YARN-5326) | Support for recurring reservations in the YARN ReservationSystem |  Major | resourcemanager | Subru Krishnan | Carlo Curino |
+| [YARN-6413](https://issues.apache.org/jira/browse/YARN-6413) | FileSystem based Yarn Registry implementation |  Major | amrmproxy, api, resourcemanager | Ellen Hui | Ellen Hui |
+| [HDFS-12482](https://issues.apache.org/jira/browse/HDFS-12482) | Provide a configuration to adjust the weight of EC recovery tasks to adjust the speed of recovery |  Minor | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [HDFS-12744](https://issues.apache.org/jira/browse/HDFS-12744) | More logs when short-circuit read is failed and disabled |  Major | datanode | Weiwei Yang | Weiwei Yang |
+| [HDFS-12771](https://issues.apache.org/jira/browse/HDFS-12771) | Add genstamp and block size to metasave Corrupt blocks list |  Minor | . | Kuhu Shukla | Kuhu Shukla |
+| [HADOOP-14987](https://issues.apache.org/jira/browse/HADOOP-14987) | Improve KMSClientProvider log around delegation token checking |  Major | . | Xiaoyu Yao | Xiaoyu Yao |
+| [MAPREDUCE-6975](https://issues.apache.org/jira/browse/MAPREDUCE-6975) | Logging task counters |  Major | task | Prabhu Joseph | Prabhu Joseph |
+| [YARN-7401](https://issues.apache.org/jira/browse/YARN-7401) | Reduce lock contention in ClusterNodeTracker#getClusterCapacity() |  Major | resourcemanager | Daniel Templeton | Daniel Templeton |
+| [HDFS-7060](https://issues.apache.org/jira/browse/HDFS-7060) | Avoid taking locks when sending heartbeats from the DataNode |  Major | . | Haohui Mai | Jiandan Yang |
+| [YARN-7413](https://issues.apache.org/jira/browse/YARN-7413) | Support resource type in SLS |  Major | scheduler-load-simulator | Yufei Gu | Yufei Gu |
+| [YARN-7386](https://issues.apache.org/jira/browse/YARN-7386) | Duplicate Strings in various places in Yarn memory |  Major | . | Misha Dmitriev | Misha Dmitriev |
+| [HADOOP-14960](https://issues.apache.org/jira/browse/HADOOP-14960) | Add GC time percentage monitor/alerter |  Major | . | Misha Dmitriev | Misha Dmitriev |
+| [HADOOP-15037](https://issues.apache.org/jira/browse/HADOOP-15037) | Add site release notes for OrgQueue and resource types |  Major | . | Andrew Wang | Andrew Wang |
+| [HADOOP-14876](https://issues.apache.org/jira/browse/HADOOP-14876) | Create downstream developer docs from the compatibility guidelines |  Critical | documentation | Daniel Templeton | Daniel Templeton |
+| [HADOOP-14112](https://issues.apache.org/jira/browse/HADOOP-14112) | Über-jira adl:// Azure Data Lake Phase I: Stabilization |  Major | fs/adl | Steve Loughran | John Zhuge |
+| [HADOOP-15104](https://issues.apache.org/jira/browse/HADOOP-15104) | AliyunOSS: change the default value of max error retry |  Major | fs/oss | wujinhu | wujinhu |
+
+
+### BUG FIXES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-7172](https://issues.apache.org/jira/browse/YARN-7172) | ResourceCalculator.fitsIn() should not take a cluster resource parameter |  Major | scheduler | Daniel Templeton | Sen Zhao |
+| [HADOOP-14901](https://issues.apache.org/jira/browse/HADOOP-14901) | ReuseObjectMapper in Hadoop Common |  Minor | . | Hanisha Koneru | Hanisha Koneru |
+| [YARN-7248](https://issues.apache.org/jira/browse/YARN-7248) | NM returns new SCHEDULED container status to older clients |  Blocker | nodemanager | Jason Lowe | Arun Suresh |
+| [HADOOP-14902](https://issues.apache.org/jira/browse/HADOOP-14902) | LoadGenerator#genFile write close timing is incorrectly calculated |  Major | fs | Jason Lowe | Hanisha Koneru |
+| [YARN-7084](https://issues.apache.org/jira/browse/YARN-7084) | TestSchedulingMonitor#testRMStarts fails sporadically |  Major | . | Jason Lowe | Jason Lowe |
+| [HDFS-12453](https://issues.apache.org/jira/browse/HDFS-12453) | TestDataNodeHotSwapVolumes fails in trunk Jenkins runs |  Critical | test | Arpit Agarwal | Lei (Eddy) Xu |
+| [HADOOP-14915](https://issues.apache.org/jira/browse/HADOOP-14915) | method name is incorrect in ConfServlet |  Minor | . | Bharat Viswanadham | Bharat Viswanadham |
+| [HADOOP-14752](https://issues.apache.org/jira/browse/HADOOP-14752) | TestCopyFromLocal#testCopyFromLocalWithThreads is fleaky |  Major | test | Andras Bokor | Andras Bokor |
+| [HDFS-12569](https://issues.apache.org/jira/browse/HDFS-12569) | Unset EC policy logs empty payload in edit log |  Blocker | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [YARN-6943](https://issues.apache.org/jira/browse/YARN-6943) | Update Yarn to YARN in documentation |  Minor | documentation | Miklos Szegedi | Chetna Chaudhari |
+| [YARN-7211](https://issues.apache.org/jira/browse/YARN-7211) | AMSimulator in SLS does't work due to refactor of responseId |  Blocker | scheduler-load-simulator | Yufei Gu | Botong Huang |
+| [HADOOP-14459](https://issues.apache.org/jira/browse/HADOOP-14459) | SerializationFactory shouldn't throw a NullPointerException if the serializations list is not defined |  Minor | . | Nandor Kollar | Nandor Kollar |
+| [YARN-7044](https://issues.apache.org/jira/browse/YARN-7044) | TestContainerAllocation#testAMContainerAllocationWhenDNSUnavailable fails |  Major | capacity scheduler, test | Wangda Tan | Akira Ajisaka |
+| [YARN-7226](https://issues.apache.org/jira/browse/YARN-7226) | Whitelisted variables do not support delayed variable expansion |  Major | nodemanager | Jason Lowe | Jason Lowe |
+| [HADOOP-14616](https://issues.apache.org/jira/browse/HADOOP-14616) | Client fails to read a block with erasure code (XOR, native) when one of the data block is lost |  Blocker | . | Ayappan | Huafeng Wang |
+| [YARN-7279](https://issues.apache.org/jira/browse/YARN-7279) | Fix typo in helper message of ContainerLauncher |  Trivial | . | Elek, Marton | Elek, Marton |
+| [YARN-7258](https://issues.apache.org/jira/browse/YARN-7258) | Add Node and Rack Hints to Opportunistic Scheduler |  Major | . | Arun Suresh | kartheek muthyala |
+| [YARN-7009](https://issues.apache.org/jira/browse/YARN-7009) | TestNMClient.testNMClientNoCleanupOnStop is flaky by design |  Major | . | Miklos Szegedi | Miklos Szegedi |
+| [HDFS-12567](https://issues.apache.org/jira/browse/HDFS-12567) | BlockPlacementPolicyRackFaultTolerant fails with racks with very few nodes |  Major | erasure-coding | Andrew Wang | Andrew Wang |
+| [HDFS-12494](https://issues.apache.org/jira/browse/HDFS-12494) | libhdfs SIGSEGV in setTLSExceptionStrings |  Major | libhdfs | John Zhuge | John Zhuge |
+| [YARN-7245](https://issues.apache.org/jira/browse/YARN-7245) | Max AM Resource column in Active Users Info section of Capacity Scheduler UI page should be updated per-user |  Major | capacity scheduler, yarn | Eric Payne | Eric Payne |
+| [HDFS-12606](https://issues.apache.org/jira/browse/HDFS-12606) | When using native decoder, DFSStripedStream#close crashes JVM after being called multiple times. |  Critical | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [HDFS-12599](https://issues.apache.org/jira/browse/HDFS-12599) | Remove Mockito dependency from DataNodeTestUtils |  Minor | test | Ted Yu | Ted Yu |
+| [YARN-7309](https://issues.apache.org/jira/browse/YARN-7309) | TestClientRMService#testUpdateApplicationPriorityRequest and TestClientRMService#testUpdatePriorityAndKillAppWithZeroClusterResource test functionality not supported by FairScheduler |  Major | test | Robert Kanter | Robert Kanter |
+| [HADOOP-14912](https://issues.apache.org/jira/browse/HADOOP-14912) | FairCallQueue may defer servicing calls |  Major | ipc | Daryn Sharp | Daryn Sharp |
+| [HDFS-12635](https://issues.apache.org/jira/browse/HDFS-12635) | Unnecessary exception declaration of the CellBuffers constructor |  Minor | . | Huafeng Wang | Huafeng Wang |
+| [HDFS-12622](https://issues.apache.org/jira/browse/HDFS-12622) | Fix enumerate in HDFSErasureCoding.md |  Minor | documentation | Akira Ajisaka | Yiqun Lin |
+| [YARN-7082](https://issues.apache.org/jira/browse/YARN-7082) | TestContainerManagerSecurity failing in trunk |  Major | . | Varun Saxena | Akira Ajisaka |
+| [HADOOP-13556](https://issues.apache.org/jira/browse/HADOOP-13556) | Change Configuration.getPropsWithPrefix to use getProps instead of iterator |  Major | . | Larry McCay | Larry McCay |
+| [HADOOP-13102](https://issues.apache.org/jira/browse/HADOOP-13102) | Update GroupsMapping documentation to reflect the new changes |  Major | documentation | Anu Engineer | Esther Kundin |
+| [YARN-7270](https://issues.apache.org/jira/browse/YARN-7270) | Fix unsafe casting from long to int for class Resource and its sub-classes |  Major | resourcemanager | Yufei Gu | Yufei Gu |
+| [YARN-7310](https://issues.apache.org/jira/browse/YARN-7310) | TestAMRMProxy#testAMRMProxyE2E fails with FairScheduler |  Major | test | Robert Kanter | Robert Kanter |
+| [YARN-7269](https://issues.apache.org/jira/browse/YARN-7269) | Tracking URL in the app state does not get redirected to ApplicationMaster for Running applications |  Critical | . | Sumana Sathish | Tan, Wangda |
+| [HDFS-12659](https://issues.apache.org/jira/browse/HDFS-12659) | Update TestDeadDatanode#testNonDFSUsedONDeadNodeReReg to increase heartbeat recheck interval |  Minor | . | Ajay Kumar | Ajay Kumar |
+| [HDFS-12485](https://issues.apache.org/jira/browse/HDFS-12485) | expunge may fail to remove trash from encryption zone |  Major | . | Wei-Chiu Chuang | Wei-Chiu Chuang |
+| [HADOOP-14949](https://issues.apache.org/jira/browse/HADOOP-14949) | TestKMS#testACLs fails intermittently |  Major | kms, test | Xiao Chen | Xiao Chen |
+| [YARN-7124](https://issues.apache.org/jira/browse/YARN-7124) | LogAggregationTFileController deletes/renames while file is open |  Critical | nodemanager | Daryn Sharp | Jason Lowe |
+| [YARN-7333](https://issues.apache.org/jira/browse/YARN-7333) | container-executor fails to remove entries from a directory that is not writable or executable |  Critical | . | Jason Lowe | Jason Lowe |
+| [YARN-7308](https://issues.apache.org/jira/browse/YARN-7308) | TestApplicationACLs fails with FairScheduler |  Major | test | Robert Kanter | Robert Kanter |
+| [HADOOP-14948](https://issues.apache.org/jira/browse/HADOOP-14948) | Document missing config key hadoop.treat.subject.external |  Minor | security | Wei-Chiu Chuang | Ajay Kumar |
+| [HDFS-12614](https://issues.apache.org/jira/browse/HDFS-12614) | FSPermissionChecker#getINodeAttrs() throws NPE when INodeAttributesProvider configured |  Major | . | Manoj Govindassamy | Manoj Govindassamy |
+| [YARN-7341](https://issues.apache.org/jira/browse/YARN-7341) | TestRouterWebServiceUtil#testMergeMetrics is flakey |  Major | federation | Robert Kanter | Robert Kanter |
+| [HDFS-12612](https://issues.apache.org/jira/browse/HDFS-12612) | DFSStripedOutputStream#close will throw if called a second time with a failed streamer |  Major | erasure-coding | Andrew Wang | Lei (Eddy) Xu |
+| [HADOOP-14958](https://issues.apache.org/jira/browse/HADOOP-14958) | CLONE - Fix source-level compatibility after HADOOP-11252 |  Blocker | . | Junping Du | Junping Du |
+| [YARN-7294](https://issues.apache.org/jira/browse/YARN-7294) | TestSignalContainer#testSignalRequestDeliveryToNM fails intermittently with Fair scheduler |  Major | . | Miklos Szegedi | Miklos Szegedi |
+| [YARN-7170](https://issues.apache.org/jira/browse/YARN-7170) | Improve bower dependencies for YARN UI v2 |  Critical | webapp | Sunil G | Sunil G |
+| [YARN-7355](https://issues.apache.org/jira/browse/YARN-7355) | TestDistributedShell should be scheduler agnostic |  Major | . | Haibo Chen | Haibo Chen |
+| [HDFS-12497](https://issues.apache.org/jira/browse/HDFS-12497) | Re-enable TestDFSStripedOutputStreamWithFailure tests |  Major | erasure-coding | Andrew Wang | Huafeng Wang |
+| [HADOOP-14942](https://issues.apache.org/jira/browse/HADOOP-14942) | DistCp#cleanup() should check whether jobFS is null |  Minor | . | Ted Yu | Andras Bokor |
+| [YARN-7318](https://issues.apache.org/jira/browse/YARN-7318) | Fix shell check warnings of SLS. |  Major | . | Wangda Tan | Gergely Novák |
+| [HDFS-12518](https://issues.apache.org/jira/browse/HDFS-12518) | Re-encryption should handle task cancellation and progress better |  Major | encryption | Xiao Chen | Xiao Chen |
+| [HADOOP-14966](https://issues.apache.org/jira/browse/HADOOP-14966) | Handle JDK-8071638 for hadoop-common |  Blocker | . | Bibin A Chundatt | Bibin A Chundatt |
+| [HDFS-12249](https://issues.apache.org/jira/browse/HDFS-12249) | dfsadmin -metaSave to output maintenance mode blocks |  Minor | namenode | Wei-Chiu Chuang | Wellington Chevreuil |
+| [HDFS-12695](https://issues.apache.org/jira/browse/HDFS-12695) | Add a link to HDFS router federation document in site.xml |  Minor | documentation | Yiqun Lin | Yiqun Lin |
+| [YARN-7382](https://issues.apache.org/jira/browse/YARN-7382) | NoSuchElementException in FairScheduler after failover causes RM crash |  Blocker | fairscheduler | Robert Kanter | Robert Kanter |
+| [YARN-7385](https://issues.apache.org/jira/browse/YARN-7385) | TestFairScheduler#testUpdateDemand and TestFSLeafQueue#testUpdateDemand are failing with NPE |  Major | test | Robert Kanter | Yufei Gu |
+| [HADOOP-14030](https://issues.apache.org/jira/browse/HADOOP-14030) | PreCommit TestKDiag failure |  Major | security | John Zhuge | Wei-Chiu Chuang |
+| [HADOOP-14979](https://issues.apache.org/jira/browse/HADOOP-14979) | Upgrade maven-dependency-plugin to 3.0.2 |  Major | build | liyunzhang | liyunzhang |
+| [HADOOP-14977](https://issues.apache.org/jira/browse/HADOOP-14977) | Xenial dockerfile needs ant in main build for findbugs |  Trivial | build, test | Allen Wittenauer | Akira Ajisaka |
+| [YARN-7339](https://issues.apache.org/jira/browse/YARN-7339) | LocalityMulticastAMRMProxyPolicy should handle cancel request properly |  Minor | . | Botong Huang | Botong Huang |
+| [HDFS-9914](https://issues.apache.org/jira/browse/HDFS-9914) | Fix configurable WebhDFS connect/read timeout |  Blocker | hdfs-client, webhdfs | Xiaoyu Yao | Xiaoyu Yao |
+| [YARN-7375](https://issues.apache.org/jira/browse/YARN-7375) | Possible NPE in RMWebapp when HA is enabled and the active RM fails |  Major | . | Chandni Singh | Chandni Singh |
+| [HDFS-12582](https://issues.apache.org/jira/browse/HDFS-12582) | Replace HdfsFileStatus constructor with a builder pattern. |  Major | . | Bharat Viswanadham | Bharat Viswanadham |
+| [HADOOP-14986](https://issues.apache.org/jira/browse/HADOOP-14986) | Enforce JDK limitations |  Major | build | Chris Douglas | Chris Douglas |
+| [HADOOP-14991](https://issues.apache.org/jira/browse/HADOOP-14991) | Add missing figures to Resource Estimator tool |  Major | . | Subru Krishnan | Rui Li |
+| [YARN-7299](https://issues.apache.org/jira/browse/YARN-7299) | Fix TestDistributedScheduler |  Major | . | Jason Lowe | Arun Suresh |
+| [YARN-6747](https://issues.apache.org/jira/browse/YARN-6747) | TestFSAppStarvation.testPreemptionEnable fails intermittently |  Major | . | Sunil G | Miklos Szegedi |
+| [YARN-7336](https://issues.apache.org/jira/browse/YARN-7336) | Unsafe cast from long to int Resource.hashCode() method |  Critical | resourcemanager | Daniel Templeton | Miklos Szegedi |
+| [YARN-7244](https://issues.apache.org/jira/browse/YARN-7244) | ShuffleHandler is not aware of disks that are added |  Major | . | Kuhu Shukla | Kuhu Shukla |
+| [HADOOP-14990](https://issues.apache.org/jira/browse/HADOOP-14990) | Clean up jdiff xml files added for 2.8.2 release |  Blocker | . | Subru Krishnan | Junping Du |
+| [HADOOP-14919](https://issues.apache.org/jira/browse/HADOOP-14919) | BZip2 drops records when reading data in splits |  Critical | . | Aki Tanaka | Jason Lowe |
+| [HDFS-12699](https://issues.apache.org/jira/browse/HDFS-12699) | TestMountTable fails with Java 7 |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-12714](https://issues.apache.org/jira/browse/HDFS-12714) | Hadoop 3 missing fix for HDFS-5169 |  Major | native | Joe McDonnell | Joe McDonnell |
+| [HDFS-12219](https://issues.apache.org/jira/browse/HDFS-12219) | Javadoc for FSNamesystem#getMaxObjects is incorrect |  Trivial | . | Erik Krogen | Erik Krogen |
+| [YARN-7412](https://issues.apache.org/jira/browse/YARN-7412) | test\_docker\_util.test\_check\_mount\_permitted() is failing |  Critical | nodemanager | Haibo Chen | Eric Badger |
+| [MAPREDUCE-6999](https://issues.apache.org/jira/browse/MAPREDUCE-6999) | Fix typo "onf" in DynamicInputChunk.java |  Trivial | . | fang zhenyi | fang zhenyi |
+| [YARN-7364](https://issues.apache.org/jira/browse/YARN-7364) | Queue dash board in new YARN UI has incorrect values |  Critical | webapp | Sunil G | Sunil G |
+| [YARN-7370](https://issues.apache.org/jira/browse/YARN-7370) | Preemption properties should be refreshable |  Major | capacity scheduler, scheduler preemption | Eric Payne | Gergely Novák |
+| [YARN-7400](https://issues.apache.org/jira/browse/YARN-7400) | incorrect log preview displayed in jobhistory server ui |  Major | yarn | Santhosh B Gowda | Xuan Gong |
+| [HADOOP-15013](https://issues.apache.org/jira/browse/HADOOP-15013) | Fix ResourceEstimator findbugs issues |  Blocker | . | Allen Wittenauer | Arun Suresh |
+| [YARN-7432](https://issues.apache.org/jira/browse/YARN-7432) | Fix DominantResourceFairnessPolicy serializable findbugs issues |  Blocker | . | Allen Wittenauer | Daniel Templeton |
+| [YARN-7434](https://issues.apache.org/jira/browse/YARN-7434) | Router getApps REST invocation fails with multiple RMs |  Critical | . | Subru Krishnan | Íñigo Goiri |
+| [HDFS-12725](https://issues.apache.org/jira/browse/HDFS-12725) | BlockPlacementPolicyRackFaultTolerant fails with very uneven racks |  Major | erasure-coding | Xiao Chen | Xiao Chen |
+| [YARN-5085](https://issues.apache.org/jira/browse/YARN-5085) | Add support for change of container ExecutionType |  Major | . | Arun Suresh | Arun Suresh |
+| [YARN-7428](https://issues.apache.org/jira/browse/YARN-7428) | Add containerId to Localizer failed logs |  Minor | nodemanager | Prabhu Joseph | Prabhu Joseph |
+| [YARN-7410](https://issues.apache.org/jira/browse/YARN-7410) | Cleanup FixedValueResource to avoid dependency to ResourceUtils |  Major | resourcemanager | Sunil G | Wangda Tan |
+| [YARN-7360](https://issues.apache.org/jira/browse/YARN-7360) | TestRM.testNMTokenSentForNormalContainer() should be scheduler agnostic |  Major | test | Haibo Chen | Haibo Chen |
+| [HADOOP-15018](https://issues.apache.org/jira/browse/HADOOP-15018) | Update JAVA\_HOME in create-release for Xenial Dockerfile |  Blocker | build | Andrew Wang | Andrew Wang |
+| [HDFS-12788](https://issues.apache.org/jira/browse/HDFS-12788) | Reset the upload button when file upload fails |  Critical | ui, webhdfs | Brahma Reddy Battula | Brahma Reddy Battula |
+| [YARN-7453](https://issues.apache.org/jira/browse/YARN-7453) | Fix issue where RM fails to switch to active after first successful start |  Blocker | resourcemanager | Rohith Sharma K S | Rohith Sharma K S |
+| [YARN-7458](https://issues.apache.org/jira/browse/YARN-7458) | TestContainerManagerSecurity is still flakey |  Major | test | Robert Kanter | Robert Kanter |
+| [HADOOP-15025](https://issues.apache.org/jira/browse/HADOOP-15025) | Ensure singleton for ResourceEstimatorService |  Major | . | Subru Krishnan | Rui Li |
+| [HDFS-12732](https://issues.apache.org/jira/browse/HDFS-12732) | Correct spellings of ramdomly to randomly in log. |  Trivial | . | hu xiaodong | hu xiaodong |
+| [YARN-7454](https://issues.apache.org/jira/browse/YARN-7454) | RMAppAttemptMetrics#getAggregateResourceUsage can NPE due to double lookup |  Minor | resourcemanager | Jason Lowe | Jason Lowe |
+| [YARN-7388](https://issues.apache.org/jira/browse/YARN-7388) | TestAMRestart should be scheduler agnostic |  Major | . | Haibo Chen | Haibo Chen |
+| [YARN-7465](https://issues.apache.org/jira/browse/YARN-7465) | start-yarn.sh fails to start ResourceManager unless running as root |  Blocker | . | Sean Mackrory |  |
+| [HADOOP-8522](https://issues.apache.org/jira/browse/HADOOP-8522) | ResetableGzipOutputStream creates invalid gzip files when finish() and resetState() are used |  Major | io | Mike Percy | Mike Percy |
+| [YARN-7475](https://issues.apache.org/jira/browse/YARN-7475) | Fix Container log link in new YARN UI |  Major | . | Sunil G | Sunil G |
+| [HADOOP-15036](https://issues.apache.org/jira/browse/HADOOP-15036) | Update LICENSE.txt for HADOOP-14840 |  Major | . | Arun Suresh | Arun Suresh |
+| [YARN-6078](https://issues.apache.org/jira/browse/YARN-6078) | Containers stuck in Localizing state |  Major | . | Jagadish | Billie Rinaldi |
+| [YARN-7469](https://issues.apache.org/jira/browse/YARN-7469) | Capacity Scheduler Intra-queue preemption: User can starve if newest app is exactly at user limit |  Major | capacity scheduler, yarn | Eric Payne | Eric Payne |
+| [HADOOP-15067](https://issues.apache.org/jira/browse/HADOOP-15067) | GC time percentage reported in JvmMetrics should be a gauge, not counter |  Major | . | Misha Dmitriev | Misha Dmitriev |
+| [YARN-7290](https://issues.apache.org/jira/browse/YARN-7290) | Method canContainerBePreempted can return true when it shouldn't |  Major | fairscheduler | Steven Rand | Steven Rand |
+| [HDFS-12754](https://issues.apache.org/jira/browse/HDFS-12754) | Lease renewal can hit a deadlock |  Major | . | Kuhu Shukla | Kuhu Shukla |
+| [YARN-6647](https://issues.apache.org/jira/browse/YARN-6647) | RM can crash during transitionToStandby due to InterruptedException |  Critical | resourcemanager | Jason Lowe | Bibin A Chundatt |
+| [HDFS-11754](https://issues.apache.org/jira/browse/HDFS-11754) | Make FsServerDefaults cache configurable. |  Minor | . | Rushabh S Shah | Mikhail Erofeev |
+| [YARN-7509](https://issues.apache.org/jira/browse/YARN-7509) | AsyncScheduleThread and ResourceCommitterService are still running after RM is transitioned to standby |  Critical | . | Tao Yang | Tao Yang |
+| [YARN-7589](https://issues.apache.org/jira/browse/YARN-7589) | TestPBImplRecords fails with NullPointerException |  Major | . | Jason Lowe | Daniel Templeton |
+| [HADOOP-15058](https://issues.apache.org/jira/browse/HADOOP-15058) | create-release site build outputs dummy shaded jars due to skipShade |  Blocker | . | Andrew Wang | Andrew Wang |
+| [YARN-7381](https://issues.apache.org/jira/browse/YARN-7381) | Enable the configuration: yarn.nodemanager.log-container-debug-info.enabled by default in yarn-default.xml |  Critical | . | Xuan Gong | Xuan Gong |
+| [HDFS-12889](https://issues.apache.org/jira/browse/HDFS-12889) | Router UI is missing robots.txt file |  Major | . | Bharat Viswanadham | Bharat Viswanadham |
+| [HDFS-12872](https://issues.apache.org/jira/browse/HDFS-12872) | EC Checksum broken when BlockAccessToken is enabled |  Critical | erasure-coding | Xiao Chen | Xiao Chen |
+| [HDFS-11576](https://issues.apache.org/jira/browse/HDFS-11576) | Block recovery will fail indefinitely if recovery time \> heartbeat interval |  Critical | datanode, hdfs, namenode | Lukas Majercak | Lukas Majercak |
+| [YARN-7607](https://issues.apache.org/jira/browse/YARN-7607) | Remove the trailing duplicated timestamp in container diagnostics message |  Minor | nodemanager | Weiwei Yang | Weiwei Yang |
+| [HDFS-12840](https://issues.apache.org/jira/browse/HDFS-12840) | Creating a file with non-default EC policy in a EC zone is not correctly serialized in the editlog |  Blocker | erasure-coding | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [HADOOP-15080](https://issues.apache.org/jira/browse/HADOOP-15080) | Aliyun OSS: update oss sdk from 2.8.1 to 2.8.3 to remove its dependency on Cat-x "json-lib" |  Blocker | fs/oss | Chris Douglas | SammiChen |
+| [HADOOP-15059](https://issues.apache.org/jira/browse/HADOOP-15059) | 3.0 deployment cannot work with old version MR tar ball which breaks rolling upgrade |  Blocker | security | Junping Du | Jason Lowe |
+
+
+### SUB-TASKS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-4081](https://issues.apache.org/jira/browse/YARN-4081) | Add support for multiple resource types in the Resource class |  Major | resourcemanager | Varun Vasudev | Varun Vasudev |
+| [YARN-4172](https://issues.apache.org/jira/browse/YARN-4172) | Extend DominantResourceCalculator to account for all resources |  Major | resourcemanager | Varun Vasudev | Varun Vasudev |
+| [YARN-4715](https://issues.apache.org/jira/browse/YARN-4715) | Add support to read resource types from a config file |  Major | nodemanager, resourcemanager | Varun Vasudev | Varun Vasudev |
+| [YARN-4829](https://issues.apache.org/jira/browse/YARN-4829) | Add support for binary units |  Major | nodemanager, resourcemanager | Varun Vasudev | Varun Vasudev |
+| [YARN-4830](https://issues.apache.org/jira/browse/YARN-4830) | Add support for resource types in the nodemanager |  Major | nodemanager | Varun Vasudev | Varun Vasudev |
+| [YARN-5242](https://issues.apache.org/jira/browse/YARN-5242) | Update DominantResourceCalculator to consider all resource types in calculations |  Major | resourcemanager | Varun Vasudev | Varun Vasudev |
+| [YARN-5586](https://issues.apache.org/jira/browse/YARN-5586) | Update the Resources class to consider all resource types |  Major | nodemanager, resourcemanager | Varun Vasudev | Varun Vasudev |
+| [YARN-5216](https://issues.apache.org/jira/browse/YARN-5216) | Expose configurable preemption policy for OPPORTUNISTIC containers running on the NM |  Major | distributed-scheduling | Arun Suresh | Hitesh Sharma |
+| [YARN-5951](https://issues.apache.org/jira/browse/YARN-5951) | Changes to allow CapacityScheduler to use configuration store |  Major | . | Jonathan Hung | Jonathan Hung |
+| [YARN-5946](https://issues.apache.org/jira/browse/YARN-5946) | Create YarnConfigurationStore interface and InMemoryConfigurationStore class |  Major | . | Jonathan Hung | Jonathan Hung |
+| [YARN-6232](https://issues.apache.org/jira/browse/YARN-6232) | Update resource usage and preempted resource calculations to take into account all resource types |  Major | resourcemanager | Varun Vasudev | Varun Vasudev |
+| [YARN-5948](https://issues.apache.org/jira/browse/YARN-5948) | Implement MutableConfigurationManager for handling storage into configuration store |  Major | . | Jonathan Hung | Jonathan Hung |
+| [YARN-5952](https://issues.apache.org/jira/browse/YARN-5952) | Create REST API for changing YARN scheduler configurations |  Major | . | Jonathan Hung | Jonathan Hung |
+| [HDFS-10882](https://issues.apache.org/jira/browse/HDFS-10882) | Federation State Store Interface API |  Major | fs | Jason Kace | Jason Kace |
+| [YARN-6445](https://issues.apache.org/jira/browse/YARN-6445) | [YARN-3926] Performance improvements in resource profile branch with respect to SLS |  Major | nodemanager, resourcemanager | Varun Vasudev | Varun Vasudev |
+| [HDFS-10630](https://issues.apache.org/jira/browse/HDFS-10630) | Federation State Store FS Implementation |  Major | hdfs | Íñigo Goiri | Jason Kace |
+| [YARN-5949](https://issues.apache.org/jira/browse/YARN-5949) | Add pluggable configuration ACL policy interface and implementation |  Major | . | Jonathan Hung | Jonathan Hung |
+| [YARN-6575](https://issues.apache.org/jira/browse/YARN-6575) | Support global configuration mutation in MutableConfProvider |  Major | . | Jonathan Hung | Jonathan Hung |
+| [YARN-5953](https://issues.apache.org/jira/browse/YARN-5953) | Create CLI for changing YARN configurations |  Major | . | Jonathan Hung | Jonathan Hung |
+| [YARN-6761](https://issues.apache.org/jira/browse/YARN-6761) | Fix build for YARN-3926 branch |  Major | nodemanager, resourcemanager | Varun Vasudev | Varun Vasudev |
+| [YARN-6786](https://issues.apache.org/jira/browse/YARN-6786) | ResourcePBImpl imports cleanup |  Trivial | resourcemanager | Daniel Templeton | Yeliang Cang |
+| [YARN-5292](https://issues.apache.org/jira/browse/YARN-5292) | NM Container lifecycle and state transitions to support for PAUSED container state. |  Major | . | Hitesh Sharma | Hitesh Sharma |
+| [HDFS-12223](https://issues.apache.org/jira/browse/HDFS-12223) | Rebasing HDFS-10467 |  Major | fs | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-10687](https://issues.apache.org/jira/browse/HDFS-10687) | Federation Membership State Store internal API |  Major | hdfs | Íñigo Goiri | Jason Kace |
+| [YARN-5947](https://issues.apache.org/jira/browse/YARN-5947) | Create LeveldbConfigurationStore class using Leveldb as backing store |  Major | . | Jonathan Hung | Jonathan Hung |
+| [YARN-6322](https://issues.apache.org/jira/browse/YARN-6322) | Disable queue refresh when configuration mutation is enabled |  Major | . | Jonathan Hung | Jonathan Hung |
+| [HDFS-11826](https://issues.apache.org/jira/browse/HDFS-11826) | Federation Namenode Heartbeat |  Major | fs | Íñigo Goiri | Íñigo Goiri |
+| [YARN-6788](https://issues.apache.org/jira/browse/YARN-6788) | Improve performance of resource profile branch |  Blocker | nodemanager, resourcemanager | Sunil G | Sunil G |
+| [HDFS-10880](https://issues.apache.org/jira/browse/HDFS-10880) | Federation Mount Table State Store internal API |  Major | fs | Jason Kace | Íñigo Goiri |
+| [HDFS-10646](https://issues.apache.org/jira/browse/HDFS-10646) | Federation admin tool |  Major | fs | Íñigo Goiri | Íñigo Goiri |
+| [YARN-6994](https://issues.apache.org/jira/browse/YARN-6994) | Remove last uses of Long from resource types code |  Minor | resourcemanager | Daniel Templeton | Daniel Templeton |
+| [YARN-6892](https://issues.apache.org/jira/browse/YARN-6892) | Improve API implementation in Resources and DominantResourceCalculator class |  Major | nodemanager, resourcemanager | Sunil G | Sunil G |
+| [HDFS-11554](https://issues.apache.org/jira/browse/HDFS-11554) | [Documentation] Router-based federation documentation |  Minor | fs | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-12312](https://issues.apache.org/jira/browse/HDFS-12312) | Rebasing HDFS-10467 (2) |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [YARN-6610](https://issues.apache.org/jira/browse/YARN-6610) | DominantResourceCalculator#getResourceAsValue dominant param is updated to handle multiple resources |  Critical | resourcemanager | Daniel Templeton | Daniel Templeton |
+| [YARN-7030](https://issues.apache.org/jira/browse/YARN-7030) | Performance optimizations in Resource and ResourceUtils class |  Critical | nodemanager, resourcemanager | Wangda Tan | Wangda Tan |
+| [YARN-7042](https://issues.apache.org/jira/browse/YARN-7042) | Clean up unit tests after YARN-6610 |  Major | test | Daniel Templeton | Daniel Templeton |
+| [YARN-6789](https://issues.apache.org/jira/browse/YARN-6789) | Add Client API to get all supported resource types from RM |  Major | nodemanager, resourcemanager | Sunil G | Sunil G |
+| [YARN-6781](https://issues.apache.org/jira/browse/YARN-6781) | ResourceUtils#initializeResourcesMap takes an unnecessary Map parameter |  Minor | resourcemanager | Daniel Templeton | Yu-Tang Lin |
+| [HDFS-10631](https://issues.apache.org/jira/browse/HDFS-10631) | Federation State Store ZooKeeper implementation |  Major | fs | Íñigo Goiri | Jason Kace |
+| [YARN-7067](https://issues.apache.org/jira/browse/YARN-7067) | Optimize ResourceType information display in UI |  Critical | nodemanager, resourcemanager | Wangda Tan | Wangda Tan |
+| [YARN-7039](https://issues.apache.org/jira/browse/YARN-7039) | Fix javac and javadoc errors in YARN-3926 branch |  Major | nodemanager, resourcemanager | Sunil G | Sunil G |
+| [YARN-7024](https://issues.apache.org/jira/browse/YARN-7024) | Fix issues on recovery in LevelDB store |  Major | . | Jonathan Hung | Jonathan Hung |
+| [YARN-7093](https://issues.apache.org/jira/browse/YARN-7093) | Improve log message in ResourceUtils |  Trivial | nodemanager, resourcemanager | Sunil G | Sunil G |
+| [YARN-6933](https://issues.apache.org/jira/browse/YARN-6933) | ResourceUtils.DISALLOWED\_NAMES check is duplicated |  Major | resourcemanager | Daniel Templeton | Manikandan R |
+| [YARN-5328](https://issues.apache.org/jira/browse/YARN-5328) | Plan/ResourceAllocation data structure enhancements required to support recurring reservations in ReservationSystem |  Major | resourcemanager | Subru Krishnan | Subru Krishnan |
+| [HDFS-12384](https://issues.apache.org/jira/browse/HDFS-12384) | Fixing compilation issue with BanDuplicateClasses |  Major | fs | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-12335](https://issues.apache.org/jira/browse/HDFS-12335) | Federation Metrics |  Major | fs | Giovanni Matteo Fumarola | Íñigo Goiri |
+| [YARN-5330](https://issues.apache.org/jira/browse/YARN-5330) | SharingPolicy enhancements required to support recurring reservations in ReservationSystem |  Major | resourcemanager | Subru Krishnan | Carlo Curino |
+| [YARN-7072](https://issues.apache.org/jira/browse/YARN-7072) | Add a new log aggregation file format controller |  Major | . | Xuan Gong | Xuan Gong |
+| [YARN-7136](https://issues.apache.org/jira/browse/YARN-7136) | Additional Performance Improvement for Resource Profile Feature |  Critical | nodemanager, resourcemanager | Wangda Tan | Wangda Tan |
+| [YARN-7137](https://issues.apache.org/jira/browse/YARN-7137) | Move newly added APIs to unstable in YARN-3926 branch |  Blocker | nodemanager, resourcemanager | Wangda Tan | Wangda Tan |
+| [YARN-6059](https://issues.apache.org/jira/browse/YARN-6059) | Update paused container state in the NM state store |  Blocker | . | Hitesh Sharma | Hitesh Sharma |
+| [HDFS-12430](https://issues.apache.org/jira/browse/HDFS-12430) | Rebasing HDFS-10467 After HDFS-12269 and HDFS-12218 |  Major | fs | Íñigo Goiri | Íñigo Goiri |
+| [YARN-7194](https://issues.apache.org/jira/browse/YARN-7194) | Log aggregation status is always Failed with the newly added log aggregation IndexedFileFormat |  Major | . | Xuan Gong | Xuan Gong |
+| [YARN-6612](https://issues.apache.org/jira/browse/YARN-6612) | Update fair scheduler policies to be aware of resource types |  Major | fairscheduler | Daniel Templeton | Daniel Templeton |
+| [HDFS-12450](https://issues.apache.org/jira/browse/HDFS-12450) | Fixing TestNamenodeHeartbeat and support non-HA |  Major | fs | Íñigo Goiri | Íñigo Goiri |
+| [YARN-6840](https://issues.apache.org/jira/browse/YARN-6840) | Implement zookeeper based store for scheduler configuration updates |  Major | . | Wangda Tan | Jonathan Hung |
+| [YARN-7046](https://issues.apache.org/jira/browse/YARN-7046) | Add closing logic to configuration store |  Major | . | Jonathan Hung | Jonathan Hung |
+| [HDFS-12381](https://issues.apache.org/jira/browse/HDFS-12381) | [Documentation] Adding configuration keys for the Router |  Minor | fs | Íñigo Goiri | Íñigo Goiri |
+| [YARN-7238](https://issues.apache.org/jira/browse/YARN-7238) | Documentation for API based scheduler configuration management |  Major | . | Jonathan Hung | Jonathan Hung |
+| [YARN-7252](https://issues.apache.org/jira/browse/YARN-7252) | Removing queue then failing over results in exception |  Critical | . | Jonathan Hung | Jonathan Hung |
+| [YARN-7251](https://issues.apache.org/jira/browse/YARN-7251) | Misc changes to YARN-5734 |  Major | . | Jonathan Hung | Jonathan Hung |
+| [YARN-7250](https://issues.apache.org/jira/browse/YARN-7250) | Update Shared cache client api to use URLs |  Minor | . | Chris Trezzo | Chris Trezzo |
+| [YARN-6509](https://issues.apache.org/jira/browse/YARN-6509) | Add a size threshold beyond which yarn logs will require a force option |  Major | . | Siddharth Seth | Xuan Gong |
+| [YARN-7259](https://issues.apache.org/jira/browse/YARN-7259) | Add size-based rolling policy to LogAggregationIndexedFileController |  Major | . | Xuan Gong | Xuan Gong |
+| [MAPREDUCE-6971](https://issues.apache.org/jira/browse/MAPREDUCE-6971) | Moving logging APIs over to slf4j in hadoop-mapreduce-client-app |  Major | . | Jinjiang Ling | Jinjiang Ling |
+| [YARN-6550](https://issues.apache.org/jira/browse/YARN-6550) | Capture launch\_container.sh logs to a separate log file |  Major | . | Wangda Tan | Suma Shivaprasad |
+| [HDFS-12580](https://issues.apache.org/jira/browse/HDFS-12580) | Rebasing HDFS-10467 after HDFS-12447 |  Major | fs | Íñigo Goiri | Íñigo Goiri |
+| [YARN-6916](https://issues.apache.org/jira/browse/YARN-6916) | Moving logging APIs over to slf4j in hadoop-yarn-server-common |  Major | . | Akira Ajisaka | Akira Ajisaka |
+| [YARN-7219](https://issues.apache.org/jira/browse/YARN-7219) | Make AllocateRequestProto compatible with branch-2/branch-2.8 |  Critical | yarn | Ray Chiang | Ray Chiang |
+| [YARN-6975](https://issues.apache.org/jira/browse/YARN-6975) | Moving logging APIs over to slf4j in hadoop-yarn-server-tests, hadoop-yarn-server-web-proxy and hadoop-yarn-server-router |  Major | . | Yeliang Cang | Yeliang Cang |
+| [YARN-2037](https://issues.apache.org/jira/browse/YARN-2037) | Add work preserving restart support for Unmanaged AMs |  Major | resourcemanager | Karthik Kambatla | Botong Huang |
+| [YARN-5329](https://issues.apache.org/jira/browse/YARN-5329) | Placement Agent enhancements required to support recurring reservations in ReservationSystem |  Blocker | resourcemanager | Subru Krishnan | Carlo Curino |
+| [YARN-6182](https://issues.apache.org/jira/browse/YARN-6182) | Fix alignment issues and missing information in new YARN UI's Queue page |  Major | yarn-ui-v2 | Akhil PB | Akhil PB |
+| [YARN-2960](https://issues.apache.org/jira/browse/YARN-2960) | Add documentation for the YARN shared cache |  Major | . | Chris Trezzo | Chris Trezzo |
+| [HDFS-12273](https://issues.apache.org/jira/browse/HDFS-12273) | Federation UI |  Major | fs | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-12577](https://issues.apache.org/jira/browse/HDFS-12577) | Rename Router tooling |  Major | fs | Íñigo Goiri | Íñigo Goiri |
+| [HDFS-12541](https://issues.apache.org/jira/browse/HDFS-12541) | Extend TestSafeModeWithStripedFile with a random EC policy |  Major | erasure-coding, test | Takanobu Asanuma | Takanobu Asanuma |
+| [YARN-7304](https://issues.apache.org/jira/browse/YARN-7304) | Merge YARN-5734 branch to branch-3.0 |  Major | . | Xuan Gong | Xuan Gong |
+| [HDFS-12547](https://issues.apache.org/jira/browse/HDFS-12547) | Extend TestQuotaWithStripedBlocks with a random EC policy |  Major | erasure-coding, test | Takanobu Asanuma | Takanobu Asanuma |
+| [YARN-7205](https://issues.apache.org/jira/browse/YARN-7205) | Log improvements for the ResourceUtils |  Major | nodemanager, resourcemanager | Jian He | Sunil G |
+| [HDFS-12637](https://issues.apache.org/jira/browse/HDFS-12637) | Extend TestDistributedFileSystemWithECFile with a random EC policy |  Major | erasure-coding, test | Takanobu Asanuma | Takanobu Asanuma |
+| [YARN-7275](https://issues.apache.org/jira/browse/YARN-7275) | NM Statestore cleanup for Container updates |  Blocker | . | Arun Suresh | kartheek muthyala |
+| [YARN-7311](https://issues.apache.org/jira/browse/YARN-7311) | Fix TestRMWebServicesReservation parametrization for fair scheduler |  Blocker | fairscheduler, reservation system | Yufei Gu | Yufei Gu |
+| [YARN-6546](https://issues.apache.org/jira/browse/YARN-6546) | SLS is slow while loading 10k queues |  Major | scheduler-load-simulator | Yufei Gu | Yufei Gu |
+| [YARN-7345](https://issues.apache.org/jira/browse/YARN-7345) | GPU Isolation: Incorrect minor device numbers written to devices.deny file |  Major | . | Jonathan Hung | Jonathan Hung |
+| [YARN-7338](https://issues.apache.org/jira/browse/YARN-7338) | Support same origin policy for cross site scripting prevention. |  Major | yarn-ui-v2 | Vrushali C | Sunil G |
+| [HDFS-12620](https://issues.apache.org/jira/browse/HDFS-12620) | Backporting HDFS-10467 to branch-2 |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [YARN-4090](https://issues.apache.org/jira/browse/YARN-4090) | Make Collections.sort() more efficient by caching resource usage |  Major | fairscheduler | Xianyin Xin | Yufei Gu |
+| [YARN-7353](https://issues.apache.org/jira/browse/YARN-7353) | Docker permitted volumes don't properly check for directories |  Major | yarn | Eric Badger | Eric Badger |
+| [YARN-6984](https://issues.apache.org/jira/browse/YARN-6984) | DominantResourceCalculator.isAnyMajorResourceZero() should test all resources |  Major | scheduler | Daniel Templeton | Sunil G |
+| [YARN-3661](https://issues.apache.org/jira/browse/YARN-3661) | Basic Federation UI |  Major | nodemanager, resourcemanager | Giovanni Matteo Fumarola | Íñigo Goiri |
+| [YARN-4827](https://issues.apache.org/jira/browse/YARN-4827) | Document configuration of ReservationSystem for FairScheduler |  Blocker | capacity scheduler | Subru Krishnan | Yufei Gu |
+| [YARN-5516](https://issues.apache.org/jira/browse/YARN-5516) | Add REST API for supporting recurring reservations |  Major | resourcemanager | Sangeetha Abdu Jyothi | Sean Po |
+| [YARN-6505](https://issues.apache.org/jira/browse/YARN-6505) | Define the strings used in SLS JSON input file format |  Major | scheduler-load-simulator | Yufei Gu | Gergely Novák |
+| [YARN-7178](https://issues.apache.org/jira/browse/YARN-7178) | Add documentation for Container Update API |  Blocker | . | Arun Suresh | Arun Suresh |
+| [YARN-7374](https://issues.apache.org/jira/browse/YARN-7374) | Improve performance of DRF comparisons for resource types in fair scheduler |  Critical | fairscheduler | Daniel Templeton | Daniel Templeton |
+| [YARN-6927](https://issues.apache.org/jira/browse/YARN-6927) | Add support for individual resource types requests in MapReduce |  Major | resourcemanager | Daniel Templeton | Gergo Repas |
+| [YARN-7407](https://issues.apache.org/jira/browse/YARN-7407) | Moving logging APIs over to slf4j in hadoop-yarn-applications |  Major | . | Yeliang Cang | Yeliang Cang |
+| [YARN-7379](https://issues.apache.org/jira/browse/YARN-7379) | Moving logging APIs over to slf4j in hadoop-yarn-client |  Major | . | Yeliang Cang | Yeliang Cang |
+| [YARN-7276](https://issues.apache.org/jira/browse/YARN-7276) | Federation Router Web Service fixes |  Major | . | Íñigo Goiri | Íñigo Goiri |
+| [HADOOP-14997](https://issues.apache.org/jira/browse/HADOOP-14997) |  Add hadoop-aliyun as dependency of hadoop-cloud-storage |  Minor | fs/oss | Genmao Yu | Genmao Yu |
+| [YARN-7289](https://issues.apache.org/jira/browse/YARN-7289) | Application lifetime does not work with FairScheduler |  Major | resourcemanager | Miklos Szegedi | Miklos Szegedi |
+| [YARN-7286](https://issues.apache.org/jira/browse/YARN-7286) | Add support for docker to have no capabilities |  Major | yarn | Eric Badger | Eric Badger |
+| [HDFS-11467](https://issues.apache.org/jira/browse/HDFS-11467) | Support ErasureCoding section in OIV XML/ReverseXML |  Blocker | tools | Wei-Chiu Chuang | Huafeng Wang |
+| [YARN-7307](https://issues.apache.org/jira/browse/YARN-7307) | Allow client/AM update supported resource types via YARN APIs |  Blocker | nodemanager, resourcemanager | Wangda Tan | Sunil G |
+| [MAPREDUCE-6997](https://issues.apache.org/jira/browse/MAPREDUCE-6997) | Moving logging APIs over to slf4j in hadoop-mapreduce-client-hs |  Major | . | Akira Ajisaka | Gergely Novák |
+| [MAPREDUCE-7001](https://issues.apache.org/jira/browse/MAPREDUCE-7001) | Moving logging APIs over to slf4j in hadoop-mapreduce-client-shuffle |  Trivial | . | Jinjiang Ling | Jinjiang Ling |
+| [YARN-7166](https://issues.apache.org/jira/browse/YARN-7166) | Container REST endpoints should report resource types |  Major | resourcemanager | Daniel Templeton | Daniel Templeton |
+| [YARN-7143](https://issues.apache.org/jira/browse/YARN-7143) | FileNotFound handling in ResourceUtils is inconsistent |  Major | resourcemanager | Daniel Templeton | Daniel Templeton |
+| [YARN-7437](https://issues.apache.org/jira/browse/YARN-7437) | Rename PlacementSet and SchedulingPlacementSet |  Major | . | Wangda Tan | Wangda Tan |
+| [YARN-6909](https://issues.apache.org/jira/browse/YARN-6909) | Use LightWeightedResource when number of resource types more than two |  Critical | resourcemanager | Daniel Templeton | Sunil G |
+| [YARN-7406](https://issues.apache.org/jira/browse/YARN-7406) | Moving logging APIs over to slf4j in hadoop-yarn-api |  Major | . | Yeliang Cang | Yeliang Cang |
+| [YARN-7442](https://issues.apache.org/jira/browse/YARN-7442) | [YARN-7069] Limit format of resource type name |  Blocker | nodemanager, resourcemanager | Wangda Tan | Wangda Tan |
+| [YARN-7369](https://issues.apache.org/jira/browse/YARN-7369) | Improve the resource types docs |  Major | docs | Daniel Templeton | Daniel Templeton |
+| [HADOOP-14993](https://issues.apache.org/jira/browse/HADOOP-14993) | AliyunOSS: Override listFiles and listLocatedStatus |  Major | fs/oss | Genmao Yu | Genmao Yu |
+| [YARN-7430](https://issues.apache.org/jira/browse/YARN-7430) | Enable user re-mapping for Docker containers by default |  Blocker | security, yarn | Eric Yang | Eric Yang |
+| [HADOOP-15024](https://issues.apache.org/jira/browse/HADOOP-15024) | AliyunOSS: support user agent configuration and include that & Hadoop version information to oss server |  Major | fs, fs/oss | SammiChen | SammiChen |
+| [YARN-7541](https://issues.apache.org/jira/browse/YARN-7541) | Node updates don't update the maximum cluster capability for resources other than CPU and memory |  Critical | resourcemanager | Daniel Templeton | Daniel Templeton |
+
+
+### OTHER:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [YARN-7317](https://issues.apache.org/jira/browse/YARN-7317) | Fix overallocation resulted from ceiling in LocalityMulticastAMRMProxyPolicy |  Minor | . | Botong Huang | Botong Huang |
+| [HDFS-12847](https://issues.apache.org/jira/browse/HDFS-12847) | Regenerate editsStored and editsStored.xml in HDFS tests |  Major | test | Lei (Eddy) Xu | Lei (Eddy) Xu |
+| [HADOOP-13493](https://issues.apache.org/jira/browse/HADOOP-13493) | Compatibility Docs should clarify the policy for what takes precedence when a conflict is found |  Critical | documentation | Robert Kanter | Daniel Templeton |
+
+

+ 139 - 0
hadoop-common-project/hadoop-common/src/site/markdown/release/3.0.0/RELEASENOTES.3.0.0.md

@@ -0,0 +1,139 @@
+
+<!---
+# 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.
+-->
+# Apache Hadoop  3.0.0 Release Notes
+
+These release notes cover new developer and user-facing incompatibilities, important issues, features, and major improvements.
+
+
+---
+
+* [YARN-7219](https://issues.apache.org/jira/browse/YARN-7219) | *Critical* | **Make AllocateRequestProto compatible with branch-2/branch-2.8**
+
+Change update\_requests field id to 7.  This matches the current field id in branch-2 and branch-2.8.
+
+
+---
+
+* [YARN-1492](https://issues.apache.org/jira/browse/YARN-1492) | *Major* | **truly shared cache for jars (jobjar/libjar)**
+
+The YARN Shared Cache provides the facility to upload and manage shared application resources to HDFS in a safe and scalable manner. YARN applications can leverage resources uploaded by other applications or previous runs of the same application without having to re-­upload and localize identical files multiple times. This will save network resources and reduce YARN application startup time.
+
+
+---
+
+* [HDFS-10467](https://issues.apache.org/jira/browse/HDFS-10467) | *Major* | **Router-based HDFS federation**
+
+HDFS Router-based Federation adds a RPC routing layer that provides a federated view of multiple HDFS namespaces.
+This is similar to the existing ViewFS and HDFS federation functionality, except the mount table is managed on the server-side by the routing layer rather than on the client.
+This simplifies access to a federated cluster for existing HDFS clients.
+
+See HDFS-10467 and the HDFS Router-based Federation documentation for more details.
+
+
+---
+
+* [YARN-5734](https://issues.apache.org/jira/browse/YARN-5734) | *Major* | **OrgQueue for easy CapacityScheduler queue configuration management**
+
+<!-- markdown -->
+
+The OrgQueue extension to the capacity scheduler provides a programmatic way to change configurations by providing a REST API that users can call to modify queue configurations. This enables automation of queue configuration management by administrators in the queue's `administer_queue` ACL.
+
+
+---
+
+* [MAPREDUCE-5951](https://issues.apache.org/jira/browse/MAPREDUCE-5951) | *Major* | **Add support for the YARN Shared Cache**
+
+MapReduce support for the YARN shared cache allows MapReduce jobs to take advantage of additional resource caching. This saves network bandwidth between the job submission client as well as within the YARN cluster itself. This will reduce job submission time and overall job runtime.
+
+
+---
+
+* [YARN-6623](https://issues.apache.org/jira/browse/YARN-6623) | *Blocker* | **Add support to turn off launching privileged containers in the container-executor**
+
+A change in configuration for launching Docker containers under YARN. Docker container capabilities, mounts, networks and allowing privileged container have to specified in the container-executor.cfg. By default, all of the above are turned off. This change will break existing setups launching Docker containers under YARN. Please refer to the Docker containers under YARN documentation for more information.
+
+
+---
+
+* [HADOOP-14816](https://issues.apache.org/jira/browse/HADOOP-14816) | *Major* | **Update Dockerfile to use Xenial**
+
+This patch changes the default build and test environment in the following ways:
+
+\* Switch from Ubuntu "Trusty" 14.04 to Ubuntu "Xenial" 16.04
+\* Switch from Oracle JDK 8 to OpenJDK 8
+\* Adds OpenJDK 9 to the build environment
+
+
+---
+
+* [HADOOP-14957](https://issues.apache.org/jira/browse/HADOOP-14957) | *Major* | **ReconfigurationTaskStatus is exposing guava Optional in its public api**
+
+ReconfigurationTaskStatus' API scope is reduced to LimitedPrivate, and its dependency on com.google.com.base.Optional is replaced by java.util.Optional.
+
+
+---
+
+* [HADOOP-14840](https://issues.apache.org/jira/browse/HADOOP-14840) | *Major* | **Tool to estimate resource requirements of an application pipeline based on prior executions**
+
+The first version of Resource Estimator service, a tool that captures the historical resource usage of an app and predicts its future resource requirement.
+
+
+---
+
+* [YARN-5326](https://issues.apache.org/jira/browse/YARN-5326) | *Major* | **Support for recurring reservations in the YARN ReservationSystem**
+
+Add native support for recurring reservations (good till cancelled) to enable  periodic allocations of the same resources.
+
+
+---
+
+* [MAPREDUCE-6983](https://issues.apache.org/jira/browse/MAPREDUCE-6983) | *Major* | **Moving logging APIs over to slf4j in hadoop-mapreduce-client-core**
+
+In hadoop-mapreduce-client-core module, the type of some public LOG variables were changed from org.apache.commons.logging.Log to org.slf4j.Logger. In the public methods that accepts logger, the logger was changed from org.apache.commons.logging.Log to org.slf4j.Logger.
+
+
+---
+
+* [HDFS-12682](https://issues.apache.org/jira/browse/HDFS-12682) | *Blocker* | **ECAdmin -listPolicies will always show SystemErasureCodingPolicies state as DISABLED**
+
+**WARNING: No release note provided for this change.**
+
+
+---
+
+* [YARN-5085](https://issues.apache.org/jira/browse/YARN-5085) | *Major* | **Add support for change of container ExecutionType**
+
+This allows the Application Master to ask the Scheduler to change the ExecutionType of a running/allocated container.
+
+
+---
+
+* [HDFS-12840](https://issues.apache.org/jira/browse/HDFS-12840) | *Blocker* | **Creating a file with non-default EC policy in a EC zone is not correctly serialized in the editlog**
+
+Add ErasureCodingPolicyId to each OP\_ADD edit log op.
+
+
+---
+
+* [HADOOP-15059](https://issues.apache.org/jira/browse/HADOOP-15059) | *Blocker* | **3.0 deployment cannot work with old version MR tar ball which breaks rolling upgrade**
+
+This change reverses the default delegation token format implemented by HADOOP-12563, but preserves the capability to read the new delegation token format.  When the new format becomes default, then MR deployment jobs runs will be compatible with releases that contain this change.
+
+
+

+ 2 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -56,7 +56,6 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.alias.CredentialProvider;
 import org.apache.hadoop.security.alias.CredentialProviderFactory;
@@ -385,11 +384,9 @@ public class TestConfiguration {
       Configuration conf = new Configuration(false);
       conf.addResource(new Path(CONFIG_MULTI_BYTE));
       assertEquals(value, conf.get(name));
-      FileOutputStream fos = new FileOutputStream(CONFIG_MULTI_BYTE_SAVED);
-      try {
+      try (FileOutputStream fos =
+               new FileOutputStream(CONFIG_MULTI_BYTE_SAVED)) {
         conf.writeXml(fos);
-      } finally {
-        IOUtils.closeStream(fos);
       }
 
       conf = new Configuration(false);

+ 4 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.InvalidPathHandleException;
 import org.apache.hadoop.fs.Options.HandleOpt;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathHandle;
@@ -241,7 +242,7 @@ public abstract class AbstractContractOpenTest
       // fail to resolve if path1 had been modified
       instream = getFileSystem().open(fd1);
       fail("Expected an exception");
-    } catch (IOException e) {
+    } catch (InvalidPathHandleException e) {
       // expected
     }
 
@@ -290,7 +291,7 @@ public abstract class AbstractContractOpenTest
       // handle should not resolve when content changed
       instream = getFileSystem().open(fd);
       fail("Failed to detect change to content");
-    } catch (IOException e) {
+    } catch (InvalidPathHandleException e) {
       // expected
     }
   }
@@ -330,7 +331,7 @@ public abstract class AbstractContractOpenTest
       // verify attempt to resolve the handle fails
       instream = getFileSystem().open(fd1);
       fail("Expected an exception");
-    } catch (IOException e) {
+    } catch (InvalidPathHandleException e) {
       // expected
     }
 

+ 2 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java

@@ -285,7 +285,7 @@ public class TestCount {
         // <----13---> <-------17------> <----13-----> <------17------->
         "    SSD_QUOTA     REM_SSD_QUOTA    DISK_QUOTA    REM_DISK_QUOTA " +
         // <----13---> <-------17------>
-        "ARCHIVE_QUOTA REM_ARCHIVE_QUOTA " +
+        "ARCHIVE_QUOTA REM_ARCHIVE_QUOTA PROVIDED_QUOTA REM_PROVIDED_QUOTA " +
         "PATHNAME";
     verify(out).println(withStorageTypeHeader);
     verifyNoMoreInteractions(out);
@@ -340,6 +340,7 @@ public class TestCount {
         "    SSD_QUOTA     REM_SSD_QUOTA " +
         "   DISK_QUOTA    REM_DISK_QUOTA " +
         "ARCHIVE_QUOTA REM_ARCHIVE_QUOTA " +
+        "PROVIDED_QUOTA REM_PROVIDED_QUOTA " +
         "PATHNAME";
     verify(out).println(withStorageTypeHeader);
     verifyNoMoreInteractions(out);

+ 3 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodec.java

@@ -362,11 +362,10 @@ public class TestCodec {
     final Path file = new Path(wd, "test" + codec.getDefaultExtension());
     final byte[] b = new byte[REC_SIZE];
     final Base64 b64 = new Base64(0, null);
-    DataOutputStream fout = null;
     Compressor cmp = CodecPool.getCompressor(codec);
-    try {
-      fout = new DataOutputStream(codec.createOutputStream(
-            fs.create(file, true), cmp));
+    try (DataOutputStream fout =
+             new DataOutputStream(codec.createOutputStream(fs.create(file,
+                 true), cmp))) {
       final DataOutputBuffer dob = new DataOutputBuffer(REC_SIZE * 4 / 3 + 4);
       int seq = 0;
       while (infLen > 0) {
@@ -382,7 +381,6 @@ public class TestCodec {
       }
       LOG.info("Wrote " + seq + " records to " + file);
     } finally {
-      IOUtils.cleanupWithLogger(LOG, fout);
       CodecPool.returnCompressor(cmp);
     }
     return file;

+ 2 - 6
hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/MiniKMS.java

@@ -105,15 +105,11 @@ public class MiniKMS {
 
   private void copyResource(String inputResourceName, File outputFile) throws
       IOException {
-    InputStream is = null;
-    OutputStream os = null;
-    try {
-      is = ThreadUtil.getResourceAsStream(inputResourceName);
-      os = new FileOutputStream(outputFile);
+    InputStream is = ThreadUtil.getResourceAsStream(inputResourceName);
+    try (OutputStream os = new FileOutputStream(outputFile)) {
       IOUtils.copy(is, os);
     } finally {
       IOUtils.closeQuietly(is);
-      IOUtils.closeQuietly(os);
     }
   }
 

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
+import org.apache.hadoop.fs.InvalidPathHandleException;
 import org.apache.hadoop.fs.PathHandle;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Options;
@@ -336,6 +337,8 @@ public class DistributedFileSystem extends FileSystem
    * the {@link PathHandle}.
    * @param fd Reference to entity in this FileSystem.
    * @param bufferSize the size of the buffer to be used.
+   * @throws InvalidPathHandleException If PathHandle constraints do not hold
+   * @throws IOException On I/O errors
    */
   @Override
   public FSDataInputStream open(PathHandle fd, int bufferSize)

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java

@@ -47,6 +47,9 @@ public final class HdfsConstants {
   public static final String WARM_STORAGE_POLICY_NAME = "WARM";
   public static final byte COLD_STORAGE_POLICY_ID = 2;
   public static final String COLD_STORAGE_POLICY_NAME = "COLD";
+  public static final byte PROVIDED_STORAGE_POLICY_ID = 1;
+  public static final String PROVIDED_STORAGE_POLICY_NAME = "PROVIDED";
+
 
   public static final int DEFAULT_DATA_SOCKET_SIZE = 0;
 

+ 7 - 9
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsPathHandle.java

@@ -23,6 +23,7 @@ import java.util.Optional;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.InvalidPathHandleException;
 import org.apache.hadoop.fs.PathHandle;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsPathHandleProto;
 
@@ -55,24 +56,21 @@ public final class HdfsPathHandle implements PathHandle {
     HdfsPathHandleProto p =
         HdfsPathHandleProto.parseFrom(ByteString.copyFrom(bytes));
     path = p.getPath();
-    mtime = p.hasMtime()
-        ? p.getMtime()
-        : null;
-    inodeId = p.hasInodeId()
-        ? p.getInodeId()
-        : null;
+    mtime   = p.hasMtime()   ? p.getMtime()   : null;
+    inodeId = p.hasInodeId() ? p.getInodeId() : null;
   }
 
   public String getPath() {
     return path;
   }
 
-  public void verify(HdfsLocatedFileStatus stat) throws IOException {
+  public void verify(HdfsLocatedFileStatus stat)
+      throws InvalidPathHandleException {
     if (mtime != null && mtime != stat.getModificationTime()) {
-      throw new IOException("Content changed");
+      throw new InvalidPathHandleException("Content changed");
     }
     if (inodeId != null && inodeId != stat.getFileId()) {
-      throw new IOException("Wrong file");
+      throw new InvalidPathHandleException("Wrong file");
     }
   }
 

+ 83 - 20
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java

@@ -17,7 +17,9 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import java.io.Serializable;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.List;
 
 import com.google.common.base.Preconditions;
@@ -39,6 +41,32 @@ import com.google.common.collect.Lists;
 @InterfaceStability.Evolving
 public class LocatedBlock {
 
+  /**
+   * Comparator that ensures that a PROVIDED storage type is greater than any
+   * other storage type. Any other storage types are considered equal.
+   */
+  private static class ProvidedLastComparator
+      implements Comparator<DatanodeInfoWithStorage>, Serializable {
+
+    private static final long serialVersionUID = 6441720011443190984L;
+
+    @Override
+    public int compare(DatanodeInfoWithStorage dns1,
+        DatanodeInfoWithStorage dns2) {
+      if (StorageType.PROVIDED.equals(dns1.getStorageType())
+          && !StorageType.PROVIDED.equals(dns2.getStorageType())) {
+        return 1;
+      }
+      if (!StorageType.PROVIDED.equals(dns1.getStorageType())
+          && StorageType.PROVIDED.equals(dns2.getStorageType())) {
+        return -1;
+      }
+      // Storage types of dns1 and dns2 are now both provided or not provided;
+      // thus, are essentially equal for the purpose of this comparator.
+      return 0;
+    }
+  }
+
   private final ExtendedBlock b;
   private long offset;  // offset of the first byte of the block in the file
   private final DatanodeInfoWithStorage[] locs;
@@ -51,6 +79,10 @@ public class LocatedBlock {
   // their locations are not part of this object
   private boolean corrupt;
   private Token<BlockTokenIdentifier> blockToken = new Token<>();
+
+  // use one instance of the Provided comparator as it uses no state.
+  private static ProvidedLastComparator providedLastComparator =
+      new ProvidedLastComparator();
   /**
    * List of cached datanode locations
    */
@@ -62,40 +94,50 @@ public class LocatedBlock {
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs) {
     // By default, startOffset is unknown(-1) and corrupt is false.
-    this(b, locs, null, null, -1, false, EMPTY_LOCS);
+    this(b, convert(locs, null, null), null, null, -1, false, EMPTY_LOCS);
   }
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs,
       String[] storageIDs, StorageType[] storageTypes) {
-    this(b, locs, storageIDs, storageTypes, -1, false, EMPTY_LOCS);
+    this(b, convert(locs, storageIDs, storageTypes),
+         storageIDs, storageTypes, -1, false, EMPTY_LOCS);
+  }
+
+  public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs,
+      String[] storageIDs, StorageType[] storageTypes, long startOffset,
+      boolean corrupt, DatanodeInfo[] cachedLocs) {
+    this(b, convert(locs, storageIDs, storageTypes),
+        storageIDs, storageTypes, startOffset, corrupt,
+        null == cachedLocs || 0 == cachedLocs.length ? EMPTY_LOCS : cachedLocs);
   }
 
-  public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs,
-      StorageType[] storageTypes, long startOffset,
+  public LocatedBlock(ExtendedBlock b, DatanodeInfoWithStorage[] locs,
+      String[] storageIDs, StorageType[] storageTypes, long startOffset,
       boolean corrupt, DatanodeInfo[] cachedLocs) {
     this.b = b;
     this.offset = startOffset;
     this.corrupt = corrupt;
-    if (locs==null) {
-      this.locs = EMPTY_LOCS;
-    } else {
-      this.locs = new DatanodeInfoWithStorage[locs.length];
-      for(int i = 0; i < locs.length; i++) {
-        DatanodeInfo di = locs[i];
-        DatanodeInfoWithStorage storage = new DatanodeInfoWithStorage(di,
-            storageIDs != null ? storageIDs[i] : null,
-            storageTypes != null ? storageTypes[i] : null);
-        this.locs[i] = storage;
-      }
-    }
+    this.locs = null == locs ? EMPTY_LOCS : locs;
     this.storageIDs = storageIDs;
     this.storageTypes = storageTypes;
+    this.cachedLocs = null == cachedLocs || 0 == cachedLocs.length
+      ? EMPTY_LOCS
+      : cachedLocs;
+  }
 
-    if (cachedLocs == null || cachedLocs.length == 0) {
-      this.cachedLocs = EMPTY_LOCS;
-    } else {
-      this.cachedLocs = cachedLocs;
+  private static DatanodeInfoWithStorage[] convert(
+      DatanodeInfo[] infos, String[] storageIDs, StorageType[] storageTypes) {
+    if (null == infos) {
+      return EMPTY_LOCS;
     }
+
+    DatanodeInfoWithStorage[] ret = new DatanodeInfoWithStorage[infos.length];
+    for(int i = 0; i < infos.length; i++) {
+      ret[i] = new DatanodeInfoWithStorage(infos[i],
+          storageIDs   != null ? storageIDs[i]   : null,
+          storageTypes != null ? storageTypes[i] : null);
+    }
+    return ret;
   }
 
   public Token<BlockTokenIdentifier> getBlockToken() {
@@ -145,6 +187,27 @@ public class LocatedBlock {
     }
   }
 
+  /**
+   * Moves all locations that have {@link StorageType}
+   * {@code PROVIDED} to the end of the locations array without
+   * changing the relative ordering of the remaining locations
+   * Only the first {@code activeLen} locations are considered.
+   * The caller must immediately invoke {@link
+   * org.apache.hadoop.hdfs.protocol.LocatedBlock#updateCachedStorageInfo}
+   * to update the cached Storage ID/Type arrays.
+   * @param activeLen
+   */
+  public void moveProvidedToEnd(int activeLen) {
+
+    if (activeLen <= 0) {
+      return;
+    }
+    // as this is a stable sort, for elements that are equal,
+    // the current order of the elements is maintained
+    Arrays.sort(locs, 0, (activeLen < locs.length) ? activeLen : locs.length,
+        providedLastComparator);
+  }
+
   public long getStartOffset() {
     return offset;
   }

+ 89 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java

@@ -0,0 +1,89 @@
+/*
+ * 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.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+
+import javax.annotation.Nonnull;
+import java.util.Arrays;
+
+/**
+ * ProvidedStorageLocation is a location in an external storage system
+ * containing the data for a block (~Replica).
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ProvidedStorageLocation {
+  private final Path path;
+  private final long offset;
+  private final long length;
+  private final byte[] nonce;
+
+  public ProvidedStorageLocation(Path path, long offset, long length,
+      byte[] nonce) {
+    this.path = path;
+    this.offset = offset;
+    this.length = length;
+    this.nonce = Arrays.copyOf(nonce, nonce.length);
+  }
+
+  public @Nonnull Path getPath() {
+    return path;
+  }
+
+  public long getOffset() {
+    return offset;
+  }
+
+  public long getLength() {
+    return length;
+  }
+
+  public @Nonnull byte[] getNonce() {
+    // create a copy of the nonce and return it.
+    return Arrays.copyOf(nonce, nonce.length);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    ProvidedStorageLocation that = (ProvidedStorageLocation) o;
+
+    if ((offset != that.offset) || (length != that.length)
+        || !path.equals(that.path)) {
+      return false;
+    }
+    return Arrays.equals(nonce, that.nonce);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = path.hashCode();
+    result = 31 * result + (int) (offset ^ (offset >>> 32));
+    result = 31 * result + (int) (length ^ (length >>> 32));
+    result = 31 * result + Arrays.hashCode(nonce);
+    return result;
+  }
+}

+ 37 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -96,6 +96,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
@@ -403,6 +404,8 @@ public class PBHelperClient {
       return StorageTypeProto.ARCHIVE;
     case RAM_DISK:
       return StorageTypeProto.RAM_DISK;
+    case PROVIDED:
+      return StorageTypeProto.PROVIDED;
     default:
       throw new IllegalStateException(
           "BUG: StorageType not found, type=" + type);
@@ -419,6 +422,8 @@ public class PBHelperClient {
       return StorageType.ARCHIVE;
     case RAM_DISK:
       return StorageType.RAM_DISK;
+    case PROVIDED:
+      return StorageType.PROVIDED;
     default:
       throw new IllegalStateException(
           "BUG: StorageTypeProto not found, type=" + type);
@@ -3223,4 +3228,35 @@ public class PBHelperClient {
     }
     return ret;
   }
-}
+
+  public static ProvidedStorageLocation convert(
+      HdfsProtos.ProvidedStorageLocationProto providedStorageLocationProto) {
+    if (providedStorageLocationProto == null) {
+      return null;
+    }
+    String path = providedStorageLocationProto.getPath();
+    long length = providedStorageLocationProto.getLength();
+    long offset = providedStorageLocationProto.getOffset();
+    ByteString nonce = providedStorageLocationProto.getNonce();
+
+    if (path == null || length == -1 || offset == -1 || nonce == null) {
+      return null;
+    } else {
+      return new ProvidedStorageLocation(new Path(path), offset, length,
+          nonce.toByteArray());
+    }
+  }
+
+  public static HdfsProtos.ProvidedStorageLocationProto convert(
+      ProvidedStorageLocation providedStorageLocation) {
+    String path = providedStorageLocation.getPath().toString();
+    return HdfsProtos.ProvidedStorageLocationProto.newBuilder()
+        .setPath(path)
+        .setLength(providedStorageLocation.getLength())
+        .setOffset(providedStorageLocation.getOffset())
+        .setNonce(ByteString.copyFrom(providedStorageLocation.getNonce()))
+        .build();
+  }
+
+
+}

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto

@@ -45,6 +45,20 @@ message ExtendedBlockProto {
                                                // here for historical reasons
 }
 
+
+/**
+* ProvidedStorageLocation will contain the exact location in the provided
+  storage. The path, offset and length will result in ranged read. The nonce
+  is there to verify that you receive what you expect.
+*/
+
+message ProvidedStorageLocationProto {
+  required string path = 1;
+  required int64 offset = 2;
+  required int64 length = 3;
+  required bytes nonce = 4;
+}
+
 /**
  * Identifies a Datanode
  */
@@ -207,6 +221,7 @@ enum StorageTypeProto {
   SSD = 2;
   ARCHIVE = 3;
   RAM_DISK = 4;
+  PROVIDED = 5;
 }
 
 /**

文件差异内容过多而无法显示
+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.0.0.xml


+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -186,7 +186,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <dependency>
       <groupId>org.fusesource.leveldbjni</groupId>
       <artifactId>leveldbjni-all</artifactId>
-      <version>1.8</version>
     </dependency>
     <dependency>
       <groupId>org.rocksdb</groupId>
@@ -240,6 +239,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
           <groupId>com.codahale.metrics</groupId>
           <artifactId>metrics-core</artifactId>
       </dependency>
+    <dependency>
+        <groupId>org.assertj</groupId>
+        <artifactId>assertj-core</artifactId>
+        <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>
@@ -377,6 +381,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                   <include>CBlockClientServerProtocol.proto</include>
                   <include>FederationProtocol.proto</include>
                   <include>RouterProtocol.proto</include>
+                  <include>AliasMapProtocol.proto</include>
                 </includes>
               </source>
             </configuration>

+ 29 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -95,6 +95,14 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY;
   public static final String  DFS_NAMENODE_BACKUP_HTTP_ADDRESS_DEFAULT = "0.0.0.0:50105";
   public static final String  DFS_NAMENODE_BACKUP_SERVICE_RPC_ADDRESS_KEY = "dfs.namenode.backup.dnrpc-address";
+  public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS = "dfs.provided.aliasmap.inmemory.dnrpc-address";
+  public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT = "0.0.0.0:50200";
+  public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR = "dfs.provided.aliasmap.inmemory.leveldb.dir";
+  public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE = "dfs.provided.aliasmap.inmemory.batch-size";
+  public static final int DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE_DEFAULT = 500;
+  public static final String DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED = "dfs.provided.aliasmap.inmemory.enabled";
+  public static final boolean DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED_DEFAULT = false;
+
   public static final String  DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_KEY =
       HdfsClientConfigKeys.DeprecatedKeys.DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_KEY;
   public static final long    DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_DEFAULT =
@@ -328,6 +336,26 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.namenode.edits.asynclogging";
   public static final boolean DFS_NAMENODE_EDITS_ASYNC_LOGGING_DEFAULT = true;
 
+  public static final String DFS_NAMENODE_PROVIDED_ENABLED = "dfs.namenode.provided.enabled";
+  public static final boolean DFS_NAMENODE_PROVIDED_ENABLED_DEFAULT = false;
+
+  public static final String DFS_PROVIDER_STORAGEUUID = "dfs.provided.storage.id";
+  public static final String DFS_PROVIDER_STORAGEUUID_DEFAULT =  "DS-PROVIDED";
+  public static final String DFS_PROVIDED_ALIASMAP_CLASS = "dfs.provided.aliasmap.class";
+  public static final String DFS_PROVIDED_ALIASMAP_LOAD_RETRIES = "dfs.provided.aliasmap.load.retries";
+
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER = "dfs.provided.aliasmap.text.delimiter";
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT = ",";
+
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_READ_FILE = "dfs.provided.aliasmap.text.read.file";
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_READ_FILE_DEFAULT = "file:///tmp/blocks.csv";
+
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_CODEC = "dfs.provided.aliasmap.text.codec";
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_WRITE_DIR = "dfs.provided.aliasmap.text.write.dir";
+  public static final String DFS_PROVIDED_ALIASMAP_TEXT_WRITE_DIR_DEFAULT = "file:///tmp/";
+
+  public static final String DFS_PROVIDED_ALIASMAP_LEVELDB_PATH = "dfs.provided.aliasmap.leveldb.path";
+
   public static final String  DFS_LIST_LIMIT = "dfs.ls.limit";
   public static final int     DFS_LIST_LIMIT_DEFAULT = 1000;
   public static final String  DFS_CONTENT_SUMMARY_LIMIT_KEY = "dfs.content-summary.limit";
@@ -1617,4 +1645,5 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   @Deprecated
   public static final long    DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT =
       HdfsClientConfigKeys.DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT;
+
 }

+ 37 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java

@@ -0,0 +1,37 @@
+/*
+ * 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.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos;
+import org.apache.hadoop.ipc.ProtocolInfo;
+
+/**
+ * Protocol between the Namenode and the Datanode to read the AliasMap
+ * used for Provided storage.
+ * TODO add Kerberos support
+ */
+@ProtocolInfo(
+    protocolName =
+        "org.apache.hadoop.hdfs.server.aliasmap.AliasMapProtocol",
+    protocolVersion = 1)
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface AliasMapProtocolPB extends
+    AliasMapProtocolProtos.AliasMapProtocolService.BlockingInterface {
+}

+ 134 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java

@@ -0,0 +1,134 @@
+/*
+ * 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.protocolPB;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.KeyValueProto;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.ReadResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.WriteRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.WriteResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMapProtocol;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.*;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.*;
+
+/**
+ * AliasMapProtocolServerSideTranslatorPB is responsible for translating RPC
+ * calls and forwarding them to the internal InMemoryAliasMap.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class AliasMapProtocolServerSideTranslatorPB
+    implements AliasMapProtocolPB {
+
+  private final InMemoryAliasMapProtocol aliasMap;
+
+  public AliasMapProtocolServerSideTranslatorPB(
+      InMemoryAliasMapProtocol aliasMap) {
+    this.aliasMap = aliasMap;
+  }
+
+  private static final WriteResponseProto VOID_WRITE_RESPONSE =
+      WriteResponseProto.newBuilder().build();
+
+  @Override
+  public WriteResponseProto write(RpcController controller,
+      WriteRequestProto request) throws ServiceException {
+    try {
+      FileRegion toWrite =
+          PBHelper.convert(request.getKeyValuePair());
+
+      aliasMap.write(toWrite.getBlock(), toWrite.getProvidedStorageLocation());
+      return VOID_WRITE_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ReadResponseProto read(RpcController controller,
+      ReadRequestProto request) throws ServiceException {
+    try {
+      Block toRead =  PBHelperClient.convert(request.getKey());
+
+      Optional<ProvidedStorageLocation> optionalResult =
+          aliasMap.read(toRead);
+
+      ReadResponseProto.Builder builder = ReadResponseProto.newBuilder();
+      if (optionalResult.isPresent()) {
+        ProvidedStorageLocation providedStorageLocation = optionalResult.get();
+        builder.setValue(PBHelperClient.convert(providedStorageLocation));
+      }
+
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ListResponseProto list(RpcController controller,
+      ListRequestProto request) throws ServiceException {
+    try {
+      BlockProto marker = request.getMarker();
+      IterationResult iterationResult;
+      if (marker.isInitialized()) {
+        iterationResult =
+            aliasMap.list(Optional.of(PBHelperClient.convert(marker)));
+      } else {
+        iterationResult = aliasMap.list(Optional.empty());
+      }
+      ListResponseProto.Builder responseBuilder =
+          ListResponseProto.newBuilder();
+      List<FileRegion> fileRegions = iterationResult.getFileRegions();
+
+      List<KeyValueProto> keyValueProtos = fileRegions.stream()
+          .map(PBHelper::convert).collect(Collectors.toList());
+      responseBuilder.addAllFileRegions(keyValueProtos);
+      Optional<Block> nextMarker = iterationResult.getNextBlock();
+      nextMarker
+          .map(m -> responseBuilder.setNextMarker(PBHelperClient.convert(m)));
+
+      return responseBuilder.build();
+
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  public BlockPoolResponseProto getBlockPoolId(RpcController controller,
+      BlockPoolRequestProto req) throws ServiceException {
+    try {
+      String bpid = aliasMap.getBlockPoolId();
+      return BlockPoolResponseProto.newBuilder().setBlockPoolId(bpid).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+}

+ 174 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java

@@ -0,0 +1,174 @@
+/*
+ * 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.protocolPB;
+
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMapProtocol;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT;
+import static org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.*;
+import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.*;
+
+/**
+ * This class is the client side translator to translate requests made to the
+ * {@link InMemoryAliasMapProtocol} interface to the RPC server implementing
+ * {@link AliasMapProtocolPB}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class InMemoryAliasMapProtocolClientSideTranslatorPB
+    implements InMemoryAliasMapProtocol {
+
+  private static final Logger LOG =
+      LoggerFactory
+          .getLogger(InMemoryAliasMapProtocolClientSideTranslatorPB.class);
+
+  private AliasMapProtocolPB rpcProxy;
+
+  public InMemoryAliasMapProtocolClientSideTranslatorPB(Configuration conf) {
+    String addr = conf.getTrimmed(DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+        DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT);
+    InetSocketAddress aliasMapAddr = NetUtils.createSocketAddr(addr);
+
+    RPC.setProtocolEngine(conf, AliasMapProtocolPB.class,
+        ProtobufRpcEngine.class);
+    LOG.info("Connecting to address: " + addr);
+    try {
+      rpcProxy = RPC.getProxy(AliasMapProtocolPB.class,
+          RPC.getProtocolVersion(AliasMapProtocolPB.class), aliasMapAddr, null,
+          conf, NetUtils.getDefaultSocketFactory(conf), 0);
+    } catch (IOException e) {
+      throw new RuntimeException(
+          "Error in connecting to " + addr + " Got: " + e);
+    }
+  }
+
+  @Override
+  public InMemoryAliasMap.IterationResult list(Optional<Block> marker)
+      throws IOException {
+    ListRequestProto.Builder builder = ListRequestProto.newBuilder();
+    if (marker.isPresent()) {
+      builder.setMarker(PBHelperClient.convert(marker.get()));
+    }
+    ListRequestProto request = builder.build();
+    try {
+      ListResponseProto response = rpcProxy.list(null, request);
+      List<KeyValueProto> fileRegionsList = response.getFileRegionsList();
+
+      List<FileRegion> fileRegions = fileRegionsList
+          .stream()
+          .map(kv -> new FileRegion(
+              PBHelperClient.convert(kv.getKey()),
+              PBHelperClient.convert(kv.getValue())
+          ))
+          .collect(Collectors.toList());
+      BlockProto nextMarker = response.getNextMarker();
+
+      if (nextMarker.isInitialized()) {
+        return new InMemoryAliasMap.IterationResult(fileRegions,
+            Optional.of(PBHelperClient.convert(nextMarker)));
+      } else {
+        return new InMemoryAliasMap.IterationResult(fileRegions,
+            Optional.empty());
+      }
+
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Nonnull
+  @Override
+  public Optional<ProvidedStorageLocation> read(@Nonnull Block block)
+      throws IOException {
+
+    ReadRequestProto request =
+        ReadRequestProto
+            .newBuilder()
+            .setKey(PBHelperClient.convert(block))
+            .build();
+    try {
+      ReadResponseProto response = rpcProxy.read(null, request);
+
+      ProvidedStorageLocationProto providedStorageLocation =
+          response.getValue();
+      if (providedStorageLocation.isInitialized()) {
+        return Optional.of(PBHelperClient.convert(providedStorageLocation));
+      }
+      return Optional.empty();
+
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void write(@Nonnull Block block,
+      @Nonnull ProvidedStorageLocation providedStorageLocation)
+      throws IOException {
+    WriteRequestProto request =
+        WriteRequestProto
+            .newBuilder()
+            .setKeyValuePair(KeyValueProto.newBuilder()
+                .setKey(PBHelperClient.convert(block))
+                .setValue(PBHelperClient.convert(providedStorageLocation))
+                .build())
+            .build();
+
+    try {
+      rpcProxy.write(null, request);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public String getBlockPoolId() throws IOException {
+    try {
+      BlockPoolResponseProto response = rpcProxy.getBlockPoolId(null,
+          BlockPoolRequestProto.newBuilder().build());
+      return response.getBlockPoolId();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  public void stop() {
+    RPC.stopProxy(rpcProxy);
+  }
+}

+ 28 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -36,6 +36,8 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.KeyValueProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECReconstructionCommandProto;
@@ -56,6 +58,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECReconstr
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ProvidedStorageLocationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
@@ -80,6 +83,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.StorageInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
@@ -1096,4 +1100,28 @@ public class PBHelper {
         DatanodeProtocol.DNA_ERASURE_CODING_RECONSTRUCTION,
         blkECReconstructionInfos);
   }
+
+  public static KeyValueProto convert(FileRegion fileRegion) {
+    return KeyValueProto
+        .newBuilder()
+        .setKey(PBHelperClient.convert(fileRegion.getBlock()))
+        .setValue(PBHelperClient.convert(
+            fileRegion.getProvidedStorageLocation()))
+        .build();
+  }
+
+  public static FileRegion
+      convert(KeyValueProto keyValueProto) {
+    BlockProto blockProto =
+        keyValueProto.getKey();
+    ProvidedStorageLocationProto providedStorageLocationProto =
+        keyValueProto.getValue();
+
+    Block block =
+        PBHelperClient.convert(blockProto);
+    ProvidedStorageLocation providedStorageLocation =
+        PBHelperClient.convert(providedStorageLocationProto);
+
+    return new FileRegion(block, providedStorageLocation);
+  }
 }

+ 222 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java

@@ -0,0 +1,222 @@
+/*
+ * 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.server.aliasmap;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ProvidedStorageLocationProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.fusesource.leveldbjni.JniDBFactory;
+import org.iq80.leveldb.DB;
+import org.iq80.leveldb.DBIterator;
+import org.iq80.leveldb.Options;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * InMemoryAliasMap is an implementation of the InMemoryAliasMapProtocol for
+ * use with LevelDB.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
+    Configurable {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(InMemoryAliasMap.class);
+
+  private final DB levelDb;
+  private Configuration conf;
+  private String blockPoolID;
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+  @VisibleForTesting
+  static String createPathErrorMessage(String directory) {
+    return new StringBuilder()
+        .append("Configured directory '")
+        .append(directory)
+        .append("' doesn't exist")
+        .toString();
+  }
+
+  public static @Nonnull InMemoryAliasMap init(Configuration conf,
+      String blockPoolID) throws IOException {
+    Options options = new Options();
+    options.createIfMissing(true);
+    String directory =
+        conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR);
+    LOG.info("Attempting to load InMemoryAliasMap from \"{}\"", directory);
+    File levelDBpath;
+    if (blockPoolID != null) {
+      levelDBpath = new File(directory, blockPoolID);
+    } else {
+      levelDBpath = new File(directory);
+    }
+    if (!levelDBpath.exists()) {
+      String error = createPathErrorMessage(directory);
+      throw new IOException(error);
+    }
+    DB levelDb = JniDBFactory.factory.open(levelDBpath, options);
+    InMemoryAliasMap aliasMap = new InMemoryAliasMap(levelDb, blockPoolID);
+    aliasMap.setConf(conf);
+    return aliasMap;
+  }
+
+  @VisibleForTesting
+  InMemoryAliasMap(DB levelDb, String blockPoolID) {
+    this.levelDb = levelDb;
+    this.blockPoolID = blockPoolID;
+  }
+
+  @Override
+  public IterationResult list(Optional<Block> marker) throws IOException {
+    try (DBIterator iterator = levelDb.iterator()) {
+      Integer batchSize =
+          conf.getInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE,
+              DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE_DEFAULT);
+      if (marker.isPresent()) {
+        iterator.seek(toProtoBufBytes(marker.get()));
+      } else {
+        iterator.seekToFirst();
+      }
+      int i = 0;
+      ArrayList<FileRegion> batch =
+          Lists.newArrayListWithExpectedSize(batchSize);
+      while (iterator.hasNext() && i < batchSize) {
+        Map.Entry<byte[], byte[]> entry = iterator.next();
+        Block block = fromBlockBytes(entry.getKey());
+        ProvidedStorageLocation providedStorageLocation =
+            fromProvidedStorageLocationBytes(entry.getValue());
+        batch.add(new FileRegion(block, providedStorageLocation));
+        ++i;
+      }
+      if (iterator.hasNext()) {
+        Block nextMarker = fromBlockBytes(iterator.next().getKey());
+        return new IterationResult(batch, Optional.of(nextMarker));
+      } else {
+        return new IterationResult(batch, Optional.empty());
+      }
+    }
+  }
+
+  public @Nonnull Optional<ProvidedStorageLocation> read(@Nonnull Block block)
+      throws IOException {
+
+    byte[] extendedBlockDbFormat = toProtoBufBytes(block);
+    byte[] providedStorageLocationDbFormat = levelDb.get(extendedBlockDbFormat);
+    if (providedStorageLocationDbFormat == null) {
+      return Optional.empty();
+    } else {
+      ProvidedStorageLocation providedStorageLocation =
+          fromProvidedStorageLocationBytes(providedStorageLocationDbFormat);
+      return Optional.of(providedStorageLocation);
+    }
+  }
+
+  public void write(@Nonnull Block block,
+      @Nonnull ProvidedStorageLocation providedStorageLocation)
+      throws IOException {
+    byte[] extendedBlockDbFormat = toProtoBufBytes(block);
+    byte[] providedStorageLocationDbFormat =
+        toProtoBufBytes(providedStorageLocation);
+    levelDb.put(extendedBlockDbFormat, providedStorageLocationDbFormat);
+  }
+
+  @Override
+  public String getBlockPoolId() {
+    return blockPoolID;
+  }
+
+  public void close() throws IOException {
+    levelDb.close();
+  }
+
+  @Nonnull
+  public static ProvidedStorageLocation fromProvidedStorageLocationBytes(
+      @Nonnull byte[] providedStorageLocationDbFormat)
+      throws InvalidProtocolBufferException {
+    ProvidedStorageLocationProto providedStorageLocationProto =
+        ProvidedStorageLocationProto
+            .parseFrom(providedStorageLocationDbFormat);
+    return PBHelperClient.convert(providedStorageLocationProto);
+  }
+
+  @Nonnull
+  public static Block fromBlockBytes(@Nonnull byte[] blockDbFormat)
+      throws InvalidProtocolBufferException {
+    BlockProto blockProto = BlockProto.parseFrom(blockDbFormat);
+    return PBHelperClient.convert(blockProto);
+  }
+
+  public static byte[] toProtoBufBytes(@Nonnull ProvidedStorageLocation
+      providedStorageLocation) throws IOException {
+    ProvidedStorageLocationProto providedStorageLocationProto =
+        PBHelperClient.convert(providedStorageLocation);
+    ByteArrayOutputStream providedStorageLocationOutputStream =
+        new ByteArrayOutputStream();
+    providedStorageLocationProto.writeTo(providedStorageLocationOutputStream);
+    return providedStorageLocationOutputStream.toByteArray();
+  }
+
+  public static byte[] toProtoBufBytes(@Nonnull Block block)
+      throws IOException {
+    BlockProto blockProto =
+        PBHelperClient.convert(block);
+    ByteArrayOutputStream blockOutputStream = new ByteArrayOutputStream();
+    blockProto.writeTo(blockOutputStream);
+    return blockOutputStream.toByteArray();
+  }
+
+  /**
+   * CheckedFunction is akin to {@link java.util.function.Function} but
+   * specifies an IOException.
+   * @param <T1> First argument type.
+   * @param <T2> Second argument type.
+   * @param <R> Return type.
+   */
+  @FunctionalInterface
+  public interface CheckedFunction2<T1, T2, R> {
+    R apply(T1 t1, T2 t2) throws IOException;
+  }
+}

+ 103 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java

@@ -0,0 +1,103 @@
+/*
+ * 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.server.aliasmap;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Protocol used by clients to read/write data about aliases of
+ * provided blocks for an in-memory implementation of the
+ * {@link org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface InMemoryAliasMapProtocol {
+
+  /**
+   * The result of a read from the in-memory aliasmap. It contains the
+   * a list of FileRegions that are returned, along with the next block
+   * from which the read operation must continue.
+   */
+  class IterationResult {
+
+    private final List<FileRegion> batch;
+    private final Optional<Block> nextMarker;
+
+    public IterationResult(List<FileRegion> batch, Optional<Block> nextMarker) {
+      this.batch = batch;
+      this.nextMarker = nextMarker;
+    }
+
+    public List<FileRegion> getFileRegions() {
+      return batch;
+    }
+
+    public Optional<Block> getNextBlock() {
+      return nextMarker;
+    }
+  }
+
+  /**
+   * List the next batch of {@link FileRegion}s in the alias map starting from
+   * the given {@code marker}. To retrieve all {@link FileRegion}s stored in the
+   * alias map, multiple calls to this function might be required.
+   * @param marker the next block to get fileregions from.
+   * @return the {@link IterationResult} with a set of
+   * FileRegions and the next marker.
+   * @throws IOException
+   */
+  InMemoryAliasMap.IterationResult list(Optional<Block> marker)
+      throws IOException;
+
+  /**
+   * Gets the {@link ProvidedStorageLocation} associated with the
+   * specified block.
+   * @param block the block to lookup
+   * @return the associated {@link ProvidedStorageLocation}.
+   * @throws IOException
+   */
+  @Nonnull
+  Optional<ProvidedStorageLocation> read(@Nonnull Block block)
+      throws IOException;
+
+  /**
+   * Stores the block and it's associated {@link ProvidedStorageLocation}
+   * in the alias map.
+   * @param block
+   * @param providedStorageLocation
+   * @throws IOException
+   */
+  void write(@Nonnull Block block,
+      @Nonnull ProvidedStorageLocation providedStorageLocation)
+      throws IOException;
+
+  /**
+   * Get the associated block pool id.
+   * @return the block pool id associated with the Namenode running
+   * the in-memory alias map.
+   */
+  String getBlockPoolId() throws IOException;
+}

+ 153 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java

@@ -0,0 +1,153 @@
+/*
+ * 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.server.aliasmap;
+
+import com.google.protobuf.BlockingService;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.protocolPB.AliasMapProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.AliasMapProtocolServerSideTranslatorPB;
+import org.apache.hadoop.ipc.RPC;
+import javax.annotation.Nonnull;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Optional;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT;
+import static org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.*;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.CheckedFunction2;
+
+/**
+ * InMemoryLevelDBAliasMapServer is the entry point from the Namenode into
+ * the {@link InMemoryAliasMap}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
+    Configurable, Closeable {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(InMemoryLevelDBAliasMapServer.class);
+  private final CheckedFunction2<Configuration, String, InMemoryAliasMap>
+      initFun;
+  private RPC.Server aliasMapServer;
+  private Configuration conf;
+  private InMemoryAliasMap aliasMap;
+  private String blockPoolId;
+
+  public InMemoryLevelDBAliasMapServer(
+          CheckedFunction2<Configuration, String, InMemoryAliasMap> initFun,
+      String blockPoolId) {
+    this.initFun = initFun;
+    this.blockPoolId = blockPoolId;
+  }
+
+  public void start() throws IOException {
+    if (UserGroupInformation.isSecurityEnabled()) {
+      throw new UnsupportedOperationException("Unable to start "
+          + "InMemoryLevelDBAliasMapServer as security is enabled");
+    }
+    RPC.setProtocolEngine(getConf(), AliasMapProtocolPB.class,
+        ProtobufRpcEngine.class);
+    AliasMapProtocolServerSideTranslatorPB aliasMapProtocolXlator =
+        new AliasMapProtocolServerSideTranslatorPB(this);
+
+    BlockingService aliasMapProtocolService =
+        AliasMapProtocolService
+            .newReflectiveBlockingService(aliasMapProtocolXlator);
+
+    String rpcAddress =
+        conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+            DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT);
+    String[] split = rpcAddress.split(":");
+    String bindHost = split[0];
+    Integer port = Integer.valueOf(split[1]);
+
+    aliasMapServer = new RPC.Builder(conf)
+        .setProtocol(AliasMapProtocolPB.class)
+        .setInstance(aliasMapProtocolService)
+        .setBindAddress(bindHost)
+        .setPort(port)
+        .setNumHandlers(1)
+        .setVerbose(true)
+        .build();
+
+    LOG.info("Starting InMemoryLevelDBAliasMapServer on {}", rpcAddress);
+    aliasMapServer.start();
+  }
+
+  @Override
+  public InMemoryAliasMap.IterationResult list(Optional<Block> marker)
+      throws IOException {
+    return aliasMap.list(marker);
+  }
+
+  @Nonnull
+  @Override
+  public Optional<ProvidedStorageLocation> read(@Nonnull Block block)
+      throws IOException {
+    return aliasMap.read(block);
+  }
+
+  @Override
+  public void write(@Nonnull Block block,
+      @Nonnull ProvidedStorageLocation providedStorageLocation)
+      throws IOException {
+    aliasMap.write(block, providedStorageLocation);
+  }
+
+  @Override
+  public String getBlockPoolId() {
+    return blockPoolId;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    try {
+      this.aliasMap = initFun.apply(conf, blockPoolId);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void close() {
+    LOG.info("Stopping InMemoryLevelDBAliasMapServer");
+    try {
+      aliasMap.close();
+    } catch (IOException e) {
+      LOG.error(e.getMessage());
+    }
+    aliasMapServer.stop();
+  }
+
+}

+ 14 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -24,6 +24,7 @@ import java.util.NoSuchElementException;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@@ -186,13 +187,23 @@ public abstract class BlockInfo extends Block
    */
   DatanodeStorageInfo findStorageInfo(DatanodeDescriptor dn) {
     int len = getCapacity();
+    DatanodeStorageInfo providedStorageInfo = null;
     for(int idx = 0; idx < len; idx++) {
       DatanodeStorageInfo cur = getStorageInfo(idx);
-      if(cur != null && cur.getDatanodeDescriptor() == dn) {
-        return cur;
+      if(cur != null) {
+        if (cur.getStorageType() == StorageType.PROVIDED) {
+          // if block resides on provided storage, only match the storage ids
+          if (dn.getStorageInfo(cur.getStorageID()) != null) {
+            // do not return here as we have to check the other
+            // DatanodeStorageInfos for this block which could be local
+            providedStorageInfo = cur;
+          }
+        } else if (cur.getDatanodeDescriptor() == dn) {
+          return cur;
+        }
       }
     }
-    return null;
+    return providedStorageInfo;
   }
 
   /**

+ 110 - 39
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -103,6 +103,8 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.hdfs.util.FoldedTreeSet;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.server.namenode.CacheManager;
@@ -435,6 +437,9 @@ public class BlockManager implements BlockStatsMXBean {
    */
   private final short minReplicationToBeInMaintenance;
 
+  /** Storages accessible from multiple DNs. */
+  private final ProvidedStorageMap providedStorageMap;
+
   public BlockManager(final Namesystem namesystem, boolean haEnabled,
       final Configuration conf) throws IOException {
     this.namesystem = namesystem;
@@ -467,6 +472,8 @@ public class BlockManager implements BlockStatsMXBean {
 
     blockTokenSecretManager = createBlockTokenSecretManager(conf);
 
+    providedStorageMap = new ProvidedStorageMap(namesystem, this, conf);
+
     this.maxCorruptFilesReturned = conf.getInt(
       DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY,
       DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED);
@@ -1144,7 +1151,7 @@ public class BlockManager implements BlockStatsMXBean {
     final long fileLength = bc.computeContentSummary(
         getStoragePolicySuite()).getLength();
     final long pos = fileLength - lastBlock.getNumBytes();
-    return createLocatedBlock(lastBlock, pos,
+    return createLocatedBlock(null, lastBlock, pos,
         BlockTokenIdentifier.AccessMode.WRITE);
   }
 
@@ -1165,8 +1172,10 @@ public class BlockManager implements BlockStatsMXBean {
     return locations;
   }
 
-  private List<LocatedBlock> createLocatedBlockList(final BlockInfo[] blocks,
-      final long offset, final long length, final int nrBlocksToReturn,
+  private void createLocatedBlockList(
+      LocatedBlockBuilder locatedBlocks,
+      final BlockInfo[] blocks,
+      final long offset, final long length,
       final AccessMode mode) throws IOException {
     int curBlk;
     long curPos = 0, blkSize = 0;
@@ -1181,21 +1190,22 @@ public class BlockManager implements BlockStatsMXBean {
     }
 
     if (nrBlocks > 0 && curBlk == nrBlocks)   // offset >= end of file
-      return Collections.emptyList();
+      return;
 
     long endOff = offset + length;
-    List<LocatedBlock> results = new ArrayList<>(blocks.length);
     do {
-      results.add(createLocatedBlock(blocks[curBlk], curPos, mode));
+      locatedBlocks.addBlock(
+          createLocatedBlock(locatedBlocks, blocks[curBlk], curPos, mode));
       curPos += blocks[curBlk].getNumBytes();
       curBlk++;
     } while (curPos < endOff 
           && curBlk < blocks.length
-          && results.size() < nrBlocksToReturn);
-    return results;
+          && !locatedBlocks.isBlockMax());
+    return;
   }
 
-  private LocatedBlock createLocatedBlock(final BlockInfo[] blocks,
+  private LocatedBlock createLocatedBlock(LocatedBlockBuilder locatedBlocks,
+      final BlockInfo[] blocks,
       final long endPos, final AccessMode mode) throws IOException {
     int curBlk;
     long curPos = 0;
@@ -1208,12 +1218,13 @@ public class BlockManager implements BlockStatsMXBean {
       curPos += blkSize;
     }
     
-    return createLocatedBlock(blocks[curBlk], curPos, mode);
+    return createLocatedBlock(locatedBlocks, blocks[curBlk], curPos, mode);
   }
 
-  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos,
-    final AccessMode mode) throws IOException {
-    final LocatedBlock lb = createLocatedBlock(blk, pos);
+  private LocatedBlock createLocatedBlock(LocatedBlockBuilder locatedBlocks,
+      final BlockInfo blk, final long pos, final AccessMode mode)
+          throws IOException {
+    final LocatedBlock lb = createLocatedBlock(locatedBlocks, blk, pos);
     if (mode != null) {
       setBlockToken(lb, mode);
     }
@@ -1221,8 +1232,8 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /** @return a LocatedBlock for the given block */
-  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos)
-      throws IOException {
+  private LocatedBlock createLocatedBlock(LocatedBlockBuilder locatedBlocks,
+      final BlockInfo blk, final long pos) throws IOException {
     if (!blk.isComplete()) {
       final BlockUnderConstructionFeature uc = blk.getUnderConstructionFeature();
       if (blk.isStriped()) {
@@ -1235,7 +1246,9 @@ public class BlockManager implements BlockStatsMXBean {
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
         final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(),
             blk);
-        return newLocatedBlock(eb, storages, pos, false);
+        return null == locatedBlocks
+            ? newLocatedBlock(eb, storages, pos, false)
+                : locatedBlocks.newLocatedBlock(eb, storages, pos, false);
       }
     }
 
@@ -1299,9 +1312,10 @@ public class BlockManager implements BlockStatsMXBean {
       " numCorrupt: " + numCorruptNodes +
       " numCorruptRepls: " + numCorruptReplicas;
     final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(), blk);
-    return blockIndices == null ?
-        newLocatedBlock(eb, machines, pos, isCorrupt) :
-        newLocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
+    return blockIndices == null
+        ? null == locatedBlocks ? newLocatedBlock(eb, machines, pos, isCorrupt)
+            : locatedBlocks.newLocatedBlock(eb, machines, pos, isCorrupt)
+        : newLocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
   }
 
   /** Create a LocatedBlocks. */
@@ -1323,27 +1337,31 @@ public class BlockManager implements BlockStatsMXBean {
         LOG.debug("blocks = {}", java.util.Arrays.asList(blocks));
       }
       final AccessMode mode = needBlockToken? BlockTokenIdentifier.AccessMode.READ: null;
-      final List<LocatedBlock> locatedblocks = createLocatedBlockList(
-          blocks, offset, length, Integer.MAX_VALUE, mode);
 
-      final LocatedBlock lastlb;
-      final boolean isComplete;
+      LocatedBlockBuilder locatedBlocks = providedStorageMap
+          .newLocatedBlocks(Integer.MAX_VALUE)
+          .fileLength(fileSizeExcludeBlocksUnderConstruction)
+          .lastUC(isFileUnderConstruction)
+          .encryption(feInfo)
+          .erasureCoding(ecPolicy);
+
+      createLocatedBlockList(locatedBlocks, blocks, offset, length, mode);
       if (!inSnapshot) {
         final BlockInfo last = blocks[blocks.length - 1];
         final long lastPos = last.isComplete()?
             fileSizeExcludeBlocksUnderConstruction - last.getNumBytes()
             : fileSizeExcludeBlocksUnderConstruction;
-        lastlb = createLocatedBlock(last, lastPos, mode);
-        isComplete = last.isComplete();
+
+        locatedBlocks
+          .lastBlock(createLocatedBlock(locatedBlocks, last, lastPos, mode))
+          .lastComplete(last.isComplete());
       } else {
-        lastlb = createLocatedBlock(blocks,
-            fileSizeExcludeBlocksUnderConstruction, mode);
-        isComplete = true;
+        locatedBlocks
+          .lastBlock(createLocatedBlock(locatedBlocks, blocks,
+              fileSizeExcludeBlocksUnderConstruction, mode))
+          .lastComplete(true);
       }
-      LocatedBlocks locations = new LocatedBlocks(
-          fileSizeExcludeBlocksUnderConstruction,
-          isFileUnderConstruction, locatedblocks, lastlb, isComplete, feInfo,
-          ecPolicy);
+      LocatedBlocks locations = locatedBlocks.build();
       // Set caching information for the located blocks.
       CacheManager cm = namesystem.getCacheManager();
       if (cm != null) {
@@ -1497,6 +1515,7 @@ public class BlockManager implements BlockStatsMXBean {
    
   /** Remove the blocks associated to the given datanode. */
   void removeBlocksAssociatedTo(final DatanodeDescriptor node) {
+    providedStorageMap.removeDatanode(node);
     for (DatanodeStorageInfo storage : node.getStorageInfos()) {
       final Iterator<BlockInfo> it = storage.getBlockIterator();
       //add the BlockInfos to a new collection as the
@@ -2130,6 +2149,22 @@ public class BlockManager implements BlockStatsMXBean {
     return datanodeDescriptors;
   }
 
+  /**
+   * Get the associated {@link DatanodeDescriptor} for the storage.
+   * If the storage is of type PROVIDED, one of the nodes that reported
+   * PROVIDED storage are returned. If not, this is equivalent to
+   * {@code storage.getDatanodeDescriptor()}.
+   * @param storage
+   * @return the associated {@link DatanodeDescriptor}.
+   */
+  private DatanodeDescriptor getDatanodeDescriptorFromStorage(
+      DatanodeStorageInfo storage) {
+    if (storage.getStorageType() == StorageType.PROVIDED) {
+      return providedStorageMap.chooseProvidedDatanode();
+    }
+    return storage.getDatanodeDescriptor();
+  }
+
   /**
    * Parse the data-nodes the block belongs to and choose a certain number
    * from them to be the recovery sources.
@@ -2178,10 +2213,14 @@ public class BlockManager implements BlockStatsMXBean {
     BitSet bitSet = isStriped ?
         new BitSet(((BlockInfoStriped) block).getTotalBlockNum()) : null;
     for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
-      final DatanodeDescriptor node = storage.getDatanodeDescriptor();
+      final DatanodeDescriptor node = getDatanodeDescriptorFromStorage(storage);
       final StoredReplicaState state = checkReplicaOnStorage(numReplicas, block,
           storage, corruptReplicas.getNodes(block), false);
       if (state == StoredReplicaState.LIVE) {
+        if (storage.getStorageType() == StorageType.PROVIDED) {
+          storage = new DatanodeStorageInfo(node, storage.getStorageID(),
+              storage.getStorageType(), storage.getState());
+        }
         nodesContainingLiveReplicas.add(storage);
       }
       containingNodes.add(node);
@@ -2384,6 +2423,21 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
 
+  public long getProvidedCapacity() {
+    return providedStorageMap.getCapacity();
+  }
+
+  public void updateHeartbeat(DatanodeDescriptor node, StorageReport[] reports,
+      long cacheCapacity, long cacheUsed, int xceiverCount, int failedVolumes,
+      VolumeFailureSummary volumeFailureSummary) {
+
+    for (StorageReport report: reports) {
+      providedStorageMap.updateStorage(node, report.getStorage());
+    }
+    node.updateHeartbeat(reports, cacheCapacity, cacheUsed, xceiverCount,
+        failedVolumes, volumeFailureSummary);
+  }
+
   /**
    * StatefulBlockInfo is used to build the "toUC" list, which is a list of
    * updates to the information about under-construction blocks.
@@ -2442,7 +2496,10 @@ public class BlockManager implements BlockStatsMXBean {
 
       // To minimize startup time, we discard any second (or later) block reports
       // that we receive while still in startup phase.
-      DatanodeStorageInfo storageInfo = node.getStorageInfo(storage.getStorageID());
+      // Register DN with provided storage, not with storage owned by DN
+      // DN should still have a ref to the DNStorageInfo.
+      DatanodeStorageInfo storageInfo =
+          providedStorageMap.getStorage(node, storage);
 
       if (storageInfo == null) {
         // We handle this for backwards compatibility.
@@ -2474,9 +2531,12 @@ public class BlockManager implements BlockStatsMXBean {
             nodeID.getDatanodeUuid());
         processFirstBlockReport(storageInfo, newReport);
       } else {
-        invalidatedBlocks = processReport(storageInfo, newReport, context);
+        // Block reports for provided storage are not
+        // maintained by DN heartbeats
+        if (!StorageType.PROVIDED.equals(storageInfo.getStorageType())) {
+          invalidatedBlocks = processReport(storageInfo, newReport, context);
+        }
       }
-      
       storageInfo.receivedBlockReport();
     } finally {
       endTime = Time.monotonicNow();
@@ -2566,7 +2626,7 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
   
-  private Collection<Block> processReport(
+  Collection<Block> processReport(
       final DatanodeStorageInfo storageInfo,
       final BlockListAsLongs report,
       BlockReportContext context) throws IOException {
@@ -2690,7 +2750,7 @@ public class BlockManager implements BlockStatsMXBean {
    * @param report - the initial block report, to be processed
    * @throws IOException 
    */
-  private void processFirstBlockReport(
+  void processFirstBlockReport(
       final DatanodeStorageInfo storageInfo,
       final BlockListAsLongs report) throws IOException {
     if (report == null) return;
@@ -4297,7 +4357,13 @@ public class BlockManager implements BlockStatsMXBean {
     Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
         .getNodes(storedBlock);
     for (DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) {
-      final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
+      if (storage.getStorageType() == StorageType.PROVIDED
+          && storage.getState() == State.NORMAL) {
+        // assume the policy is satisfied for blocks on PROVIDED storage
+        // as long as the storage is in normal state.
+        return true;
+      }
+      final DatanodeDescriptor cur = getDatanodeDescriptorFromStorage(storage);
       // Nodes under maintenance should be counted as valid replicas from
       // rack policy point of view.
       if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()
@@ -4902,4 +4968,9 @@ public class BlockManager implements BlockStatsMXBean {
   public void setBlockRecoveryTimeout(long blockRecoveryTimeout) {
     pendingRecoveryBlocks.setRecoveryTimeoutInterval(blockRecoveryTimeout);
   }
+
+  @VisibleForTesting
+  public ProvidedStorageMap getProvidedStorageMap() {
+    return providedStorageMap;
+  }
 }

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java

@@ -82,6 +82,12 @@ public class BlockStoragePolicySuite {
         HdfsConstants.COLD_STORAGE_POLICY_NAME,
         new StorageType[]{StorageType.ARCHIVE}, StorageType.EMPTY_ARRAY,
         StorageType.EMPTY_ARRAY);
+    final byte providedId = HdfsConstants.PROVIDED_STORAGE_POLICY_ID;
+    policies[providedId] = new BlockStoragePolicy(providedId,
+      HdfsConstants.PROVIDED_STORAGE_POLICY_NAME,
+      new StorageType[]{StorageType.PROVIDED, StorageType.DISK},
+      new StorageType[]{StorageType.PROVIDED, StorageType.DISK},
+      new StorageType[]{StorageType.PROVIDED, StorageType.DISK});
     return new BlockStoragePolicySuite(hotId, policies);
   }
 

+ 41 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -151,7 +151,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private final LeavingServiceStatus leavingServiceStatus =
       new LeavingServiceStatus();
 
-  private final Map<String, DatanodeStorageInfo> storageMap =
+  protected final Map<String, DatanodeStorageInfo> storageMap =
       new HashMap<>();
 
   /**
@@ -322,6 +322,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
   boolean hasStaleStorages() {
     synchronized (storageMap) {
       for (DatanodeStorageInfo storage : storageMap.values()) {
+        if (StorageType.PROVIDED.equals(storage.getStorageType())) {
+          // to verify provided storage participated in this hb, requires
+          // check to pass DNDesc.
+          // e.g., storageInfo.verifyBlockReportId(this, curBlockReportId)
+          continue;
+        }
         if (storage.areBlockContentsStale()) {
           return true;
         }
@@ -443,12 +449,19 @@ public class DatanodeDescriptor extends DatanodeInfo {
     this.volumeFailures = volFailures;
     this.volumeFailureSummary = volumeFailureSummary;
     for (StorageReport report : reports) {
-      DatanodeStorageInfo storage = updateStorage(report.getStorage());
+
+      DatanodeStorageInfo storage =
+          storageMap.get(report.getStorage().getStorageID());
       if (checkFailedStorages) {
         failedStorageInfos.remove(storage);
       }
 
       storage.receivedHeartbeat(report);
+      // skip accounting for capacity of PROVIDED storages!
+      if (StorageType.PROVIDED.equals(storage.getStorageType())) {
+        continue;
+      }
+
       totalCapacity += report.getCapacity();
       totalRemaining += report.getRemaining();
       totalBlockPoolUsed += report.getBlockPoolUsed();
@@ -474,6 +487,29 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
+  void injectStorage(DatanodeStorageInfo s) {
+    synchronized (storageMap) {
+      DatanodeStorageInfo storage = storageMap.get(s.getStorageID());
+      if (null == storage) {
+        LOG.info("Adding new storage ID {} for DN {}", s.getStorageID(),
+            getXferAddr());
+        DFSTopologyNodeImpl parent = null;
+        if (getParent() instanceof DFSTopologyNodeImpl) {
+          parent = (DFSTopologyNodeImpl) getParent();
+        }
+        StorageType type = s.getStorageType();
+        if (!hasStorageType(type) && parent != null) {
+          // we are about to add a type this node currently does not have,
+          // inform the parent that a new type is added to this datanode
+          parent.childAddStorage(getName(), type);
+        }
+        storageMap.put(s.getStorageID(), s);
+      } else {
+        assert storage == s : "found " + storage + " expected " + s;
+      }
+    }
+  }
+
   /**
    * Remove stale storages from storageMap. We must not remove any storages
    * as long as they have associated block replicas.
@@ -883,7 +919,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
     
     // must re-process IBR after re-registration
     for(DatanodeStorageInfo storage : getStorageInfos()) {
-      storage.setBlockReportCount(0);
+      if (storage.getStorageType() != StorageType.PROVIDED) {
+        storage.setBlockReportCount(0);
+      }
     }
     heartbeatedSinceRegistration = false;
     forceRegistration = false;

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -532,6 +532,8 @@ public class DatanodeManager {
     } else {
       networktopology.sortByDistance(client, lb.getLocations(), activeLen);
     }
+    // move PROVIDED storage to the end to prefer local replicas.
+    lb.moveProvidedToEnd(activeLen);
     // must update cache since we modified locations array
     lb.updateCachedStorageInfo();
   }

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java

@@ -77,4 +77,7 @@ public interface DatanodeStatistics {
 
   /** @return Storage Tier statistics*/
   Map<StorageType, StorageTypeStats> getStorageTypeStats();
+
+  /** @return the provided capacity */
+  public long getProvidedCapacity();
 }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStats.java

@@ -183,7 +183,7 @@ class DatanodeStats {
       StorageTypeStats storageTypeStats =
           storageTypeStatsMap.get(storageType);
       if (storageTypeStats == null) {
-        storageTypeStats = new StorageTypeStats();
+        storageTypeStats = new StorageTypeStats(storageType);
         storageTypeStatsMap.put(storageType, storageTypeStats);
       }
       storageTypeStats.addNode(node);
@@ -194,7 +194,7 @@ class DatanodeStats {
       StorageTypeStats storageTypeStats =
           storageTypeStatsMap.get(info.getStorageType());
       if (storageTypeStats == null) {
-        storageTypeStats = new StorageTypeStats();
+        storageTypeStats = new StorageTypeStats(info.getStorageType());
         storageTypeStatsMap.put(info.getStorageType(), storageTypeStats);
       }
       storageTypeStats.addStorage(info, node);

+ 12 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java

@@ -120,10 +120,15 @@ public class DatanodeStorageInfo {
   private boolean blockContentsStale = true;
 
   DatanodeStorageInfo(DatanodeDescriptor dn, DatanodeStorage s) {
+    this(dn, s.getStorageID(), s.getStorageType(), s.getState());
+  }
+
+  DatanodeStorageInfo(DatanodeDescriptor dn, String storageID,
+      StorageType storageType, State state) {
     this.dn = dn;
-    this.storageID = s.getStorageID();
-    this.storageType = s.getStorageType();
-    this.state = s.getState();
+    this.storageID = storageID;
+    this.storageType = storageType;
+    this.state = state;
   }
 
   public int getBlockReportCount() {
@@ -172,6 +177,10 @@ public class DatanodeStorageInfo {
     this.state = state;
   }
 
+  void setHeartbeatedSinceFailover(boolean value) {
+    heartbeatedSinceFailover = value;
+  }
+
   boolean areBlocksOnFailedStorage() {
     return getState() == State.FAILED && !blocks.isEmpty();
   }

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java

@@ -195,6 +195,11 @@ class HeartbeatManager implements DatanodeStatistics {
     return stats.getStatsMap();
   }
 
+  @Override
+  public long getProvidedCapacity() {
+    return blockManager.getProvidedCapacity();
+  }
+
   synchronized void register(final DatanodeDescriptor d) {
     if (!d.isAlive()) {
       addDatanode(d);
@@ -232,8 +237,8 @@ class HeartbeatManager implements DatanodeStatistics {
       int xceiverCount, int failedVolumes,
       VolumeFailureSummary volumeFailureSummary) {
     stats.subtract(node);
-    node.updateHeartbeat(reports, cacheCapacity, cacheUsed,
-      xceiverCount, failedVolumes, volumeFailureSummary);
+    blockManager.updateHeartbeat(node, reports, cacheCapacity, cacheUsed,
+        xceiverCount, failedVolumes, volumeFailureSummary);
     stats.add(node);
   }
 

+ 109 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LocatedBlockBuilder.java

@@ -0,0 +1,109 @@
+/**
+ * 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.server.blockmanagement;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+class LocatedBlockBuilder {
+
+  protected long flen;
+  protected List<LocatedBlock> blocks = Collections.<LocatedBlock>emptyList();
+  protected boolean isUC;
+  protected LocatedBlock last;
+  protected boolean lastComplete;
+  protected FileEncryptionInfo feInfo;
+  private final int maxBlocks;
+  protected ErasureCodingPolicy ecPolicy;
+
+  LocatedBlockBuilder(int maxBlocks) {
+    this.maxBlocks = maxBlocks;
+  }
+
+  boolean isBlockMax() {
+    return blocks.size() >= maxBlocks;
+  }
+
+  LocatedBlockBuilder fileLength(long fileLength) {
+    flen = fileLength;
+    return this;
+  }
+
+  LocatedBlockBuilder addBlock(LocatedBlock block) {
+    if (blocks.isEmpty()) {
+      blocks = new ArrayList<>();
+    }
+    blocks.add(block);
+    return this;
+  }
+
+  // return new block so tokens can be set
+  LocatedBlock newLocatedBlock(ExtendedBlock eb,
+      DatanodeStorageInfo[] storage,
+      long pos, boolean isCorrupt) {
+    LocatedBlock blk =
+        BlockManager.newLocatedBlock(eb, storage, pos, isCorrupt);
+    return blk;
+  }
+
+  LocatedBlockBuilder lastUC(boolean underConstruction) {
+    isUC = underConstruction;
+    return this;
+  }
+
+  LocatedBlockBuilder lastBlock(LocatedBlock block) {
+    last = block;
+    return this;
+  }
+
+  LocatedBlockBuilder lastComplete(boolean complete) {
+    lastComplete = complete;
+    return this;
+  }
+
+  LocatedBlockBuilder encryption(FileEncryptionInfo fileEncryptionInfo) {
+    feInfo = fileEncryptionInfo;
+    return this;
+  }
+
+  LocatedBlockBuilder erasureCoding(ErasureCodingPolicy codingPolicy) {
+    ecPolicy = codingPolicy;
+    return this;
+  }
+
+  LocatedBlocks build(DatanodeDescriptor client) {
+    return build();
+  }
+
+  LocatedBlocks build() {
+    return new LocatedBlocks(flen, isUC, blocks, last,
+        lastComplete, feInfo, ecPolicy);
+  }
+
+}

+ 540 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java

@@ -0,0 +1,540 @@
+/**
+ * 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.server.blockmanagement;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap;
+import org.apache.hadoop.hdfs.server.common.BlockAlias;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
+import org.apache.hadoop.hdfs.util.RwLock;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.protobuf.ByteString;
+
+/**
+ * This class allows us to manage and multiplex between storages local to
+ * datanodes, and provided storage.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ProvidedStorageMap {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ProvidedStorageMap.class);
+
+  // limit to a single provider for now
+  private RwLock lock;
+  private BlockManager bm;
+  private BlockAliasMap aliasMap;
+
+  private final String storageId;
+  private final ProvidedDescriptor providedDescriptor;
+  private final DatanodeStorageInfo providedStorageInfo;
+  private boolean providedEnabled;
+  private long capacity;
+  private int defaultReplication;
+
+  ProvidedStorageMap(RwLock lock, BlockManager bm, Configuration conf)
+      throws IOException {
+
+    storageId = conf.get(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID,
+        DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT);
+
+    providedEnabled = conf.getBoolean(
+        DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED,
+        DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED_DEFAULT);
+
+    if (!providedEnabled) {
+      // disable mapping
+      aliasMap = null;
+      providedDescriptor = null;
+      providedStorageInfo = null;
+      return;
+    }
+
+    DatanodeStorage ds = new DatanodeStorage(
+        storageId, State.NORMAL, StorageType.PROVIDED);
+    providedDescriptor = new ProvidedDescriptor();
+    providedStorageInfo = providedDescriptor.createProvidedStorage(ds);
+    this.defaultReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
+        DFSConfigKeys.DFS_REPLICATION_DEFAULT);
+
+    this.bm = bm;
+    this.lock = lock;
+
+    // load block reader into storage
+    Class<? extends BlockAliasMap> aliasMapClass = conf.getClass(
+            DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+            TextFileRegionAliasMap.class, BlockAliasMap.class);
+    aliasMap = ReflectionUtils.newInstance(aliasMapClass, conf);
+
+    LOG.info("Loaded alias map class: " +
+        aliasMap.getClass() + " storage: " + providedStorageInfo);
+  }
+
+  /**
+   * @param dn datanode descriptor
+   * @param s data node storage
+   * @return the {@link DatanodeStorageInfo} for the specified datanode.
+   * If {@code s} corresponds to a provided storage, the storage info
+   * representing provided storage is returned.
+   * @throws IOException
+   */
+  DatanodeStorageInfo getStorage(DatanodeDescriptor dn, DatanodeStorage s)
+      throws IOException {
+    if (providedEnabled && storageId.equals(s.getStorageID())) {
+      if (StorageType.PROVIDED.equals(s.getStorageType())) {
+        if (providedStorageInfo.getState() == State.FAILED
+            && s.getState() == State.NORMAL) {
+          providedStorageInfo.setState(State.NORMAL);
+          LOG.info("Provided storage transitioning to state " + State.NORMAL);
+        }
+        if (dn.getStorageInfo(s.getStorageID()) == null) {
+          dn.injectStorage(providedStorageInfo);
+        }
+        processProvidedStorageReport();
+        return providedDescriptor.getProvidedStorage(dn, s);
+      }
+      LOG.warn("Reserved storage {} reported as non-provided from {}", s, dn);
+    }
+    return dn.getStorageInfo(s.getStorageID());
+  }
+
+  private void processProvidedStorageReport()
+      throws IOException {
+    assert lock.hasWriteLock() : "Not holding write lock";
+    if (providedStorageInfo.getBlockReportCount() == 0
+        || providedDescriptor.activeProvidedDatanodes() == 0) {
+      LOG.info("Calling process first blk report from storage: "
+          + providedStorageInfo);
+      // first pass; periodic refresh should call bm.processReport
+      BlockAliasMap.Reader<BlockAlias> reader =
+          aliasMap.getReader(null, bm.getBlockPoolId());
+      if (reader != null) {
+        bm.processFirstBlockReport(providedStorageInfo,
+                new ProvidedBlockList(reader.iterator()));
+      }
+    }
+  }
+
+  @VisibleForTesting
+  public DatanodeStorageInfo getProvidedStorageInfo() {
+    return providedStorageInfo;
+  }
+
+  public LocatedBlockBuilder newLocatedBlocks(int maxValue) {
+    if (!providedEnabled) {
+      return new LocatedBlockBuilder(maxValue);
+    }
+    return new ProvidedBlocksBuilder(maxValue);
+  }
+
+  public void removeDatanode(DatanodeDescriptor dnToRemove) {
+    if (providedEnabled) {
+      assert lock.hasWriteLock() : "Not holding write lock";
+      providedDescriptor.remove(dnToRemove);
+      // if all datanodes fail, set the block report count to 0
+      if (providedDescriptor.activeProvidedDatanodes() == 0) {
+        providedStorageInfo.setBlockReportCount(0);
+      }
+    }
+  }
+
+  public long getCapacity() {
+    if (providedStorageInfo == null) {
+      return 0;
+    }
+    return providedStorageInfo.getCapacity();
+  }
+
+  public void updateStorage(DatanodeDescriptor node, DatanodeStorage storage) {
+    if (isProvidedStorage(storage.getStorageID())) {
+      if (StorageType.PROVIDED.equals(storage.getStorageType())) {
+        node.injectStorage(providedStorageInfo);
+        return;
+      } else {
+        LOG.warn("Reserved storage {} reported as non-provided from {}",
+            storage, node);
+      }
+    }
+    node.updateStorage(storage);
+  }
+
+  private boolean isProvidedStorage(String dnStorageId) {
+    return providedEnabled && storageId.equals(dnStorageId);
+  }
+
+  /**
+   * Choose a datanode that reported a volume of {@link StorageType} PROVIDED.
+   *
+   * @return the {@link DatanodeDescriptor} corresponding to a datanode that
+   *         reported a volume with {@link StorageType} PROVIDED. If multiple
+   *         datanodes report a PROVIDED volume, one is chosen uniformly at
+   *         random.
+   */
+  public DatanodeDescriptor chooseProvidedDatanode() {
+    return providedDescriptor.chooseRandom();
+  }
+
+  /**
+   * Builder used for creating {@link LocatedBlocks} when a block is provided.
+   */
+  class ProvidedBlocksBuilder extends LocatedBlockBuilder {
+
+    ProvidedBlocksBuilder(int maxBlocks) {
+      super(maxBlocks);
+    }
+
+    private DatanodeDescriptor chooseProvidedDatanode(
+        Set<String> excludedUUids) {
+      DatanodeDescriptor dn = providedDescriptor.choose(null, excludedUUids);
+      if (dn == null) {
+        dn = providedDescriptor.choose(null);
+      }
+      return dn;
+    }
+
+    @Override
+    LocatedBlock newLocatedBlock(ExtendedBlock eb,
+        DatanodeStorageInfo[] storages, long pos, boolean isCorrupt) {
+
+      List<DatanodeInfoWithStorage> locs = new ArrayList<>();
+      List<String> sids = new ArrayList<>();
+      List<StorageType> types = new ArrayList<>();
+      boolean isProvidedBlock = false;
+      Set<String> excludedUUids = new HashSet<>();
+
+      for (int i = 0; i < storages.length; ++i) {
+        DatanodeStorageInfo currInfo = storages[i];
+        StorageType storageType = currInfo.getStorageType();
+        sids.add(currInfo.getStorageID());
+        types.add(storageType);
+        if (StorageType.PROVIDED.equals(storageType)) {
+          // Provided location will be added to the list of locations after
+          // examining all local locations.
+          isProvidedBlock = true;
+        } else {
+          locs.add(new DatanodeInfoWithStorage(
+              currInfo.getDatanodeDescriptor(),
+              currInfo.getStorageID(), storageType));
+          excludedUUids.add(currInfo.getDatanodeDescriptor().getDatanodeUuid());
+        }
+      }
+
+      int numLocations = locs.size();
+      if (isProvidedBlock) {
+        // add the first datanode here
+        DatanodeDescriptor dn = chooseProvidedDatanode(excludedUUids);
+        locs.add(
+            new DatanodeInfoWithStorage(dn, storageId, StorageType.PROVIDED));
+        excludedUUids.add(dn.getDatanodeUuid());
+        numLocations++;
+        // add more replicas until we reach the defaultReplication
+        for (int count = numLocations + 1;
+            count <= defaultReplication && count <= providedDescriptor
+                .activeProvidedDatanodes(); count++) {
+          dn = chooseProvidedDatanode(excludedUUids);
+          locs.add(new DatanodeInfoWithStorage(
+              dn, storageId, StorageType.PROVIDED));
+          sids.add(storageId);
+          types.add(StorageType.PROVIDED);
+          excludedUUids.add(dn.getDatanodeUuid());
+        }
+      }
+      return new LocatedBlock(eb,
+          locs.toArray(new DatanodeInfoWithStorage[locs.size()]),
+          sids.toArray(new String[sids.size()]),
+          types.toArray(new StorageType[types.size()]),
+          pos, isCorrupt, null);
+    }
+
+    @Override
+    LocatedBlocks build(DatanodeDescriptor client) {
+      // TODO choose provided locations close to the client.
+      return new LocatedBlocks(
+          flen, isUC, blocks, last, lastComplete, feInfo, ecPolicy);
+    }
+
+    @Override
+    LocatedBlocks build() {
+      return build(providedDescriptor.chooseRandom());
+    }
+  }
+
+  /**
+   * An abstract DatanodeDescriptor to track datanodes with provided storages.
+   * NOTE: never resolved through registerDatanode, so not in the topology.
+   */
+  public static class ProvidedDescriptor extends DatanodeDescriptor {
+
+    private final NavigableMap<String, DatanodeDescriptor> dns =
+        new ConcurrentSkipListMap<>();
+    // maintain a separate list of the datanodes with provided storage
+    // to efficiently choose Datanodes when required.
+    private final List<DatanodeDescriptor> dnR = new ArrayList<>();
+    public final static String NETWORK_LOCATION = "/REMOTE";
+    public final static String NAME = "PROVIDED";
+
+    ProvidedDescriptor() {
+      super(new DatanodeID(
+            null,                         // String ipAddr,
+            null,                         // String hostName,
+            UUID.randomUUID().toString(), // String datanodeUuid,
+            0,                            // int xferPort,
+            0,                            // int infoPort,
+            0,                            // int infoSecurePort,
+            0));                          // int ipcPort
+    }
+
+    DatanodeStorageInfo getProvidedStorage(
+        DatanodeDescriptor dn, DatanodeStorage s) {
+      dns.put(dn.getDatanodeUuid(), dn);
+      dnR.add(dn);
+      return storageMap.get(s.getStorageID());
+    }
+
+    DatanodeStorageInfo createProvidedStorage(DatanodeStorage ds) {
+      assert null == storageMap.get(ds.getStorageID());
+      DatanodeStorageInfo storage = new ProvidedDatanodeStorageInfo(this, ds);
+      storage.setHeartbeatedSinceFailover(true);
+      storageMap.put(storage.getStorageID(), storage);
+      return storage;
+    }
+
+    DatanodeDescriptor choose(DatanodeDescriptor client) {
+      return choose(client, Collections.<String>emptySet());
+    }
+
+    DatanodeDescriptor choose(DatanodeDescriptor client,
+        Set<String> excludedUUids) {
+      // exact match for now
+      if (client != null && !excludedUUids.contains(client.getDatanodeUuid())) {
+        DatanodeDescriptor dn = dns.get(client.getDatanodeUuid());
+        if (dn != null) {
+          return dn;
+        }
+      }
+      // prefer live nodes first.
+      DatanodeDescriptor dn = chooseRandomNode(excludedUUids, true);
+      if (dn == null) {
+        dn = chooseRandomNode(excludedUUids, false);
+      }
+      return dn;
+    }
+
+    private DatanodeDescriptor chooseRandomNode(Set<String> excludedUUids,
+        boolean preferLiveNodes) {
+      Random r = new Random();
+      for (int i = dnR.size() - 1; i >= 0; --i) {
+        int pos = r.nextInt(i + 1);
+        DatanodeDescriptor node = dnR.get(pos);
+        String uuid = node.getDatanodeUuid();
+        if (!excludedUUids.contains(uuid)) {
+          if (!preferLiveNodes || node.getAdminState() == AdminStates.NORMAL) {
+            return node;
+          }
+        }
+        Collections.swap(dnR, i, pos);
+      }
+      return null;
+    }
+
+    DatanodeDescriptor chooseRandom(DatanodeStorageInfo... excludedStorages) {
+      Set<String> excludedNodes = new HashSet<>();
+      if (excludedStorages != null) {
+        for (int i = 0; i < excludedStorages.length; i++) {
+          DatanodeDescriptor dn = excludedStorages[i].getDatanodeDescriptor();
+          String uuid = dn.getDatanodeUuid();
+          excludedNodes.add(uuid);
+        }
+      }
+      return choose(null, excludedNodes);
+    }
+
+    @Override
+    void addBlockToBeReplicated(Block block, DatanodeStorageInfo[] targets) {
+      // pick a random datanode, delegate to it
+      DatanodeDescriptor node = chooseRandom(targets);
+      if (node != null) {
+        node.addBlockToBeReplicated(block, targets);
+      } else {
+        LOG.error("Cannot find a source node to replicate block: "
+            + block + " from");
+      }
+    }
+
+    int remove(DatanodeDescriptor dnToRemove) {
+      // this operation happens under the FSNamesystem lock;
+      // no additional synchronization required.
+      if (dnToRemove != null) {
+        DatanodeDescriptor storedDN = dns.get(dnToRemove.getDatanodeUuid());
+        if (storedDN != null) {
+          dns.remove(dnToRemove.getDatanodeUuid());
+          dnR.remove(dnToRemove);
+        }
+      }
+      return dns.size();
+    }
+
+    int activeProvidedDatanodes() {
+      return dns.size();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return (this == obj) || super.equals(obj);
+    }
+
+    @Override
+    public int hashCode() {
+      return super.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return "PROVIDED-LOCATION";
+    }
+
+    @Override
+    public String getNetworkLocation() {
+      return NETWORK_LOCATION;
+    }
+
+    @Override
+    public String getName() {
+      return NAME;
+    }
+  }
+
+  /**
+   * The DatanodeStorageInfo used for the provided storage.
+   */
+  static class ProvidedDatanodeStorageInfo extends DatanodeStorageInfo {
+
+    ProvidedDatanodeStorageInfo(ProvidedDescriptor dn, DatanodeStorage ds) {
+      super(dn, ds);
+    }
+
+    @Override
+    boolean removeBlock(BlockInfo b) {
+      ProvidedDescriptor dn = (ProvidedDescriptor) getDatanodeDescriptor();
+      if (dn.activeProvidedDatanodes() == 0) {
+        return super.removeBlock(b);
+      } else {
+        return false;
+      }
+    }
+
+    @Override
+    void setState(DatanodeStorage.State state) {
+      if (state == State.FAILED) {
+        // The state should change to FAILED only when there are no active
+        // datanodes with PROVIDED storage.
+        ProvidedDescriptor dn = (ProvidedDescriptor) getDatanodeDescriptor();
+        if (dn.activeProvidedDatanodes() == 0) {
+          LOG.info("Provided storage {} transitioning to state {}",
+              this, State.FAILED);
+          super.setState(state);
+        }
+      } else {
+        super.setState(state);
+      }
+    }
+
+    @Override
+    public String toString() {
+      return "PROVIDED-STORAGE";
+    }
+  }
+
+  /**
+   * Used to emulate block reports for provided blocks.
+   */
+  static class ProvidedBlockList extends BlockListAsLongs {
+
+    private final Iterator<BlockAlias> inner;
+
+    ProvidedBlockList(Iterator<BlockAlias> inner) {
+      this.inner = inner;
+    }
+
+    @Override
+    public Iterator<BlockReportReplica> iterator() {
+      return new Iterator<BlockReportReplica>() {
+        @Override
+        public BlockReportReplica next() {
+          return new BlockReportReplica(inner.next().getBlock());
+        }
+        @Override
+        public boolean hasNext() {
+          return inner.hasNext();
+        }
+        @Override
+        public void remove() {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
+
+    @Override
+    public int getNumberOfBlocks() {
+      // is ignored for ProvidedBlockList.
+      return -1;
+    }
+
+    @Override
+    public ByteString getBlocksBuffer() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long[] getBlockListAsLongs() {
+      // should only be used for backwards compat, DN.ver > NN.ver
+      throw new UnsupportedOperationException();
+    }
+  }
+}

+ 32 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/StorageTypeStats.java

@@ -22,6 +22,7 @@ import java.beans.ConstructorProperties;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.StorageType;
 
 /**
  * Statistics per StorageType.
@@ -36,6 +37,7 @@ public class StorageTypeStats {
   private long capacityRemaining = 0L;
   private long blockPoolUsed = 0L;
   private int nodesInService = 0;
+  private StorageType storageType;
 
   @ConstructorProperties({"capacityTotal", "capacityUsed", "capacityNonDfsUsed",
       "capacityRemaining", "blockPoolUsed", "nodesInService"})
@@ -51,22 +53,47 @@ public class StorageTypeStats {
   }
 
   public long getCapacityTotal() {
+    // for PROVIDED storage, avoid counting the same storage
+    // across multiple datanodes
+    if (storageType == StorageType.PROVIDED && nodesInService > 0) {
+      return capacityTotal/nodesInService;
+    }
     return capacityTotal;
   }
 
   public long getCapacityUsed() {
+    // for PROVIDED storage, avoid counting the same storage
+    // across multiple datanodes
+    if (storageType == StorageType.PROVIDED && nodesInService > 0) {
+      return capacityUsed/nodesInService;
+    }
     return capacityUsed;
   }
 
   public long getCapacityNonDfsUsed() {
+    // for PROVIDED storage, avoid counting the same storage
+    // across multiple datanodes
+    if (storageType == StorageType.PROVIDED && nodesInService > 0) {
+      return capacityNonDfsUsed/nodesInService;
+    }
     return capacityNonDfsUsed;
   }
 
   public long getCapacityRemaining() {
+    // for PROVIDED storage, avoid counting the same storage
+    // across multiple datanodes
+    if (storageType == StorageType.PROVIDED && nodesInService > 0) {
+      return capacityRemaining/nodesInService;
+    }
     return capacityRemaining;
   }
 
   public long getBlockPoolUsed() {
+    // for PROVIDED storage, avoid counting the same storage
+    // across multiple datanodes
+    if (storageType == StorageType.PROVIDED && nodesInService > 0) {
+      return blockPoolUsed/nodesInService;
+    }
     return blockPoolUsed;
   }
 
@@ -74,7 +101,9 @@ public class StorageTypeStats {
     return nodesInService;
   }
 
-  StorageTypeStats() {}
+  StorageTypeStats(StorageType storageType) {
+    this.storageType = storageType;
+  }
 
   StorageTypeStats(StorageTypeStats other) {
     capacityTotal = other.capacityTotal;
@@ -87,6 +116,7 @@ public class StorageTypeStats {
 
   void addStorage(final DatanodeStorageInfo info,
       final DatanodeDescriptor node) {
+    assert storageType == info.getStorageType();
     capacityUsed += info.getDfsUsed();
     capacityNonDfsUsed += info.getNonDfsUsed();
     blockPoolUsed += info.getBlockPoolUsed();
@@ -106,6 +136,7 @@ public class StorageTypeStats {
 
   void subtractStorage(final DatanodeStorageInfo info,
       final DatanodeDescriptor node) {
+    assert storageType == info.getStorageType();
     capacityUsed -= info.getDfsUsed();
     capacityNonDfsUsed -= info.getNonDfsUsed();
     blockPoolUsed -= info.getBlockPoolUsed();

+ 33 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockAlias.java

@@ -0,0 +1,33 @@
+/**
+ * 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.server.common;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.Block;
+
+/**
+ * Interface used to load provided blocks.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface BlockAlias {
+
+  Block getBlock();
+
+}

+ 85 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java

@@ -0,0 +1,85 @@
+/**
+ * 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.server.common;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+
+/**
+ * This class is used to represent provided blocks that are file regions,
+ * i.e., can be described using (path, offset, length).
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class FileRegion implements BlockAlias {
+
+  private final Pair<Block, ProvidedStorageLocation> pair;
+
+  public FileRegion(long blockId, Path path, long offset,
+      long length, long genStamp) {
+    this(blockId, path, offset, length, genStamp, new byte[0]);
+  }
+
+  public FileRegion(long blockId, Path path, long offset,
+                    long length, long genStamp, byte[] nonce) {
+    this(new Block(blockId, length, genStamp),
+            new ProvidedStorageLocation(path, offset, length, nonce));
+  }
+
+  public FileRegion(long blockId, Path path, long offset, long length) {
+    this(blockId, path, offset, length,
+        HdfsConstants.GRANDFATHER_GENERATION_STAMP);
+  }
+
+  public FileRegion(Block block,
+      ProvidedStorageLocation providedStorageLocation) {
+    this.pair  = Pair.of(block, providedStorageLocation);
+  }
+
+  public Block getBlock() {
+    return pair.getKey();
+  }
+
+  public ProvidedStorageLocation getProvidedStorageLocation() {
+    return pair.getValue();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    FileRegion that = (FileRegion) o;
+
+    return pair.equals(that.pair);
+  }
+
+  @Override
+  public int hashCode() {
+    return pair.hashCode();
+  }
+}

+ 66 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -40,6 +40,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
@@ -196,7 +197,10 @@ public abstract class Storage extends StorageInfo {
     Iterator<StorageDirectory> it =
       (dirType == null) ? dirIterator() : dirIterator(dirType);
     for ( ;it.hasNext(); ) {
-      list.add(new File(it.next().getCurrentDir(), fileName));
+      File currentDir = it.next().getCurrentDir();
+      if (currentDir != null) {
+        list.add(new File(currentDir, fileName));
+      }
     }
     return list;
   }
@@ -328,10 +332,20 @@ public abstract class Storage extends StorageInfo {
      */
     public StorageDirectory(String bpid, StorageDirType dirType,
         boolean isShared, StorageLocation location) {
-      this(new File(location.getBpURI(bpid, STORAGE_DIR_CURRENT)), dirType,
+      this(getBlockPoolCurrentDir(bpid, location), dirType,
           isShared, location);
     }
 
+    private static File getBlockPoolCurrentDir(String bpid,
+        StorageLocation location) {
+      if (location == null ||
+          location.getStorageType() == StorageType.PROVIDED) {
+        return null;
+      } else {
+        return new File(location.getBpURI(bpid, STORAGE_DIR_CURRENT));
+      }
+    }
+
     private StorageDirectory(File dir, StorageDirType dirType,
         boolean isShared, StorageLocation location) {
       this.root = dir;
@@ -347,7 +361,8 @@ public abstract class Storage extends StorageInfo {
     }
 
     private static File getStorageLocationFile(StorageLocation location) {
-      if (location == null) {
+      if (location == null ||
+          location.getStorageType() == StorageType.PROVIDED) {
         return null;
       }
       try {
@@ -406,6 +421,10 @@ public abstract class Storage extends StorageInfo {
      */
     public void clearDirectory() throws IOException {
       File curDir = this.getCurrentDir();
+      if (curDir == null) {
+        // if the directory is null, there is nothing to do.
+        return;
+      }
       if (curDir.exists()) {
         File[] files = FileUtil.listFiles(curDir);
         LOG.info("Will remove files: " + Arrays.toString(files));
@@ -423,6 +442,9 @@ public abstract class Storage extends StorageInfo {
      * @return the directory path
      */
     public File getCurrentDir() {
+      if (root == null) {
+        return null;
+      }
       return new File(root, STORAGE_DIR_CURRENT);
     }
 
@@ -443,6 +465,9 @@ public abstract class Storage extends StorageInfo {
      * @return the version file path
      */
     public File getVersionFile() {
+      if (root == null) {
+        return null;
+      }
       return new File(new File(root, STORAGE_DIR_CURRENT), STORAGE_FILE_VERSION);
     }
 
@@ -452,6 +477,9 @@ public abstract class Storage extends StorageInfo {
      * @return the previous version file path
      */
     public File getPreviousVersionFile() {
+      if (root == null) {
+        return null;
+      }
       return new File(new File(root, STORAGE_DIR_PREVIOUS), STORAGE_FILE_VERSION);
     }
 
@@ -462,6 +490,9 @@ public abstract class Storage extends StorageInfo {
      * @return the directory path
      */
     public File getPreviousDir() {
+      if (root == null) {
+        return null;
+      }
       return new File(root, STORAGE_DIR_PREVIOUS);
     }
 
@@ -476,6 +507,9 @@ public abstract class Storage extends StorageInfo {
      * @return the directory path
      */
     public File getPreviousTmp() {
+      if (root == null) {
+        return null;
+      }
       return new File(root, STORAGE_TMP_PREVIOUS);
     }
 
@@ -490,6 +524,9 @@ public abstract class Storage extends StorageInfo {
      * @return the directory path
      */
     public File getRemovedTmp() {
+      if (root == null) {
+        return null;
+      }
       return new File(root, STORAGE_TMP_REMOVED);
     }
 
@@ -503,6 +540,9 @@ public abstract class Storage extends StorageInfo {
      * @return the directory path
      */
     public File getFinalizedTmp() {
+      if (root == null) {
+        return null;
+      }
       return new File(root, STORAGE_TMP_FINALIZED);
     }
 
@@ -517,6 +557,9 @@ public abstract class Storage extends StorageInfo {
      * @return the directory path
      */
     public File getLastCheckpointTmp() {
+      if (root == null) {
+        return null;
+      }
       return new File(root, STORAGE_TMP_LAST_CKPT);
     }
 
@@ -530,6 +573,9 @@ public abstract class Storage extends StorageInfo {
      * @return the directory path
      */
     public File getPreviousCheckpoint() {
+      if (root == null) {
+        return null;
+      }
       return new File(root, STORAGE_PREVIOUS_CKPT);
     }
 
@@ -543,7 +589,7 @@ public abstract class Storage extends StorageInfo {
     private void checkEmptyCurrent() throws InconsistentFSStateException,
         IOException {
       File currentDir = getCurrentDir();
-      if(!currentDir.exists()) {
+      if(currentDir == null || !currentDir.exists()) {
         // if current/ does not exist, it's safe to format it.
         return;
       }
@@ -589,6 +635,13 @@ public abstract class Storage extends StorageInfo {
     public StorageState analyzeStorage(StartupOption startOpt, Storage storage,
         boolean checkCurrentIsEmpty)
         throws IOException {
+
+      if (location != null &&
+          location.getStorageType() == StorageType.PROVIDED) {
+        // currently we assume that PROVIDED storages are always NORMAL
+        return StorageState.NORMAL;
+      }
+
       assert root != null : "root is null";
       boolean hadMkdirs = false;
       String rootPath = root.getCanonicalPath();
@@ -710,6 +763,10 @@ public abstract class Storage extends StorageInfo {
      */
     public void doRecover(StorageState curState) throws IOException {
       File curDir = getCurrentDir();
+      if (curDir == null || root == null) {
+        // at this point, we do not support recovery on PROVIDED storages
+        return;
+      }
       String rootPath = root.getCanonicalPath();
       switch(curState) {
       case COMPLETE_UPGRADE:  // mv previous.tmp -> previous
@@ -883,7 +940,8 @@ public abstract class Storage extends StorageInfo {
     
     @Override
     public String toString() {
-      return "Storage Directory " + this.root;
+      return "Storage Directory root= " + this.root +
+          "; location= " + this.location;
     }
 
     /**
@@ -1153,6 +1211,9 @@ public abstract class Storage extends StorageInfo {
   }
   
   public void writeProperties(File to, StorageDirectory sd) throws IOException {
+    if (to == null) {
+      return;
+    }
     Properties props = new Properties();
     setPropertiesFromFields(props, sd);
     writeProperties(to, props);

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java

@@ -152,6 +152,9 @@ public class StorageInfo {
    */
   protected void setFieldsFromProperties(
       Properties props, StorageDirectory sd) throws IOException {
+    if (props == null) {
+      return;
+    }
     setLayoutVersion(props, sd);
     setNamespaceID(props, sd);
     setcTime(props, sd);
@@ -241,6 +244,9 @@ public class StorageInfo {
   }
 
   public static Properties readPropertiesFile(File from) throws IOException {
+    if (from == null) {
+      return null;
+    }
     RandomAccessFile file = new RandomAccessFile(from, "rws");
     FileInputStream in = null;
     Properties props = new Properties();

+ 113 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java

@@ -0,0 +1,113 @@
+/**
+ * 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.server.common.blockaliasmap;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Optional;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.BlockAlias;
+
+/**
+ * An abstract class used to read and write block maps for provided blocks.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class BlockAliasMap<T extends BlockAlias> {
+
+  /**
+   * ImmutableIterator is an Iterator that does not support the remove
+   * operation. This could inherit {@link java.util.Enumeration} but Iterator
+   * is supported by more APIs and Enumeration's javadoc even suggests using
+   * Iterator instead.
+   */
+  public abstract class ImmutableIterator implements Iterator<T> {
+    public void remove() {
+      throw new UnsupportedOperationException(
+          "Remove is not supported for provided storage");
+    }
+  }
+
+  /**
+   * An abstract class that is used to read {@link BlockAlias}es
+   * for provided blocks.
+   */
+  public static abstract class Reader<U extends BlockAlias>
+      implements Iterable<U>, Closeable {
+
+    /**
+     * reader options.
+     */
+    public interface Options { }
+
+    /**
+     * @param ident block to resolve
+     * @return BlockAlias corresponding to the provided block.
+     * @throws IOException
+     */
+    public abstract Optional<U> resolve(Block ident) throws IOException;
+  }
+
+  /**
+   * Returns a reader to the alias map.
+   * @param opts reader options
+   * @param blockPoolID block pool id to use
+   * @return {@link Reader} to the alias map. If a Reader for the blockPoolID
+   * cannot be created, this will return null.
+   * @throws IOException
+   */
+  public abstract Reader<T> getReader(Reader.Options opts, String blockPoolID)
+      throws IOException;
+
+  /**
+   * An abstract class used as a writer for the provided block map.
+   */
+  public static abstract class Writer<U extends BlockAlias>
+      implements Closeable {
+    /**
+     * writer options.
+     */
+    public interface Options { }
+
+    public abstract void store(U token) throws IOException;
+
+  }
+
+  /**
+   * Returns the writer for the alias map.
+   * @param opts writer options.
+   * @param blockPoolID block pool id to use
+   * @return {@link Writer} to the alias map.
+   * @throws IOException
+   */
+  public abstract Writer<T> getWriter(Writer.Options opts, String blockPoolID)
+      throws IOException;
+
+  /**
+   * Refresh the alias map.
+   * @throws IOException
+   */
+  public abstract void refresh() throws IOException;
+
+  public abstract void close() throws IOException;
+
+}

+ 178 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java

@@ -0,0 +1,178 @@
+/*
+ * 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.server.common.blockaliasmap.impl;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.protocolPB.InMemoryAliasMapProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+
+/**
+ * InMemoryLevelDBAliasMapClient is the client for the InMemoryAliasMapServer.
+ * This is used by the Datanode and fs2img to store and retrieve FileRegions
+ * based on the given Block.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class InMemoryLevelDBAliasMapClient extends BlockAliasMap<FileRegion>
+    implements Configurable {
+
+  private Configuration conf;
+  private InMemoryAliasMapProtocolClientSideTranslatorPB aliasMap;
+  private String blockPoolID;
+
+  @Override
+  public void close() {
+    aliasMap.stop();
+  }
+
+  class LevelDbReader extends BlockAliasMap.Reader<FileRegion> {
+
+    @Override
+    public Optional<FileRegion> resolve(Block block) throws IOException {
+      Optional<ProvidedStorageLocation> read = aliasMap.read(block);
+      return read.map(psl -> new FileRegion(block, psl));
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+
+    private class LevelDbIterator
+        extends BlockAliasMap<FileRegion>.ImmutableIterator {
+
+      private Iterator<FileRegion> iterator;
+      private Optional<Block> nextMarker;
+
+      LevelDbIterator()  {
+        batch(Optional.empty());
+      }
+
+      private void batch(Optional<Block> newNextMarker) {
+        try {
+          InMemoryAliasMap.IterationResult iterationResult =
+              aliasMap.list(newNextMarker);
+          List<FileRegion> fileRegions = iterationResult.getFileRegions();
+          this.iterator = fileRegions.iterator();
+          this.nextMarker = iterationResult.getNextBlock();
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+
+      @Override
+      public boolean hasNext() {
+        return iterator.hasNext() || nextMarker.isPresent();
+      }
+
+      @Override
+      public FileRegion next() {
+        if (iterator.hasNext()) {
+          return iterator.next();
+        } else {
+          if (nextMarker.isPresent()) {
+            batch(nextMarker);
+            return next();
+          } else {
+            throw new NoSuchElementException();
+          }
+        }
+      }
+    }
+
+    @Override
+    public Iterator<FileRegion> iterator() {
+      return new LevelDbIterator();
+    }
+  }
+
+  class LevelDbWriter extends BlockAliasMap.Writer<FileRegion> {
+    @Override
+    public void store(FileRegion fileRegion) throws IOException {
+      aliasMap.write(fileRegion.getBlock(),
+          fileRegion.getProvidedStorageLocation());
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+  }
+
+  InMemoryLevelDBAliasMapClient() {
+    if (UserGroupInformation.isSecurityEnabled()) {
+      throw new UnsupportedOperationException("Unable to start "
+          + "InMemoryLevelDBAliasMapClient as security is enabled");
+    }
+  }
+
+
+  @Override
+  public Reader<FileRegion> getReader(Reader.Options opts, String blockPoolID)
+      throws IOException {
+    if (this.blockPoolID == null) {
+      this.blockPoolID = aliasMap.getBlockPoolId();
+    }
+    // if a block pool id has been supplied, and doesn't match the associated
+    // block pool id, return null.
+    if (blockPoolID != null && this.blockPoolID != null
+        && !this.blockPoolID.equals(blockPoolID)) {
+      return null;
+    }
+    return new LevelDbReader();
+  }
+
+  @Override
+  public Writer<FileRegion> getWriter(Writer.Options opts, String blockPoolID)
+      throws IOException {
+    if (this.blockPoolID == null) {
+      this.blockPoolID = aliasMap.getBlockPoolId();
+    }
+    if (blockPoolID != null && !this.blockPoolID.equals(blockPoolID)) {
+      return null;
+    }
+    return new LevelDbWriter();
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    this.aliasMap = new InMemoryAliasMapProtocolClientSideTranslatorPB(conf);
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void refresh() throws IOException {
+  }
+
+}

+ 274 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java

@@ -0,0 +1,274 @@
+/**
+ * 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.server.common.blockaliasmap.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.iq80.leveldb.DB;
+import org.iq80.leveldb.DBIterator;
+import static org.fusesource.leveldbjni.JniDBFactory.factory;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LEVELDB_PATH;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.fromBlockBytes;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.fromProvidedStorageLocationBytes;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.toProtoBufBytes;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A LevelDB based implementation of {@link BlockAliasMap}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class LevelDBFileRegionAliasMap
+      extends BlockAliasMap<FileRegion> implements Configurable {
+
+  private Configuration conf;
+  private LevelDBOptions opts = new LevelDBOptions();
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(LevelDBFileRegionAliasMap.class);
+
+  @Override
+  public void setConf(Configuration conf) {
+    opts.setConf(conf);
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public Reader<FileRegion> getReader(Reader.Options opts, String blockPoolID)
+      throws IOException {
+    if (null == opts) {
+      opts = this.opts;
+    }
+    if (!(opts instanceof LevelDBOptions)) {
+      throw new IllegalArgumentException("Invalid options " + opts.getClass());
+    }
+    LevelDBOptions o = (LevelDBOptions) opts;
+    return new LevelDBFileRegionAliasMap.LevelDBReader(
+        createDB(o.levelDBPath, false, blockPoolID));
+  }
+
+  @Override
+  public Writer<FileRegion> getWriter(Writer.Options opts, String blockPoolID)
+      throws IOException {
+    if (null == opts) {
+      opts = this.opts;
+    }
+    if (!(opts instanceof LevelDBOptions)) {
+      throw new IllegalArgumentException("Invalid options " + opts.getClass());
+    }
+    LevelDBOptions o = (LevelDBOptions) opts;
+    return new LevelDBFileRegionAliasMap.LevelDBWriter(
+        createDB(o.levelDBPath, true, blockPoolID));
+  }
+
+  private static DB createDB(String levelDBPath, boolean createIfMissing,
+      String blockPoolID) throws IOException {
+    if (levelDBPath == null || levelDBPath.length() == 0) {
+      throw new IllegalArgumentException(
+          "A valid path needs to be specified for "
+              + LevelDBFileRegionAliasMap.class + " using the parameter "
+              + DFS_PROVIDED_ALIASMAP_LEVELDB_PATH);
+    }
+    org.iq80.leveldb.Options options = new org.iq80.leveldb.Options();
+    options.createIfMissing(createIfMissing);
+    File dbFile;
+    if (blockPoolID != null) {
+      dbFile = new File(levelDBPath, blockPoolID);
+    } else {
+      dbFile = new File(levelDBPath);
+    }
+    if (createIfMissing && !dbFile.exists()) {
+      if (!dbFile.mkdirs()) {
+        throw new IOException("Unable to create " + dbFile);
+      }
+    }
+    return factory.open(dbFile, options);
+  }
+
+  @Override
+  public void refresh() throws IOException {
+  }
+
+  @Override
+  public void close() throws IOException {
+    // Do nothing.
+  }
+
+  /**
+   * Class specifying reader options for the {@link LevelDBFileRegionAliasMap}.
+   */
+  public static class LevelDBOptions implements LevelDBReader.Options,
+      LevelDBWriter.Options, Configurable {
+    private Configuration conf;
+    private String levelDBPath;
+
+    @Override
+    public void setConf(Configuration conf) {
+      this.conf = conf;
+      this.levelDBPath = conf.get(DFS_PROVIDED_ALIASMAP_LEVELDB_PATH);
+    }
+
+    @Override
+    public Configuration getConf() {
+      return conf;
+    }
+
+    @Override
+    public LevelDBOptions filename(String levelDBPath) {
+      this.levelDBPath = levelDBPath;
+      return this;
+    }
+  }
+
+  /**
+   * This class is used as a reader for block maps which
+   * are stored as LevelDB files.
+   */
+  public static class LevelDBReader extends Reader<FileRegion> {
+
+    /**
+     * Options for {@link LevelDBReader}.
+     */
+    public interface Options extends Reader.Options {
+      Options filename(String levelDBPath);
+    }
+
+    private DB db;
+
+    LevelDBReader(DB db) {
+      this.db = db;
+    }
+
+    @Override
+    public Optional<FileRegion> resolve(Block block) throws IOException {
+      if (db == null) {
+        return Optional.empty();
+      }
+      // consider layering index w/ composable format
+      byte[] key = toProtoBufBytes(block);
+      byte[] value = db.get(key);
+      ProvidedStorageLocation psl = fromProvidedStorageLocationBytes(value);
+      return Optional.of(new FileRegion(block, psl));
+    }
+
+    static class FRIterator implements Iterator<FileRegion> {
+      private final DBIterator internal;
+
+      FRIterator(DBIterator internal) {
+        this.internal = internal;
+      }
+
+      @Override
+      public boolean hasNext() {
+        return internal.hasNext();
+      }
+
+      @Override
+      public FileRegion next() {
+        Map.Entry<byte[], byte[]> entry = internal.next();
+        if (entry == null) {
+          return null;
+        }
+        try {
+          Block block = fromBlockBytes(entry.getKey());
+          ProvidedStorageLocation psl =
+              fromProvidedStorageLocationBytes(entry.getValue());
+          return new FileRegion(block, psl);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    }
+
+    public Iterator<FileRegion> iterator() {
+      if (db == null) {
+        return null;
+      }
+      DBIterator iterator = db.iterator();
+      iterator.seekToFirst();
+      return new FRIterator(iterator);
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (db != null) {
+        db.close();
+      }
+    }
+  }
+
+  /**
+   * This class is used as a writer for block maps which
+   * are stored as LevelDB files.
+   */
+  public static class LevelDBWriter extends Writer<FileRegion> {
+
+    /**
+     * Interface for Writer options.
+     */
+    public interface Options extends Writer.Options {
+      Options filename(String levelDBPath);
+    }
+
+    private final DB db;
+
+    LevelDBWriter(DB db) {
+      this.db = db;
+    }
+
+    @Override
+    public void store(FileRegion token) throws IOException {
+      byte[] key = toProtoBufBytes(token.getBlock());
+      byte[] value = toProtoBufBytes(token.getProvidedStorageLocation());
+      db.put(key, value);
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (db != null) {
+        db.close();
+      }
+    }
+  }
+}

+ 490 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java

@@ -0,0 +1,490 @@
+/**
+ * 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.server.common.blockaliasmap.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.io.MultipleIOException;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This class is used for block maps stored as text files,
+ * with a specified delimiter.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class TextFileRegionAliasMap
+    extends BlockAliasMap<FileRegion> implements Configurable {
+
+  private Configuration conf;
+  private ReaderOptions readerOpts = TextReader.defaults();
+  private WriterOptions writerOpts = TextWriter.defaults();
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TextFileRegionAliasMap.class);
+  @Override
+  public void setConf(Configuration conf) {
+    readerOpts.setConf(conf);
+    writerOpts.setConf(conf);
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public Reader<FileRegion> getReader(Reader.Options opts, String blockPoolID)
+      throws IOException {
+    if (null == opts) {
+      opts = readerOpts;
+    }
+    if (!(opts instanceof ReaderOptions)) {
+      throw new IllegalArgumentException("Invalid options " + opts.getClass());
+    }
+    ReaderOptions o = (ReaderOptions) opts;
+    Configuration readerConf = (null == o.getConf())
+        ? new Configuration()
+            : o.getConf();
+    return createReader(o.file, o.delim, readerConf, blockPoolID);
+  }
+
+  @VisibleForTesting
+  TextReader createReader(Path file, String delim, Configuration cfg,
+      String blockPoolID) throws IOException {
+    FileSystem fs = file.getFileSystem(cfg);
+    if (fs instanceof LocalFileSystem) {
+      fs = ((LocalFileSystem)fs).getRaw();
+    }
+    CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
+    CompressionCodec codec = factory.getCodec(file);
+    String filename = fileNameFromBlockPoolID(blockPoolID);
+    if (codec != null) {
+      filename = filename + codec.getDefaultExtension();
+    }
+    Path bpidFilePath = new Path(file.getParent(), filename);
+    return new TextReader(fs, bpidFilePath, codec, delim);
+  }
+
+  @Override
+  public Writer<FileRegion> getWriter(Writer.Options opts, String blockPoolID)
+      throws IOException {
+    if (null == opts) {
+      opts = writerOpts;
+    }
+    if (!(opts instanceof WriterOptions)) {
+      throw new IllegalArgumentException("Invalid options " + opts.getClass());
+    }
+    WriterOptions o = (WriterOptions) opts;
+    Configuration cfg = (null == o.getConf())
+        ? new Configuration()
+            : o.getConf();
+    String baseName = fileNameFromBlockPoolID(blockPoolID);
+    Path blocksFile = new Path(o.dir, baseName);
+    if (o.codec != null) {
+      CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
+      CompressionCodec codec = factory.getCodecByName(o.codec);
+      blocksFile = new Path(o.dir, baseName + codec.getDefaultExtension());
+      return createWriter(blocksFile, codec, o.delim, cfg);
+    }
+    return createWriter(blocksFile, null, o.delim, conf);
+  }
+
+  @VisibleForTesting
+  TextWriter createWriter(Path file, CompressionCodec codec, String delim,
+      Configuration cfg) throws IOException {
+    FileSystem fs = file.getFileSystem(cfg);
+    if (fs instanceof LocalFileSystem) {
+      fs = ((LocalFileSystem)fs).getRaw();
+    }
+    OutputStream tmp = fs.create(file);
+    java.io.Writer out = new BufferedWriter(new OutputStreamWriter(
+          (null == codec) ? tmp : codec.createOutputStream(tmp), "UTF-8"));
+    return new TextWriter(out, delim);
+  }
+
+  /**
+   * Class specifying reader options for the {@link TextFileRegionAliasMap}.
+   */
+  public static class ReaderOptions
+      implements TextReader.Options, Configurable {
+
+    private Configuration conf;
+    private String delim =
+        DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT;
+    private Path file = new Path(
+        new File(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_FILE_DEFAULT)
+            .toURI().toString());
+
+    @Override
+    public void setConf(Configuration conf) {
+      this.conf = conf;
+      String tmpfile =
+          conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_FILE,
+              DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_FILE_DEFAULT);
+      file = new Path(tmpfile);
+      delim = conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER,
+          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT);
+      LOG.info("TextFileRegionAliasMap: read path {}", tmpfile);
+    }
+
+    @Override
+    public Configuration getConf() {
+      return conf;
+    }
+
+    @Override
+    public ReaderOptions filename(Path file) {
+      this.file = file;
+      return this;
+    }
+
+    @Override
+    public ReaderOptions delimiter(String delim) {
+      this.delim = delim;
+      return this;
+    }
+  }
+
+  /**
+   * Class specifying writer options for the {@link TextFileRegionAliasMap}.
+   */
+  public static class WriterOptions
+      implements TextWriter.Options, Configurable {
+
+    private Configuration conf;
+    private String codec = null;
+    private Path dir =
+        new Path(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_DIR_DEFAULT);
+    private String delim =
+        DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT;
+
+    @Override
+    public void setConf(Configuration conf) {
+      this.conf = conf;
+      String tmpDir = conf.get(
+          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_DIR, dir.toString());
+      dir = new Path(tmpDir);
+      codec = conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_CODEC);
+      delim = conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER,
+          DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER_DEFAULT);
+    }
+
+    @Override
+    public Configuration getConf() {
+      return conf;
+    }
+
+    @Override
+    public WriterOptions dirName(Path dir) {
+      this.dir = dir;
+      return this;
+    }
+
+    public String getCodec() {
+      return codec;
+    }
+
+    public Path getDir() {
+      return dir;
+    }
+
+    @Override
+    public WriterOptions codec(String codec) {
+      this.codec = codec;
+      return this;
+    }
+
+    @Override
+    public WriterOptions delimiter(String delim) {
+      this.delim = delim;
+      return this;
+    }
+
+  }
+
+  /**
+   * This class is used as a reader for block maps which
+   * are stored as delimited text files.
+   */
+  public static class TextReader extends Reader<FileRegion> {
+
+    /**
+     * Options for {@link TextReader}.
+     */
+    public interface Options extends Reader.Options {
+      Options filename(Path file);
+      Options delimiter(String delim);
+    }
+
+    public static ReaderOptions defaults() {
+      return new ReaderOptions();
+    }
+
+    private final Path file;
+    private final String delim;
+    private final FileSystem fs;
+    private final CompressionCodec codec;
+    private final Map<FRIterator, BufferedReader> iterators;
+    private final String blockPoolID;
+
+    protected TextReader(FileSystem fs, Path file, CompressionCodec codec,
+        String delim) {
+      this(fs, file, codec, delim,
+          new IdentityHashMap<FRIterator, BufferedReader>());
+    }
+
+    TextReader(FileSystem fs, Path file, CompressionCodec codec, String delim,
+        Map<FRIterator, BufferedReader> iterators) {
+      this.fs = fs;
+      this.file = file;
+      this.codec = codec;
+      this.delim = delim;
+      this.iterators = Collections.synchronizedMap(iterators);
+      this.blockPoolID = blockPoolIDFromFileName(file);
+    }
+
+    @Override
+    public Optional<FileRegion> resolve(Block ident) throws IOException {
+      // consider layering index w/ composable format
+      Iterator<FileRegion> i = iterator();
+      try {
+        while (i.hasNext()) {
+          FileRegion f = i.next();
+          if (f.getBlock().equals(ident)) {
+            return Optional.of(f);
+          }
+        }
+      } finally {
+        BufferedReader r = iterators.remove(i);
+        if (r != null) {
+          // null on last element
+          r.close();
+        }
+      }
+      return Optional.empty();
+    }
+
+    class FRIterator implements Iterator<FileRegion> {
+
+      private FileRegion pending;
+
+      @Override
+      public boolean hasNext() {
+        return pending != null;
+      }
+
+      @Override
+      public FileRegion next() {
+        if (null == pending) {
+          throw new NoSuchElementException();
+        }
+        FileRegion ret = pending;
+        try {
+          pending = nextInternal(this);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+        return ret;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    }
+
+    private FileRegion nextInternal(Iterator<FileRegion> i) throws IOException {
+      BufferedReader r = iterators.get(i);
+      if (null == r) {
+        throw new IllegalStateException();
+      }
+      String line = r.readLine();
+      if (null == line) {
+        iterators.remove(i);
+        return null;
+      }
+      String[] f = line.split(delim);
+      if (f.length != 5 && f.length != 6) {
+        throw new IOException("Invalid line: " + line);
+      }
+      byte[] nonce = new byte[0];
+      if (f.length == 6) {
+        nonce = f[5].getBytes(Charset.forName("UTF-8"));
+      }
+      return new FileRegion(Long.parseLong(f[0]), new Path(f[1]),
+          Long.parseLong(f[2]), Long.parseLong(f[3]), Long.parseLong(f[4]),
+          nonce);
+    }
+
+    public InputStream createStream() throws IOException {
+      InputStream i = fs.open(file);
+      if (codec != null) {
+        i = codec.createInputStream(i);
+      }
+      return i;
+    }
+
+    @Override
+    public Iterator<FileRegion> iterator() {
+      FRIterator i = new FRIterator();
+      try {
+        BufferedReader r =
+            new BufferedReader(new InputStreamReader(createStream(), "UTF-8"));
+        iterators.put(i, r);
+        i.pending = nextInternal(i);
+      } catch (IOException e) {
+        iterators.remove(i);
+        throw new RuntimeException(e);
+      }
+      return i;
+    }
+
+    @Override
+    public void close() throws IOException {
+      ArrayList<IOException> ex = new ArrayList<>();
+      synchronized (iterators) {
+        for (Iterator<BufferedReader> i = iterators.values().iterator();
+             i.hasNext();) {
+          try {
+            BufferedReader r = i.next();
+            r.close();
+          } catch (IOException e) {
+            ex.add(e);
+          } finally {
+            i.remove();
+          }
+        }
+        iterators.clear();
+      }
+      if (!ex.isEmpty()) {
+        throw MultipleIOException.createIOException(ex);
+      }
+    }
+  }
+
+  /**
+   * This class is used as a writer for block maps which
+   * are stored as delimited text files.
+   */
+  public static class TextWriter extends Writer<FileRegion> {
+
+    /**
+     * Interface for Writer options.
+     */
+    public interface Options extends Writer.Options {
+      Options codec(String codec);
+      Options dirName(Path dir);
+      Options delimiter(String delim);
+    }
+
+    public static WriterOptions defaults() {
+      return new WriterOptions();
+    }
+
+    private final String delim;
+    private final java.io.Writer out;
+
+    public TextWriter(java.io.Writer out, String delim) {
+      this.out = out;
+      this.delim = delim;
+    }
+
+    @Override
+    public void store(FileRegion token) throws IOException {
+      final Block block = token.getBlock();
+      final ProvidedStorageLocation psl = token.getProvidedStorageLocation();
+
+      out.append(String.valueOf(block.getBlockId())).append(delim);
+      out.append(psl.getPath().toString()).append(delim);
+      out.append(Long.toString(psl.getOffset())).append(delim);
+      out.append(Long.toString(psl.getLength())).append(delim);
+      out.append(Long.toString(block.getGenerationStamp()));
+      if (psl.getNonce().length > 0) {
+        out.append(delim)
+            .append(new String(psl.getNonce(), Charset.forName("UTF-8")));
+      }
+      out.append("\n");
+    }
+
+    @Override
+    public void close() throws IOException {
+      out.close();
+    }
+
+  }
+
+  @Override
+  public void refresh() throws IOException {
+    throw new UnsupportedOperationException(
+        "Refresh not supported by " + getClass());
+  }
+
+  @Override
+  public void close() throws IOException {
+    // nothing to do;
+  }
+
+  @VisibleForTesting
+  public static String blockPoolIDFromFileName(Path file) {
+    if (file == null) {
+      return "";
+    }
+    String fileName = file.getName();
+    return fileName.substring("blocks_".length()).split("\\.")[0];
+  }
+
+  @VisibleForTesting
+  public static String fileNameFromBlockPoolID(String blockPoolID) {
+    return "blocks_" + blockPoolID + ".csv";
+  }
+}

+ 27 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/package-info.java

@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+package org.apache.hadoop.hdfs.server.common.blockaliasmap;
+
+/**
+ * The AliasMap defines mapping of PROVIDED HDFS blocks to data in remote
+ * storage systems.
+ */
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 19 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
@@ -360,6 +361,9 @@ public class BlockPoolSliceStorage extends Storage {
   private boolean doTransition(StorageDirectory sd, NamespaceInfo nsInfo,
       StartupOption startOpt, List<Callable<StorageDirectory>> callables,
       Configuration conf) throws IOException {
+    if (sd.getStorageLocation().getStorageType() == StorageType.PROVIDED) {
+      return false; // regular startup for PROVIDED storage directories
+    }
     if (startOpt == StartupOption.ROLLBACK && sd.getPreviousDir().exists()) {
       Preconditions.checkState(!getTrashRootDir(sd).exists(),
           sd.getPreviousDir() + " and " + getTrashRootDir(sd) + " should not " +
@@ -439,6 +443,10 @@ public class BlockPoolSliceStorage extends Storage {
         LayoutVersion.Feature.FEDERATION, layoutVersion)) {
       return;
     }
+    // no upgrades for storage directories that are PROVIDED
+    if (bpSd.getRoot() == null) {
+      return;
+    }
     final int oldLV = getLayoutVersion();
     LOG.info("Upgrading block pool storage directory " + bpSd.getRoot()
         + ".\n   old LV = " + oldLV
@@ -589,8 +597,9 @@ public class BlockPoolSliceStorage extends Storage {
       throws IOException {
     File prevDir = bpSd.getPreviousDir();
     // regular startup if previous dir does not exist
-    if (!prevDir.exists())
+    if (prevDir == null || !prevDir.exists()) {
       return;
+    }
     // read attributes out of the VERSION file of previous directory
     BlockPoolSliceStorage prevInfo = new BlockPoolSliceStorage();
     prevInfo.readPreviousVersionProperties(bpSd);
@@ -631,6 +640,9 @@ public class BlockPoolSliceStorage extends Storage {
    * that holds the snapshot.
    */
   void doFinalize(File dnCurDir) throws IOException {
+    if (dnCurDir == null) {
+      return; //we do nothing if the directory is null
+    }
     File bpRoot = getBpRoot(blockpoolID, dnCurDir);
     StorageDirectory bpSd = new StorageDirectory(bpRoot);
     // block pool level previous directory
@@ -841,6 +853,9 @@ public class BlockPoolSliceStorage extends Storage {
   public void setRollingUpgradeMarkers(List<StorageDirectory> dnStorageDirs)
       throws IOException {
     for (StorageDirectory sd : dnStorageDirs) {
+      if (sd.getCurrentDir() == null) {
+        return;
+      }
       File bpRoot = getBpRoot(blockpoolID, sd.getCurrentDir());
       File markerFile = new File(bpRoot, ROLLING_UPGRADE_MARKER_FILE);
       if (!storagesWithRollingUpgradeMarker.contains(bpRoot.toString())) {
@@ -863,6 +878,9 @@ public class BlockPoolSliceStorage extends Storage {
   public void clearRollingUpgradeMarkers(List<StorageDirectory> dnStorageDirs)
       throws IOException {
     for (StorageDirectory sd : dnStorageDirs) {
+      if (sd.getCurrentDir() == null) {
+        continue;
+      }
       File bpRoot = getBpRoot(blockpoolID, sd.getCurrentDir());
       File markerFile = new File(bpRoot, ROLLING_UPGRADE_MARKER_FILE);
       if (!storagesWithoutRollingUpgradeMarker.contains(bpRoot.toString())) {

+ 31 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -48,6 +48,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -129,22 +130,31 @@ public class DataStorage extends Storage {
     this.datanodeUuid = newDatanodeUuid;
   }
 
-  private static boolean createStorageID(StorageDirectory sd, int lv) {
+  private static boolean createStorageID(StorageDirectory sd, int lv,
+      Configuration conf) {
     // Clusters previously upgraded from layout versions earlier than
     // ADD_DATANODE_AND_STORAGE_UUIDS failed to correctly generate a
     // new storage ID. We check for that and fix it now.
     final boolean haveValidStorageId = DataNodeLayoutVersion.supports(
         LayoutVersion.Feature.ADD_DATANODE_AND_STORAGE_UUIDS, lv)
         && DatanodeStorage.isValidStorageId(sd.getStorageUuid());
-    return createStorageID(sd, !haveValidStorageId);
+    return createStorageID(sd, !haveValidStorageId, conf);
   }
 
   /** Create an ID for this storage.
    * @return true if a new storage ID was generated.
    * */
   public static boolean createStorageID(
-      StorageDirectory sd, boolean regenerateStorageIds) {
+      StorageDirectory sd, boolean regenerateStorageIds, Configuration conf) {
     final String oldStorageID = sd.getStorageUuid();
+    if (sd.getStorageLocation() != null &&
+        sd.getStorageLocation().getStorageType() == StorageType.PROVIDED) {
+      // Only one provided storage id is supported.
+      // TODO support multiple provided storage ids
+      sd.setStorageUuid(conf.get(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID,
+          DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT));
+      return false;
+    }
     if (oldStorageID == null || regenerateStorageIds) {
       sd.setStorageUuid(DatanodeStorage.generateUuid());
       LOG.info("Generated new storageID " + sd.getStorageUuid() +
@@ -273,7 +283,7 @@ public class DataStorage extends Storage {
         LOG.info("Storage directory with location " + location
             + " is not formatted for namespace " + nsInfo.getNamespaceID()
             + ". Formatting...");
-        format(sd, nsInfo, datanode.getDatanodeUuid());
+        format(sd, nsInfo, datanode.getDatanodeUuid(), datanode.getConf());
         break;
       default:  // recovery part is common
         sd.doRecover(curState);
@@ -547,15 +557,15 @@ public class DataStorage extends Storage {
   }
 
   void format(StorageDirectory sd, NamespaceInfo nsInfo,
-              String datanodeUuid) throws IOException {
+              String newDatanodeUuid, Configuration conf) throws IOException {
     sd.clearDirectory(); // create directory
     this.layoutVersion = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
     this.clusterID = nsInfo.getClusterID();
     this.namespaceID = nsInfo.getNamespaceID();
     this.cTime = 0;
-    setDatanodeUuid(datanodeUuid);
+    setDatanodeUuid(newDatanodeUuid);
 
-    createStorageID(sd, false);
+    createStorageID(sd, false, conf);
     writeProperties(sd);
   }
 
@@ -600,6 +610,9 @@ public class DataStorage extends Storage {
 
   private void setFieldsFromProperties(Properties props, StorageDirectory sd,
       boolean overrideLayoutVersion, int toLayoutVersion) throws IOException {
+    if (props == null) {
+      return;
+    }
     if (overrideLayoutVersion) {
       this.layoutVersion = toLayoutVersion;
     } else {
@@ -694,6 +707,10 @@ public class DataStorage extends Storage {
   private boolean doTransition(StorageDirectory sd, NamespaceInfo nsInfo,
       StartupOption startOpt, List<Callable<StorageDirectory>> callables,
       Configuration conf) throws IOException {
+    if (sd.getStorageLocation().getStorageType() == StorageType.PROVIDED) {
+      createStorageID(sd, layoutVersion, conf);
+      return false; // regular start up for PROVIDED storage directories
+    }
     if (startOpt == StartupOption.ROLLBACK) {
       doRollback(sd, nsInfo); // rollback if applicable
     }
@@ -724,7 +741,7 @@ public class DataStorage extends Storage {
 
     // regular start up.
     if (this.layoutVersion == HdfsServerConstants.DATANODE_LAYOUT_VERSION) {
-      createStorageID(sd, layoutVersion);
+      createStorageID(sd, layoutVersion, conf);
       return false; // need to write properties
     }
 
@@ -733,7 +750,7 @@ public class DataStorage extends Storage {
       if (federationSupported) {
         // If the existing on-disk layout version supports federation,
         // simply update the properties.
-        upgradeProperties(sd);
+        upgradeProperties(sd, conf);
       } else {
         doUpgradePreFederation(sd, nsInfo, callables, conf);
       }
@@ -829,15 +846,16 @@ public class DataStorage extends Storage {
 
     // 4. Write version file under <SD>/current
     clusterID = nsInfo.getClusterID();
-    upgradeProperties(sd);
+    upgradeProperties(sd, conf);
     
     // 5. Rename <SD>/previous.tmp to <SD>/previous
     rename(tmpDir, prevDir);
     LOG.info("Upgrade of " + sd.getRoot()+ " is complete");
   }
 
-  void upgradeProperties(StorageDirectory sd) throws IOException {
-    createStorageID(sd, layoutVersion);
+  void upgradeProperties(StorageDirectory sd, Configuration conf)
+      throws IOException {
+    createStorageID(sd, layoutVersion, conf);
     LOG.info("Updating layout version from " + layoutVersion
         + " to " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
         + " for storage " + sd.getRoot());
@@ -989,7 +1007,7 @@ public class DataStorage extends Storage {
     // then finalize it. Else finalize the corresponding BP.
     for (StorageDirectory sd : getStorageDirs()) {
       File prevDir = sd.getPreviousDir();
-      if (prevDir.exists()) {
+      if (prevDir != null && prevDir.exists()) {
         // data node level storage finalize
         doFinalize(sd);
       } else {

+ 18 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java

@@ -44,6 +44,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.AutoCloseableLock;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
@@ -105,7 +106,7 @@ public class DirectoryScanner implements Runnable {
    * @param b whether to retain diffs
    */
   @VisibleForTesting
-  void setRetainDiffs(boolean b) {
+  public void setRetainDiffs(boolean b) {
     retainDiffs = b;
   }
 
@@ -215,7 +216,8 @@ public class DirectoryScanner implements Runnable {
    * @param dataset the dataset to scan
    * @param conf the Configuration object
    */
-  DirectoryScanner(DataNode datanode, FsDatasetSpi<?> dataset, Configuration conf) {
+  public DirectoryScanner(DataNode datanode, FsDatasetSpi<?> dataset,
+      Configuration conf) {
     this.datanode = datanode;
     this.dataset = dataset;
     int interval = (int) conf.getTimeDuration(
@@ -369,15 +371,14 @@ public class DirectoryScanner implements Runnable {
    * Reconcile differences between disk and in-memory blocks
    */
   @VisibleForTesting
-  void reconcile() throws IOException {
+  public void reconcile() throws IOException {
     scan();
     for (Entry<String, LinkedList<ScanInfo>> entry : diffs.entrySet()) {
       String bpid = entry.getKey();
       LinkedList<ScanInfo> diff = entry.getValue();
       
       for (ScanInfo info : diff) {
-        dataset.checkAndUpdate(bpid, info.getBlockId(), info.getBlockFile(),
-            info.getMetaFile(), info.getVolume());
+        dataset.checkAndUpdate(bpid, info);
       }
     }
     if (!retainDiffs) clear();
@@ -429,11 +430,12 @@ public class DirectoryScanner implements Runnable {
           }
           // Block file and/or metadata file exists on the disk
           // Block exists in memory
-          if (info.getBlockFile() == null) {
+          if (info.getVolume().getStorageType() != StorageType.PROVIDED &&
+              info.getBlockFile() == null) {
             // Block metadata file exits and block file is missing
             addDifference(diffRecord, statsRecord, info);
           } else if (info.getGenStamp() != memBlock.getGenerationStamp()
-              || info.getBlockFileLength() != memBlock.getNumBytes()) {
+              || info.getBlockLength() != memBlock.getNumBytes()) {
             // Block metadata file is missing or has wrong generation stamp,
             // or block file length is different than expected
             statsRecord.mismatchBlocks++;
@@ -513,7 +515,8 @@ public class DirectoryScanner implements Runnable {
    *
    * @return a map of sorted arrays of block information
    */
-  private Map<String, ScanInfo[]> getDiskReport() {
+  @VisibleForTesting
+  public Map<String, ScanInfo[]> getDiskReport() {
     ScanInfoPerBlockPool list = new ScanInfoPerBlockPool();
     ScanInfoPerBlockPool[] dirReports = null;
     // First get list of data directories
@@ -528,6 +531,10 @@ public class DirectoryScanner implements Runnable {
           new HashMap<Integer, Future<ScanInfoPerBlockPool>>();
 
       for (int i = 0; i < volumes.size(); i++) {
+        if (volumes.get(i).getStorageType() == StorageType.PROVIDED) {
+          // Disable scanning PROVIDED volumes to keep overhead low
+          continue;
+        }
         ReportCompiler reportCompiler =
             new ReportCompiler(datanode, volumes.get(i));
         Future<ScanInfoPerBlockPool> result =
@@ -611,6 +618,9 @@ public class DirectoryScanner implements Runnable {
       for (String bpid : bpList) {
         LinkedList<ScanInfo> report = new LinkedList<>();
 
+        perfTimer.reset().start();
+        throttleTimer.reset().start();
+
         try {
           result.put(bpid, volume.compileReport(bpid, report, this));
         } catch (InterruptedException ex) {

+ 122 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java

@@ -0,0 +1,122 @@
+/**
+ * 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.server.datanode;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathHandle;
+import org.apache.hadoop.fs.RawPathHandle;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+
+/**
+ * This class is used for provided replicas that are finalized.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class FinalizedProvidedReplica extends ProvidedReplica {
+
+  public FinalizedProvidedReplica(long blockId, URI fileURI, long fileOffset,
+      long blockLen, long genStamp, PathHandle pathHandle, FsVolumeSpi volume,
+      Configuration conf, FileSystem remoteFS) {
+    super(blockId, fileURI, fileOffset, blockLen, genStamp, pathHandle, volume,
+        conf, remoteFS);
+  }
+
+  public FinalizedProvidedReplica(FileRegion fileRegion, FsVolumeSpi volume,
+      Configuration conf, FileSystem remoteFS) {
+    super(fileRegion.getBlock().getBlockId(),
+        fileRegion.getProvidedStorageLocation().getPath().toUri(),
+        fileRegion.getProvidedStorageLocation().getOffset(),
+        fileRegion.getBlock().getNumBytes(),
+        fileRegion.getBlock().getGenerationStamp(),
+        new RawPathHandle(ByteBuffer
+            .wrap(fileRegion.getProvidedStorageLocation().getNonce())),
+        volume, conf, remoteFS);
+  }
+
+  public FinalizedProvidedReplica(long blockId, Path pathPrefix,
+      String pathSuffix, long fileOffset, long blockLen, long genStamp,
+      PathHandle pathHandle, FsVolumeSpi volume, Configuration conf,
+      FileSystem remoteFS) {
+    super(blockId, pathPrefix, pathSuffix, fileOffset, blockLen,
+        genStamp, pathHandle, volume, conf, remoteFS);
+  }
+
+  @Override
+  public ReplicaState getState() {
+    return ReplicaState.FINALIZED;
+  }
+
+  @Override
+  public long getBytesOnDisk() {
+    return getNumBytes();
+  }
+
+  @Override
+  public long getVisibleLength() {
+    return getNumBytes(); //all bytes are visible
+  }
+
+  @Override  // Object
+  public boolean equals(Object o) {
+    return super.equals(o);
+  }
+
+  @Override  // Object
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return super.toString();
+  }
+
+  @Override
+  public ReplicaInfo getOriginalReplica() {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support getOriginalReplica");
+  }
+
+  @Override
+  public long getRecoveryID() {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support getRecoveryID");
+  }
+
+  @Override
+  public void setRecoveryID(long recoveryId) {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support setRecoveryID");
+  }
+
+  @Override
+  public ReplicaRecoveryInfo createInfo() {
+    throw new UnsupportedOperationException("Replica of type " + getState() +
+        " does not support createInfo");
+  }
+}

+ 350 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java

@@ -0,0 +1,350 @@
+/**
+ * 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.server.datanode;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.io.input.BoundedInputStream;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathHandle;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetUtil;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+
+/**
+ * This abstract class is used as a base class for provided replicas.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public abstract class ProvidedReplica extends ReplicaInfo {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ProvidedReplica.class);
+
+  // Null checksum information for provided replicas.
+  // Shared across all replicas.
+  static final byte[] NULL_CHECKSUM_ARRAY =
+      FsDatasetUtil.createNullChecksumByteArray();
+  private URI fileURI;
+  private Path pathPrefix;
+  private String pathSuffix;
+  private long fileOffset;
+  private Configuration conf;
+  private PathHandle pathHandle;
+  private FileSystem remoteFS;
+
+  /**
+   * Constructor.
+   *
+   * @param blockId block id
+   * @param fileURI remote URI this block is to be read from
+   * @param fileOffset the offset in the remote URI
+   * @param blockLen the length of the block
+   * @param genStamp the generation stamp of the block
+   * @param volume the volume this block belongs to
+   * @param conf the configuration
+   * @param remoteFS reference to the remote filesystem to use for this replica.
+   */
+  public ProvidedReplica(long blockId, URI fileURI, long fileOffset,
+      long blockLen, long genStamp, PathHandle pathHandle, FsVolumeSpi volume,
+      Configuration conf, FileSystem remoteFS) {
+    super(volume, blockId, blockLen, genStamp);
+    this.fileURI = fileURI;
+    this.fileOffset = fileOffset;
+    this.conf = conf;
+    this.pathHandle = pathHandle;
+    if (remoteFS != null) {
+      this.remoteFS = remoteFS;
+    } else {
+      LOG.warn(
+          "Creating an reference to the remote FS for provided block " + this);
+      try {
+        this.remoteFS = FileSystem.get(fileURI, this.conf);
+      } catch (IOException e) {
+        LOG.warn("Failed to obtain filesystem for " + fileURI);
+        this.remoteFS = null;
+      }
+    }
+  }
+
+  /**
+   * Constructor.
+   *
+   * @param blockId block id
+   * @param pathPrefix A prefix of the {@link Path} associated with this replica
+   *          on the remote {@link FileSystem}.
+   * @param pathSuffix A suffix of the {@link Path} associated with this replica
+   *          on the remote {@link FileSystem}. Resolving the {@code pathSuffix}
+   *          against the {@code pathPrefix} should provide the exact
+   *          {@link Path} of the data associated with this replica on the
+   *          remote {@link FileSystem}.
+   * @param fileOffset the offset in the remote URI
+   * @param blockLen the length of the block
+   * @param genStamp the generation stamp of the block
+   * @param volume the volume this block belongs to
+   * @param conf the configuration
+   * @param remoteFS reference to the remote filesystem to use for this replica.
+   */
+  public ProvidedReplica(long blockId, Path pathPrefix, String pathSuffix,
+      long fileOffset, long blockLen, long genStamp, PathHandle pathHandle,
+      FsVolumeSpi volume, Configuration conf, FileSystem remoteFS) {
+    super(volume, blockId, blockLen, genStamp);
+    this.fileURI = null;
+    this.pathPrefix = pathPrefix;
+    this.pathSuffix = pathSuffix;
+    this.fileOffset = fileOffset;
+    this.conf = conf;
+    this.pathHandle = pathHandle;
+    if (remoteFS != null) {
+      this.remoteFS = remoteFS;
+    } else {
+      LOG.warn(
+          "Creating an reference to the remote FS for provided block " + this);
+      try {
+        this.remoteFS = FileSystem.get(pathPrefix.toUri(), this.conf);
+      } catch (IOException e) {
+        LOG.warn("Failed to obtain filesystem for " + pathPrefix);
+        this.remoteFS = null;
+      }
+    }
+  }
+
+  public ProvidedReplica(ProvidedReplica r) {
+    super(r);
+    this.fileURI = r.fileURI;
+    this.fileOffset = r.fileOffset;
+    this.conf = r.conf;
+    this.remoteFS = r.remoteFS;
+    this.pathHandle = r.pathHandle;
+    this.pathPrefix = r.pathPrefix;
+    this.pathSuffix = r.pathSuffix;
+  }
+
+  @Override
+  public URI getBlockURI() {
+    return getRemoteURI();
+  }
+
+  @VisibleForTesting
+  public String getPathSuffix() {
+    return pathSuffix;
+  }
+
+  @VisibleForTesting
+  public Path getPathPrefix() {
+    return pathPrefix;
+  }
+
+  private URI getRemoteURI() {
+    if (fileURI != null) {
+      return fileURI;
+    } else if (pathPrefix == null) {
+      return new Path(pathSuffix).toUri();
+    } else {
+      return new Path(pathPrefix, pathSuffix).toUri();
+    }
+  }
+
+  @Override
+  public InputStream getDataInputStream(long seekOffset) throws IOException {
+    if (remoteFS != null) {
+      FSDataInputStream ins;
+      try {
+        if (pathHandle != null) {
+          ins = remoteFS.open(pathHandle, conf.getInt(IO_FILE_BUFFER_SIZE_KEY,
+              IO_FILE_BUFFER_SIZE_DEFAULT));
+        } else {
+          ins = remoteFS.open(new Path(getRemoteURI()));
+        }
+      } catch (UnsupportedOperationException e) {
+        throw new IOException("PathHandle specified, but unsuported", e);
+      }
+
+      ins.seek(fileOffset + seekOffset);
+      return new BoundedInputStream(
+          new FSDataInputStream(ins), getBlockDataLength());
+    } else {
+      throw new IOException("Remote filesystem for provided replica " + this +
+          " does not exist");
+    }
+  }
+
+  @Override
+  public OutputStream getDataOutputStream(boolean append) throws IOException {
+    throw new UnsupportedOperationException(
+        "OutputDataStream is not implemented for ProvidedReplica");
+  }
+
+  @Override
+  public URI getMetadataURI() {
+    return null;
+  }
+
+  @Override
+  public OutputStream getMetadataOutputStream(boolean append)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public boolean blockDataExists() {
+    if(remoteFS != null) {
+      try {
+        return remoteFS.exists(new Path(getRemoteURI()));
+      } catch (IOException e) {
+        return false;
+      }
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public boolean deleteBlockData() {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not support deleting block data");
+  }
+
+  @Override
+  public long getBlockDataLength() {
+    return this.getNumBytes();
+  }
+
+  @Override
+  public LengthInputStream getMetadataInputStream(long offset)
+      throws IOException {
+    return new LengthInputStream(new ByteArrayInputStream(NULL_CHECKSUM_ARRAY),
+        NULL_CHECKSUM_ARRAY.length);
+  }
+
+  @Override
+  public boolean metadataExists() {
+    return NULL_CHECKSUM_ARRAY == null ? false : true;
+  }
+
+  @Override
+  public boolean deleteMetadata() {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not support deleting metadata");
+  }
+
+  @Override
+  public long getMetadataLength() {
+    return NULL_CHECKSUM_ARRAY == null ? 0 : NULL_CHECKSUM_ARRAY.length;
+  }
+
+  @Override
+  public boolean renameMeta(URI destURI) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not support renaming metadata");
+  }
+
+  @Override
+  public boolean renameData(URI destURI) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not support renaming data");
+  }
+
+  @Override
+  public boolean getPinning(LocalFileSystem localFS) throws IOException {
+    return false;
+  }
+
+  @Override
+  public void setPinning(LocalFileSystem localFS) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not support pinning");
+  }
+
+  @Override
+  public void bumpReplicaGS(long newGS) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not yet support writes");
+  }
+
+  @Override
+  public boolean breakHardLinksIfNeeded() throws IOException {
+    return false;
+  }
+
+  @Override
+  public ReplicaRecoveryInfo createInfo()
+      throws UnsupportedOperationException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not yet support writes");
+  }
+
+  @Override
+  public int compareWith(ScanInfo info) {
+    if (info.getFileRegion().equals(
+        new FileRegion(this.getBlockId(), new Path(getRemoteURI()),
+            fileOffset, this.getNumBytes(), this.getGenerationStamp()))) {
+      return 0;
+    } else {
+      return (int) (info.getBlockLength() - getNumBytes());
+    }
+  }
+
+  @Override
+  public void truncateBlock(long newLength) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not yet support truncate");
+  }
+
+  @Override
+  public void updateWithReplica(StorageLocation replicaLocation) {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not yet support update");
+  }
+
+  @Override
+  public void copyMetadata(URI destination) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not yet support copy metadata");
+  }
+
+  @Override
+  public void copyBlockdata(URI destination) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedReplica does not yet support copy data");
+  }
+
+  @VisibleForTesting
+  public void setPathHandle(PathHandle pathHandle) {
+    this.pathHandle = pathHandle;
+  }
+}

+ 136 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java

@@ -18,9 +18,16 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.File;
+import java.net.URI;
 
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathHandle;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 
 /**
@@ -42,12 +49,26 @@ public class ReplicaBuilder {
 
   private ReplicaInfo fromReplica;
 
+  private URI uri;
+  private long offset;
+  private Configuration conf;
+  private FileRegion fileRegion;
+  private FileSystem remoteFS;
+  private PathHandle pathHandle;
+  private String pathSuffix;
+  private Path pathPrefix;
+
   public ReplicaBuilder(ReplicaState state) {
     volume = null;
     writer = null;
     block = null;
     length = -1;
+    fileRegion = null;
+    conf = null;
+    fromReplica = null;
+    uri = null;
     this.state = state;
+    pathHandle = null;
   }
 
   public ReplicaBuilder setState(ReplicaState state) {
@@ -105,6 +126,58 @@ public class ReplicaBuilder {
     return this;
   }
 
+  public ReplicaBuilder setURI(URI uri) {
+    this.uri = uri;
+    return this;
+  }
+
+  public ReplicaBuilder setConf(Configuration conf) {
+    this.conf = conf;
+    return this;
+  }
+
+  public ReplicaBuilder setOffset(long offset) {
+    this.offset = offset;
+    return this;
+  }
+
+  public ReplicaBuilder setFileRegion(FileRegion fileRegion) {
+    this.fileRegion = fileRegion;
+    return this;
+  }
+
+  public ReplicaBuilder setRemoteFS(FileSystem remoteFS) {
+    this.remoteFS = remoteFS;
+    return this;
+  }
+
+  /**
+   * Set the suffix of the {@link Path} associated with the replica.
+   * Intended to be use only for {@link ProvidedReplica}s.
+   * @param suffix the path suffix.
+   * @return the builder with the path suffix set.
+   */
+  public ReplicaBuilder setPathSuffix(String suffix) {
+    this.pathSuffix = suffix;
+    return this;
+  }
+
+  /**
+   * Set the prefix of the {@link Path} associated with the replica.
+   * Intended to be use only for {@link ProvidedReplica}s.
+   * @param prefix the path prefix.
+   * @return the builder with the path prefix set.
+   */
+  public ReplicaBuilder setPathPrefix(Path prefix) {
+    this.pathPrefix = prefix;
+    return this;
+  }
+
+  public ReplicaBuilder setPathHandle(PathHandle pathHandle) {
+    this.pathHandle = pathHandle;
+    return this;
+  }
+
   public LocalReplicaInPipeline buildLocalReplicaInPipeline()
       throws IllegalArgumentException {
     LocalReplicaInPipeline info = null;
@@ -176,7 +249,7 @@ public class ReplicaBuilder {
     }
   }
 
-  private ReplicaInfo buildFinalizedReplica() throws IllegalArgumentException {
+  private LocalReplica buildFinalizedReplica() throws IllegalArgumentException {
     if (null != fromReplica &&
         fromReplica.getState() == ReplicaState.FINALIZED) {
       return new FinalizedReplica((FinalizedReplica)fromReplica);
@@ -193,7 +266,7 @@ public class ReplicaBuilder {
     }
   }
 
-  private ReplicaInfo buildRWR() throws IllegalArgumentException {
+  private LocalReplica buildRWR() throws IllegalArgumentException {
 
     if (null != fromReplica && fromReplica.getState() == ReplicaState.RWR) {
       return new ReplicaWaitingToBeRecovered(
@@ -211,7 +284,7 @@ public class ReplicaBuilder {
     }
   }
 
-  private ReplicaInfo buildRUR() throws IllegalArgumentException {
+  private LocalReplica buildRUR() throws IllegalArgumentException {
     if (null == fromReplica) {
       throw new IllegalArgumentException(
           "Missing a valid replica to recover from");
@@ -228,8 +301,54 @@ public class ReplicaBuilder {
     }
   }
 
-  public ReplicaInfo build() throws IllegalArgumentException {
-    ReplicaInfo info = null;
+  private ProvidedReplica buildProvidedFinalizedReplica()
+      throws IllegalArgumentException {
+    ProvidedReplica info = null;
+    if (fromReplica != null) {
+      throw new IllegalArgumentException("Finalized PROVIDED replica " +
+          "cannot be constructed from another replica");
+    }
+    if (fileRegion == null && uri == null &&
+        (pathPrefix == null || pathSuffix == null)) {
+      throw new IllegalArgumentException(
+          "Trying to construct a provided replica on " + volume +
+          " without enough information");
+    }
+    if (fileRegion == null) {
+      if (uri != null) {
+        info = new FinalizedProvidedReplica(blockId, uri, offset,
+            length, genStamp, pathHandle, volume, conf, remoteFS);
+      } else {
+        info = new FinalizedProvidedReplica(blockId, pathPrefix, pathSuffix,
+            offset, length, genStamp, pathHandle, volume, conf, remoteFS);
+      }
+    } else {
+      info = new FinalizedProvidedReplica(fileRegion, volume, conf, remoteFS);
+    }
+    return info;
+  }
+
+  private ProvidedReplica buildProvidedReplica()
+      throws IllegalArgumentException {
+    ProvidedReplica info = null;
+    switch(this.state) {
+    case FINALIZED:
+      info = buildProvidedFinalizedReplica();
+      break;
+    case RWR:
+    case RUR:
+    case RBW:
+    case TEMPORARY:
+    default:
+      throw new IllegalArgumentException("Unknown replica state " +
+          state + " for PROVIDED replica");
+    }
+    return info;
+  }
+
+  private LocalReplica buildLocalReplica()
+      throws IllegalArgumentException {
+    LocalReplica info = null;
     switch(this.state) {
     case FINALIZED:
       info = buildFinalizedReplica();
@@ -249,4 +368,16 @@ public class ReplicaBuilder {
     }
     return info;
   }
+
+  public ReplicaInfo build() throws IllegalArgumentException {
+
+    ReplicaInfo info = null;
+    if(volume != null && volume.getStorageType() == StorageType.PROVIDED) {
+      info = buildProvidedReplica();
+    } else {
+      info = buildLocalReplica();
+    }
+
+    return info;
+  }
 }

+ 19 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java

@@ -49,6 +49,17 @@ abstract public class ReplicaInfo extends Block
   private static final FileIoProvider DEFAULT_FILE_IO_PROVIDER =
       new FileIoProvider(null, null);
 
+  /**
+   * Constructor.
+   * @param block a block
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   */
+  ReplicaInfo(Block block, FsVolumeSpi vol) {
+    this(vol, block.getBlockId(), block.getNumBytes(),
+        block.getGenerationStamp());
+  }
+
   /**
   * Constructor
   * @param vol volume where replica is located
@@ -62,7 +73,14 @@ abstract public class ReplicaInfo extends Block
   }
   
   /**
-   * Get the volume where this replica is located on disk.
+   * Copy constructor.
+   * @param from where to copy from
+   */
+  ReplicaInfo(ReplicaInfo from) {
+    this(from, from.getVolume());
+  }
+
+  /**
    * @return the volume where this replica is located on disk
    */
   public FsVolumeSpi getVolume() {

+ 25 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.nio.channels.ServerSocketChannel;
+import java.net.BindException;
 
 /**
  * Utility class to start a datanode in a secure cluster, first obtaining 
@@ -102,7 +103,13 @@ public class SecureDataNodeStarter implements Daemon {
 
     ServerSocket ss = (socketWriteTimeout > 0) ? 
         ServerSocketChannel.open().socket() : new ServerSocket();
-    ss.bind(streamingAddr, backlogLength);
+    try {
+      ss.bind(streamingAddr, backlogLength);
+    } catch (BindException e) {
+      BindException newBe = appendMessageToBindException(e,
+          streamingAddr.toString());
+      throw newBe;
+    }
 
     // Check that we got the port we need
     if (ss.getLocalPort() != streamingAddr.getPort()) {
@@ -126,13 +133,20 @@ public class SecureDataNodeStarter implements Daemon {
     if (policy.isHttpEnabled()) {
       httpChannel = ServerSocketChannel.open();
       InetSocketAddress infoSocAddr = DataNode.getInfoAddr(conf);
-      httpChannel.socket().bind(infoSocAddr);
+      try {
+        httpChannel.socket().bind(infoSocAddr);
+      } catch (BindException e) {
+        BindException newBe = appendMessageToBindException(e,
+            infoSocAddr.toString());
+        throw newBe;
+      }
       InetSocketAddress localAddr = (InetSocketAddress) httpChannel.socket()
         .getLocalSocketAddress();
 
       if (localAddr.getPort() != infoSocAddr.getPort()) {
-        throw new RuntimeException("Unable to bind on specified info port in secure " +
-            "context. Needed " + streamingAddr.getPort() + ", got " + ss.getLocalPort());
+        throw new RuntimeException("Unable to bind on specified info port in " +
+            "secure context. Needed " + infoSocAddr.getPort() + ", got " +
+             ss.getLocalPort());
       }
       System.err.println("Successfully obtained privileged resources (streaming port = "
           + ss + " ) (http listener port = " + localAddr.getPort() +")");
@@ -149,4 +163,11 @@ public class SecureDataNodeStarter implements Daemon {
     return new SecureResources(ss, httpChannel);
   }
 
+  private static BindException appendMessageToBindException(BindException e,
+      String msg) {
+    BindException newBe = new BindException(e.getMessage() + " " + msg);
+    newBe.initCause(e.getCause());
+    newBe.setStackTrace(e.getStackTrace());
+    return newBe;
+  }
 }

+ 39 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java

@@ -64,21 +64,25 @@ public class StorageLocation
     this.storageType = storageType;
     if (uri.getScheme() == null || uri.getScheme().equals("file")) {
       // make sure all URIs that point to a file have the same scheme
-      try {
-        File uriFile = new File(uri.getPath());
-        String uriStr = uriFile.toURI().normalize().toString();
-        if (uriStr.endsWith("/")) {
-          uriStr = uriStr.substring(0, uriStr.length() - 1);
-        }
-        uri = new URI(uriStr);
-      } catch (URISyntaxException e) {
-        throw new IllegalArgumentException(
-            "URI: " + uri + " is not in the expected format");
-      }
+      uri = normalizeFileURI(uri);
     }
     baseURI = uri;
   }
 
+  public static URI normalizeFileURI(URI uri) {
+    try {
+      File uriFile = new File(uri.getPath());
+      String uriStr = uriFile.toURI().normalize().toString();
+      if (uriStr.endsWith("/")) {
+        uriStr = uriStr.substring(0, uriStr.length() - 1);
+      }
+      return new URI(uriStr);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(
+              "URI: " + uri + " is not in the expected format");
+    }
+  }
+
   public StorageType getStorageType() {
     return this.storageType;
   }
@@ -98,6 +102,16 @@ public class StorageLocation
 
   public boolean matchesStorageDirectory(StorageDirectory sd,
       String bpid) throws IOException {
+    if (sd.getStorageLocation().getStorageType() == StorageType.PROVIDED &&
+        storageType == StorageType.PROVIDED) {
+      return matchesStorageDirectory(sd);
+    }
+    if (sd.getStorageLocation().getStorageType() == StorageType.PROVIDED ||
+        storageType == StorageType.PROVIDED) {
+      // only one PROVIDED storage directory can exist; so this cannot match!
+      return false;
+    }
+    // both storage directories are local
     return this.getBpURI(bpid, Storage.STORAGE_DIR_CURRENT).normalize()
         .equals(sd.getRoot().toURI().normalize());
   }
@@ -197,6 +211,12 @@ public class StorageLocation
     if (conf == null) {
       conf = new HdfsConfiguration();
     }
+    if (storageType == StorageType.PROVIDED) {
+      // skip creation if the storage type is PROVIDED
+      Storage.LOG.info("Skipping creating directory for block pool "
+          + blockPoolID + " for PROVIDED storage location " + this);
+      return;
+    }
 
     LocalFileSystem localFS = FileSystem.getLocal(conf);
     FsPermission permission = new FsPermission(conf.get(
@@ -213,10 +233,14 @@ public class StorageLocation
 
   @Override  // Checkable
   public VolumeCheckResult check(CheckContext context) throws IOException {
-    DiskChecker.checkDir(
-        context.localFileSystem,
-        new Path(baseURI),
-        context.expectedPermission);
+    // assume provided storage locations are always healthy,
+    // and check only for local storages.
+    if (storageType != StorageType.PROVIDED) {
+      DiskChecker.checkDir(
+          context.localFileSystem,
+          new Path(baseURI),
+          context.expectedPermission);
+    }
     return VolumeCheckResult.HEALTHY;
   }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockChecksumReconstructor.java

@@ -111,7 +111,7 @@ public class StripedBlockChecksumReconstructor extends StripedReconstructor {
     // case-1) length of data bytes which is fraction of bytesPerCRC
     // case-2) length of data bytes which is less than bytesPerCRC
     if (requestedLen <= toReconstructLen) {
-      int remainingLen = (int) requestedLen;
+      int remainingLen = Math.toIntExact(requestedLen);
       outputData = Arrays.copyOf(targetBuffer.array(), remainingLen);
 
       int partialLength = remainingLen % getChecksum().getBytesPerChecksum();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java

@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.UnexpectedReplicaStateException;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
@@ -252,8 +253,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * and, in case that they are not matched, update the record or mark it
    * as corrupted.
    */
-  void checkAndUpdate(String bpid, long blockId, File diskFile,
-      File diskMetaFile, FsVolumeSpi vol) throws IOException;
+  void checkAndUpdate(String bpid, ScanInfo info) throws IOException;
 
   /**
    * @param b - the block

+ 32 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.FileIoProvider;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
 import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.checker.Checkable;
@@ -241,10 +242,11 @@ public interface FsVolumeSpi
 
     private final FsVolumeSpi volume;
 
+    private final FileRegion fileRegion;
     /**
      * Get the file's length in async block scan
      */
-    private final long blockFileLength;
+    private final long blockLength;
 
     private final static Pattern CONDENSED_PATH_REGEX =
         Pattern.compile("(?<!^)(\\\\|/){2,}");
@@ -297,10 +299,10 @@ public interface FsVolumeSpi
       this.blockId = blockId;
       String condensedVolPath =
           (vol == null || vol.getBaseURI() == null) ? null :
-            getCondensedPath(new File(vol.getBaseURI()).getAbsolutePath());
+              getCondensedPath(new File(vol.getBaseURI()).getAbsolutePath());
       this.blockSuffix = blockFile == null ? null :
-        getSuffix(blockFile, condensedVolPath);
-      this.blockFileLength = (blockFile != null) ? blockFile.length() : 0;
+              getSuffix(blockFile, condensedVolPath);
+      this.blockLength = (blockFile != null) ? blockFile.length() : 0;
       if (metaFile == null) {
         this.metaSuffix = null;
       } else if (blockFile == null) {
@@ -310,6 +312,26 @@ public interface FsVolumeSpi
             condensedVolPath + blockSuffix);
       }
       this.volume = vol;
+      this.fileRegion = null;
+    }
+
+    /**
+     * Create a ScanInfo object for a block. This constructor will examine
+     * the block data and meta-data files.
+     *
+     * @param blockId the block ID
+     * @param vol the volume that contains the block
+     * @param fileRegion the file region (for provided blocks)
+     * @param length the length of the block data
+     */
+    public ScanInfo(long blockId, FsVolumeSpi vol, FileRegion fileRegion,
+        long length) {
+      this.blockId = blockId;
+      this.blockLength = length;
+      this.volume = vol;
+      this.fileRegion = fileRegion;
+      this.blockSuffix = null;
+      this.metaSuffix = null;
     }
 
     /**
@@ -328,8 +350,8 @@ public interface FsVolumeSpi
      *
      * @return the length of the data block
      */
-    public long getBlockFileLength() {
-      return blockFileLength;
+    public long getBlockLength() {
+      return blockLength;
     }
 
     /**
@@ -399,6 +421,10 @@ public interface FsVolumeSpi
           getMetaFile().getName()) :
             HdfsConstants.GRANDFATHER_GENERATION_STAMP;
     }
+
+    public FileRegion getFileRegion() {
+      return fileRegion;
+    }
   }
 
   /**

+ 58 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -86,6 +86,7 @@ import org.apache.hadoop.hdfs.server.datanode.UnexpectedReplicaStateException;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
@@ -1088,8 +1089,10 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       // calculate and write the last crc
       checksum.calculateChunkedSums(data, 0, offset, crcs, 0);
       metaOut.write(crcs, 0, 4);
+      metaOut.close();
+      metaOut = null;
     } finally {
-      IOUtils.cleanup(null, metaOut);
+      IOUtils.closeStream(metaOut);
     }
   }
 
@@ -1511,6 +1514,13 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     }
   }
 
+  private boolean isReplicaProvided(ReplicaInfo replicaInfo) {
+    if (replicaInfo == null) {
+      return false;
+    }
+    return replicaInfo.getVolume().getStorageType() == StorageType.PROVIDED;
+  }
+
   @Override // FsDatasetSpi
   public ReplicaHandler createTemporary(StorageType storageType,
       String storageId, ExtendedBlock b, boolean isTransfer)
@@ -1529,12 +1539,14 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           isInPipeline = currentReplicaInfo.getState() == ReplicaState.TEMPORARY
               || currentReplicaInfo.getState() == ReplicaState.RBW;
           /*
-           * If the current block is old, reject.
+           * If the current block is not PROVIDED and old, reject.
            * else If transfer request, then accept it.
            * else if state is not RBW/Temporary, then reject
+           * If current block is PROVIDED, ignore the replica.
            */
-          if ((currentReplicaInfo.getGenerationStamp() >= b.getGenerationStamp())
-              || (!isTransfer && !isInPipeline)) {
+          if (((currentReplicaInfo.getGenerationStamp() >= b
+              .getGenerationStamp()) || (!isTransfer && !isInPipeline))
+              && !isReplicaProvided(currentReplicaInfo)) {
             throw new ReplicaAlreadyExistsException("Block " + b
                 + " already exists in state " + currentReplicaInfo.getState()
                 + " and thus cannot be created.");
@@ -1554,11 +1566,17 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
             + " after " + writerStopMs + " miniseconds.");
       }
 
+      // if lastFoundReplicaInfo is PROVIDED and FINALIZED,
+      // stopWriter isn't required.
+      if (isReplicaProvided(lastFoundReplicaInfo) &&
+          lastFoundReplicaInfo.getState() == ReplicaState.FINALIZED) {
+        continue;
+      }
       // Stop the previous writer
       ((ReplicaInPipeline)lastFoundReplicaInfo).stopWriter(writerStopTimeoutMs);
     } while (true);
-
-    if (lastFoundReplicaInfo != null) {
+    if (lastFoundReplicaInfo != null
+        && !isReplicaProvided(lastFoundReplicaInfo)) {
       // Old blockfile should be deleted synchronously as it might collide
       // with the new block if allocated in same volume.
       // Do the deletion outside of lock as its DISK IO.
@@ -1750,6 +1768,10 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
       Set<String> missingVolumesReported = new HashSet<>();
       for (ReplicaInfo b : volumeMap.replicas(bpid)) {
+        // skip PROVIDED replicas.
+        if (b.getVolume().getStorageType() == StorageType.PROVIDED) {
+          continue;
+        }
         String volStorageID = b.getVolume().getStorageID();
         if (!builders.containsKey(volStorageID)) {
           if (!missingVolumesReported.contains(volStorageID)) {
@@ -1885,7 +1907,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     try (AutoCloseableLock lock = datasetLock.acquire()) {
       r = volumeMap.get(bpid, blockId);
     }
-
     if (r != null) {
       if (r.blockDataExists()) {
         return r;
@@ -2238,13 +2259,20 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
    * @param vol Volume of the block file
    */
   @Override
-  public void checkAndUpdate(String bpid, long blockId, File diskFile,
-      File diskMetaFile, FsVolumeSpi vol) throws IOException {
+  public void checkAndUpdate(String bpid, ScanInfo scanInfo)
+      throws IOException {
+
+    long blockId = scanInfo.getBlockId();
+    File diskFile = scanInfo.getBlockFile();
+    File diskMetaFile = scanInfo.getMetaFile();
+    FsVolumeSpi vol = scanInfo.getVolume();
+
     Block corruptBlock = null;
     ReplicaInfo memBlockInfo;
     try (AutoCloseableLock lock = datasetLock.acquire()) {
       memBlockInfo = volumeMap.get(bpid, blockId);
-      if (memBlockInfo != null && memBlockInfo.getState() != ReplicaState.FINALIZED) {
+      if (memBlockInfo != null &&
+          memBlockInfo.getState() != ReplicaState.FINALIZED) {
         // Block is not finalized - ignore the difference
         return;
       }
@@ -2259,6 +2287,26 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           Block.getGenerationStamp(diskMetaFile.getName()) :
           HdfsConstants.GRANDFATHER_GENERATION_STAMP;
 
+      if (vol.getStorageType() == StorageType.PROVIDED) {
+        if (memBlockInfo == null) {
+          // replica exists on provided store but not in memory
+          ReplicaInfo diskBlockInfo =
+              new ReplicaBuilder(ReplicaState.FINALIZED)
+              .setFileRegion(scanInfo.getFileRegion())
+              .setFsVolume(vol)
+              .setConf(conf)
+              .build();
+
+          volumeMap.add(bpid, diskBlockInfo);
+          LOG.warn("Added missing block to memory " + diskBlockInfo);
+        } else {
+          // replica exists in memory but not in the provided store
+          volumeMap.remove(bpid, blockId);
+          LOG.warn("Deleting missing provided block " + memBlockInfo);
+        }
+        return;
+      }
+
       if (!diskFileExists) {
         if (memBlockInfo == null) {
           // Block file does not exist and block does not exist in memory
@@ -3034,7 +3082,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           newReplicaInfo =
               replicaState.getLazyPersistVolume().activateSavedReplica(bpid,
                   replicaInfo, replicaState);
-
           // Update the volumeMap entry.
           volumeMap.add(bpid, newReplicaInfo);
 

+ 23 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileDescriptor;
 import java.io.FileInputStream;
@@ -32,10 +34,12 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.DataChecksum;
 
 /** Utility methods. */
 @InterfaceAudience.Private
@@ -44,6 +48,22 @@ public class FsDatasetUtil {
     return f.getName().endsWith(DatanodeUtil.UNLINK_BLOCK_SUFFIX);
   }
 
+  public static byte[] createNullChecksumByteArray() {
+    DataChecksum csum =
+        DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 512);
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    DataOutputStream dataOut = new DataOutputStream(out);
+    try {
+      BlockMetadataHeader.writeHeader(dataOut, csum);
+      dataOut.close();
+    } catch (IOException e) {
+      FsVolumeImpl.LOG.error(
+          "Exception in creating null checksum stream: " + e);
+      return null;
+    }
+    return out.toByteArray();
+  }
+
   static File getOrigFile(File unlinkTmpFile) {
     final String name = unlinkTmpFile.getName();
     if (!name.endsWith(DatanodeUtil.UNLINK_BLOCK_SUFFIX)) {
@@ -135,8 +155,9 @@ public class FsDatasetUtil {
    * Compute the checksum for a block file that does not already have
    * its checksum computed, and save it to dstMeta file.
    */
-  public static void computeChecksum(File srcMeta, File dstMeta, File blockFile,
-      int smallBufferSize, Configuration conf) throws IOException {
+  public static void computeChecksum(File srcMeta, File dstMeta,
+      File blockFile, int smallBufferSize, Configuration conf)
+          throws IOException {
     Preconditions.checkNotNull(srcMeta);
     Preconditions.checkNotNull(dstMeta);
     Preconditions.checkNotNull(blockFile);

+ 13 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java

@@ -154,18 +154,24 @@ public class FsVolumeImpl implements FsVolumeSpi {
     this.reservedForReplicas = new AtomicLong(0L);
     this.storageLocation = sd.getStorageLocation();
     this.currentDir = sd.getCurrentDir();
-    File parent = currentDir.getParentFile();
-    this.usage = new DF(parent, conf);
     this.storageType = storageLocation.getStorageType();
     this.reserved = conf.getLong(DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY
         + "." + StringUtils.toLowerCase(storageType.toString()), conf.getLong(
         DFSConfigKeys.DFS_DATANODE_DU_RESERVED_KEY,
         DFSConfigKeys.DFS_DATANODE_DU_RESERVED_DEFAULT));
     this.configuredCapacity = -1;
+    if (currentDir != null) {
+      File parent = currentDir.getParentFile();
+      this.usage = new DF(parent, conf);
+      cacheExecutor = initializeCacheExecutor(parent);
+      this.metrics = DataNodeVolumeMetrics.create(conf, parent.getPath());
+    } else {
+      this.usage = null;
+      cacheExecutor = null;
+      this.metrics = null;
+    }
     this.conf = conf;
     this.fileIoProvider = fileIoProvider;
-    cacheExecutor = initializeCacheExecutor(parent);
-    this.metrics = DataNodeVolumeMetrics.create(conf, getBaseURI().getPath());
   }
 
   protected ThreadPoolExecutor initializeCacheExecutor(File parent) {
@@ -440,7 +446,8 @@ public class FsVolumeImpl implements FsVolumeSpi {
   /**
    * Unplanned Non-DFS usage, i.e. Extra usage beyond reserved.
    *
-   * @return
+   * @return Disk usage excluding space used by HDFS and excluding space
+   * reserved for blocks open for write.
    * @throws IOException
    */
   public long getNonDfsUsed() throws IOException {
@@ -518,7 +525,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
   public String[] getBlockPoolList() {
     return bpSlices.keySet().toArray(new String[bpSlices.keySet().size()]);   
   }
-    
+
   /**
    * Temporary files. They get moved to the finalized block directory when
    * the block is finalized.

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.datanode.FileIoProvider;
 
@@ -67,6 +68,11 @@ public class FsVolumeImplBuilder {
   }
 
   FsVolumeImpl build() throws IOException {
+    if (sd.getStorageLocation().getStorageType() == StorageType.PROVIDED) {
+      return new ProvidedVolumeImpl(dataset, storageID, sd,
+          fileIoProvider != null ? fileIoProvider :
+            new FileIoProvider(null, null), conf);
+    }
     return new FsVolumeImpl(
         dataset, storageID, sd,
         fileIoProvider != null ? fileIoProvider :

+ 718 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java

@@ -0,0 +1,718 @@
+/**
+ * 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.server.datanode.fsdataset.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathHandle;
+import org.apache.hadoop.fs.RawPathHandle;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipeline;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.FileIoProvider;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaBuilder;
+import org.apache.hadoop.util.Timer;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+import org.apache.hadoop.util.AutoCloseableLock;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.ObjectReader;
+import org.codehaus.jackson.map.ObjectWriter;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Time;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LOAD_RETRIES;
+
+/**
+ * This class is used to create provided volumes.
+ */
+@InterfaceAudience.Private
+class ProvidedVolumeImpl extends FsVolumeImpl {
+
+  /**
+   * Get a suffix of the full path, excluding the given prefix.
+   *
+   * @param prefix a prefix of the path.
+   * @param fullPath the full path whose suffix is needed.
+   * @return the suffix of the path, which when resolved against {@code prefix}
+   *         gets back the {@code fullPath}.
+   */
+  @VisibleForTesting
+  protected static String getSuffix(final Path prefix, final Path fullPath) {
+    String prefixStr = prefix.toString();
+    String pathStr = fullPath.toString();
+    if (!pathStr.startsWith(prefixStr)) {
+      LOG.debug("Path {} is not a prefix of the path {}", prefix, fullPath);
+      return pathStr;
+    }
+    String suffix = pathStr.replaceFirst("^" + prefixStr, "");
+    if (suffix.startsWith("/")) {
+      suffix = suffix.substring(1);
+    }
+    return suffix;
+  }
+
+  /**
+   * Class to keep track of the capacity usage statistics for provided volumes.
+   */
+  public static class ProvidedVolumeDF {
+
+    private AtomicLong used = new AtomicLong();
+
+    public long getSpaceUsed() {
+      return used.get();
+    }
+
+    public void decDfsUsed(long value) {
+      used.addAndGet(-value);
+    }
+
+    public void incDfsUsed(long value) {
+      used.addAndGet(value);
+    }
+
+    public long getCapacity() {
+      return getSpaceUsed();
+    }
+  }
+
+  static class ProvidedBlockPoolSlice {
+    private ProvidedVolumeImpl providedVolume;
+
+    private BlockAliasMap<FileRegion> aliasMap;
+    private Configuration conf;
+    private String bpid;
+    private ReplicaMap bpVolumeMap;
+    private ProvidedVolumeDF df;
+    private AtomicLong numOfBlocks = new AtomicLong();
+    private int numRetries;
+
+    ProvidedBlockPoolSlice(String bpid, ProvidedVolumeImpl volume,
+        Configuration conf) {
+      this.providedVolume = volume;
+      bpVolumeMap = new ReplicaMap(new AutoCloseableLock());
+      Class<? extends BlockAliasMap> fmt =
+          conf.getClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+              TextFileRegionAliasMap.class, BlockAliasMap.class);
+      aliasMap = ReflectionUtils.newInstance(fmt, conf);
+      this.conf = conf;
+      this.bpid = bpid;
+      this.df = new ProvidedVolumeDF();
+      bpVolumeMap.initBlockPool(bpid);
+      this.numRetries = conf.getInt(DFS_PROVIDED_ALIASMAP_LOAD_RETRIES, 0);
+      LOG.info("Created alias map using class: " + aliasMap.getClass());
+    }
+
+    BlockAliasMap<FileRegion> getBlockAliasMap() {
+      return aliasMap;
+    }
+
+    @VisibleForTesting
+    void setFileRegionProvider(BlockAliasMap<FileRegion> blockAliasMap) {
+      this.aliasMap = blockAliasMap;
+    }
+
+    void fetchVolumeMap(ReplicaMap volumeMap,
+        RamDiskReplicaTracker ramDiskReplicaMap, FileSystem remoteFS)
+        throws IOException {
+      BlockAliasMap.Reader<FileRegion> reader = null;
+      int tries = 1;
+      do {
+        try {
+          reader = aliasMap.getReader(null, bpid);
+          break;
+        } catch (IOException e) {
+          tries++;
+          reader = null;
+        }
+      } while (tries <= numRetries);
+
+      if (reader == null) {
+        LOG.error("Got null reader from BlockAliasMap " + aliasMap
+            + "; no blocks will be populated");
+        return;
+      }
+      Path blockPrefixPath = new Path(providedVolume.getBaseURI());
+      for (FileRegion region : reader) {
+        if (containsBlock(providedVolume.baseURI,
+            region.getProvidedStorageLocation().getPath().toUri())) {
+          String blockSuffix = getSuffix(blockPrefixPath,
+              new Path(region.getProvidedStorageLocation().getPath().toUri()));
+          PathHandle pathHandle = null;
+          if (region.getProvidedStorageLocation().getNonce().length > 0) {
+            pathHandle = new RawPathHandle(ByteBuffer
+                .wrap(region.getProvidedStorageLocation().getNonce()));
+          }
+          ReplicaInfo newReplica = new ReplicaBuilder(ReplicaState.FINALIZED)
+              .setBlockId(region.getBlock().getBlockId())
+              .setPathPrefix(blockPrefixPath)
+              .setPathSuffix(blockSuffix)
+              .setOffset(region.getProvidedStorageLocation().getOffset())
+              .setLength(region.getBlock().getNumBytes())
+              .setGenerationStamp(region.getBlock().getGenerationStamp())
+              .setPathHandle(pathHandle)
+              .setFsVolume(providedVolume)
+              .setConf(conf)
+              .setRemoteFS(remoteFS)
+              .build();
+          ReplicaInfo oldReplica =
+              volumeMap.get(bpid, newReplica.getBlockId());
+          if (oldReplica == null) {
+            volumeMap.add(bpid, newReplica);
+            bpVolumeMap.add(bpid, newReplica);
+            incrNumBlocks();
+            incDfsUsed(region.getBlock().getNumBytes());
+          } else {
+            LOG.warn("A block with id " + newReplica.getBlockId()
+                + " exists locally. Skipping PROVIDED replica");
+          }
+        }
+      }
+    }
+
+    private void incrNumBlocks() {
+      numOfBlocks.incrementAndGet();
+    }
+
+    public boolean isEmpty() {
+      return bpVolumeMap.replicas(bpid).size() == 0;
+    }
+
+    public void shutdown(BlockListAsLongs blocksListsAsLongs) {
+      // nothing to do!
+    }
+
+    public void compileReport(LinkedList<ScanInfo> report,
+        ReportCompiler reportCompiler)
+            throws IOException, InterruptedException {
+      /* refresh the aliasMap and return the list of blocks found.
+       * the assumption here is that the block ids in the external
+       * block map, after the refresh, are consistent with those
+       * from before the refresh, i.e., for blocks which did not change,
+       * the ids remain the same.
+       */
+      aliasMap.refresh();
+      BlockAliasMap.Reader<FileRegion> reader = aliasMap.getReader(null, bpid);
+      for (FileRegion region : reader) {
+        reportCompiler.throttle();
+        report.add(new ScanInfo(region.getBlock().getBlockId(),
+            providedVolume, region,
+            region.getProvidedStorageLocation().getLength()));
+      }
+    }
+
+    public long getNumOfBlocks() {
+      return numOfBlocks.get();
+    }
+
+    long getDfsUsed() throws IOException {
+      return df.getSpaceUsed();
+    }
+
+    void incDfsUsed(long value) {
+      df.incDfsUsed(value);
+    }
+  }
+
+  private URI baseURI;
+  private final Map<String, ProvidedBlockPoolSlice> bpSlices =
+      new ConcurrentHashMap<String, ProvidedBlockPoolSlice>();
+
+  private ProvidedVolumeDF df;
+  // the remote FileSystem to which this ProvidedVolume points to.
+  private FileSystem remoteFS;
+
+  ProvidedVolumeImpl(FsDatasetImpl dataset, String storageID,
+      StorageDirectory sd, FileIoProvider fileIoProvider,
+      Configuration conf) throws IOException {
+    super(dataset, storageID, sd, fileIoProvider, conf);
+    assert getStorageLocation().getStorageType() == StorageType.PROVIDED:
+      "Only provided storages must use ProvidedVolume";
+
+    baseURI = getStorageLocation().getUri();
+    df = new ProvidedVolumeDF();
+    remoteFS = FileSystem.get(baseURI, conf);
+  }
+
+  @Override
+  public String[] getBlockPoolList() {
+    return bpSlices.keySet().toArray(new String[bpSlices.keySet().size()]);
+  }
+
+  @Override
+  public long getCapacity() {
+    try {
+      // default to whatever is the space used!
+      return getDfsUsed();
+    } catch (IOException e) {
+      LOG.warn("Exception when trying to get capacity of ProvidedVolume: {}",
+          e);
+    }
+    return 0L;
+  }
+
+  @Override
+  public long getDfsUsed() throws IOException {
+    long dfsUsed = 0;
+    synchronized(getDataset()) {
+      for(ProvidedBlockPoolSlice s : bpSlices.values()) {
+        dfsUsed += s.getDfsUsed();
+      }
+    }
+    return dfsUsed;
+  }
+
+  @Override
+  long getBlockPoolUsed(String bpid) throws IOException {
+    return getProvidedBlockPoolSlice(bpid).getDfsUsed();
+  }
+
+  @Override
+  public long getAvailable() throws IOException {
+    long remaining = getCapacity() - getDfsUsed();
+    // do not report less than 0 remaining space for PROVIDED storage
+    // to prevent marking it as over capacity on NN
+    if (remaining < 0L) {
+      LOG.warn("Volume {} has less than 0 available space", this);
+      return 0L;
+    }
+    return remaining;
+  }
+
+  @Override
+  long getActualNonDfsUsed() throws IOException {
+    return 0L;
+  }
+
+  @Override
+  public long getNonDfsUsed() throws IOException {
+    return 0L;
+  }
+
+  @Override
+  long getNumBlocks() {
+    long numBlocks = 0;
+    for (ProvidedBlockPoolSlice s : bpSlices.values()) {
+      numBlocks += s.getNumOfBlocks();
+    }
+    return numBlocks;
+  }
+
+  @Override
+  void incDfsUsedAndNumBlocks(String bpid, long value) {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public URI getBaseURI() {
+    return baseURI;
+  }
+
+  @Override
+  public File getFinalizedDir(String bpid) throws IOException {
+    return null;
+  }
+
+  @Override
+  public void reserveSpaceForReplica(long bytesToReserve) {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public void releaseReservedSpace(long bytesToRelease) {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  private static final ObjectWriter WRITER =
+      new ObjectMapper().writerWithDefaultPrettyPrinter();
+  private static final ObjectReader READER =
+      new ObjectMapper().reader(ProvidedBlockIteratorState.class);
+
+  private static class ProvidedBlockIteratorState {
+    ProvidedBlockIteratorState() {
+      iterStartMs = Time.now();
+      lastSavedMs = iterStartMs;
+      atEnd = false;
+      lastBlockId = -1;
+    }
+
+    // The wall-clock ms since the epoch at which this iterator was last saved.
+    @JsonProperty
+    private long lastSavedMs;
+
+    // The wall-clock ms since the epoch at which this iterator was created.
+    @JsonProperty
+    private long iterStartMs;
+
+    @JsonProperty
+    private boolean atEnd;
+
+    // The id of the last block read when the state of the iterator is saved.
+    // This implementation assumes that provided blocks are returned
+    // in sorted order of the block ids.
+    @JsonProperty
+    private long lastBlockId;
+  }
+
+  private class ProviderBlockIteratorImpl
+      implements FsVolumeSpi.BlockIterator {
+
+    private String bpid;
+    private String name;
+    private BlockAliasMap<FileRegion> blockAliasMap;
+    private Iterator<FileRegion> blockIterator;
+    private ProvidedBlockIteratorState state;
+
+    ProviderBlockIteratorImpl(String bpid, String name,
+        BlockAliasMap<FileRegion> blockAliasMap) {
+      this.bpid = bpid;
+      this.name = name;
+      this.blockAliasMap = blockAliasMap;
+      rewind();
+    }
+
+    @Override
+    public void close() throws IOException {
+      blockAliasMap.close();
+    }
+
+    @Override
+    public ExtendedBlock nextBlock() throws IOException {
+      if (null == blockIterator || !blockIterator.hasNext()) {
+        return null;
+      }
+      FileRegion nextRegion = null;
+      while (null == nextRegion && blockIterator.hasNext()) {
+        FileRegion temp = blockIterator.next();
+        if (temp.getBlock().getBlockId() < state.lastBlockId) {
+          continue;
+        }
+        nextRegion = temp;
+      }
+      if (null == nextRegion) {
+        return null;
+      }
+      state.lastBlockId = nextRegion.getBlock().getBlockId();
+      return new ExtendedBlock(bpid, nextRegion.getBlock());
+    }
+
+    @Override
+    public boolean atEnd() {
+      return blockIterator != null ? !blockIterator.hasNext(): true;
+    }
+
+    @Override
+    public void rewind() {
+      BlockAliasMap.Reader<FileRegion> reader = null;
+      try {
+        reader = blockAliasMap.getReader(null, bpid);
+      } catch (IOException e) {
+        LOG.warn("Exception in getting reader from provided alias map");
+      }
+      if (reader != null) {
+        blockIterator = reader.iterator();
+      } else {
+        blockIterator = null;
+      }
+      state = new ProvidedBlockIteratorState();
+    }
+
+    @Override
+    public void save() throws IOException {
+      // We do not persist the state of this iterator locally.
+      // We just re-scan provided volumes as necessary.
+      state.lastSavedMs = Time.now();
+    }
+
+    @Override
+    public void setMaxStalenessMs(long maxStalenessMs) {
+      // do not use max staleness
+    }
+
+    @Override
+    public long getIterStartMs() {
+      return state.iterStartMs;
+    }
+
+    @Override
+    public long getLastSavedMs() {
+      return state.lastSavedMs;
+    }
+
+    @Override
+    public String getBlockPoolId() {
+      return bpid;
+    }
+
+    public void load() throws IOException {
+      // on load, we just rewind the iterator for provided volumes.
+      rewind();
+      LOG.trace("load({}, {}): loaded iterator {}: {}", getStorageID(),
+          bpid, name, WRITER.writeValueAsString(state));
+    }
+  }
+
+  @Override
+  public BlockIterator newBlockIterator(String bpid, String name) {
+    return new ProviderBlockIteratorImpl(bpid, name,
+        bpSlices.get(bpid).getBlockAliasMap());
+  }
+
+  @Override
+  public BlockIterator loadBlockIterator(String bpid, String name)
+      throws IOException {
+    ProviderBlockIteratorImpl iter = new ProviderBlockIteratorImpl(bpid, name,
+        bpSlices.get(bpid).getBlockAliasMap());
+    iter.load();
+    return iter;
+  }
+
+  @Override
+  ReplicaInfo addFinalizedBlock(String bpid, Block b,
+      ReplicaInfo replicaInfo, long bytesReserved) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public VolumeCheckResult check(VolumeCheckContext ignored)
+      throws DiskErrorException {
+    return VolumeCheckResult.HEALTHY;
+  }
+
+  @Override
+  void getVolumeMap(ReplicaMap volumeMap,
+      final RamDiskReplicaTracker ramDiskReplicaMap)
+          throws IOException {
+    LOG.info("Creating volumemap for provided volume " + this);
+    for(ProvidedBlockPoolSlice s : bpSlices.values()) {
+      s.fetchVolumeMap(volumeMap, ramDiskReplicaMap, remoteFS);
+    }
+  }
+
+  private ProvidedBlockPoolSlice getProvidedBlockPoolSlice(String bpid)
+      throws IOException {
+    ProvidedBlockPoolSlice bp = bpSlices.get(bpid);
+    if (bp == null) {
+      throw new IOException("block pool " + bpid + " is not found");
+    }
+    return bp;
+  }
+
+  @Override
+  void getVolumeMap(String bpid, ReplicaMap volumeMap,
+      final RamDiskReplicaTracker ramDiskReplicaMap)
+          throws IOException {
+    getProvidedBlockPoolSlice(bpid).fetchVolumeMap(volumeMap, ramDiskReplicaMap,
+        remoteFS);
+  }
+
+  @VisibleForTesting
+  BlockAliasMap<FileRegion> getBlockFormat(String bpid) throws IOException {
+    return getProvidedBlockPoolSlice(bpid).getBlockAliasMap();
+  }
+
+  @Override
+  public String toString() {
+    return this.baseURI.toString();
+  }
+
+  @Override
+  void addBlockPool(String bpid, Configuration conf) throws IOException {
+    addBlockPool(bpid, conf, null);
+  }
+
+  @Override
+  void addBlockPool(String bpid, Configuration conf, Timer timer)
+      throws IOException {
+    LOG.info("Adding block pool " + bpid +
+        " to volume with id " + getStorageID());
+    ProvidedBlockPoolSlice bp;
+    bp = new ProvidedBlockPoolSlice(bpid, this, conf);
+    bpSlices.put(bpid, bp);
+  }
+
+  void shutdown() {
+    if (cacheExecutor != null) {
+      cacheExecutor.shutdown();
+    }
+    Set<Entry<String, ProvidedBlockPoolSlice>> set = bpSlices.entrySet();
+    for (Entry<String, ProvidedBlockPoolSlice> entry : set) {
+      entry.getValue().shutdown(null);
+    }
+  }
+
+  @Override
+  void shutdownBlockPool(String bpid, BlockListAsLongs blocksListsAsLongs) {
+    ProvidedBlockPoolSlice bp = bpSlices.get(bpid);
+    if (bp != null) {
+      bp.shutdown(blocksListsAsLongs);
+    }
+    bpSlices.remove(bpid);
+  }
+
+  @Override
+  boolean isBPDirEmpty(String bpid) throws IOException {
+    return getProvidedBlockPoolSlice(bpid).isEmpty();
+  }
+
+  @Override
+  void deleteBPDirectories(String bpid, boolean force) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public LinkedList<ScanInfo> compileReport(String bpid,
+      LinkedList<ScanInfo> report, ReportCompiler reportCompiler)
+      throws InterruptedException, IOException {
+    LOG.info("Compiling report for volume: " + this + " bpid " + bpid);
+    if(bpSlices.containsKey(bpid)) {
+      bpSlices.get(bpid).compileReport(report, reportCompiler);
+    }
+    return report;
+  }
+
+  @Override
+  public ReplicaInPipeline append(String bpid, ReplicaInfo replicaInfo,
+      long newGS, long estimateBlockLen) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public ReplicaInPipeline createRbw(ExtendedBlock b) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public ReplicaInPipeline convertTemporaryToRbw(ExtendedBlock b,
+      ReplicaInfo temp) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public ReplicaInPipeline createTemporary(ExtendedBlock b)
+      throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public ReplicaInPipeline updateRURCopyOnTruncate(ReplicaInfo rur,
+      String bpid, long newBlockId, long recoveryId, long newlength)
+          throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public ReplicaInfo moveBlockToTmpLocation(ExtendedBlock block,
+      ReplicaInfo replicaInfo, int smallBufferSize,
+      Configuration conf) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  @Override
+  public File[] copyBlockToLazyPersistLocation(String bpId, long blockId,
+      long genStamp, ReplicaInfo replicaInfo, int smallBufferSize,
+      Configuration conf) throws IOException {
+    throw new UnsupportedOperationException(
+        "ProvidedVolume does not yet support writes");
+  }
+
+  private static URI getAbsoluteURI(URI uri) {
+    if (!uri.isAbsolute()) {
+      // URI is not absolute implies it is for a local file
+      // normalize the URI
+      return StorageLocation.normalizeFileURI(uri);
+    } else {
+      return uri;
+    }
+  }
+  /**
+   * @param volumeURI URI of the volume
+   * @param blockURI URI of the block
+   * @return true if the {@code blockURI} can belong to the volume or both URIs
+   * are null.
+   */
+  @VisibleForTesting
+  public static boolean containsBlock(URI volumeURI, URI blockURI) {
+    if (volumeURI == null && blockURI == null){
+      return true;
+    }
+    if (volumeURI == null || blockURI == null) {
+      return false;
+    }
+    volumeURI = getAbsoluteURI(volumeURI);
+    blockURI = getAbsoluteURI(blockURI);
+    return !volumeURI.relativize(blockURI).equals(blockURI);
+  }
+
+  @VisibleForTesting
+  BlockAliasMap<FileRegion> getFileRegionProvider(String bpid) throws
+      IOException {
+    return getProvidedBlockPoolSlice(bpid).getBlockAliasMap();
+  }
+
+  @VisibleForTesting
+  void setFileRegionProvider(String bpid,
+      BlockAliasMap<FileRegion> blockAliasMap) throws IOException {
+    ProvidedBlockPoolSlice bp = bpSlices.get(bpid);
+    if (bp == null) {
+      throw new IOException("block pool " + bpid + " is not found");
+    }
+    bp.setFileRegionProvider(blockAliasMap);
+  }
+}

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java

@@ -64,6 +64,12 @@ public interface FederationMBean {
    */
   long getRemainingCapacity();
 
+  /**
+   * Get the total remote storage capacity mounted in the federated cluster.
+   * @return Remote capacity of the federated cluster.
+   */
+  long getProvidedSpace();
+
   /**
    * Get the number of nameservices in the federation.
    * @return Number of nameservices in the federation.

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java

@@ -271,6 +271,11 @@ public class FederationMetrics implements FederationMBean {
     return getNameserviceAggregatedLong(MembershipStats::getAvailableSpace);
   }
 
+  @Override
+  public long getProvidedSpace() {
+    return getNameserviceAggregatedLong(MembershipStats::getProvidedSpace);
+  }
+
   @Override
   public long getUsedCapacity() {
     return getTotalCapacity() - getRemainingCapacity();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java

@@ -34,7 +34,7 @@ import org.apache.hadoop.metrics2.lib.MutableRate;
  * Implementation of the RPC metrics collector.
  */
 @Metrics(name = "RouterRPCActivity", about = "Router RPC Activity",
-    context = "router")
+    context = "dfs")
 public class FederationRPCMetrics implements FederationRPCMBean {
 
   private final MetricsRegistry registry = new MetricsRegistry("router");

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java

@@ -168,6 +168,11 @@ public class NamenodeBeanMetrics
     return getFederationMetrics().getTotalCapacity();
   }
 
+  @Override
+  public long getProvidedCapacity() {
+    return getFederationMetrics().getProvidedSpace();
+  }
+
   @Override
   public String getSafemode() {
     // We assume that the global federated view is never in safe mode
@@ -449,6 +454,11 @@ public class NamenodeBeanMetrics
     return this.getUsed();
   }
 
+  @Override
+  public long getProvidedCapacityTotal() {
+    return getProvidedCapacity();
+  }
+
   @Override
   public long getFilesTotal() {
     return getFederationMetrics().getNumFiles();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/StateStoreMetrics.java

@@ -38,7 +38,7 @@ import com.google.common.annotations.VisibleForTesting;
  * Implementations of the JMX interface for the State Store metrics.
  */
 @Metrics(name = "StateStoreActivity", about = "Router metrics",
-    context = "router")
+    context = "dfs")
 public final class StateStoreMetrics implements StateStoreMBean {
 
   private final MetricsRegistry registry = new MetricsRegistry("router");

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java

@@ -236,6 +236,7 @@ public class MembershipNamenodeResolver
           report.getNumOfBlocksPendingDeletion());
       stats.setAvailableSpace(report.getAvailableSpace());
       stats.setTotalSpace(report.getTotalSpace());
+      stats.setProvidedSpace(report.getProvidedSpace());
       stats.setNumOfDecommissioningDatanodes(
           report.getNumDecommissioningDatanodes());
       stats.setNumOfActiveDatanodes(report.getNumLiveDatanodes());

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java

@@ -58,6 +58,7 @@ public class NamenodeStatusReport {
   private long numOfBlocksUnderReplicated = -1;
   private long numOfBlocksPendingDeletion = -1;
   private long totalSpace = -1;
+  private long providedSpace = -1;
 
   /** If the fields are valid. */
   private boolean registrationValid = false;
@@ -296,7 +297,7 @@ public class NamenodeStatusReport {
   public void setNamesystemInfo(long available, long total,
       long numFiles, long numBlocks, long numBlocksMissing,
       long numBlocksPendingReplication, long numBlocksUnderReplicated,
-      long numBlocksPendingDeletion) {
+      long numBlocksPendingDeletion, long providedSpace) {
     this.totalSpace = total;
     this.availableSpace = available;
     this.numOfBlocks = numBlocks;
@@ -306,6 +307,7 @@ public class NamenodeStatusReport {
     this.numOfBlocksPendingDeletion = numBlocksPendingDeletion;
     this.numOfFiles = numFiles;
     this.statsValid = true;
+    this.providedSpace = providedSpace;
   }
 
   /**
@@ -344,6 +346,14 @@ public class NamenodeStatusReport {
     return this.availableSpace;
   }
 
+  /**
+   * Get the space occupied by provided storage.
+   *
+   * @return the provided capacity.
+   */
+  public long getProvidedSpace() {
+    return this.providedSpace;
+  }
   /**
    * Get the number of missing blocks.
    *

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java

@@ -350,7 +350,8 @@ public class NamenodeHeartbeatService extends PeriodicService {
                 jsonObject.getLong("MissingBlocks"),
                 jsonObject.getLong("PendingReplicationBlocks"),
                 jsonObject.getLong("UnderReplicatedBlocks"),
-                jsonObject.getLong("PendingDeletionBlocks"));
+                jsonObject.getLong("PendingDeletionBlocks"),
+                jsonObject.getLong("ProvidedCapacityTotal"));
           }
         }
       }

+ 71 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java

@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
@@ -35,9 +38,12 @@ import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableE
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC.Server;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.AbstractService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -65,6 +71,14 @@ public class RouterAdminServer extends AbstractService
   private final Server adminServer;
   private final InetSocketAddress adminAddress;
 
+  /**
+   * Permission related info used for constructing new router permission
+   * checker instance.
+   */
+  private static String routerOwner;
+  private static String superGroup;
+  private static boolean isPermissionEnabled;
+
   public RouterAdminServer(Configuration conf, Router router)
       throws IOException {
     super(RouterAdminServer.class.getName());
@@ -96,6 +110,7 @@ public class RouterAdminServer extends AbstractService
     LOG.info("Admin server binding to {}:{}",
         bindHost, confRpcAddress.getPort());
 
+    initializePermissionSettings(this.conf);
     this.adminServer = new RPC.Builder(this.conf)
         .setProtocol(RouterAdminProtocolPB.class)
         .setInstance(clientNNPbService)
@@ -112,6 +127,22 @@ public class RouterAdminServer extends AbstractService
     router.setAdminServerAddress(this.adminAddress);
   }
 
+  /**
+   * Initialize permission related settings.
+   *
+   * @param routerConf
+   * @throws IOException
+   */
+  private static void initializePermissionSettings(Configuration routerConf)
+      throws IOException {
+    routerOwner = UserGroupInformation.getCurrentUser().getShortUserName();
+    superGroup = routerConf.get(
+        DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY,
+        DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT);
+    isPermissionEnabled = routerConf.getBoolean(DFS_PERMISSIONS_ENABLED_KEY,
+        DFS_PERMISSIONS_ENABLED_DEFAULT);
+  }
+
   /** Allow access to the client RPC server for testing. */
   @VisibleForTesting
   Server getAdminServer() {
@@ -180,4 +211,44 @@ public class RouterAdminServer extends AbstractService
       GetMountTableEntriesRequest request) throws IOException {
     return getMountTableStore().getMountTableEntries(request);
   }
+
+  /**
+   * Get a new permission checker used for making mount table access
+   * control. This method will be invoked during each RPC call in router
+   * admin server.
+   *
+   * @return Router permission checker
+   * @throws AccessControlException
+   */
+  public static RouterPermissionChecker getPermissionChecker()
+      throws AccessControlException {
+    if (!isPermissionEnabled) {
+      return null;
+    }
+
+    try {
+      return new RouterPermissionChecker(routerOwner, superGroup,
+          NameNode.getRemoteUser());
+    } catch (IOException e) {
+      throw new AccessControlException(e);
+    }
+  }
+
+  /**
+   * Get super user name.
+   *
+   * @return String super user name.
+   */
+  public static String getSuperUser() {
+    return routerOwner;
+  }
+
+  /**
+   * Get super group name.
+   *
+   * @return String super group name.
+   */
+  public static String getSuperGroup(){
+    return superGroup;
+  }
 }

+ 82 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterPermissionChecker.java

@@ -0,0 +1,82 @@
+/**
+ * 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.server.federation.router;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * Class that helps in checking permissions in Router-based federation.
+ */
+public class RouterPermissionChecker extends FSPermissionChecker {
+  static final Log LOG = LogFactory.getLog(RouterPermissionChecker.class);
+
+  /** Mount table default permission. */
+  public static final short MOUNT_TABLE_PERMISSION_DEFAULT = 00755;
+
+  public RouterPermissionChecker(String routerOwner, String supergroup,
+      UserGroupInformation callerUgi) {
+    super(routerOwner, supergroup, callerUgi, null);
+  }
+
+  /**
+   * Whether a mount table entry can be accessed by the current context.
+   *
+   * @param mountTable
+   *          MountTable being accessed
+   * @param access
+   *          type of action being performed on the cache pool
+   * @throws AccessControlException
+   *           if mount table cannot be accessed
+   */
+  public void checkPermission(MountTable mountTable, FsAction access)
+      throws AccessControlException {
+    if (isSuperUser()) {
+      return;
+    }
+
+    FsPermission mode = mountTable.getMode();
+    if (getUser().equals(mountTable.getOwnerName())
+        && mode.getUserAction().implies(access)) {
+      return;
+    }
+
+    if (isMemberOfGroup(mountTable.getGroupName())
+        && mode.getGroupAction().implies(access)) {
+      return;
+    }
+
+    if (!getUser().equals(mountTable.getOwnerName())
+        && !isMemberOfGroup(mountTable.getGroupName())
+        && mode.getOtherAction().implies(access)) {
+      return;
+    }
+
+    throw new AccessControlException(
+        "Permission denied while accessing mount table "
+            + mountTable.getSourcePath()
+            + ": user " + getUser() + " does not have " + access.toString()
+            + " permissions.");
+  }
+}

+ 45 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MountTableStoreImpl.java

@@ -24,6 +24,9 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hdfs.server.federation.router.RouterAdminServer;
+import org.apache.hadoop.hdfs.server.federation.router.RouterPermissionChecker;
 import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
@@ -36,6 +39,7 @@ import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableE
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 import org.apache.hadoop.hdfs.server.federation.store.records.Query;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.Time;
 
 /**
@@ -52,7 +56,15 @@ public class MountTableStoreImpl extends MountTableStore {
   @Override
   public AddMountTableEntryResponse addMountTableEntry(
       AddMountTableEntryRequest request) throws IOException {
-    boolean status = getDriver().put(request.getEntry(), false, true);
+    MountTable mountTable = request.getEntry();
+    if (mountTable != null) {
+      RouterPermissionChecker pc = RouterAdminServer.getPermissionChecker();
+      if (pc != null) {
+        pc.checkPermission(mountTable, FsAction.WRITE);
+      }
+    }
+
+    boolean status = getDriver().put(mountTable, false, true);
     AddMountTableEntryResponse response =
         AddMountTableEntryResponse.newInstance();
     response.setStatus(status);
@@ -62,8 +74,15 @@ public class MountTableStoreImpl extends MountTableStore {
   @Override
   public UpdateMountTableEntryResponse updateMountTableEntry(
       UpdateMountTableEntryRequest request) throws IOException {
-    MountTable entry = request.getEntry();
-    boolean status = getDriver().put(entry, true, true);
+    MountTable mountTable = request.getEntry();
+    if (mountTable != null) {
+      RouterPermissionChecker pc = RouterAdminServer.getPermissionChecker();
+      if (pc != null) {
+        pc.checkPermission(mountTable, FsAction.WRITE);
+      }
+    }
+
+    boolean status = getDriver().put(mountTable, true, true);
     UpdateMountTableEntryResponse response =
         UpdateMountTableEntryResponse.newInstance();
     response.setStatus(status);
@@ -77,8 +96,17 @@ public class MountTableStoreImpl extends MountTableStore {
     final MountTable partial = MountTable.newInstance();
     partial.setSourcePath(srcPath);
     final Query<MountTable> query = new Query<>(partial);
-    int removedRecords = getDriver().remove(getRecordClass(), query);
-    boolean status = (removedRecords == 1);
+    final MountTable deleteEntry = getDriver().get(getRecordClass(), query);
+
+    boolean status = false;
+    if (deleteEntry != null) {
+      RouterPermissionChecker pc = RouterAdminServer.getPermissionChecker();
+      if (pc != null) {
+        pc.checkPermission(deleteEntry, FsAction.WRITE);
+      }
+      status = getDriver().remove(deleteEntry);
+    }
+
     RemoveMountTableEntryResponse response =
         RemoveMountTableEntryResponse.newInstance();
     response.setStatus(status);
@@ -88,12 +116,13 @@ public class MountTableStoreImpl extends MountTableStore {
   @Override
   public GetMountTableEntriesResponse getMountTableEntries(
       GetMountTableEntriesRequest request) throws IOException {
-
+    RouterPermissionChecker pc =
+        RouterAdminServer.getPermissionChecker();
     // Get all values from the cache
     List<MountTable> records = getCachedRecords();
 
     // Sort and filter
-    Collections.sort(records);
+    Collections.sort(records, MountTable.SOURCE_COMPARATOR);
     String reqSrcPath = request.getSrcPath();
     if (reqSrcPath != null && !reqSrcPath.isEmpty()) {
       // Return only entries beneath this path
@@ -103,6 +132,15 @@ public class MountTableStoreImpl extends MountTableStore {
         String srcPath = record.getSourcePath();
         if (!srcPath.startsWith(reqSrcPath)) {
           it.remove();
+        } else if (pc != null) {
+          // do the READ permission check
+          try {
+            pc.checkPermission(record, FsAction.READ);
+          } catch (AccessControlException ignored) {
+            // Remove this mount table entry if it cannot
+            // be accessed by current user.
+            it.remove();
+          }
         }
       }
     }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.java

@@ -45,6 +45,10 @@ public abstract class MembershipStats extends BaseRecord {
 
   public abstract long getAvailableSpace();
 
+  public abstract void setProvidedSpace(long capacity);
+
+  public abstract long getProvidedSpace();
+
   public abstract void setNumOfFiles(long files);
 
   public abstract long getNumOfFiles();

+ 68 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java

@@ -28,9 +28,13 @@ import java.util.TreeMap;
 
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
 import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
+import org.apache.hadoop.hdfs.server.federation.router.RouterPermissionChecker;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -127,6 +131,15 @@ public abstract class MountTable extends BaseRecord {
     // Set the serialized dest string
     record.setDestinations(locations);
 
+    // Set permission fields
+    UserGroupInformation ugi = NameNode.getRemoteUser();
+    record.setOwnerName(ugi.getShortUserName());
+    String group = ugi.getGroups().isEmpty() ? ugi.getShortUserName()
+        : ugi.getPrimaryGroupName();
+    record.setGroupName(group);
+    record.setMode(new FsPermission(
+        RouterPermissionChecker.MOUNT_TABLE_PERMISSION_DEFAULT));
+
     // Validate
     record.validate();
     return record;
@@ -193,6 +206,48 @@ public abstract class MountTable extends BaseRecord {
    */
   public abstract void setDestOrder(DestinationOrder order);
 
+  /**
+   * Get owner name of this mount table entry.
+   *
+   * @return Owner name
+   */
+  public abstract String getOwnerName();
+
+  /**
+   * Set owner name of this mount table entry.
+   *
+   * @param owner Owner name for mount table entry
+   */
+  public abstract void setOwnerName(String owner);
+
+  /**
+   * Get group name of this mount table entry.
+   *
+   * @return Group name
+   */
+  public abstract String getGroupName();
+
+  /**
+   * Set group name of this mount table entry.
+   *
+   * @param group Group name for mount table entry
+   */
+  public abstract void setGroupName(String group);
+
+  /**
+   * Get permission of this mount table entry.
+   *
+   * @return FsPermission permission mode
+   */
+  public abstract FsPermission getMode();
+
+  /**
+   * Set permission for this mount table entry.
+   *
+   * @param mode Permission for mount table entry
+   */
+  public abstract void setMode(FsPermission mode);
+
   /**
    * Get the default location.
    * @return The default location.
@@ -235,6 +290,19 @@ public abstract class MountTable extends BaseRecord {
     if (this.isReadOnly()) {
       sb.append("[RO]");
     }
+
+    if (this.getOwnerName() != null) {
+      sb.append("[owner:").append(this.getOwnerName()).append("]");
+    }
+
+    if (this.getGroupName() != null) {
+      sb.append("[group:").append(this.getGroupName()).append("]");
+    }
+
+    if (this.getMode() != null) {
+      sb.append("[mode:").append(this.getMode()).append("]");
+    }
+
     return sb.toString();
   }
 

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java

@@ -77,6 +77,16 @@ public class MembershipStatsPBImpl extends MembershipStats
     return this.translator.getProtoOrBuilder().getAvailableSpace();
   }
 
+  @Override
+  public void setProvidedSpace(long capacity) {
+    this.translator.getBuilder().setProvidedSpace(capacity);
+  }
+
+  @Override
+  public long getProvidedSpace() {
+    return this.translator.getProtoOrBuilder().getProvidedSpace();
+  }
+
   @Override
   public void setNumOfFiles(long files) {
     this.translator.getBuilder().setNumOfFiles(files);

+ 61 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MountTablePBImpl.java

@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
 
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.MountTableRecordProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.MountTableRecordProto.Builder;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.MountTableRecordProto.DestOrder;
@@ -28,6 +29,8 @@ import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProt
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoteLocationProto;
 import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
 import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
+import org.apache.hadoop.hdfs.server.federation.router.RouterAdminServer;
+import org.apache.hadoop.hdfs.server.federation.router.RouterPermissionChecker;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.FederationProtocolPBTranslator;
 import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 
@@ -189,6 +192,64 @@ public class MountTablePBImpl extends MountTable implements PBRecord {
     }
   }
 
+  @Override
+  public String getOwnerName() {
+    MountTableRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    if (!proto.hasOwnerName()) {
+      return RouterAdminServer.getSuperUser();
+    }
+    return proto.getOwnerName();
+  }
+
+  @Override
+  public void setOwnerName(String owner) {
+    Builder builder = this.translator.getBuilder();
+    if (owner == null) {
+      builder.clearOwnerName();
+    } else {
+      builder.setOwnerName(owner);
+    }
+  }
+
+  @Override
+  public String getGroupName() {
+    MountTableRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    if (!proto.hasGroupName()) {
+      return RouterAdminServer.getSuperGroup();
+    }
+    return proto.getGroupName();
+  }
+
+  @Override
+  public void setGroupName(String group) {
+    Builder builder = this.translator.getBuilder();
+    if (group == null) {
+      builder.clearGroupName();
+    } else {
+      builder.setGroupName(group);
+    }
+  }
+
+  @Override
+  public FsPermission getMode() {
+    MountTableRecordProtoOrBuilder proto = this.translator.getProtoOrBuilder();
+    short mode = RouterPermissionChecker.MOUNT_TABLE_PERMISSION_DEFAULT;
+    if (proto.hasMode()) {
+      mode = (short) proto.getMode();
+    }
+    return new FsPermission(mode);
+  }
+
+  @Override
+  public void setMode(FsPermission mode) {
+    Builder builder = this.translator.getBuilder();
+    if (mode == null) {
+      builder.clearMode();
+    } else {
+      builder.setMode(mode.toShort());
+    }
+  }
+
   private DestinationOrder convert(DestOrder order) {
     switch (order) {
     case LOCAL:

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java

@@ -686,7 +686,7 @@ public class Mover {
     }
   }
 
-  static class Cli extends Configured implements Tool {
+  public static class Cli extends Configured implements Tool {
     private static final String USAGE = "Usage: hdfs mover "
         + "[-p <files/dirs> | -f <local file>]"
         + "\n\t-p <files/dirs>\ta space separated list of HDFS files/dirs to migrate."

+ 8 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -656,7 +656,14 @@ public class FSDirectory implements Closeable {
     byte[][] components = INode.getPathComponents(src);
     boolean isRaw = isReservedRawName(components);
     if (isPermissionEnabled && pc != null && isRaw) {
-      pc.checkSuperuserPrivilege();
+      switch(dirOp) {
+        case READ_LINK:
+        case READ:
+          break;
+        default:
+          pc.checkSuperuserPrivilege();
+          break;
+      }
     }
     components = resolveComponents(components, this);
     INodesInPath iip = INodesInPath.resolve(rootDir, components, isRaw);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageCompression.java

@@ -39,7 +39,7 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-class FSImageCompression {
+public class FSImageCompression {
 
   /** Codec to use to save or load image, or null if the image is not compressed */
   private CompressionCodec imageCodec;

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -4166,6 +4166,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return datanodeStatistics.getCapacityRemaining();
   }
 
+  @Override // FSNamesystemMBean
+  @Metric({"ProvidedCapacityTotal",
+      "Total space used in PROVIDED storage in bytes" })
+  public long getProvidedCapacityTotal() {
+    return datanodeStatistics.getProvidedCapacity();
+  }
+
   @Metric({"CapacityRemainingGB", "Remaining capacity in GB"})
   public float getCapacityRemainingGB() {
     return DFSUtil.roundBytesToGB(getCapacityRemaining());
@@ -5729,6 +5736,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return this.getCapacityTotal();
   }
 
+  @Override // NameNodeMXBean
+  public long getProvidedCapacity() {
+    return this.getProvidedCapacityTotal();
+  }
+
   @Override // NameNodeMXBean
   public String getSafemode() {
     if (!this.isInSafeMode())

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java

@@ -46,7 +46,7 @@ import org.apache.hadoop.security.UserGroupInformation;
  * 
  * Some of the helper methods are gaurded by {@link FSNamesystem#readLock()}.
  */
-class FSPermissionChecker implements AccessControlEnforcer {
+public class FSPermissionChecker implements AccessControlEnforcer {
   static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
 
   private static String getPath(byte[][] components, int start, int end) {
@@ -86,7 +86,7 @@ class FSPermissionChecker implements AccessControlEnforcer {
   private final INodeAttributeProvider attributeProvider;
 
 
-  FSPermissionChecker(String fsOwner, String supergroup,
+  protected FSPermissionChecker(String fsOwner, String supergroup,
       UserGroupInformation callerUgi,
       INodeAttributeProvider attributeProvider) {
     this.fsOwner = fsOwner;

+ 9 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java

@@ -658,6 +658,10 @@ public class NNStorage extends Storage implements Closeable,
   void readProperties(StorageDirectory sd, StartupOption startupOption)
       throws IOException {
     Properties props = readPropertiesFile(sd.getVersionFile());
+    if (props == null) {
+      throw new IOException(
+          "Properties not found  for storage directory " + sd);
+    }
     if (HdfsServerConstants.RollingUpgradeStartupOption.ROLLBACK
         .matches(startupOption)) {
       int lv = Integer.parseInt(getProperty(props, sd, "layoutVersion"));
@@ -975,7 +979,11 @@ public class NNStorage extends Storage implements Closeable,
       StorageDirectory sd = sdit.next();
       try {
         Properties props = readPropertiesFile(sd.getVersionFile());
-        cid = props.getProperty("clusterID");
+        if (props == null) {
+          cid = null;
+        } else {
+          cid = props.getProperty("clusterID");
+        }
         LOG.info("current cluster id for sd="+sd.getCurrentDir() + 
             ";lv=" + layoutVersion + ";cid=" + cid);
         

+ 21 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -45,6 +45,8 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
@@ -208,6 +210,8 @@ public class NameNode extends ReconfigurableBase implements
     HdfsConfiguration.init();
   }
 
+  private InMemoryLevelDBAliasMapServer levelDBAliasMapServer;
+
   /**
    * Categories of operations supported by the namenode.
    */
@@ -725,6 +729,7 @@ public class NameNode extends ReconfigurableBase implements
     }
 
     loadNamesystem(conf);
+    startAliasMapServerIfNecessary(conf);
 
     rpcServer = createRpcServer(conf);
 
@@ -747,6 +752,19 @@ public class NameNode extends ReconfigurableBase implements
     startMetricsLogger(conf);
   }
 
+  private void startAliasMapServerIfNecessary(Configuration conf)
+      throws IOException {
+    if (conf.getBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED,
+        DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED_DEFAULT)
+        && conf.getBoolean(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED,
+            DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED_DEFAULT)) {
+      levelDBAliasMapServer = new InMemoryLevelDBAliasMapServer(
+          InMemoryAliasMap::init, namesystem.getBlockPoolId());
+      levelDBAliasMapServer.setConf(conf);
+      levelDBAliasMapServer.start();
+    }
+  }
+
   private void initReconfigurableBackoffKey() {
     ipcClientRPCBackoffEnable = buildBackoffEnableKey(rpcServer
         .getClientRpcServer().getPort());
@@ -1027,6 +1045,9 @@ public class NameNode extends ReconfigurableBase implements
         MBeans.unregister(nameNodeStatusBeanName);
         nameNodeStatusBeanName = null;
       }
+      if (levelDBAliasMapServer != null) {
+        levelDBAliasMapServer.close();
+      }
     }
     tracer.close();
   }

+ 8 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java

@@ -65,8 +65,14 @@ public interface NameNodeMXBean {
    * @return the total raw bytes including non-dfs used space
    */
   public long getTotal();
-  
-  
+
+  /**
+   * Gets capacity of the provided storage mounted, in bytes.
+   *
+   * @return the total raw bytes present in the provided storage.
+   */
+  public long getProvidedCapacity();
+
   /**
    * Gets the safemode status
    * 

部分文件因为文件数量过多而无法显示