瀏覽代碼

Merge branch 'HDFS-9806' into trunk

Chris Douglas 7 年之前
父節點
當前提交
fc7ec80d85
共有 100 個文件被更改,包括 7896 次插入203 次删除
  1. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
  2. 2 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
  3. 3 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
  4. 83 20
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
  5. 89 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ProvidedStorageLocation.java
  6. 36 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  7. 15 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
  8. 6 1
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  9. 29 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  10. 37 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java
  11. 134 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolServerSideTranslatorPB.java
  12. 174 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InMemoryAliasMapProtocolClientSideTranslatorPB.java
  13. 28 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  14. 222 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
  15. 103 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMapProtocol.java
  16. 153 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
  17. 14 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  18. 110 39
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  19. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
  20. 41 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  21. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  22. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStatistics.java
  23. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStats.java
  24. 12 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
  25. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
  26. 109 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LocatedBlockBuilder.java
  27. 540 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
  28. 32 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/StorageTypeStats.java
  29. 33 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/BlockAlias.java
  30. 85 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java
  31. 66 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
  32. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
  33. 113 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/BlockAliasMap.java
  34. 178 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java
  35. 274 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java
  36. 490 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
  37. 27 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/package-info.java
  38. 19 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
  39. 31 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  40. 18 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  41. 122 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
  42. 350 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
  43. 136 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
  44. 19 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
  45. 39 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
  46. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
  47. 32 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
  48. 55 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  49. 23 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
  50. 13 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
  51. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImplBuilder.java
  52. 718 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
  53. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java
  54. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMetrics.java
  55. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java
  56. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java
  57. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java
  58. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java
  59. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.java
  60. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java
  61. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
  62. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageCompression.java
  63. 12 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  64. 9 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  65. 21 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  66. 8 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
  67. 6 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java
  68. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
  69. 68 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto
  70. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto
  71. 119 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  72. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
  73. 247 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsProvidedStorage.md
  74. 29 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  75. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
  76. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
  77. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
  78. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
  79. 10 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
  80. 129 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/ITestInMemoryAliasMap.java
  81. 45 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestInMemoryAliasMap.java
  82. 49 17
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
  83. 120 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
  84. 344 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java
  85. 117 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java
  86. 121 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDbMockAliasMapClient.java
  87. 200 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java
  88. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
  89. 162 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java
  90. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
  91. 10 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
  92. 649 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
  93. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestFederationMetrics.java
  94. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
  95. 6 2
      hadoop-project/pom.xml
  96. 28 0
      hadoop-tools/hadoop-fs2img/dev-support/findbugs-exclude.xml
  97. 93 0
      hadoop-tools/hadoop-fs2img/pom.xml
  98. 99 0
      hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockResolver.java
  99. 109 0
      hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java
  100. 152 0
      hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java

+ 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 - 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 - 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;
 

+ 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;
+  }
+}

+ 36 - 0
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
  */
@@ -205,6 +219,7 @@ enum StorageTypeProto {
   SSD = 2;
   ARCHIVE = 3;
   RAM_DISK = 4;
+  PROVIDED = 5;
 }
 
 /**

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

@@ -191,7 +191,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>
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <dependency>
@@ -208,6 +207,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <artifactId>curator-test</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+        <groupId>org.assertj</groupId>
+        <artifactId>assertj-core</artifactId>
+        <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>
@@ -341,6 +345,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                   <include>fsimage.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() {

+ 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;
   }
 

+ 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;
+    }
   }
 
   /**

+ 55 - 10
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;
@@ -1511,6 +1512,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 +1537,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 +1564,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.
@@ -1744,6 +1760,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)) {
@@ -1879,7 +1899,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     try (AutoCloseableLock lock = datasetLock.acquire()) {
       r = volumeMap.get(bpid, blockId);
     }
-
     if (r != null) {
       if (r.blockDataExists()) {
         return r;
@@ -2232,13 +2251,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;
       }
@@ -2253,6 +2279,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
@@ -3028,7 +3074,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();

+ 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 - 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"));
           }
         }
       }

+ 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();

+ 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);

+ 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."

+ 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())

+ 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
    * 

+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java

@@ -69,7 +69,12 @@ public interface FSNamesystemMBean {
    * @return -  used capacity in bytes
    */
   public long getCapacityUsed();
- 
+
+  /**
+   * Total PROVIDED storage capacity.
+   * @return -  total PROVIDED storage capacity in bytes
+   */
+  public long getProvidedCapacityTotal();
 
   /**
    * Total number of files and directories

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java

@@ -160,6 +160,14 @@ public class NamespaceInfo extends StorageInfo {
     return state;
   }
 
+  public void setClusterID(String clusterID) {
+    this.clusterID = clusterID;
+  }
+
+  public void setBlockPoolID(String blockPoolID) {
+    this.blockPoolID = blockPoolID;
+  }
+
   @Override
   public String toString(){
     return super.toString() + ";bpid=" + blockPoolID;

+ 68 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/AliasMapProtocol.proto

@@ -0,0 +1,68 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "AliasMapProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+import "hdfs.proto";
+
+message KeyValueProto {
+  optional BlockProto key = 1;
+  optional ProvidedStorageLocationProto value = 2;
+}
+
+message WriteRequestProto {
+  required KeyValueProto keyValuePair = 1;
+}
+
+message WriteResponseProto {
+}
+
+message ReadRequestProto {
+  required BlockProto key = 1;
+}
+
+message ReadResponseProto {
+  optional ProvidedStorageLocationProto value = 1;
+}
+
+message ListRequestProto {
+  optional BlockProto marker = 1;
+}
+
+message ListResponseProto {
+  repeated KeyValueProto fileRegions = 1;
+  optional BlockProto nextMarker = 2;
+}
+
+message BlockPoolRequestProto {
+}
+
+message BlockPoolResponseProto {
+  required string blockPoolId = 1;
+}
+
+service AliasMapProtocolService {
+  rpc write(WriteRequestProto) returns(WriteResponseProto);
+  rpc read(ReadRequestProto) returns(ReadResponseProto);
+  rpc list(ListRequestProto) returns(ListResponseProto);
+  rpc getBlockPoolId(BlockPoolRequestProto) returns(BlockPoolResponseProto);
+}

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto

@@ -30,6 +30,7 @@ package hadoop.hdfs;
 message NamenodeMembershipStatsRecordProto {
   optional uint64 totalSpace = 1;
   optional uint64 availableSpace = 2;
+  optional uint64 providedSpace = 3;
 
   optional uint64 numOfFiles = 10;
   optional uint64 numOfBlocks = 11;

+ 119 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -4621,6 +4621,125 @@
     </description>
   </property>
 
+  <property>
+    <name>dfs.namenode.provided.enabled</name>
+    <value>false</value>
+    <description>
+      Enables the Namenode to handle provided storages.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.storage.id</name>
+    <value>DS-PROVIDED</value>
+    <description>
+      The storage ID used for provided stores.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.class</name>
+    <value>org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap</value>
+    <description>
+      The class that is used to specify the input format of the blocks on
+      provided storages. The default is
+      org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap which uses
+      file regions to describe blocks. The file regions are specified as a
+      delimited text file. Each file region is a 6-tuple containing the
+      block id, remote file path, offset into file, length of block, the
+      block pool id containing the block, and the generation stamp of the
+      block.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.batch-size</name>
+    <value>500</value>
+    <description>
+      The batch size when iterating over the database backing the aliasmap
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.dnrpc-address</name>
+    <value>0.0.0.0:50200</value>
+    <description>
+      The address where the aliasmap server will be running
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.leveldb.dir</name>
+    <value>/tmp</value>
+    <description>
+      The directory where the leveldb files will be kept
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.enabled</name>
+    <value>false</value>
+    <description>
+      Don't use the aliasmap by default. Some tests will fail
+      because they try to start the namenode twice with the
+      same parameters if you turn it on.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.text.delimiter</name>
+    <value>,</value>
+    <description>
+        The delimiter used when the provided block map is specified as
+        a text file.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.text.read.file</name>
+    <value></value>
+    <description>
+        The path specifying the provided block map as a text file, specified as
+        a URI.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.text.codec</name>
+    <value></value>
+    <description>
+        The codec used to de-compress the provided block map.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.text.write.dir</name>
+    <value></value>
+    <description>
+        The path to which the provided block map should be written as a text
+        file, specified as a URI.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.leveldb.path</name>
+    <value></value>
+    <description>
+      The read/write path for the leveldb-based alias map
+      (org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.LevelDBFileRegionAliasMap).
+      The path has to be explicitly configured when this alias map is used.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.load.retries</name>
+    <value>0</value>
+    <description>
+      The number of retries on the Datanode to load the provided aliasmap;
+      defaults to 0.
+    </description>
+  </property>
+
   <property>
     <name>dfs.lock.suppress.warning.interval</name>
     <value>10s</value>

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html

@@ -162,6 +162,7 @@
 {#nn}
 <table class="table table-bordered table-striped">
   <tr><th> Configured Capacity:</th><td>{Total|fmt_bytes}</td></tr>
+  <tr><th> Configured Remote Capacity:</th><td>{ProvidedCapacity|fmt_bytes}</td></tr>
   <tr><th> DFS Used:</th><td>{Used|fmt_bytes} ({PercentUsed|fmt_percentage})</td></tr>
   <tr><th> Non DFS Used:</th><td>{NonDfsUsedSpace|fmt_bytes}</td></tr>
   <tr><th> DFS Remaining:</th><td>{Free|fmt_bytes} ({PercentRemaining|fmt_percentage})</td></tr>

+ 247 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsProvidedStorage.md

@@ -0,0 +1,247 @@
+<!---
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+HDFS Provided Storage
+=====================
+
+Provided storage allows data *stored outside HDFS* to be mapped to and addressed
+from HDFS. It builds on [heterogeneous storage](./ArchivalStorage.html) by
+introducing a new storage type, `PROVIDED`, to the set of media in a datanode.
+Clients accessing data in
+`PROVIDED` storages can cache replicas in local media, enforce HDFS invariants
+(e.g., security, quotas), and address more data than the cluster could persist
+in the storage attached to DataNodes. This architecture is particularly useful
+in scenarios where HDFS clusters are ephemeral (e.g., cloud scenarios), and/or
+require to read data that lives in other storage systems (e.g., blob stores).
+
+Provided storage is an experimental feature in HDFS.
+
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
+
+Introduction
+------------
+
+As of this writing, support for mounting external storage as `PROVIDED` blocks
+is limited to creating a *read-only image* of a remote namespace that implements the
+`org.apache.hadoop.fs.FileSystem` interface, and starting a NameNode
+to serve the image. Specifically, reads from a snapshot of a remote namespace are
+supported. Adding a remote namespace to an existing/running namenode, refreshing the
+remote snapshot, unmounting, and writes are not available in this release. One
+can use [ViewFs](./ViewFs.html) and [RBF](HDFSRouterFederation.html) to
+integrate namespaces with `PROVIDED` storage into an existing deployment.
+
+Creating HDFS Clusters with `PROVIDED` Storage
+----------------------------------------------
+
+One can create snapshots of the remote namespace using the `fs2img` tool. Given
+a path to a remote `FileSystem`, the tool creates an _image_ mirroring the
+namespace and an _alias map_ that maps blockIDs in the generated image to a
+`FileRegion` in the remote filesystem. A `FileRegion` contains sufficient information to
+address a fixed sequence of bytes in the remote `FileSystem` (e.g., file, offset, length)
+and a nonce to verify that the region is unchanged since the image was generated.
+
+After the NameNode image and alias map are created, the NameNode and DataNodes
+must be configured to consistently reference this address space. When a DataNode
+registers with an attached, `PROVIDED` storage, the NameNode considers all the
+external blocks to be addressable through that DataNode, and may begin to direct
+clients to it. Symmetrically, the DataNode must be able to map every block in
+the `PROVIDED` storage to remote data.
+
+Deployment details vary depending on the configured alias map implementation.
+
+### `PROVIDED` Configuration
+
+Each NameNode supports one alias map. When `PROVIDED` storage is enabled,
+the storage ID configured on the NameNode and DataNodes must match.
+All other details are internal to the alias map implementation.
+
+The configuration to enable `PROVIDED` storage is as follows.
+The configuration options available for the alias map implementations are
+available below.
+
+```xml
+<configuration>
+
+  <property>
+    <name>dfs.namenode.provided.enabled</name>
+    <value>true</value>
+    <description>Enabled provided storage on the Namenode</description>
+  </property>
+
+  <property>
+     <name>dfs.datanode.data.dir</name>
+     <value>[DISK]/local/path/to/blocks/, [PROVIDED]remoteFS://remoteFS-authority/path/to/data/</value>
+  </property>
+
+  <property>
+      <name>dfs.provided.storage.id</name>
+      <value>DS-PROVIDED</value>
+      <description>The storage ID used for provided storages in the cluster.</description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.class</name>
+    <value>org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap</value>
+  </property>
+
+</configuration>
+```
+
+### fs2img tool
+
+The `fs2img` tool "walks" over a remote namespace by recursively enumerating
+children of a remote URI to produce an FSImage. Some attributes can be
+controlled by plugins, such as owner/group mappings from the remote filesystem
+to HDFS and the mapping of files to HDFS blocks.
+
+The various options available in running the tool are:
+
+| Option                  | Property                    | Default           | Description |
+|:------------------------|:--------------------------- |:----------------- |:---- |
+| `-o`, `--outdir`        | dfs.namenode.name.dir       | file://${hadoop.tmp.dir}/dfs/name | Output directory |
+| `-b`, `--blockclass`    | dfs.provided.aliasmap.class | NullBlocksMap     | Block output class |
+| `-u`, `--ugiclass`      | hdfs.image.writer.ugi.class | SingleUGIResolver | UGI resolver class |
+| `-i`, `--blockidclass`  | hdfs.image.writer.blockresolver.class | FixedBlockResolver | Block resolver class |
+| `-c`, `--cachedirs`     | hdfs.image.writer.cache.entries | 100           | Max active dirents |
+| `-cid`, `--clusterID`   |                             |                   | Cluster ID |
+| `-bpid`, `--blockPoolID`|                             |                   | Block pool ID |
+
+#### Examples
+
+Assign all files to be owned by "rmarathe", write to gzip compressed text:
+```
+hadoop org.apache.hadoop.hdfs.server.namenode.FileSystemImage \
+  -Dhdfs.image.writer.ugi.single.user=rmarathe \
+  -Ddfs.provided.aliasmap.text.codec=gzip \
+  -Ddfs.provided.aliasmap.text.write.dir=file:///tmp/
+  -b org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap \
+  -u org.apache.hadoop.hdfs.server.namenode.SingleUGIResolver \
+  -o file:///tmp/name \
+  hdfs://afreast/projects/ydau/onan
+```
+
+Assign ownership based on a custom `UGIResolver`, in LevelDB:
+```
+hadoop org.apache.hadoop.hdfs.server.namenode.FileSystemImage \
+  -Ddfs.provided.aliasmap.leveldb.path=/path/to/leveldb/map/dingos.db \
+  -b org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.LevelDBFileRegionAliasMap \
+  -o file:///tmp/name \
+  -u CustomResolver \
+  hdfs://enfield/projects/ywqmd/incandenza
+```
+
+
+Alias Map Implementations
+-------------------------
+
+The alias map implementation to use is configured using the `dfs.provided.aliasmap.class` parameter.
+Currently, the following two types of alias maps are supported.
+
+### InMemoryAliasMap
+
+This is a LevelDB-based alias map that runs as a separate server in Namenode.
+The alias map itself can be created using the `fs2img` tool using the option
+`-Ddfs.provided.aliasmap.leveldb.path=file:///path/to/leveldb/map/dingos.db -o org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.LevelDBFileRegionAliasMap`
+as in the example above.
+
+Datanodes contact this alias map using the `org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMapProtocol` protocol.
+
+#### Configuration
+
+
+```xml
+<configuration>
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.batch-size</name>
+    <value>500</value>
+    <description>
+      The batch size when iterating over the database backing the aliasmap
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.dnrpc-address</name>
+    <value>namenode:rpc-port</value>
+    <description>
+      The address where the aliasmap server will be running
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.leveldb.dir</name>
+    <value>/path/to/leveldb/map/dingos.db</value>
+    <description>
+      The directory where the leveldb files will be kept
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.enabled</name>
+    <value>true</value>
+    <description>Enable the inmemory alias map on the NameNode. Defaults to false.</description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.class</name>
+    <value>org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.InMemoryLevelDBAliasMapClient</value>
+  </property>
+</configuration>
+```
+
+### TextFileRegionAliasMap
+
+This alias map implementation stores the mapping from `blockID`s to `FileRegion`s
+in a delimited text file. This format is useful for test environments,
+particularly single-node.
+
+#### Configuration
+```xml
+<configuration>
+  <property>
+    <name>dfs.provided.aliasmap.text.delimiter</name>
+    <value>,</value>
+    <description>
+        The delimiter used when the alias map is specified as
+        a text file.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.text.read.file</name>
+    <value>file:///path/to/aliasmap/blocks_blocPoolID.csv</value>
+    <description>
+        The path specifying the alias map as a text file,
+        specified as a URI.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.text.codec</name>
+    <value></value>
+    <description>
+        The codec used to de-compress the alias map. Default value is empty.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.text.write.dir</name>
+    <value>file:///path/to/aliasmap/</value>
+    <description>
+        The path to which the alias map should be written as a text
+        file, specified as a URI.
+    </description>
+  </property>
+</configuration>
+```
+

+ 29 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -147,6 +147,9 @@ public class MiniDFSCluster implements AutoCloseable {
       GenericTestUtils.SYSPROP_TEST_DATA_DIR;
   /** Configuration option to set the data dir: {@value} */
   public static final String HDFS_MINIDFS_BASEDIR = "hdfs.minidfs.basedir";
+  /** Configuration option to set the provided data dir: {@value} */
+  public static final String HDFS_MINIDFS_BASEDIR_PROVIDED =
+      "hdfs.minidfs.basedir.provided";
   public static final String  DFS_NAMENODE_SAFEMODE_EXTENSION_TESTING_KEY
       = DFS_NAMENODE_SAFEMODE_EXTENSION_KEY + ".testing";
   public static final String  DFS_NAMENODE_DECOMMISSION_INTERVAL_TESTING_KEY
@@ -1397,7 +1400,12 @@ public class MiniDFSCluster implements AutoCloseable {
       if ((storageTypes != null) && (j >= storageTypes.length)) {
         break;
       }
-      File dir = getInstanceStorageDir(dnIndex, j);
+      File dir;
+      if (storageTypes != null && storageTypes[j] == StorageType.PROVIDED) {
+        dir = getProvidedStorageDir(dnIndex, j);
+      } else {
+        dir = getInstanceStorageDir(dnIndex, j);
+      }
       dir.mkdirs();
       if (!dir.isDirectory()) {
         throw new IOException("Mkdirs failed to create directory for DataNode " + dir);
@@ -2846,6 +2854,26 @@ public class MiniDFSCluster implements AutoCloseable {
     return new File(base_dir, getStorageDirPath(dnIndex, dirIndex));
   }
 
+  /**
+   * Get a storage directory for PROVIDED storages.
+   * The PROVIDED directory to return can be set by using the configuration
+   * parameter {@link #HDFS_MINIDFS_BASEDIR_PROVIDED}. If this parameter is
+   * not set, this function behaves exactly the same as
+   * {@link #getInstanceStorageDir(int, int)}. Currently, the two parameters
+   * are ignored as only one PROVIDED storage is supported in HDFS-9806.
+   *
+   * @param dnIndex datanode index (starts from 0)
+   * @param dirIndex directory index
+   * @return Storage directory
+   */
+  public File getProvidedStorageDir(int dnIndex, int dirIndex) {
+    String base = conf.get(HDFS_MINIDFS_BASEDIR_PROVIDED, null);
+    if (base == null) {
+      return getInstanceStorageDir(dnIndex, dirIndex);
+    }
+    return new File(base);
+  }
+
   /**
    * Get a storage directory for a datanode.
    * <ol>

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java

@@ -84,6 +84,7 @@ public class TestBlockStoragePolicy {
   static final byte ONESSD  = HdfsConstants.ONESSD_STORAGE_POLICY_ID;
   static final byte ALLSSD  = HdfsConstants.ALLSSD_STORAGE_POLICY_ID;
   static final byte LAZY_PERSIST  = HdfsConstants.MEMORY_STORAGE_POLICY_ID;
+  static final byte PROVIDED  = HdfsConstants.PROVIDED_STORAGE_POLICY_ID;
 
   @Test (timeout=300000)
   public void testConfigKeyEnabled() throws IOException {
@@ -143,6 +144,11 @@ public class TestBlockStoragePolicy {
     expectedPolicyStrings.put(ALLSSD, "BlockStoragePolicy{ALL_SSD:" + ALLSSD +
         ", storageTypes=[SSD], creationFallbacks=[DISK], " +
         "replicationFallbacks=[DISK]}");
+    expectedPolicyStrings.put(PROVIDED,
+        "BlockStoragePolicy{PROVIDED:" + PROVIDED
+            + ", storageTypes=[PROVIDED, DISK], "
+            + "creationFallbacks=[PROVIDED, DISK], "
+            + "replicationFallbacks=[PROVIDED, DISK]}");
 
     for(byte i = 1; i < 16; i++) {
       final BlockStoragePolicy policy = POLICY_SUITE.getPolicy(i); 

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java

@@ -208,7 +208,7 @@ public class TestDFSRollback {
       UpgradeUtilities.createDataNodeVersionFile(
           dataCurrentDirs,
           storageInfo,
-          UpgradeUtilities.getCurrentBlockPoolID(cluster));
+          UpgradeUtilities.getCurrentBlockPoolID(cluster), conf);
 
       cluster.startDataNodes(conf, 1, false, StartupOption.ROLLBACK, null);
       assertTrue(cluster.isDataNodeUp());
@@ -256,7 +256,7 @@ public class TestDFSRollback {
           NodeType.DATA_NODE);
       
       UpgradeUtilities.createDataNodeVersionFile(baseDirs, storageInfo,
-          UpgradeUtilities.getCurrentBlockPoolID(cluster));
+          UpgradeUtilities.getCurrentBlockPoolID(cluster), conf);
       
       startBlockPoolShouldFail(StartupOption.ROLLBACK, 
           cluster.getNamesystem().getBlockPoolId());
@@ -283,7 +283,7 @@ public class TestDFSRollback {
           NodeType.DATA_NODE);
      
       UpgradeUtilities.createDataNodeVersionFile(baseDirs, storageInfo,
-          UpgradeUtilities.getCurrentBlockPoolID(cluster));
+          UpgradeUtilities.getCurrentBlockPoolID(cluster), conf);
       
       startBlockPoolShouldFail(StartupOption.ROLLBACK, 
           cluster.getNamesystem().getBlockPoolId());

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java

@@ -265,7 +265,7 @@ public class TestDFSStartupVersions {
           conf.getStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY), "current");
       log("DataNode version info", DATA_NODE, i, versions[i]);
       UpgradeUtilities.createDataNodeVersionFile(storage,
-          versions[i].storageInfo, bpid, versions[i].blockPoolId);
+          versions[i].storageInfo, bpid, versions[i].blockPoolId, conf);
       try {
         cluster.startDataNodes(conf, 1, false, StartupOption.REGULAR, null);
       } catch (Exception ignore) {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java

@@ -290,7 +290,7 @@ public class TestDFSUpgrade {
           UpgradeUtilities.getCurrentFsscTime(cluster), NodeType.DATA_NODE);
       
       UpgradeUtilities.createDataNodeVersionFile(baseDirs, storageInfo,
-          UpgradeUtilities.getCurrentBlockPoolID(cluster));
+          UpgradeUtilities.getCurrentBlockPoolID(cluster), conf);
       
       startBlockPoolShouldFail(StartupOption.REGULAR, UpgradeUtilities
           .getCurrentBlockPoolID(null));
@@ -308,7 +308,7 @@ public class TestDFSUpgrade {
           NodeType.DATA_NODE);
           
       UpgradeUtilities.createDataNodeVersionFile(baseDirs, storageInfo, 
-          UpgradeUtilities.getCurrentBlockPoolID(cluster));
+          UpgradeUtilities.getCurrentBlockPoolID(cluster), conf);
       // Ensure corresponding block pool failed to initialized
       startBlockPoolShouldFail(StartupOption.REGULAR, UpgradeUtilities
           .getCurrentBlockPoolID(null));

+ 10 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java

@@ -384,8 +384,10 @@ public class UpgradeUtilities {
           new File(datanodeStorage.toString()));
       sd.setStorageUuid(DatanodeStorage.generateUuid());
       Properties properties = Storage.readPropertiesFile(sd.getVersionFile());
-      properties.setProperty("storageID", sd.getStorageUuid());
-      Storage.writeProperties(sd.getVersionFile(), properties);
+      if (properties != null) {
+        properties.setProperty("storageID", sd.getStorageUuid());
+        Storage.writeProperties(sd.getVersionFile(), properties);
+      }
 
       retVal[i] = newDir;
     }
@@ -461,8 +463,9 @@ public class UpgradeUtilities {
    * @param bpid Block pool Id
    */
   public static void createDataNodeVersionFile(File[] parent,
-      StorageInfo version, String bpid) throws IOException {
-    createDataNodeVersionFile(parent, version, bpid, bpid);
+      StorageInfo version, String bpid, Configuration conf)
+          throws IOException {
+    createDataNodeVersionFile(parent, version, bpid, bpid, conf);
   }
   
   /**
@@ -477,7 +480,8 @@ public class UpgradeUtilities {
    * @param bpidToWrite Block pool Id to write into the version file
    */
   public static void createDataNodeVersionFile(File[] parent,
-      StorageInfo version, String bpid, String bpidToWrite) throws IOException {
+      StorageInfo version, String bpid, String bpidToWrite, Configuration conf)
+          throws IOException {
     DataStorage storage = new DataStorage(version);
     storage.setDatanodeUuid("FixedDatanodeUuid");
 
@@ -485,7 +489,7 @@ public class UpgradeUtilities {
     for (int i = 0; i < parent.length; i++) {
       File versionFile = new File(parent[i], "VERSION");
       StorageDirectory sd = new StorageDirectory(parent[i].getParentFile());
-      DataStorage.createStorageID(sd, false);
+      DataStorage.createStorageID(sd, false, conf);
       storage.writeProperties(versionFile, sd);
       versionFiles[i] = versionFile;
       File bpDir = BlockPoolSliceStorage.getBpRoot(bpid, parent[i]);

+ 129 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/ITestInMemoryAliasMap.java

@@ -0,0 +1,129 @@
+/*
+ * 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.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.Optional;
+
+/**
+ * ITestInMemoryAliasMap is an integration test that writes and reads to
+ * an AliasMap. This is an integration test because it can't be run in parallel
+ * like normal unit tests since there is conflict over the port being in use.
+ */
+public class ITestInMemoryAliasMap {
+  private InMemoryAliasMap aliasMap;
+  private File tempDirectory;
+  private static String bpid = "bpid-0";
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    File temp = Files.createTempDirectory("seagull").toFile();
+    tempDirectory = new File(temp, bpid);
+    tempDirectory.mkdirs();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        temp.getAbsolutePath());
+    aliasMap = InMemoryAliasMap.init(conf, bpid);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    aliasMap.close();
+    FileUtils.deleteDirectory(tempDirectory);
+  }
+
+  @Test
+  public void readNotFoundReturnsNothing() throws IOException {
+    Block block = new Block(42, 43, 44);
+
+    Optional<ProvidedStorageLocation> actualProvidedStorageLocationOpt
+        = aliasMap.read(block);
+
+    assertFalse(actualProvidedStorageLocationOpt.isPresent());
+  }
+
+  @Test
+  public void readWrite() throws Exception {
+    Block block = new Block(42, 43, 44);
+
+    Path path = new Path("eagle", "mouse");
+    long offset = 47;
+    long length = 48;
+    int nonceSize = 4;
+    byte[] nonce = new byte[nonceSize];
+    Arrays.fill(nonce, 0, (nonceSize - 1), Byte.parseByte("0011", 2));
+
+    ProvidedStorageLocation expectedProvidedStorageLocation =
+        new ProvidedStorageLocation(path, offset, length, nonce);
+
+    aliasMap.write(block, expectedProvidedStorageLocation);
+
+    Optional<ProvidedStorageLocation> actualProvidedStorageLocationOpt
+        = aliasMap.read(block);
+
+    assertTrue(actualProvidedStorageLocationOpt.isPresent());
+    assertEquals(expectedProvidedStorageLocation,
+        actualProvidedStorageLocationOpt.get());
+
+  }
+
+  @Test
+  public void list() throws IOException {
+    Block block1 = new Block(42, 43, 44);
+    Block block2 = new Block(43, 44, 45);
+    Block block3 = new Block(44, 45, 46);
+
+    Path path = new Path("eagle", "mouse");
+    int nonceSize = 4;
+    byte[] nonce = new byte[nonceSize];
+    Arrays.fill(nonce, 0, (nonceSize - 1), Byte.parseByte("0011", 2));
+    ProvidedStorageLocation expectedProvidedStorageLocation1 =
+        new ProvidedStorageLocation(path, 47, 48, nonce);
+    ProvidedStorageLocation expectedProvidedStorageLocation2 =
+        new ProvidedStorageLocation(path, 48, 49, nonce);
+    ProvidedStorageLocation expectedProvidedStorageLocation3 =
+        new ProvidedStorageLocation(path, 49, 50, nonce);
+
+    aliasMap.write(block1, expectedProvidedStorageLocation1);
+    aliasMap.write(block2, expectedProvidedStorageLocation2);
+    aliasMap.write(block3, expectedProvidedStorageLocation3);
+
+    InMemoryAliasMap.IterationResult list = aliasMap.list(Optional.empty());
+    // we should have 3 results
+    assertEquals(3, list.getFileRegions().size());
+    // no more results expected
+    assertFalse(list.getNextBlock().isPresent());
+  }
+}
+

+ 45 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestInMemoryAliasMap.java

@@ -0,0 +1,45 @@
+/*
+ * 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.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+
+/**
+ * TestInMemoryAliasMap tests the initialization of an AliasMap. Most of the
+ * rest of the tests are in ITestInMemoryAliasMap since the tests are not
+ * thread safe (there is competition for the port).
+ */
+public class TestInMemoryAliasMap {
+
+  @Test
+  public void testInit() {
+    String nonExistingDirectory = "non-existing-directory";
+    Configuration conf = new Configuration();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        nonExistingDirectory);
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() -> InMemoryAliasMap.init(conf, "bpid")).withMessage(
+            InMemoryAliasMap.createPathErrorMessage(nonExistingDirectory));
+  }
+}

+ 49 - 17
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java

@@ -300,7 +300,7 @@ public class TestDatanodeManager {
    */
   @Test
   public void testSortLocatedBlocks() throws IOException, URISyntaxException {
-    HelperFunction(null);
+    HelperFunction(null, 0);
   }
 
   /**
@@ -312,7 +312,7 @@ public class TestDatanodeManager {
    */
   @Test
   public void testgoodScript() throws IOException, URISyntaxException {
-    HelperFunction("/" + Shell.appendScriptExtension("topology-script"));
+    HelperFunction("/" + Shell.appendScriptExtension("topology-script"), 0);
   }
 
 
@@ -325,7 +325,22 @@ public class TestDatanodeManager {
    */
   @Test
   public void testBadScript() throws IOException, URISyntaxException {
-    HelperFunction("/"+ Shell.appendScriptExtension("topology-broken-script"));
+    HelperFunction("/" + Shell.appendScriptExtension("topology-broken-script"),
+        0);
+  }
+
+  /**
+   * Test with different sorting functions but include datanodes.
+   * with provided storage
+   * @throws IOException
+   * @throws URISyntaxException
+   */
+  @Test
+  public void testWithProvidedTypes() throws IOException, URISyntaxException {
+    HelperFunction(null, 1);
+    HelperFunction(null, 3);
+    HelperFunction("/" + Shell.appendScriptExtension("topology-script"), 1);
+    HelperFunction("/" + Shell.appendScriptExtension("topology-script"), 2);
   }
 
   /**
@@ -333,11 +348,12 @@ public class TestDatanodeManager {
    * we invoke this function with and without topology scripts
    *
    * @param scriptFileName - Script Name or null
+   * @param providedStorages - number of provided storages to add
    *
    * @throws URISyntaxException
    * @throws IOException
    */
-  public void HelperFunction(String scriptFileName)
+  public void HelperFunction(String scriptFileName, int providedStorages)
     throws URISyntaxException, IOException {
     // create the DatanodeManager which will be tested
     Configuration conf = new Configuration();
@@ -352,17 +368,25 @@ public class TestDatanodeManager {
     }
     DatanodeManager dm = mockDatanodeManager(fsn, conf);
 
+    int totalDNs = 5 + providedStorages;
+
     // register 5 datanodes, each with different storage ID and type
-    DatanodeInfo[] locs = new DatanodeInfo[5];
-    String[] storageIDs = new String[5];
-    StorageType[] storageTypes = new StorageType[]{
-      StorageType.ARCHIVE,
-      StorageType.DEFAULT,
-      StorageType.DISK,
-      StorageType.RAM_DISK,
-      StorageType.SSD
-    };
-    for (int i = 0; i < 5; i++) {
+    DatanodeInfo[] locs = new DatanodeInfo[totalDNs];
+    String[] storageIDs = new String[totalDNs];
+    List<StorageType> storageTypesList = new ArrayList<>(
+        Arrays.asList(StorageType.ARCHIVE,
+            StorageType.DEFAULT,
+            StorageType.DISK,
+            StorageType.RAM_DISK,
+            StorageType.SSD));
+
+    for (int i = 0; i < providedStorages; i++) {
+      storageTypesList.add(StorageType.PROVIDED);
+    }
+
+    StorageType[] storageTypes= storageTypesList.toArray(new StorageType[0]);
+
+    for (int i = 0; i < totalDNs; i++) {
       // register new datanode
       String uuid = "UUID-" + i;
       String ip = "IP-" + i;
@@ -398,9 +422,9 @@ public class TestDatanodeManager {
     DatanodeInfo[] sortedLocs = block.getLocations();
     storageIDs = block.getStorageIDs();
     storageTypes = block.getStorageTypes();
-    assertThat(sortedLocs.length, is(5));
-    assertThat(storageIDs.length, is(5));
-    assertThat(storageTypes.length, is(5));
+    assertThat(sortedLocs.length, is(totalDNs));
+    assertThat(storageIDs.length, is(totalDNs));
+    assertThat(storageTypes.length, is(totalDNs));
     for (int i = 0; i < sortedLocs.length; i++) {
       assertThat(((DatanodeInfoWithStorage) sortedLocs[i]).getStorageID(),
         is(storageIDs[i]));
@@ -414,6 +438,14 @@ public class TestDatanodeManager {
       is(DatanodeInfo.AdminStates.DECOMMISSIONED));
     assertThat(sortedLocs[sortedLocs.length - 2].getAdminState(),
       is(DatanodeInfo.AdminStates.DECOMMISSIONED));
+    // check that the StorageType of datanoodes immediately
+    // preceding the decommissioned datanodes is PROVIDED
+    for (int i = 0; i < providedStorages; i++) {
+      assertThat(
+          ((DatanodeInfoWithStorage)
+              sortedLocs[sortedLocs.length - 3 - i]).getStorageType(),
+          is(StorageType.PROVIDED));
+    }
   }
 
   /**

+ 120 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java

@@ -0,0 +1,120 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TestProvidedImpl;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.util.RwLock;
+import org.junit.Before;
+import org.junit.Test;
+import java.io.IOException;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * This class tests the {@link ProvidedStorageMap}.
+ */
+public class TestProvidedStorageMap {
+
+  private Configuration conf;
+  private BlockManager bm;
+  private RwLock nameSystemLock;
+  private String providedStorageID;
+  private String blockPoolID;
+
+  @Before
+  public void setup() {
+    providedStorageID = DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT;
+    conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID,
+            providedStorageID);
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, true);
+    conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+        TestProvidedImpl.TestFileRegionBlockAliasMap.class,
+        BlockAliasMap.class);
+    blockPoolID = "BP-12344-10.1.1.2-12344";
+    bm = mock(BlockManager.class);
+    when(bm.getBlockPoolId()).thenReturn(blockPoolID);
+    nameSystemLock = mock(RwLock.class);
+  }
+
+  private DatanodeDescriptor createDatanodeDescriptor(int port) {
+    return DFSTestUtil.getDatanodeDescriptor("127.0.0.1", port, "defaultRack",
+        "localhost");
+  }
+
+  @Test
+  public void testProvidedStorageMap() throws IOException {
+    ProvidedStorageMap providedMap = new ProvidedStorageMap(
+        nameSystemLock, bm, conf);
+    DatanodeStorageInfo providedMapStorage =
+        providedMap.getProvidedStorageInfo();
+    // the provided storage cannot be null
+    assertNotNull(providedMapStorage);
+
+    // create a datanode
+    DatanodeDescriptor dn1 = createDatanodeDescriptor(5000);
+
+    // associate two storages to the datanode
+    DatanodeStorage dn1ProvidedStorage = new DatanodeStorage(
+        providedStorageID,
+        DatanodeStorage.State.NORMAL,
+        StorageType.PROVIDED);
+    DatanodeStorage dn1DiskStorage = new DatanodeStorage(
+        "sid-1", DatanodeStorage.State.NORMAL, StorageType.DISK);
+
+    when(nameSystemLock.hasWriteLock()).thenReturn(true);
+    DatanodeStorageInfo dns1Provided =
+        providedMap.getStorage(dn1, dn1ProvidedStorage);
+    DatanodeStorageInfo dns1Disk = providedMap.getStorage(dn1, dn1DiskStorage);
+
+    assertTrue("The provided storages should be equal",
+        dns1Provided == providedMapStorage);
+    assertTrue("Disk storage has not yet been registered with block manager",
+        dns1Disk == null);
+    // add the disk storage to the datanode.
+    DatanodeStorageInfo dnsDisk = new DatanodeStorageInfo(dn1, dn1DiskStorage);
+    dn1.injectStorage(dnsDisk);
+    assertTrue("Disk storage must match the injected storage info",
+        dnsDisk == providedMap.getStorage(dn1, dn1DiskStorage));
+
+    // create a 2nd datanode
+    DatanodeDescriptor dn2 = createDatanodeDescriptor(5010);
+    // associate a provided storage with the datanode
+    DatanodeStorage dn2ProvidedStorage = new DatanodeStorage(
+        providedStorageID,
+        DatanodeStorage.State.NORMAL,
+        StorageType.PROVIDED);
+
+    DatanodeStorageInfo dns2Provided = providedMap.getStorage(
+        dn2, dn2ProvidedStorage);
+    assertTrue("The provided storages should be equal",
+        dns2Provided == providedMapStorage);
+    assertTrue("The DatanodeDescriptor should contain the provided storage",
+        dn2.getStorageInfo(providedStorageID) == providedMapStorage);
+  }
+}

+ 344 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java

@@ -0,0 +1,344 @@
+/*
+ * 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 com.google.common.collect.Lists;
+import com.google.common.io.Files;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+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.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+
+/**
+ * Tests the {@link InMemoryLevelDBAliasMapClient}.
+ */
+public class TestInMemoryLevelDBAliasMapClient {
+
+  private InMemoryLevelDBAliasMapServer levelDBAliasMapServer;
+  private InMemoryLevelDBAliasMapClient inMemoryLevelDBAliasMapClient;
+  private File tempDir;
+  private Configuration conf;
+  private final static String BPID = "BPID-0";
+
+  @Before
+  public void setUp() throws IOException {
+    conf = new Configuration();
+    int port = 9876;
+
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+        "localhost:" + port);
+    tempDir = Files.createTempDir();
+    File levelDBDir = new File(tempDir, BPID);
+    levelDBDir.mkdirs();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        tempDir.getAbsolutePath());
+    levelDBAliasMapServer =
+        new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init, BPID);
+    inMemoryLevelDBAliasMapClient = new InMemoryLevelDBAliasMapClient();
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    levelDBAliasMapServer.close();
+    inMemoryLevelDBAliasMapClient.close();
+    FileUtils.deleteDirectory(tempDir);
+  }
+
+  @Test
+  public void writeRead() throws Exception {
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+    Block block = new Block(42, 43, 44);
+    byte[] nonce = "blackbird".getBytes();
+    ProvidedStorageLocation providedStorageLocation
+        = new ProvidedStorageLocation(new Path("cuckoo"),
+        45, 46, nonce);
+    BlockAliasMap.Writer<FileRegion> writer =
+        inMemoryLevelDBAliasMapClient.getWriter(null, BPID);
+    writer.store(new FileRegion(block, providedStorageLocation));
+
+    BlockAliasMap.Reader<FileRegion> reader =
+        inMemoryLevelDBAliasMapClient.getReader(null, BPID);
+    Optional<FileRegion> fileRegion = reader.resolve(block);
+    assertEquals(new FileRegion(block, providedStorageLocation),
+        fileRegion.get());
+  }
+
+  @Test
+  public void iterateSingleBatch() throws Exception {
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+    Block block1 = new Block(42, 43, 44);
+    Block block2 = new Block(43, 44, 45);
+    byte[] nonce1 = "blackbird".getBytes();
+    byte[] nonce2 = "cuckoo".getBytes();
+    ProvidedStorageLocation providedStorageLocation1 =
+        new ProvidedStorageLocation(new Path("eagle"),
+        46, 47, nonce1);
+    ProvidedStorageLocation providedStorageLocation2 =
+        new ProvidedStorageLocation(new Path("falcon"),
+            46, 47, nonce2);
+    BlockAliasMap.Writer<FileRegion> writer1 =
+        inMemoryLevelDBAliasMapClient.getWriter(null, BPID);
+    writer1.store(new FileRegion(block1, providedStorageLocation1));
+    BlockAliasMap.Writer<FileRegion> writer2 =
+        inMemoryLevelDBAliasMapClient.getWriter(null, BPID);
+    writer2.store(new FileRegion(block2, providedStorageLocation2));
+
+    BlockAliasMap.Reader<FileRegion> reader =
+        inMemoryLevelDBAliasMapClient.getReader(null, BPID);
+    List<FileRegion> actualFileRegions =
+        Lists.newArrayListWithCapacity(2);
+    for (FileRegion fileRegion : reader) {
+      actualFileRegions.add(fileRegion);
+    }
+
+    assertArrayEquals(
+        new FileRegion[] {new FileRegion(block1, providedStorageLocation1),
+            new FileRegion(block2, providedStorageLocation2)},
+        actualFileRegions.toArray());
+  }
+
+  @Test
+  public void iterateThreeBatches() throws Exception {
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE, "2");
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+    Block block1 = new Block(42, 43, 44);
+    Block block2 = new Block(43, 44, 45);
+    Block block3 = new Block(44, 45, 46);
+    Block block4 = new Block(47, 48, 49);
+    Block block5 = new Block(50, 51, 52);
+    Block block6 = new Block(53, 54, 55);
+    byte[] nonce1 = "blackbird".getBytes();
+    byte[] nonce2 = "cuckoo".getBytes();
+    byte[] nonce3 = "sparrow".getBytes();
+    byte[] nonce4 = "magpie".getBytes();
+    byte[] nonce5 = "seagull".getBytes();
+    byte[] nonce6 = "finch".getBytes();
+    ProvidedStorageLocation providedStorageLocation1 =
+        new ProvidedStorageLocation(new Path("eagle"),
+            46, 47, nonce1);
+    ProvidedStorageLocation providedStorageLocation2 =
+        new ProvidedStorageLocation(new Path("falcon"),
+            48, 49, nonce2);
+    ProvidedStorageLocation providedStorageLocation3 =
+        new ProvidedStorageLocation(new Path("robin"),
+            50, 51, nonce3);
+    ProvidedStorageLocation providedStorageLocation4 =
+        new ProvidedStorageLocation(new Path("parakeet"),
+            52, 53, nonce4);
+    ProvidedStorageLocation providedStorageLocation5 =
+        new ProvidedStorageLocation(new Path("heron"),
+            54, 55, nonce5);
+    ProvidedStorageLocation providedStorageLocation6 =
+        new ProvidedStorageLocation(new Path("duck"),
+            56, 57, nonce6);
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null, BPID)
+        .store(new FileRegion(block1, providedStorageLocation1));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null, BPID)
+        .store(new FileRegion(block2, providedStorageLocation2));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null, BPID)
+        .store(new FileRegion(block3, providedStorageLocation3));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null, BPID)
+        .store(new FileRegion(block4, providedStorageLocation4));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null, BPID)
+        .store(new FileRegion(block5, providedStorageLocation5));
+    inMemoryLevelDBAliasMapClient
+        .getWriter(null, BPID)
+        .store(new FileRegion(block6, providedStorageLocation6));
+
+    BlockAliasMap.Reader<FileRegion> reader =
+        inMemoryLevelDBAliasMapClient.getReader(null, BPID);
+    List<FileRegion> actualFileRegions =
+        Lists.newArrayListWithCapacity(6);
+    for (FileRegion fileRegion : reader) {
+      actualFileRegions.add(fileRegion);
+    }
+
+    FileRegion[] expectedFileRegions =
+        new FileRegion[] {new FileRegion(block1, providedStorageLocation1),
+            new FileRegion(block2, providedStorageLocation2),
+            new FileRegion(block3, providedStorageLocation3),
+            new FileRegion(block4, providedStorageLocation4),
+            new FileRegion(block5, providedStorageLocation5),
+            new FileRegion(block6, providedStorageLocation6)};
+    assertArrayEquals(expectedFileRegions, actualFileRegions.toArray());
+  }
+
+
+  class ReadThread implements Runnable {
+    private final Block block;
+    private final BlockAliasMap.Reader<FileRegion> reader;
+    private int delay;
+    private Optional<FileRegion> fileRegionOpt;
+
+    ReadThread(Block block, BlockAliasMap.Reader<FileRegion> reader,
+        int delay) {
+      this.block = block;
+      this.reader = reader;
+      this.delay = delay;
+    }
+
+    public Optional<FileRegion> getFileRegion() {
+      return fileRegionOpt;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Thread.sleep(delay);
+        fileRegionOpt = reader.resolve(block);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  class WriteThread implements Runnable {
+    private final Block block;
+    private final BlockAliasMap.Writer<FileRegion> writer;
+    private final ProvidedStorageLocation providedStorageLocation;
+    private int delay;
+
+    WriteThread(Block block, ProvidedStorageLocation providedStorageLocation,
+        BlockAliasMap.Writer<FileRegion> writer, int delay) {
+      this.block = block;
+      this.writer = writer;
+      this.providedStorageLocation = providedStorageLocation;
+      this.delay = delay;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Thread.sleep(delay);
+        writer.store(new FileRegion(block, providedStorageLocation));
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  public FileRegion generateRandomFileRegion(int seed) {
+    Block block = new Block(seed, seed + 1, seed + 2);
+    Path path = new Path("koekoek");
+    byte[] nonce = new byte[0];
+    ProvidedStorageLocation providedStorageLocation =
+        new ProvidedStorageLocation(path, seed + 3, seed + 4, nonce);
+    return new FileRegion(block, providedStorageLocation);
+  }
+
+  @Test
+  public void multipleReads() throws IOException {
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+
+    Random r = new Random();
+    List<FileRegion> expectedFileRegions = r.ints(0, 200)
+        .limit(50)
+        .boxed()
+        .map(i -> generateRandomFileRegion(i))
+        .collect(Collectors.toList());
+
+
+    BlockAliasMap.Reader<FileRegion> reader =
+        inMemoryLevelDBAliasMapClient.getReader(null, BPID);
+    BlockAliasMap.Writer<FileRegion> writer =
+        inMemoryLevelDBAliasMapClient.getWriter(null, BPID);
+
+    ExecutorService executor = Executors.newCachedThreadPool();
+
+    List<ReadThread> readThreads = expectedFileRegions
+        .stream()
+        .map(fileRegion -> new ReadThread(fileRegion.getBlock(),
+            reader,
+            4000))
+        .collect(Collectors.toList());
+
+
+    List<? extends Future<?>> readFutures =
+        readThreads.stream()
+            .map(readThread -> executor.submit(readThread))
+            .collect(Collectors.toList());
+
+    List<? extends Future<?>> writeFutures = expectedFileRegions
+        .stream()
+        .map(fileRegion -> new WriteThread(fileRegion.getBlock(),
+            fileRegion.getProvidedStorageLocation(),
+            writer,
+            1000))
+        .map(writeThread -> executor.submit(writeThread))
+        .collect(Collectors.toList());
+
+    readFutures.stream()
+        .map(readFuture -> {
+          try {
+            return readFuture.get();
+          } catch (InterruptedException e) {
+            throw new RuntimeException(e);
+          } catch (ExecutionException e) {
+            throw new RuntimeException(e);
+          }
+        })
+        .collect(Collectors.toList());
+
+    List<FileRegion> actualFileRegions = readThreads.stream()
+        .map(readThread -> readThread.getFileRegion().get())
+        .collect(Collectors.toList());
+
+    assertThat(actualFileRegions).containsExactlyInAnyOrder(
+        expectedFileRegions.toArray(new FileRegion[0]));
+  }
+}

+ 117 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDBFileRegionAliasMap.java

@@ -0,0 +1,117 @@
+/**
+ * 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.fs.Path;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.junit.Test;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.util.Iterator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+/**
+ * Tests for the {@link LevelDBFileRegionAliasMap}.
+ */
+public class TestLevelDBFileRegionAliasMap {
+
+  private static final String BPID = "BPID-0";
+
+  /**
+   * A basic test to verify that we can write data and read it back again.
+   * @throws Exception
+   */
+  @Test
+  public void testReadBack() throws Exception {
+    File dbFile = Files.createTempDirectory("fileregionformat")
+        .toFile();
+    try {
+      LevelDBFileRegionAliasMap frf = new LevelDBFileRegionAliasMap();
+      LevelDBFileRegionAliasMap.LevelDBOptions opts =
+          new LevelDBFileRegionAliasMap.LevelDBOptions()
+              .filename(dbFile.getAbsolutePath());
+      BlockAliasMap.Writer<FileRegion> writer = frf.getWriter(opts, BPID);
+
+      FileRegion fr = new FileRegion(1, new Path("/file"), 1, 1, 1);
+      writer.store(fr);
+      writer.close();
+
+      BlockAliasMap.Reader<FileRegion> reader = frf.getReader(opts, BPID);
+      FileRegion fr2 = reader.resolve(new Block(1, 1, 1)).get();
+      assertEquals(fr, fr2);
+      reader.close();
+    } finally {
+      dbFile.delete();
+    }
+  }
+
+  @Test
+  /**
+   * A basic test to verify that we can read a bunch of data that we've written.
+   */
+  public void testIterate() throws Exception {
+    FileRegion[] regions = new FileRegion[10];
+    regions[0] = new FileRegion(1, new Path("/file1"), 0, 1024, 1);
+    regions[1] = new FileRegion(2, new Path("/file1"), 1024, 1024, 1);
+    regions[2] = new FileRegion(3, new Path("/file1"), 2048, 1024, 1);
+    regions[3] = new FileRegion(4, new Path("/file2"), 0, 1024, 1);
+    regions[4] = new FileRegion(5, new Path("/file2"), 1024, 1024, 1);
+    regions[5] = new FileRegion(6, new Path("/file2"), 2048, 1024, 1);
+    regions[6] = new FileRegion(7, new Path("/file2"), 3072, 1024, 1);
+    regions[7] = new FileRegion(8, new Path("/file3"), 0, 1024, 1);
+    regions[8] = new FileRegion(9, new Path("/file4"), 0, 1024, 1);
+    regions[9] = new FileRegion(10, new Path("/file5"), 0, 1024,  1);
+    File dbFile = Files.createTempDirectory("fileregionformat")
+        .toFile();
+    try {
+      LevelDBFileRegionAliasMap frf = new LevelDBFileRegionAliasMap();
+      LevelDBFileRegionAliasMap.LevelDBOptions opts =
+          new LevelDBFileRegionAliasMap.LevelDBOptions()
+              .filename(dbFile.getAbsolutePath());
+      BlockAliasMap.Writer<FileRegion> writer = frf.getWriter(opts, BPID);
+
+      for (FileRegion fr : regions) {
+        writer.store(fr);
+      }
+      writer.close();
+
+      BlockAliasMap.Reader<FileRegion> reader = frf.getReader(opts, BPID);
+      Iterator<FileRegion> it = reader.iterator();
+      int last = -1;
+      int count = 0;
+      while(it.hasNext()) {
+        FileRegion fr = it.next();
+        int blockId = (int)fr.getBlock().getBlockId();
+        assertEquals(regions[blockId-1], fr);
+        assertNotEquals(blockId, last);
+        last = blockId;
+        count++;
+      }
+      assertEquals(count, 10);
+
+      reader.close();
+    } finally {
+      dbFile.delete();
+    }
+  }
+}

+ 121 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDbMockAliasMapClient.java

@@ -0,0 +1,121 @@
+/*
+ * 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 com.google.common.io.Files;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+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.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.iq80.leveldb.DBException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import java.io.File;
+import java.io.IOException;
+
+import static org.assertj.core.api.AssertionsForClassTypes.assertThatExceptionOfType;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests the in-memory alias map with a mock level-db implementation.
+ */
+public class TestLevelDbMockAliasMapClient {
+  private InMemoryLevelDBAliasMapServer levelDBAliasMapServer;
+  private InMemoryLevelDBAliasMapClient inMemoryLevelDBAliasMapClient;
+  private File tempDir;
+  private Configuration conf;
+  private InMemoryAliasMap aliasMapMock;
+  private final String bpid = "BPID-0";
+
+  @Before
+  public void setUp() throws IOException {
+    aliasMapMock = mock(InMemoryAliasMap.class);
+    when(aliasMapMock.getBlockPoolId()).thenReturn(bpid);
+    levelDBAliasMapServer = new InMemoryLevelDBAliasMapServer(
+        (config, blockPoolID) -> aliasMapMock, bpid);
+    conf = new Configuration();
+    int port = 9877;
+
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+        "localhost:" + port);
+    tempDir = Files.createTempDir();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        tempDir.getAbsolutePath());
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+    inMemoryLevelDBAliasMapClient = new InMemoryLevelDBAliasMapClient();
+    inMemoryLevelDBAliasMapClient.setConf(conf);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    levelDBAliasMapServer.close();
+    inMemoryLevelDBAliasMapClient.close();
+    FileUtils.deleteDirectory(tempDir);
+  }
+
+  @Test
+  public void readFailure() throws Exception {
+    Block block = new Block(42, 43, 44);
+    doThrow(new IOException())
+        .doThrow(new DBException())
+        .when(aliasMapMock)
+        .read(block);
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() ->
+            inMemoryLevelDBAliasMapClient.getReader(null, bpid)
+                .resolve(block));
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() ->
+            inMemoryLevelDBAliasMapClient.getReader(null, bpid)
+                .resolve(block));
+  }
+
+  @Test
+  public void writeFailure() throws IOException {
+    Block block = new Block(42, 43, 44);
+    byte[] nonce = new byte[0];
+    Path path = new Path("koekoek");
+    ProvidedStorageLocation providedStorageLocation =
+        new ProvidedStorageLocation(path, 45, 46, nonce);
+
+    doThrow(new IOException())
+        .when(aliasMapMock)
+        .write(block, providedStorageLocation);
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() ->
+            inMemoryLevelDBAliasMapClient.getWriter(null, bpid)
+                .store(new FileRegion(block, providedStorageLocation)));
+
+    assertThatExceptionOfType(IOException.class)
+        .isThrownBy(() ->
+            inMemoryLevelDBAliasMapClient.getWriter(null, bpid)
+                .store(new FileRegion(block, providedStorageLocation)));
+  }
+
+}

+ 200 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestTextBlockAliasMap.java

@@ -0,0 +1,200 @@
+/**
+ * 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.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.util.Iterator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap.*;
+import org.apache.hadoop.hdfs.server.common.FileRegion;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.compress.CompressionCodec;
+
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.junit.Test;
+
+import static org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap.fileNameFromBlockPoolID;
+import static org.junit.Assert.*;
+
+/**
+ * Test for the text based block format for provided block maps.
+ */
+public class TestTextBlockAliasMap {
+
+  static final String OUTFILE_PATH = "hdfs://dummyServer:0000/";
+  static final String OUTFILE_BASENAME = "dummyFile";
+  static final Path OUTFILE = new Path(OUTFILE_PATH, OUTFILE_BASENAME + "txt");
+  static final String BPID = "BPID-0";
+
+  void check(TextWriter.Options opts, final Path vp,
+      final Class<? extends CompressionCodec> vc) throws IOException {
+    TextFileRegionAliasMap mFmt = new TextFileRegionAliasMap() {
+      @Override
+      public TextWriter createWriter(Path file, CompressionCodec codec,
+          String delim, Configuration conf) throws IOException {
+        assertEquals(vp, file);
+        if (null == vc) {
+          assertNull(codec);
+        } else {
+          assertEquals(vc, codec.getClass());
+        }
+        return null; // ignored
+      }
+    };
+    mFmt.getWriter(opts, BPID);
+  }
+
+  void check(TextReader.Options opts, final Path vp,
+      final Class<? extends CompressionCodec> vc) throws IOException {
+    TextFileRegionAliasMap aliasMap = new TextFileRegionAliasMap() {
+      @Override
+      public TextReader createReader(Path file, String delim, Configuration cfg,
+          String blockPoolID) throws IOException {
+        assertEquals(vp, file);
+        if (null != vc) {
+          CompressionCodecFactory factory = new CompressionCodecFactory(cfg);
+          CompressionCodec codec = factory.getCodec(file);
+          assertEquals(vc, codec.getClass());
+        }
+        return null; // ignored
+      }
+    };
+    aliasMap.getReader(opts, BPID);
+  }
+
+  @Test
+  public void testWriterOptions() throws Exception {
+    TextWriter.Options opts = TextWriter.defaults();
+    assertTrue(opts instanceof WriterOptions);
+    WriterOptions wopts = (WriterOptions) opts;
+    Path def =
+        new Path(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_DIR_DEFAULT);
+    assertEquals(def, wopts.getDir());
+    assertNull(wopts.getCodec());
+
+    Path cp = new Path(OUTFILE_PATH, "blocks_" + BPID + ".csv");
+    opts.dirName(new Path(OUTFILE_PATH));
+    check(opts, cp, null);
+
+    opts.codec("gzip");
+    cp = new Path(OUTFILE_PATH, "blocks_" + BPID + ".csv.gz");
+    check(opts, cp, org.apache.hadoop.io.compress.GzipCodec.class);
+  }
+
+  @Test
+  public void testReaderOptions() throws Exception {
+    TextReader.Options opts = TextReader.defaults();
+    assertTrue(opts instanceof ReaderOptions);
+    ReaderOptions ropts = (ReaderOptions) opts;
+
+    Path cp = new Path(OUTFILE_PATH, fileNameFromBlockPoolID(BPID));
+    opts.filename(cp);
+    check(opts, cp, null);
+
+    cp = new Path(OUTFILE_PATH, "blocks_" + BPID + ".csv.gz");
+    opts.filename(cp);
+    check(opts, cp, org.apache.hadoop.io.compress.GzipCodec.class);
+  }
+
+  @Test
+  public void testCSVReadWrite() throws Exception {
+    final DataOutputBuffer out = new DataOutputBuffer();
+    FileRegion r1 = new FileRegion(4344L, OUTFILE, 0, 1024);
+    FileRegion r2 = new FileRegion(4345L, OUTFILE, 1024, 1024);
+    FileRegion r3 = new FileRegion(4346L, OUTFILE, 2048, 512);
+    try (TextWriter csv = new TextWriter(new OutputStreamWriter(out), ",")) {
+      csv.store(r1);
+      csv.store(r2);
+      csv.store(r3);
+    }
+    Iterator<FileRegion> i3;
+    try (TextReader csv = new TextReader(null, null, null, ",") {
+      @Override
+      public InputStream createStream() {
+        DataInputBuffer in = new DataInputBuffer();
+        in.reset(out.getData(), 0, out.getLength());
+        return in;
+        }}) {
+      Iterator<FileRegion> i1 = csv.iterator();
+      assertEquals(r1, i1.next());
+      Iterator<FileRegion> i2 = csv.iterator();
+      assertEquals(r1, i2.next());
+      assertEquals(r2, i2.next());
+      assertEquals(r3, i2.next());
+      assertEquals(r2, i1.next());
+      assertEquals(r3, i1.next());
+
+      assertFalse(i1.hasNext());
+      assertFalse(i2.hasNext());
+      i3 = csv.iterator();
+    }
+    try {
+      i3.next();
+    } catch (IllegalStateException e) {
+      return;
+    }
+    fail("Invalid iterator");
+  }
+
+  @Test
+  public void testCSVReadWriteTsv() throws Exception {
+    final DataOutputBuffer out = new DataOutputBuffer();
+    FileRegion r1 = new FileRegion(4344L, OUTFILE, 0, 1024);
+    FileRegion r2 = new FileRegion(4345L, OUTFILE, 1024, 1024);
+    FileRegion r3 = new FileRegion(4346L, OUTFILE, 2048, 512);
+    try (TextWriter csv = new TextWriter(new OutputStreamWriter(out), "\t")) {
+      csv.store(r1);
+      csv.store(r2);
+      csv.store(r3);
+    }
+    Iterator<FileRegion> i3;
+    try (TextReader csv = new TextReader(null, null, null, "\t") {
+      @Override
+      public InputStream createStream() {
+        DataInputBuffer in = new DataInputBuffer();
+        in.reset(out.getData(), 0, out.getLength());
+        return in;
+      }}) {
+      Iterator<FileRegion> i1 = csv.iterator();
+      assertEquals(r1, i1.next());
+      Iterator<FileRegion> i2 = csv.iterator();
+      assertEquals(r1, i2.next());
+      assertEquals(r2, i2.next());
+      assertEquals(r3, i2.next());
+      assertEquals(r2, i1.next());
+      assertEquals(r3, i1.next());
+
+      assertFalse(i1.hasNext());
+      assertFalse(i2.hasNext());
+      i3 = csv.iterator();
+    }
+    try {
+      i3.next();
+    } catch (IllegalStateException e) {
+      return;
+    }
+    fail("Invalid iterator");
+  }
+
+}

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

@@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.DataNodeVolumeMetrics;
 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;
@@ -616,7 +617,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     this.datanode = datanode;
     if (storage != null) {
       for (int i = 0; i < storage.getNumStorageDirs(); ++i) {
-        DataStorage.createStorageID(storage.getStorageDir(i), false);
+        DataStorage.createStorageID(storage.getStorageDir(i), false, conf);
       }
       this.datanodeUuid = storage.getDatanodeUuid();
     } else {
@@ -1352,8 +1353,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   @Override
-  public void checkAndUpdate(String bpid, long blockId, File diskFile,
-      File diskMetaFile, FsVolumeSpi vol) throws IOException {
+  public void checkAndUpdate(String bpid, ScanInfo info) throws IOException {
     throw new UnsupportedOperationException();
   }
 

+ 162 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java

@@ -0,0 +1,162 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.io.input.BoundedInputStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests the implementation of {@link ProvidedReplica}.
+ */
+public class TestProvidedReplicaImpl {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestProvidedReplicaImpl.class);
+  private static final String BASE_DIR =
+      new FileSystemTestHelper().getTestRootDir();
+  private static final String FILE_NAME = "provided-test";
+  // length of the file that is associated with the provided blocks.
+  private static final long FILE_LEN = 128 * 1024 * 10L + 64 * 1024;
+  // length of each provided block.
+  private static final long BLK_LEN = 128 * 1024L;
+
+  private static List<ProvidedReplica> replicas;
+
+  private static void createFileIfNotExists(String baseDir) throws IOException {
+    File newFile = new File(baseDir, FILE_NAME);
+    newFile.getParentFile().mkdirs();
+    if(!newFile.exists()) {
+      newFile.createNewFile();
+      OutputStream writer = new FileOutputStream(newFile.getAbsolutePath());
+      byte[] bytes = new byte[1];
+      bytes[0] = (byte) 0;
+      for(int i=0; i< FILE_LEN; i++) {
+        writer.write(bytes);
+      }
+      writer.flush();
+      writer.close();
+      LOG.info("Created provided file " + newFile +
+          " of length " + newFile.length());
+    }
+  }
+
+  private static void createProvidedReplicas(Configuration conf) {
+    long numReplicas = (long) Math.ceil((double) FILE_LEN/BLK_LEN);
+    File providedFile = new File(BASE_DIR, FILE_NAME);
+    replicas = new ArrayList<ProvidedReplica>();
+
+    LOG.info("Creating " + numReplicas + " provided replicas");
+    for (int i=0; i<numReplicas; i++) {
+      long currentReplicaLength =
+          FILE_LEN >= (i+1)*BLK_LEN ? BLK_LEN : FILE_LEN - i*BLK_LEN;
+      replicas.add(
+          new FinalizedProvidedReplica(i, providedFile.toURI(), i*BLK_LEN,
+          currentReplicaLength, 0, null, null, conf, null));
+    }
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    createFileIfNotExists(new File(BASE_DIR).getAbsolutePath());
+    createProvidedReplicas(new Configuration());
+  }
+
+  /**
+   * Checks if {@code ins} matches the provided file from offset
+   * {@code fileOffset} for length {@ dataLength}.
+   * @param file the local file
+   * @param ins input stream to compare against
+   * @param fileOffset offset
+   * @param dataLength length
+   * @throws IOException
+   */
+  public static void verifyReplicaContents(File file,
+      InputStream ins, long fileOffset, long dataLength)
+          throws IOException {
+
+    InputStream fileIns = new FileInputStream(file);
+    fileIns.skip(fileOffset);
+
+    try (ReadableByteChannel i =
+        Channels.newChannel(new BoundedInputStream(fileIns, dataLength))) {
+      try (ReadableByteChannel j = Channels.newChannel(ins)) {
+        ByteBuffer ib = ByteBuffer.allocate(4096);
+        ByteBuffer jb = ByteBuffer.allocate(4096);
+        while (true) {
+          int il = i.read(ib);
+          int jl = j.read(jb);
+          if (il < 0 || jl < 0) {
+            assertEquals(il, jl);
+            break;
+          }
+          ib.flip();
+          jb.flip();
+          int cmp = Math.min(ib.remaining(), jb.remaining());
+          for (int k = 0; k < cmp; ++k) {
+            assertEquals(ib.get(), jb.get());
+          }
+          ib.compact();
+          jb.compact();
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testProvidedReplicaRead() throws IOException {
+
+    File providedFile = new File(BASE_DIR, FILE_NAME);
+    for (int i = 0; i < replicas.size(); i++) {
+      ProvidedReplica replica = replicas.get(i);
+      // block data should exist!
+      assertTrue(replica.blockDataExists());
+      assertEquals(providedFile.toURI(), replica.getBlockURI());
+      verifyReplicaContents(providedFile, replica.getDataInputStream(0),
+          BLK_LEN*i, replica.getBlockDataLength());
+    }
+    LOG.info("All replica contents verified");
+
+    providedFile.delete();
+    // the block data should no longer be found!
+    for(int i=0; i < replicas.size(); i++) {
+      ProvidedReplica replica = replicas.get(i);
+      assertTrue(!replica.blockDataExists());
+    }
+  }
+
+}

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.*;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 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;
@@ -94,8 +95,8 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
   }
 
   @Override
-  public void checkAndUpdate(String bpid, long blockId, File diskFile,
-      File diskMetaFile, FsVolumeSpi vol) {
+  public void checkAndUpdate(String bpid, ScanInfo info) {
+    return;
   }
 
   @Override

+ 10 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java

@@ -119,11 +119,12 @@ public class TestFsDatasetImpl {
   
   private final static String BLOCKPOOL = "BP-TEST";
 
-  private static Storage.StorageDirectory createStorageDirectory(File root)
+  private static Storage.StorageDirectory createStorageDirectory(File root,
+      Configuration conf)
       throws SecurityException, IOException {
     Storage.StorageDirectory sd = new Storage.StorageDirectory(
         StorageLocation.parse(root.toURI().toString()));
-    DataStorage.createStorageID(sd, false);
+    DataStorage.createStorageID(sd, false, conf);
     return sd;
   }
 
@@ -137,7 +138,7 @@ public class TestFsDatasetImpl {
       File loc = new File(BASE_DIR + "/data" + i);
       dirStrings.add(new Path(loc.toString()).toUri().toString());
       loc.mkdirs();
-      dirs.add(createStorageDirectory(loc));
+      dirs.add(createStorageDirectory(loc, conf));
       when(storage.getStorageDir(i)).thenReturn(dirs.get(i));
     }
 
@@ -197,7 +198,8 @@ public class TestFsDatasetImpl {
       String pathUri = new Path(path).toUri().toString();
       expectedVolumes.add(new File(pathUri).getAbsolutePath());
       StorageLocation loc = StorageLocation.parse(pathUri);
-      Storage.StorageDirectory sd = createStorageDirectory(new File(path));
+      Storage.StorageDirectory sd = createStorageDirectory(
+          new File(path), conf);
       DataStorage.VolumeBuilder builder =
           new DataStorage.VolumeBuilder(storage, sd);
       when(storage.prepareVolume(eq(datanode), eq(loc),
@@ -315,7 +317,8 @@ public class TestFsDatasetImpl {
     String newVolumePath = BASE_DIR + "/newVolumeToRemoveLater";
     StorageLocation loc = StorageLocation.parse(newVolumePath);
 
-    Storage.StorageDirectory sd = createStorageDirectory(new File(newVolumePath));
+    Storage.StorageDirectory sd = createStorageDirectory(
+        new File(newVolumePath), conf);
     DataStorage.VolumeBuilder builder =
         new DataStorage.VolumeBuilder(storage, sd);
     when(storage.prepareVolume(eq(datanode), eq(loc),
@@ -348,7 +351,7 @@ public class TestFsDatasetImpl {
         any(ReplicaMap.class),
         any(RamDiskReplicaLruTracker.class));
 
-    Storage.StorageDirectory sd = createStorageDirectory(badDir);
+    Storage.StorageDirectory sd = createStorageDirectory(badDir, conf);
     sd.lock();
     DataStorage.VolumeBuilder builder = new DataStorage.VolumeBuilder(storage, sd);
     when(storage.prepareVolume(eq(datanode),
@@ -492,7 +495,7 @@ public class TestFsDatasetImpl {
     String path = BASE_DIR + "/newData0";
     String pathUri = new Path(path).toUri().toString();
     StorageLocation loc = StorageLocation.parse(pathUri);
-    Storage.StorageDirectory sd = createStorageDirectory(new File(path));
+    Storage.StorageDirectory sd = createStorageDirectory(new File(path), conf);
     DataStorage.VolumeBuilder builder =
         new DataStorage.VolumeBuilder(storage, sd);
     when(

+ 649 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java

@@ -0,0 +1,649 @@
+/**
+ * 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 static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathHandle;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+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.common.FileRegion;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
+import org.apache.hadoop.hdfs.server.datanode.DNConf;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedProvidedReplica;
+import org.apache.hadoop.hdfs.server.datanode.ProvidedReplica;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdfs.server.datanode.TestProvidedReplicaImpl;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.BlockIterator;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.AutoCloseableLock;
+import org.apache.hadoop.util.StringUtils;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Basic test cases for provided implementation.
+ */
+public class TestProvidedImpl {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestFsDatasetImpl.class);
+  private static final String BASE_DIR =
+      new FileSystemTestHelper().getTestRootDir();
+  private static final int NUM_LOCAL_INIT_VOLUMES = 1;
+  // only support one provided volume for now.
+  private static final int NUM_PROVIDED_INIT_VOLUMES = 1;
+  private static final String[] BLOCK_POOL_IDS = {"bpid-0", "bpid-1"};
+  private static final int NUM_PROVIDED_BLKS = 10;
+  private static final long BLK_LEN = 128 * 1024;
+  private static final int MIN_BLK_ID = 0;
+  private static final int CHOSEN_BP_ID = 0;
+
+  private static String providedBasePath = BASE_DIR;
+
+  private Configuration conf;
+  private DataNode datanode;
+  private DataStorage storage;
+  private FsDatasetImpl dataset;
+  private static Map<Long, String> blkToPathMap;
+  private static List<FsVolumeImpl> providedVolumes;
+  private static long spaceUsed = 0;
+
+  /**
+   * A simple FileRegion iterator for tests.
+   */
+  public static class TestFileRegionIterator implements Iterator<FileRegion> {
+
+    private int numBlocks;
+    private int currentCount;
+    private String basePath;
+
+    public TestFileRegionIterator(String basePath, int minID, int numBlocks) {
+      this.currentCount = minID;
+      this.numBlocks = numBlocks;
+      this.basePath = basePath;
+    }
+
+    @Override
+    public boolean hasNext() {
+      return currentCount < numBlocks;
+    }
+
+    @Override
+    public FileRegion next() {
+      FileRegion region = null;
+      if (hasNext()) {
+        File newFile = new File(basePath, "file" + currentCount);
+        if(!newFile.exists()) {
+          try {
+            LOG.info("Creating file for blkid " + currentCount);
+            blkToPathMap.put((long) currentCount, newFile.getAbsolutePath());
+            LOG.info("Block id " + currentCount + " corresponds to file " +
+                newFile.getAbsolutePath());
+            newFile.createNewFile();
+            Writer writer = new OutputStreamWriter(
+                new FileOutputStream(newFile.getAbsolutePath()), "utf-8");
+            for(int i=0; i< BLK_LEN/(Integer.SIZE/8); i++) {
+              writer.write(currentCount);
+            }
+            writer.flush();
+            writer.close();
+            spaceUsed += BLK_LEN;
+          } catch (IOException e) {
+            e.printStackTrace();
+          }
+        }
+        region = new FileRegion(currentCount, new Path(newFile.toString()),
+            0, BLK_LEN);
+        currentCount++;
+      }
+      return region;
+    }
+
+    @Override
+    public void remove() {
+      // do nothing.
+    }
+
+    public void resetMinBlockId(int minId) {
+      currentCount = minId;
+    }
+
+    public void resetBlockCount(int numBlocks) {
+      this.numBlocks = numBlocks;
+    }
+
+  }
+
+  /**
+   * A simple FileRegion BlockAliasMap for tests.
+   */
+  public static class TestFileRegionBlockAliasMap
+      extends BlockAliasMap<FileRegion> {
+
+    private Configuration conf;
+    private int minId;
+    private int numBlocks;
+    private Iterator<FileRegion> suppliedIterator;
+
+    TestFileRegionBlockAliasMap() {
+      this(null, MIN_BLK_ID, NUM_PROVIDED_BLKS);
+    }
+
+    TestFileRegionBlockAliasMap(Iterator<FileRegion> iterator, int minId,
+                                int numBlocks) {
+      this.suppliedIterator = iterator;
+      this.minId = minId;
+      this.numBlocks = numBlocks;
+    }
+
+    @Override
+    public Reader<FileRegion> getReader(Reader.Options opts, String blockPoolId)
+        throws IOException {
+
+      if (!blockPoolId.equals(BLOCK_POOL_IDS[CHOSEN_BP_ID])) {
+        return null;
+      }
+      BlockAliasMap.Reader<FileRegion> reader =
+          new BlockAliasMap.Reader<FileRegion>() {
+            @Override
+            public Iterator<FileRegion> iterator() {
+              if (suppliedIterator == null) {
+                return new TestFileRegionIterator(providedBasePath, minId,
+                    numBlocks);
+              } else {
+                return suppliedIterator;
+              }
+            }
+
+            @Override
+            public void close() throws IOException {
+
+            }
+
+            @Override
+            public Optional<FileRegion> resolve(Block ident)
+                throws IOException {
+              return null;
+            }
+          };
+      return reader;
+    }
+
+    @Override
+    public Writer<FileRegion> getWriter(Writer.Options opts, String blockPoolId)
+        throws IOException {
+      // not implemented
+      return null;
+    }
+
+    @Override
+    public void refresh() throws IOException {
+      // do nothing!
+    }
+
+    @Override
+    public void close() throws IOException {
+      // do nothing
+    }
+  }
+
+  private static Storage.StorageDirectory createLocalStorageDirectory(
+      File root, Configuration conf)
+      throws SecurityException, IOException {
+    Storage.StorageDirectory sd =
+        new Storage.StorageDirectory(
+            StorageLocation.parse(root.toURI().toString()));
+    DataStorage.createStorageID(sd, false, conf);
+    return sd;
+  }
+
+  private static Storage.StorageDirectory createProvidedStorageDirectory(
+      String confString, Configuration conf)
+      throws SecurityException, IOException {
+    Storage.StorageDirectory sd =
+        new Storage.StorageDirectory(StorageLocation.parse(confString));
+    DataStorage.createStorageID(sd, false, conf);
+    return sd;
+  }
+
+  private static void createStorageDirs(DataStorage storage,
+      Configuration conf, int numDirs, int numProvidedDirs)
+          throws IOException {
+    List<Storage.StorageDirectory> dirs =
+        new ArrayList<Storage.StorageDirectory>();
+    List<String> dirStrings = new ArrayList<String>();
+    FileUtils.deleteDirectory(new File(BASE_DIR));
+    for (int i = 0; i < numDirs; i++) {
+      File loc = new File(BASE_DIR, "data" + i);
+      dirStrings.add(new Path(loc.toString()).toUri().toString());
+      loc.mkdirs();
+      dirs.add(createLocalStorageDirectory(loc, conf));
+      when(storage.getStorageDir(i)).thenReturn(dirs.get(i));
+    }
+
+    for (int i = numDirs; i < numDirs + numProvidedDirs; i++) {
+      File loc = new File(BASE_DIR, "data" + i);
+      providedBasePath = loc.getAbsolutePath();
+      loc.mkdirs();
+      String dirString = "[PROVIDED]" +
+          new Path(loc.toString()).toUri().toString();
+      dirStrings.add(dirString);
+      dirs.add(createProvidedStorageDirectory(dirString, conf));
+      when(storage.getStorageDir(i)).thenReturn(dirs.get(i));
+    }
+
+    String dataDir = StringUtils.join(",", dirStrings);
+    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDir);
+    when(storage.dirIterator()).thenReturn(dirs.iterator());
+    when(storage.getNumStorageDirs()).thenReturn(numDirs + numProvidedDirs);
+  }
+
+  private int getNumVolumes() {
+    try (FsDatasetSpi.FsVolumeReferences volumes =
+        dataset.getFsVolumeReferences()) {
+      return volumes.size();
+    } catch (IOException e) {
+      return 0;
+    }
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    datanode = mock(DataNode.class);
+    storage = mock(DataStorage.class);
+    conf = new Configuration();
+    conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 0);
+
+    when(datanode.getConf()).thenReturn(conf);
+    final DNConf dnConf = new DNConf(datanode);
+    when(datanode.getDnConf()).thenReturn(dnConf);
+    // reset the space used
+    spaceUsed = 0;
+
+    final BlockScanner disabledBlockScanner = new BlockScanner(datanode, conf);
+    when(datanode.getBlockScanner()).thenReturn(disabledBlockScanner);
+    final ShortCircuitRegistry shortCircuitRegistry =
+        new ShortCircuitRegistry(conf);
+    when(datanode.getShortCircuitRegistry()).thenReturn(shortCircuitRegistry);
+
+    this.conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+        TestFileRegionBlockAliasMap.class, BlockAliasMap.class);
+
+    blkToPathMap = new HashMap<Long, String>();
+    providedVolumes = new LinkedList<FsVolumeImpl>();
+
+    createStorageDirs(
+        storage, conf, NUM_LOCAL_INIT_VOLUMES, NUM_PROVIDED_INIT_VOLUMES);
+
+    dataset = new FsDatasetImpl(datanode, storage, conf);
+    FsVolumeReferences volumes = dataset.getFsVolumeReferences();
+    for (int i = 0; i < volumes.size(); i++) {
+      FsVolumeSpi vol = volumes.get(i);
+      if (vol.getStorageType() == StorageType.PROVIDED) {
+        providedVolumes.add((FsVolumeImpl) vol);
+      }
+    }
+
+    for (String bpid : BLOCK_POOL_IDS) {
+      dataset.addBlockPool(bpid, conf);
+    }
+  }
+
+  @Test
+  public void testProvidedVolumeImpl() throws IOException {
+
+    assertEquals(NUM_LOCAL_INIT_VOLUMES + NUM_PROVIDED_INIT_VOLUMES,
+        getNumVolumes());
+    assertEquals(NUM_PROVIDED_INIT_VOLUMES, providedVolumes.size());
+    assertEquals(0, dataset.getNumFailedVolumes());
+
+    for (int i = 0; i < providedVolumes.size(); i++) {
+      // check basic information about provided volume
+      assertEquals(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT,
+          providedVolumes.get(i).getStorageID());
+      assertEquals(StorageType.PROVIDED,
+          providedVolumes.get(i).getStorageType());
+
+      long space = providedVolumes.get(i).getBlockPoolUsed(
+              BLOCK_POOL_IDS[CHOSEN_BP_ID]);
+      // check the df stats of the volume
+      assertEquals(spaceUsed, space);
+      assertEquals(NUM_PROVIDED_BLKS, providedVolumes.get(i).getNumBlocks());
+
+      providedVolumes.get(i).shutdownBlockPool(
+          BLOCK_POOL_IDS[1 - CHOSEN_BP_ID], null);
+      try {
+        assertEquals(0, providedVolumes.get(i)
+            .getBlockPoolUsed(BLOCK_POOL_IDS[1 - CHOSEN_BP_ID]));
+        // should not be triggered
+        assertTrue(false);
+      } catch (IOException e) {
+        LOG.info("Expected exception: " + e);
+      }
+
+    }
+  }
+
+  @Test
+  public void testBlockLoad() throws IOException {
+    for (int i = 0; i < providedVolumes.size(); i++) {
+      FsVolumeImpl vol = providedVolumes.get(i);
+      ReplicaMap volumeMap = new ReplicaMap(new AutoCloseableLock());
+      vol.getVolumeMap(volumeMap, null);
+
+      assertEquals(vol.getBlockPoolList().length, BLOCK_POOL_IDS.length);
+      for (int j = 0; j < BLOCK_POOL_IDS.length; j++) {
+        if (j != CHOSEN_BP_ID) {
+          // this block pool should not have any blocks
+          assertEquals(null, volumeMap.replicas(BLOCK_POOL_IDS[j]));
+        }
+      }
+      assertEquals(NUM_PROVIDED_BLKS,
+          volumeMap.replicas(BLOCK_POOL_IDS[CHOSEN_BP_ID]).size());
+    }
+  }
+
+  @Test
+  public void testProvidedBlockRead() throws IOException {
+    for (int id = 0; id < NUM_PROVIDED_BLKS; id++) {
+      ExtendedBlock eb = new ExtendedBlock(
+          BLOCK_POOL_IDS[CHOSEN_BP_ID], id, BLK_LEN,
+          HdfsConstants.GRANDFATHER_GENERATION_STAMP);
+      InputStream ins = dataset.getBlockInputStream(eb, 0);
+      String filepath = blkToPathMap.get((long) id);
+      TestProvidedReplicaImpl.verifyReplicaContents(new File(filepath), ins, 0,
+          BLK_LEN);
+    }
+  }
+
+  @Test
+  public void testProvidedBlockIterator() throws IOException {
+    for (int i = 0; i < providedVolumes.size(); i++) {
+      FsVolumeImpl vol = providedVolumes.get(i);
+      BlockIterator iter =
+          vol.newBlockIterator(BLOCK_POOL_IDS[CHOSEN_BP_ID], "temp");
+      Set<Long> blockIdsUsed = new HashSet<Long>();
+
+      assertEquals(BLOCK_POOL_IDS[CHOSEN_BP_ID], iter.getBlockPoolId());
+      while(!iter.atEnd()) {
+        ExtendedBlock eb = iter.nextBlock();
+        long blkId = eb.getBlockId();
+        assertTrue(blkId >= MIN_BLK_ID && blkId < NUM_PROVIDED_BLKS);
+        // all block ids must be unique!
+        assertTrue(!blockIdsUsed.contains(blkId));
+        blockIdsUsed.add(blkId);
+      }
+      assertEquals(NUM_PROVIDED_BLKS, blockIdsUsed.size());
+
+      // rewind the block iterator
+      iter.rewind();
+      while(!iter.atEnd()) {
+        ExtendedBlock eb = iter.nextBlock();
+        long blkId = eb.getBlockId();
+        // the block should have already appeared in the first scan.
+        assertTrue(blockIdsUsed.contains(blkId));
+        blockIdsUsed.remove(blkId);
+      }
+      // none of the blocks should remain in blockIdsUsed
+      assertEquals(0, blockIdsUsed.size());
+
+      // the other block pool should not contain any blocks!
+      BlockIterator nonProvidedBpIter =
+          vol.newBlockIterator(BLOCK_POOL_IDS[1 - CHOSEN_BP_ID], "temp");
+      assertEquals(null, nonProvidedBpIter.nextBlock());
+    }
+  }
+
+  private int getBlocksInProvidedVolumes(String basePath, int numBlocks,
+      int minBlockId) throws IOException {
+    TestFileRegionIterator fileRegionIterator =
+        new TestFileRegionIterator(basePath, minBlockId, numBlocks);
+    int totalBlocks = 0;
+    for (int i = 0; i < providedVolumes.size(); i++) {
+      ProvidedVolumeImpl vol = (ProvidedVolumeImpl) providedVolumes.get(i);
+      vol.setFileRegionProvider(BLOCK_POOL_IDS[CHOSEN_BP_ID],
+          new TestFileRegionBlockAliasMap(fileRegionIterator, minBlockId,
+              numBlocks));
+      ReplicaMap volumeMap = new ReplicaMap(new AutoCloseableLock());
+      vol.getVolumeMap(BLOCK_POOL_IDS[CHOSEN_BP_ID], volumeMap, null);
+      totalBlocks += volumeMap.size(BLOCK_POOL_IDS[CHOSEN_BP_ID]);
+    }
+    return totalBlocks;
+  }
+
+  /**
+   * Tests if the FileRegions provided by the FileRegionProvider
+   * can belong to the Providevolume.
+   * @throws IOException
+   */
+  @Test
+  public void testProvidedVolumeContents() throws IOException {
+    int expectedBlocks = 5;
+    int minId = 0;
+    // use a path which has the same prefix as providedBasePath
+    // all these blocks can belong to the provided volume
+    int blocksFound = getBlocksInProvidedVolumes(providedBasePath + "/test1/",
+        expectedBlocks, minId);
+    assertEquals(
+        "Number of blocks in provided volumes should be " + expectedBlocks,
+        expectedBlocks, blocksFound);
+    blocksFound = getBlocksInProvidedVolumes(
+        "file:/" + providedBasePath + "/test1/", expectedBlocks, minId);
+    assertEquals(
+        "Number of blocks in provided volumes should be " + expectedBlocks,
+        expectedBlocks, blocksFound);
+    // use a path that is entirely different from the providedBasePath
+    // none of these blocks can belong to the volume
+    blocksFound =
+        getBlocksInProvidedVolumes("randomtest1/", expectedBlocks, minId);
+    assertEquals("Number of blocks in provided volumes should be 0", 0,
+        blocksFound);
+  }
+
+  @Test
+  public void testProvidedVolumeContainsBlock() throws URISyntaxException {
+    assertEquals(true, ProvidedVolumeImpl.containsBlock(null, null));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("file:/a"), null));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("file:/a/b/c/"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("/a/b/c/"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("/a/b/c"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("/a/b/c/"),
+            new URI("/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("file:/a/b/c/"),
+            new URI("/a/b/c/d/e.file")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("/a/b/e"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("file:/a/b/e"),
+            new URI("file:/a/b/c/d/e.file")));
+    assertEquals(true,
+        ProvidedVolumeImpl.containsBlock(new URI("s3a:/bucket1/dir1/"),
+            new URI("s3a:/bucket1/dir1/temp.txt")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("s3a:/bucket2/dir1/"),
+            new URI("s3a:/bucket1/dir1/temp.txt")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("s3a:/bucket1/dir1/"),
+            new URI("s3a:/bucket1/temp.txt")));
+    assertEquals(false,
+        ProvidedVolumeImpl.containsBlock(new URI("/bucket1/dir1/"),
+            new URI("s3a:/bucket1/dir1/temp.txt")));
+  }
+
+  @Test
+  public void testProvidedReplicaSuffixExtraction() {
+    assertEquals("B.txt", ProvidedVolumeImpl.getSuffix(
+        new Path("file:///A/"), new Path("file:///A/B.txt")));
+    assertEquals("B/C.txt", ProvidedVolumeImpl.getSuffix(
+        new Path("file:///A/"), new Path("file:///A/B/C.txt")));
+    assertEquals("B/C/D.txt", ProvidedVolumeImpl.getSuffix(
+        new Path("file:///A/"), new Path("file:///A/B/C/D.txt")));
+    assertEquals("D.txt", ProvidedVolumeImpl.getSuffix(
+        new Path("file:///A/B/C/"), new Path("file:///A/B/C/D.txt")));
+    assertEquals("file:/A/B/C/D.txt", ProvidedVolumeImpl.getSuffix(
+        new Path("file:///X/B/C/"), new Path("file:///A/B/C/D.txt")));
+    assertEquals("D.txt", ProvidedVolumeImpl.getSuffix(
+        new Path("/A/B/C"), new Path("/A/B/C/D.txt")));
+    assertEquals("D.txt", ProvidedVolumeImpl.getSuffix(
+        new Path("/A/B/C/"), new Path("/A/B/C/D.txt")));
+
+    assertEquals("data/current.csv", ProvidedVolumeImpl.getSuffix(
+        new Path("wasb:///users/alice/"),
+        new Path("wasb:///users/alice/data/current.csv")));
+    assertEquals("current.csv", ProvidedVolumeImpl.getSuffix(
+        new Path("wasb:///users/alice/data"),
+        new Path("wasb:///users/alice/data/current.csv")));
+
+    assertEquals("wasb:/users/alice/data/current.csv",
+        ProvidedVolumeImpl.getSuffix(
+            new Path("wasb:///users/bob/"),
+            new Path("wasb:///users/alice/data/current.csv")));
+  }
+
+  @Test
+  public void testProvidedReplicaPrefix() throws Exception {
+    for (int i = 0; i < providedVolumes.size(); i++) {
+      FsVolumeImpl vol = providedVolumes.get(i);
+      ReplicaMap volumeMap = new ReplicaMap(new AutoCloseableLock());
+      vol.getVolumeMap(volumeMap, null);
+
+      Path expectedPrefix = new Path(
+          StorageLocation.normalizeFileURI(new File(providedBasePath).toURI()));
+      for (ReplicaInfo info : volumeMap
+          .replicas(BLOCK_POOL_IDS[CHOSEN_BP_ID])) {
+        ProvidedReplica pInfo = (ProvidedReplica) info;
+        assertEquals(expectedPrefix, pInfo.getPathPrefix());
+      }
+    }
+  }
+
+  @Test
+  public void testScannerWithProvidedVolumes() throws Exception {
+    DirectoryScanner scanner = new DirectoryScanner(datanode, dataset, conf);
+    Map<String, FsVolumeSpi.ScanInfo[]> report = scanner.getDiskReport();
+    // no blocks should be reported for the Provided volume as long as
+    // the directoryScanner is disabled.
+    assertEquals(0, report.get(BLOCK_POOL_IDS[CHOSEN_BP_ID]).length);
+  }
+
+  /**
+   * Tests that a ProvidedReplica supports path handles.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testProvidedReplicaWithPathHandle() throws Exception {
+
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    cluster.waitActive();
+
+    DistributedFileSystem fs = cluster.getFileSystem();
+
+    // generate random data
+    int chunkSize = 512;
+    Random r = new Random(12345L);
+    byte[] data = new byte[chunkSize];
+    r.nextBytes(data);
+
+    Path file = new Path("/testfile");
+    try (FSDataOutputStream fout = fs.create(file)) {
+      fout.write(data);
+    }
+
+    PathHandle pathHandle = fs.getPathHandle(fs.getFileStatus(file),
+        Options.HandleOpt.changed(true), Options.HandleOpt.moved(true));
+    FinalizedProvidedReplica replica = new FinalizedProvidedReplica(0,
+        file.toUri(), 0, chunkSize, 0, pathHandle, null, conf, fs);
+    byte[] content = new byte[chunkSize];
+    IOUtils.readFully(replica.getDataInputStream(0), content, 0, chunkSize);
+    assertArrayEquals(data, content);
+
+    fs.rename(file, new Path("/testfile.1"));
+    // read should continue succeeding after the rename operation
+    IOUtils.readFully(replica.getDataInputStream(0), content, 0, chunkSize);
+    assertArrayEquals(data, content);
+
+    replica.setPathHandle(null);
+    try {
+      // expected to fail as URI of the provided replica is no longer valid.
+      replica.getDataInputStream(0);
+      fail("Expected an exception");
+    } catch (IOException e) {
+      LOG.info("Expected exception " + e);
+    }
+  }
+}

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestFederationMetrics.java

@@ -190,6 +190,8 @@ public class TestFederationMetrics extends TestMetricsBase {
           json.getLong("numOfDecomActiveDatanodes"));
       assertEquals(stats.getNumOfDecomDeadDatanodes(),
           json.getLong("numOfDecomDeadDatanodes"));
+      assertEquals(stats.getProvidedSpace(),
+          json.getLong("providedSpace"));
       nameservicesFound++;
     }
     assertEquals(getNameservices().size(), nameservicesFound);

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java

@@ -68,7 +68,10 @@ public class TestClusterId {
       fsImage.getStorage().dirIterator(NNStorage.NameNodeDirType.IMAGE);
     StorageDirectory sd = sdit.next();
     Properties props = Storage.readPropertiesFile(sd.getVersionFile());
-    String cid = props.getProperty("clusterID");
+    String cid = null;
+    if (props != null) {
+      cid = props.getProperty("clusterID");
+    }
     LOG.info("successfully formated : sd="+sd.getCurrentDir() + ";cid="+cid);
     return cid;
   }

+ 6 - 2
hadoop-project/pom.xml

@@ -1336,7 +1336,6 @@
           <artifactId>mssql-jdbc</artifactId>
           <version>${mssql.version}</version>
         </dependency>
-
         <dependency>
           <groupId>io.swagger</groupId>
           <artifactId>swagger-annotations</artifactId>
@@ -1352,7 +1351,12 @@
           <artifactId>snakeyaml</artifactId>
           <version>${snakeyaml.version}</version>
         </dependency>
-
+        <dependency>
+          <groupId>org.assertj</groupId>
+          <artifactId>assertj-core</artifactId>
+          <version>3.8.0</version>
+          <scope>test</scope>
+        </dependency>
     </dependencies>
   </dependencyManagement>
 

+ 28 - 0
hadoop-tools/hadoop-fs2img/dev-support/findbugs-exclude.xml

@@ -0,0 +1,28 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   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.
+-->
+<FindBugsFilter>
+
+  <!-- Spotbugs 3.1.x not detecting that IOUtils::cleanupWithLogger
+       closes streams safely -->
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.server.namenode.ImageWriter" />
+    <Method name="&lt;init&gt;" />
+    <Bug pattern="OBL_UNSATISFIED_OBLIGATION_EXCEPTION_EDGE" />
+  </Match>
+
+</FindBugsFilter>

+ 93 - 0
hadoop-tools/hadoop-fs2img/pom.xml

@@ -0,0 +1,93 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project>
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.1.0-SNAPSHOT</version>
+    <relativePath>../../hadoop-project</relativePath>
+  </parent>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-fs2img</artifactId>
+  <version>3.1.0-SNAPSHOT</version>
+  <description>Apache Hadoop Image Generation Tool</description>
+  <name>Apache Hadoop Image Generation Tool</name>
+  <packaging>jar</packaging>
+
+  <properties>
+    <hadoop.log.dir>${project.build.directory}/log</hadoop.log.dir>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-cli</groupId>
+      <artifactId>commons-cli</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.assertj</groupId>
+      <artifactId>assertj-core</artifactId>
+      <version>3.8.0</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+         <configuration>
+          <archive>
+           <manifest>
+            <mainClass>org.apache.hadoop.hdfs.server.namenode.FileSystemImage</mainClass>
+           </manifest>
+         </archive>
+        </configuration>
+       </plugin>
+    </plugins>
+  </build>
+
+</project>

+ 99 - 0
hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/BlockResolver.java

@@ -0,0 +1,99 @@
+/**
+ * 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.namenode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+
+/**
+ * Given an external reference, create a sequence of blocks and associated
+ * metadata.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class BlockResolver {
+
+  protected BlockProto buildBlock(long blockId, long bytes) {
+    return buildBlock(blockId, bytes, 1001);
+  }
+
+  protected BlockProto buildBlock(long blockId, long bytes, long genstamp) {
+    BlockProto.Builder b = BlockProto.newBuilder()
+        .setBlockId(blockId)
+        .setNumBytes(bytes)
+        .setGenStamp(genstamp);
+    return b.build();
+  }
+
+  /**
+   * @param s the external reference.
+   * @return sequence of blocks that make up the reference.
+   */
+  public Iterable<BlockProto> resolve(FileStatus s) {
+    List<Long> lengths = blockLengths(s);
+    ArrayList<BlockProto> ret = new ArrayList<>(lengths.size());
+    long tot = 0;
+    for (long l : lengths) {
+      tot += l;
+      ret.add(buildBlock(nextId(), l));
+    }
+    if (tot != s.getLen()) {
+      // log a warning?
+      throw new IllegalStateException(
+          "Expected " + s.getLen() + " found " + tot);
+    }
+    return ret;
+  }
+
+  /**
+   * @return the next block id.
+   */
+  public abstract long nextId();
+
+  /**
+   * @return the maximum sequentially allocated block ID for this filesystem.
+   */
+  protected abstract long lastId();
+
+  /**
+   * @param status the external reference.
+   * @return the lengths of the resultant blocks.
+   */
+  protected abstract List<Long> blockLengths(FileStatus status);
+
+
+  /**
+   * @param status the external reference.
+   * @return the block size to assign to this external reference.
+   */
+  public long preferredBlockSize(FileStatus status) {
+    return status.getBlockSize();
+  }
+
+  /**
+   * @param status the external reference.
+   * @return the replication to assign to this external reference.
+   */
+  public abstract int getReplication(FileStatus status);
+
+}

+ 109 - 0
hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.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.namenode;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.ConcurrentModificationException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Traversal of an external FileSystem.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class FSTreeWalk extends TreeWalk {
+
+  private final Path root;
+  private final FileSystem fs;
+
+  public FSTreeWalk(Path root, Configuration conf) throws IOException {
+    this.root = root;
+    fs = root.getFileSystem(conf);
+  }
+
+  @Override
+  protected Iterable<TreePath> getChildren(TreePath path, long id,
+      TreeIterator i) {
+    // TODO symlinks
+    if (!path.getFileStatus().isDirectory()) {
+      return Collections.emptyList();
+    }
+    try {
+      ArrayList<TreePath> ret = new ArrayList<>();
+      for (FileStatus s : fs.listStatus(path.getFileStatus().getPath())) {
+        ret.add(new TreePath(s, id, i, fs));
+      }
+      return ret;
+    } catch (FileNotFoundException e) {
+      throw new ConcurrentModificationException("FS modified");
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  class FSTreeIterator extends TreeIterator {
+
+    private FSTreeIterator() {
+    }
+
+    FSTreeIterator(TreePath p) {
+      getPendingQueue().addFirst(
+          new TreePath(p.getFileStatus(), p.getParentId(), this, fs));
+    }
+
+    FSTreeIterator(Path p) throws IOException {
+      try {
+        FileStatus s = fs.getFileStatus(root);
+        getPendingQueue().addFirst(new TreePath(s, -1L, this, fs));
+      } catch (FileNotFoundException e) {
+        if (p.equals(root)) {
+          throw e;
+        }
+        throw new ConcurrentModificationException("FS modified");
+      }
+    }
+
+    @Override
+    public TreeIterator fork() {
+      if (getPendingQueue().isEmpty()) {
+        return new FSTreeIterator();
+      }
+      return new FSTreeIterator(getPendingQueue().removeFirst());
+    }
+
+  }
+
+  @Override
+  public TreeIterator iterator() {
+    try {
+      return new FSTreeIterator(root);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+}

+ 152 - 0
hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java

@@ -0,0 +1,152 @@
+/**
+ * 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.namenode;
+
+import java.io.File;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.PosixParser;
+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.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * Create FSImage from an external namespace.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class FileSystemImage implements Tool {
+
+  private Configuration conf;
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    // require absolute URI to write anywhere but local
+    FileSystem.setDefaultUri(conf, new File(".").toURI().toString());
+  }
+
+  protected void printUsage() {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp("fs2img [OPTIONS] URI", new Options());
+    formatter.setSyntaxPrefix("");
+    formatter.printHelp("Options", options());
+    ToolRunner.printGenericCommandUsage(System.out);
+  }
+
+  static Options options() {
+    Options options = new Options();
+    options.addOption("o", "outdir", true, "Output directory");
+    options.addOption("u", "ugiclass", true, "UGI resolver class");
+    options.addOption("b", "blockclass", true, "Block output class");
+    options.addOption("i", "blockidclass", true, "Block resolver class");
+    options.addOption("c", "cachedirs", true, "Max active dirents");
+    options.addOption("cid", "clusterID", true, "Cluster ID");
+    options.addOption("bpid", "blockPoolID", true, "Block Pool ID");
+    options.addOption("h", "help", false, "Print usage");
+    return options;
+  }
+
+  @Override
+  public int run(String[] argv) throws Exception {
+    Options options = options();
+    CommandLineParser parser = new PosixParser();
+    CommandLine cmd;
+    try {
+      cmd = parser.parse(options, argv);
+    } catch (ParseException e) {
+      System.out.println(
+          "Error parsing command-line options: " + e.getMessage());
+      printUsage();
+      return -1;
+    }
+
+    if (cmd.hasOption("h")) {
+      printUsage();
+      return -1;
+    }
+
+    ImageWriter.Options opts =
+        ReflectionUtils.newInstance(ImageWriter.Options.class, getConf());
+    for (Option o : cmd.getOptions()) {
+      switch (o.getOpt()) {
+      case "o":
+        opts.output(o.getValue());
+        break;
+      case "u":
+        opts.ugi(Class.forName(o.getValue()).asSubclass(UGIResolver.class));
+        break;
+      case "b":
+        opts.blocks(
+            Class.forName(o.getValue()).asSubclass(BlockAliasMap.class));
+        break;
+      case "i":
+        opts.blockIds(
+            Class.forName(o.getValue()).asSubclass(BlockResolver.class));
+        break;
+      case "c":
+        opts.cache(Integer.parseInt(o.getValue()));
+        break;
+      case "cid":
+        opts.clusterID(o.getValue());
+        break;
+      case "bpid":
+        opts.blockPoolID(o.getValue());
+        break;
+      default:
+        throw new UnsupportedOperationException(
+            "Unknown option: " + o.getOpt());
+      }
+    }
+
+    String[] rem = cmd.getArgs();
+    if (rem.length != 1) {
+      printUsage();
+      return -1;
+    }
+
+    try (ImageWriter w = new ImageWriter(opts)) {
+      for (TreePath e : new FSTreeWalk(new Path(rem[0]), getConf())) {
+        w.accept(e); // add and continue
+      }
+    }
+    return 0;
+  }
+
+  public static void main(String[] argv) throws Exception {
+    int ret = ToolRunner.run(new FileSystemImage(), argv);
+    System.exit(ret);
+  }
+
+}

Some files were not shown because too many files changed in this diff