浏览代码

Merge branch 'trunk' into HDFS-6581

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StorageType.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
	hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
arp 10 年之前
父节点
当前提交
f8bbf80067
共有 100 个文件被更改,包括 6338 次插入776 次删除
  1. 4 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 9 0
      hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
  3. 3 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
  4. 17 19
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
  5. 10 13
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopologyWithNodeGroup.java
  6. 4 8
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java
  7. 25 1
      hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm
  8. 87 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  9. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
  10. 7 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
  11. 21 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/conf/blockStoragePolicy-site.xml
  12. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hdfs-site.xml
  13. 419 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStoragePolicy.java
  14. 19 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  15. 15 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  16. 48 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  17. 33 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  18. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StorageType.java
  19. 14 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  20. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
  21. 8 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
  22. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
  23. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
  24. 16 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  25. 16 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  26. 31 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  27. 7 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  28. 168 73
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
  29. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Matcher.java
  30. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/MovedBlocks.java
  31. 69 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
  32. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
  33. 72 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  34. 13 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
  35. 250 127
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  36. 41 32
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java
  37. 40 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  38. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  39. 22 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
  40. 6 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
  41. 8 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
  42. 160 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  43. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
  44. 115 26
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  45. 644 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
  46. 93 29
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  47. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  48. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  49. 67 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  50. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
  51. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  52. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
  53. 124 101
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  54. 16 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  55. 27 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  56. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryAttributes.java
  57. 45 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  58. 11 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
  59. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
  60. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  61. 12 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
  62. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
  63. 7 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  64. 16 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java
  65. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
  66. 2 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  67. 69 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
  68. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/EnumCounters.java
  69. 7 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  70. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  71. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
  72. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  73. 118 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/blockStoragePolicy-default.xml
  74. 2 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  75. 302 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/ArchivalStorage.apt.vm
  76. 35 8
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSCommands.apt.vm
  77. 114 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  78. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  79. 1075 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
  80. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
  81. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
  82. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
  83. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
  84. 46 106
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java
  85. 81 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStoragePolicyCommands.java
  86. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
  87. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java
  88. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java
  89. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithNodeGroup.java
  90. 18 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
  91. 19 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
  92. 14 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyConsiderLoad.java
  93. 11 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java
  94. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
  95. 423 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
  96. 9 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
  97. 222 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
  98. 766 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
  99. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java
  100. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java

+ 4 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -834,6 +834,10 @@ Release 2.6.0 - UNRELEASED
     HADOOP-11105. MetricsSystemImpl could leak memory in registered callbacks.
     (Chuan Liu via cnauroth)
 
+    HADOOP-10982. KMS: Support for multiple Kerberos principals. (tucu)
+
+    HADOOP-11109. Site build is broken. (Jian He via atm)
+
 Release 2.5.1 - 2014-09-05
 
   INCOMPATIBLE CHANGES

+ 9 - 0
hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh

@@ -328,6 +328,15 @@ esac
 #
 # export HADOOP_BALANCER_OPTS=""
 
+###
+# HDFS Mover specific parameters
+###
+# Specify the JVM options to be used when starting the HDFS Mover.
+# These options will be appended to the options specified as HADOOP_OPTS
+# and therefore may override any similar flags set in HADOOP_OPTS
+#
+# export HADOOP_MOVER_OPTS=""
+
 ###
 # Advanced Users Only!
 ###

+ 3 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java

@@ -45,6 +45,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
+import java.lang.reflect.UndeclaredThrowableException;
 import java.net.HttpURLConnection;
 import java.net.SocketTimeoutException;
 import java.net.URI;
@@ -400,6 +401,8 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
       });
     } catch (IOException ex) {
       throw ex;
+    } catch (UndeclaredThrowableException ex) {
+      throw new IOException(ex.getUndeclaredThrowable());
     } catch (Exception ex) {
       throw new IOException(ex);
     }

+ 17 - 19
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java

@@ -26,6 +26,7 @@ import java.util.TreeMap;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -689,6 +690,12 @@ public class NetworkTopology {
     return rand;
   }
 
+  @VisibleForTesting
+  void setRandomSeed(long seed) {
+    Random rand = getRandom();
+    rand.setSeed(seed);
+  }
+
   /** randomly choose one node from <i>scope</i>
    * if scope starts with ~, choose one from the all nodes except for the
    * ones in <i>scope</i>; otherwise, choose one from <i>scope</i>
@@ -870,21 +877,19 @@ public class NetworkTopology {
   /**
    * Sort nodes array by network distance to <i>reader</i>.
    * <p/>
-   * In a three-level topology, a node can be either local, on the same rack, or
-   * on a different rack from the reader. Sorting the nodes based on network
-   * distance from the reader reduces network traffic and improves performance.
+   * In a three-level topology, a node can be either local, on the same rack,
+   * or on a different rack from the reader. Sorting the nodes based on network
+   * distance from the reader reduces network traffic and improves
+   * performance.
    * <p/>
    * As an additional twist, we also randomize the nodes at each network
-   * distance using the provided random seed. This helps with load balancing
-   * when there is data skew.
-   * 
-   * @param reader Node where data will be read
-   * @param nodes Available replicas with the requested data
-   * @param seed Used to seed the pseudo-random generator that randomizes the
-   *          set of nodes at each network distance.
+   * distance. This helps with load balancing when there is data skew.
+   *
+   * @param reader    Node where data will be read
+   * @param nodes     Available replicas with the requested data
+   * @param activeLen Number of active nodes at the front of the array
    */
-  public void sortByDistance(Node reader, Node[] nodes, int activeLen,
-      long seed, boolean randomizeBlockLocationsPerBlock) {
+  public void sortByDistance(Node reader, Node[] nodes, int activeLen) {
     /** Sort weights for the nodes array */
     int[] weights = new int[activeLen];
     for (int i=0; i<activeLen; i++) {
@@ -903,14 +908,7 @@ public class NetworkTopology {
       list.add(node);
     }
 
-    // Seed is normally the block id
-    // This means we use the same pseudo-random order for each block, for
-    // potentially better page cache usage.
-    // Seed is not used if we want to randomize block location for every block
     Random rand = getRandom();
-    if (!randomizeBlockLocationsPerBlock) {
-      rand.setSeed(seed);
-    }
     int idx = 0;
     for (List<Node> list: tree.values()) {
       if (list != null) {

+ 10 - 13
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopologyWithNodeGroup.java

@@ -268,19 +268,17 @@ public class NetworkTopologyWithNodeGroup extends NetworkTopology {
   /**
    * Sort nodes array by their distances to <i>reader</i>.
    * <p/>
-   * This is the same as
-   * {@link NetworkTopology#sortByDistance(Node, Node[], long)} except with a
-   * four-level network topology which contains the additional network distance
-   * of a "node group" which is between local and same rack.
-   * 
-   * @param reader Node where data will be read
-   * @param nodes Available replicas with the requested data
-   * @param seed Used to seed the pseudo-random generator that randomizes the
-   *          set of nodes at each network distance.
+   * This is the same as {@link NetworkTopology#sortByDistance(Node, Node[],
+   * int)} except with a four-level network topology which contains the
+   * additional network distance of a "node group" which is between local and
+   * same rack.
+   *
+   * @param reader    Node where data will be read
+   * @param nodes     Available replicas with the requested data
+   * @param activeLen Number of active nodes at the front of the array
    */
   @Override
-  public void sortByDistance(Node reader, Node[] nodes, int activeLen,
-      long seed, boolean randomizeBlockLocationsPerBlock) {
+  public void sortByDistance(Node reader, Node[] nodes, int activeLen) {
     // If reader is not a datanode (not in NetworkTopology tree), we need to
     // replace this reader with a sibling leaf node in tree.
     if (reader != null && !this.contains(reader)) {
@@ -293,8 +291,7 @@ public class NetworkTopologyWithNodeGroup extends NetworkTopology {
         return;
       }
     }
-    super.sortByDistance(reader, nodes, activeLen, seed,
-        randomizeBlockLocationsPerBlock);
+    super.sortByDistance(reader, nodes, activeLen);
   }
 
   /** InnerNodeWithNodeGroup represents a switch/router of a data center, rack

+ 4 - 8
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java

@@ -104,8 +104,7 @@ public class TestNetworkTopologyWithNodeGroup {
     testNodes[1] = dataNodes[2];
     testNodes[2] = dataNodes[3];
     testNodes[3] = dataNodes[0];
-    cluster.sortByDistance(dataNodes[0], testNodes,
-        testNodes.length, 0xDEADBEEF, false);
+    cluster.sortByDistance(dataNodes[0], testNodes, testNodes.length);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[1]);
     assertTrue(testNodes[2] == dataNodes[2]);
@@ -116,8 +115,7 @@ public class TestNetworkTopologyWithNodeGroup {
     testNodes[1] = dataNodes[4];
     testNodes[2] = dataNodes[1];
     testNodes[3] = dataNodes[0];
-    cluster.sortByDistance(dataNodes[0], testNodes,
-        testNodes.length, 0xDEADBEEF, false);
+    cluster.sortByDistance(dataNodes[0], testNodes, testNodes.length);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[1]);
 
@@ -126,8 +124,7 @@ public class TestNetworkTopologyWithNodeGroup {
     testNodes[1] = dataNodes[3];
     testNodes[2] = dataNodes[2];
     testNodes[3] = dataNodes[0];
-    cluster.sortByDistance(dataNodes[0], testNodes,
-        testNodes.length, 0xDEADBEEF, false);
+    cluster.sortByDistance(dataNodes[0], testNodes, testNodes.length);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[2]);
 
@@ -136,8 +133,7 @@ public class TestNetworkTopologyWithNodeGroup {
     testNodes[1] = dataNodes[7];
     testNodes[2] = dataNodes[2];
     testNodes[3] = dataNodes[0];
-    cluster.sortByDistance(computeNode, testNodes,
-        testNodes.length, 0xDEADBEEF, false);
+    cluster.sortByDistance(computeNode, testNodes, testNodes.length);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[2]);
   }

+ 25 - 1
hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm

@@ -602,7 +602,31 @@ $ keytool -genkey -alias tomcat -keyalg RSA
 
 *** HTTP Kerberos Principals Configuration
 
-  TBD
+  When KMS instances are behind a load-balancer or VIP, clients will use the
+  hostname of the VIP. For Kerberos SPNEGO authentication, the hostname of the
+  URL is used to construct the Kerberos service name of the server,
+  <<<HTTP/#HOSTNAME#>>>. This means that all KMS instances must have a Kerberos
+  service name with the load-balancer or VIP hostname.
+
+  In order to be able to access directly a specific KMS instance, the KMS
+  instance must also have Keberos service name with its own hostname. This is
+  required for monitoring and admin purposes.
+
+  Both Kerberos service principal credentials (for the load-balancer/VIP
+  hostname and for the actual KMS instance hostname) must be in the keytab file
+  configured for authentication. And the principal name specified in the
+  configuration must be '*'. For example:
+
++---+
+  <property>
+    <name>hadoop.kms.authentication.kerberos.principal</name>
+    <value>*</value>
+  </property>
++---+
+
+  <<NOTE:>> If using HTTPS, the SSL certificate used by the KMS instance must
+  be configured to support multiple hostnames (see Java 7
+  <<<keytool>>> SAN extension support for details on how to do this).
 
 *** HTTP Authentication Signature
 

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

@@ -18,6 +18,9 @@ Trunk (Unreleased)
 
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
 
+    HDFS-6584. Support archival storage.  (See breakdown of tasks below for
+    features and contributors)
+
   IMPROVEMENTS
 
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
@@ -261,6 +264,82 @@ Trunk (Unreleased)
 
     HDFS-6981. Fix DN upgrade with layout version change. (Arpit Agarwal)
 
+  BREAKDOWN OF HDFS-6584 ARCHIVAL STORAGE
+
+    HDFS-6677. Change INodeFile and FSImage to support storage policy ID.
+    (szetszwo)
+
+    HDFS-6670. Add block storage policy support with default HOT, WARM and COLD
+    policies.  (szetszwo)
+
+    HDFS-6671. Change BlockPlacementPolicy to consider block storage policy
+    in replicaiton.  (szetszwo)
+
+    HDFS-6710. Change BlockPlacementPolicy to consider block storage policy
+    in replica deletion.  (szetszwo)
+
+    HDFS-6679. Bump NameNodeLayoutVersion and update editsStored test files.
+    (vinayakumarb via szetszwo)
+
+    HDFS-6686. Change BlockPlacementPolicy to use fallback when some storage
+    types are unavailable.  (szetszwo)
+
+    HDFS-6835. Add a new API to set storage policy.  (jing9) 
+
+    HDFS-6847. Support storage policy on directories and include storage policy 
+    in HdfsFileStatus.  (Jing Zhao via szetszwo)
+
+    HDFS-6801. Add a new data migration tool, Mover, for archiving data.
+    (szetszwo via jing9)
+
+    HDFS-6863. Support migration for snapshot paths. (jing9)
+
+    HDFS-6906. Add more tests for BlockStoragePolicy.  (szetszwo via jing9)
+
+    HDFS-6911. check if a block is already scheduled in Mover. 
+    (szetszwo via jing9)
+
+    HDFS-6920. Check the storage type of delNodeHintStorage when deleting
+    a replica.  (szetszwo via jing9)
+
+    HDFS-6944. Add retry and termination logic for Mover. (jing9)
+
+    HDFS-6969. INode#getStoragePolicyID should always return the latest
+    storage policy.  (jing9)
+
+    HDFS-6961. BlockPlacementPolicy#chooseTarget should check each valid
+    storage type in each choosing round.  (jing9)
+
+    HDFS-6876. support set/get storage policy in DFSAdmin. (jing9)
+
+    HDFS-6997. Add more tests for data migration and replicaion. (szetszwo)
+
+    HDFS-6875. Support migration for a list of specified paths. (jing9)
+
+    HDFS-7027. Mover does not terminate when some storage type is out of space.
+    (szetszwo via jing9)
+
+    HDFS-7029. Fix TestDFSInotifyEventInputStream and TestDistributedFileSystem.
+    (szetszwo via jing9)
+
+    HDFS-7028. FSDirectory should not get storage policy id from symlinks.
+    (szetszwo)
+
+    HDFS-7034. Fix TestBlockPlacement and TestStorageMover. (jing9)
+
+    HDFS-7039. Fix Balancer tests.  (szetszwo via jing9)
+
+    HDFS-7062. Skip under construction block for migration. (jing9)
+
+    HDFS-7052. Add Mover into hdfs script. (jing9)
+
+    HDFS-7072. Fix TestBlockManager and TestStorageMover.  (jing9 via szetszwo)
+
+    HDFS-6864. Archival Storage: add user documentation. (szetszwo via jing9)
+
+    HDFS-7088. Archival Storage: fix TestBalancer and
+    TestBalancerWithMultipleNameNodes. (szetszwo via jing9)
+
 Release 2.6.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -480,6 +559,11 @@ Release 2.6.0 - UNRELEASED
     HDFS-7003. Add NFS Gateway support for reading and writing to
     encryption zones. (clamb via wang)
 
+    HDFS-6727. Refresh data volumes on DataNode based on configuration changes
+    (Lei Xu via cmccabe)
+
+    HDFS-6970. Move startFile EDEK retries to the DFSClient. (wang)
+
   OPTIMIZATIONS
 
     HDFS-6690. Deduplicate xattr names in memory. (wang)
@@ -687,6 +771,9 @@ Release 2.6.0 - UNRELEASED
 
     HDFS-7078. Fix listEZs to work correctly with snapshots. (wang)
 
+    HDFS-6840. Clients are always sent to the same datanode when read
+    is off rack. (wang)
+
     BREAKDOWN OF HDFS-6134 AND HADOOP-10150 SUBTASKS AND RELATED JIRAS
   
       HDFS-6387. HDFS CLI admin tool for creating & deleting an

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs

@@ -36,6 +36,8 @@ function hadoop_usage
   echo "  journalnode          run the DFS journalnode"
   echo "  lsSnapshottableDir   list all snapshottable dirs owned by the current user"
   echo "                               Use -help to see options"
+  echo "  mover                run a utility to move block replicas across"
+  echo "                       storage types"
   echo "  namenode             run the DFS namenode"
   echo "                               Use -format to initialize the DFS filesystem"
   echo "  nfs3                 run an NFS version 3 gateway"
@@ -159,6 +161,11 @@ case ${COMMAND} in
   lsSnapshottableDir)
     CLASS=org.apache.hadoop.hdfs.tools.snapshot.LsSnapshottableDir
   ;;
+  mover)
+    CLASS=org.apache.hadoop.hdfs.server.mover.Mover
+    hadoop_debug "Appending HADOOP_MOVER_OPTS onto HADOOP_OPTS"
+    HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_MOVER_OPTS}"
+  ;;
   namenode)
     daemon="true"
     CLASS='org.apache.hadoop.hdfs.server.namenode.NameNode'

+ 7 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd

@@ -47,7 +47,7 @@ if "%1" == "--config" (
       goto print_usage
   )
 
-  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin
+  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover
   for %%i in ( %hdfscommands% ) do (
     if %hdfs-command% == %%i set hdfscommand=true
   )
@@ -150,6 +150,11 @@ goto :eof
   set CLASS=org.apache.hadoop.hdfs.tools.CacheAdmin
   goto :eof
 
+:mover
+  set CLASS=org.apache.hadoop.hdfs.server.mover.Mover
+  set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_MOVER_OPTS%
+  goto :eof
+
 @rem This changes %1, %2 etc. Hence those cannot be used after calling this.
 :make_command_arguments
   if "%1" == "--config" (
@@ -198,6 +203,7 @@ goto :eof
   @echo   lsSnapshottableDir   list all snapshottable dirs owned by the current user
   @echo 						Use -help to see options
   @echo   cacheadmin           configure the HDFS cache
+  @echo   mover                run a utility to move block replicas across storage types
   @echo.
   @echo Most commands print help when invoked w/o parameters.
 

+ 21 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/conf/blockStoragePolicy-site.xml

@@ -0,0 +1,21 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  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.
+-->
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration> 
+
+</configuration>

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hdfs-site.xml

@@ -16,6 +16,7 @@
 
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration>
+<configuration xmlns:xi="http://www.w3.org/2001/XInclude"> 
+  <xi:include href="blockStoragePolicy-site.xml" />
 
 </configuration>

+ 419 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStoragePolicy.java

@@ -0,0 +1,419 @@
+/**
+ * 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;
+
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.LinkedList;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttr.NameSpace;
+
+/**
+ * A block storage policy describes how to select the storage types
+ * for the replicas of a block.
+ */
+@InterfaceAudience.Private
+public class BlockStoragePolicy {
+  public static final Log LOG = LogFactory.getLog(BlockStoragePolicy.class);
+
+  public static final String DFS_BLOCK_STORAGE_POLICIES_KEY
+      = "dfs.block.storage.policies";
+  public static final String DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX
+      = "dfs.block.storage.policy.";
+  public static final String DFS_BLOCK_STORAGE_POLICY_CREATION_FALLBACK_KEY_PREFIX
+      = "dfs.block.storage.policy.creation-fallback.";
+  public static final String DFS_BLOCK_STORAGE_POLICY_REPLICATION_FALLBACK_KEY_PREFIX
+      = "dfs.block.storage.policy.replication-fallback.";
+  public static final String STORAGE_POLICY_XATTR_NAME = "bsp";
+  /** set the namespace to TRUSTED so that only privilege users can access */
+  public static final NameSpace XAttrNS = NameSpace.TRUSTED;
+
+  public static final int ID_BIT_LENGTH = 4;
+  public static final int ID_MAX = (1 << ID_BIT_LENGTH) - 1;
+  public static final byte ID_UNSPECIFIED = 0;
+
+  private static final Suite DEFAULT_SUITE = createDefaultSuite();
+
+  private static Suite createDefaultSuite() {
+    final BlockStoragePolicy[] policies = new BlockStoragePolicy[1 << ID_BIT_LENGTH];
+    final StorageType[] storageTypes = {StorageType.DISK};
+    final byte defaultPolicyId = 12;
+    policies[defaultPolicyId] = new BlockStoragePolicy(defaultPolicyId, "HOT",
+        storageTypes, StorageType.EMPTY_ARRAY, StorageType.EMPTY_ARRAY);
+    return new Suite(defaultPolicyId, policies);
+  }
+
+  /** A block storage policy suite. */
+  public static class Suite {
+    private final byte defaultPolicyID;
+    private final BlockStoragePolicy[] policies;
+    
+    private Suite(byte defaultPolicyID, BlockStoragePolicy[] policies) {
+      this.defaultPolicyID = defaultPolicyID;
+      this.policies = policies;
+    }
+    
+    /** @return the corresponding policy. */
+    public BlockStoragePolicy getPolicy(byte id) {
+      // id == 0 means policy not specified. 
+      return id == 0? getDefaultPolicy(): policies[id];
+    }
+
+    /** @return the default policy. */
+    public BlockStoragePolicy getDefaultPolicy() {
+      return getPolicy(defaultPolicyID);
+    }
+
+    public BlockStoragePolicy getPolicy(String policyName) {
+      if (policies != null) {
+        for (BlockStoragePolicy policy : policies) {
+          if (policy != null && policy.name.equals(policyName)) {
+            return policy;
+          }
+        }
+      }
+      return null;
+    }
+  }
+
+  /** A 4-bit policy ID */
+  private final byte id;
+  /** Policy name */
+  private final String name;
+
+  /** The storage types to store the replicas of a new block. */
+  private final StorageType[] storageTypes;
+  /** The fallback storage type for block creation. */
+  private final StorageType[] creationFallbacks;
+  /** The fallback storage type for replication. */
+  private final StorageType[] replicationFallbacks;
+
+  @VisibleForTesting
+  public BlockStoragePolicy(byte id, String name, StorageType[] storageTypes,
+      StorageType[] creationFallbacks, StorageType[] replicationFallbacks) {
+    this.id = id;
+    this.name = name;
+    this.storageTypes = storageTypes;
+    this.creationFallbacks = creationFallbacks;
+    this.replicationFallbacks = replicationFallbacks;
+  }
+
+  /**
+   * @return a list of {@link StorageType}s for storing the replicas of a block.
+   */
+  public List<StorageType> chooseStorageTypes(final short replication) {
+    final List<StorageType> types = new LinkedList<StorageType>();
+    int i = 0;
+    for(; i < replication && i < storageTypes.length; i++) {
+      types.add(storageTypes[i]);
+    }
+    final StorageType last = storageTypes[storageTypes.length - 1];
+    for(; i < replication; i++) {
+      types.add(last);
+    }
+    return types;
+  }
+
+  /**
+   * Choose the storage types for storing the remaining replicas, given the
+   * replication number and the storage types of the chosen replicas.
+   *
+   * @param replication the replication number.
+   * @param chosen the storage types of the chosen replicas.
+   * @return a list of {@link StorageType}s for storing the replicas of a block.
+   */
+  public List<StorageType> chooseStorageTypes(final short replication,
+      final Iterable<StorageType> chosen) {
+    return chooseStorageTypes(replication, chosen, null);
+  }
+
+  private List<StorageType> chooseStorageTypes(final short replication,
+      final Iterable<StorageType> chosen, final List<StorageType> excess) {
+    final List<StorageType> types = chooseStorageTypes(replication);
+    diff(types, chosen, excess);
+    return types;
+  }
+
+  /**
+   * Choose the storage types for storing the remaining replicas, given the
+   * replication number, the storage types of the chosen replicas and
+   * the unavailable storage types.  It uses fallback storage in case that
+   * the desired storage type is unavailable.  
+   *
+   * @param replication the replication number.
+   * @param chosen the storage types of the chosen replicas.
+   * @param unavailables the unavailable storage types.
+   * @param isNewBlock Is it for new block creation?
+   * @return a list of {@link StorageType}s for storing the replicas of a block.
+   */
+  public List<StorageType> chooseStorageTypes(final short replication,
+      final Iterable<StorageType> chosen,
+      final EnumSet<StorageType> unavailables,
+      final boolean isNewBlock) {
+    final List<StorageType> excess = new LinkedList<StorageType>();
+    final List<StorageType> storageTypes = chooseStorageTypes(
+        replication, chosen, excess);
+    final int expectedSize = storageTypes.size() - excess.size();
+    final List<StorageType> removed = new LinkedList<StorageType>();
+    for(int i = storageTypes.size() - 1; i >= 0; i--) {
+      // replace/remove unavailable storage types.
+      final StorageType t = storageTypes.get(i);
+      if (unavailables.contains(t)) {
+        final StorageType fallback = isNewBlock?
+            getCreationFallback(unavailables)
+            : getReplicationFallback(unavailables);
+        if (fallback == null) {
+          removed.add(storageTypes.remove(i));
+        } else {
+          storageTypes.set(i, fallback);
+        }
+      }
+    }
+    // remove excess storage types after fallback replacement.
+    diff(storageTypes, excess, null);
+    if (storageTypes.size() < expectedSize) {
+      LOG.warn("Failed to place enough replicas: expected size is " + expectedSize 
+          + " but only " + storageTypes.size() + " storage types can be selected "
+          + "(replication=" + replication
+          + ", selected=" + storageTypes
+          + ", unavailable=" + unavailables
+          + ", removed=" + removed
+          + ", policy=" + this + ")");
+    }
+    return storageTypes;
+  }
+
+  /**
+   * Compute the list difference t = t - c.
+   * Further, if e is not null, set e = e + c - t;
+   */
+  private static void diff(List<StorageType> t, Iterable<StorageType> c,
+      List<StorageType> e) {
+    for(StorageType storagetype : c) {
+      final int i = t.indexOf(storagetype);
+      if (i >= 0) {
+        t.remove(i);
+      } else if (e != null) {
+        e.add(storagetype);
+      }
+    }
+  }
+
+  /**
+   * Choose excess storage types for deletion, given the
+   * replication number and the storage types of the chosen replicas.
+   *
+   * @param replication the replication number.
+   * @param chosen the storage types of the chosen replicas.
+   * @return a list of {@link StorageType}s for deletion.
+   */
+  public List<StorageType> chooseExcess(final short replication,
+      final Iterable<StorageType> chosen) {
+    final List<StorageType> types = chooseStorageTypes(replication);
+    final List<StorageType> excess = new LinkedList<StorageType>();
+    diff(types, chosen, excess);
+    return excess;
+  }
+
+  /** @return the fallback {@link StorageType} for creation. */
+  public StorageType getCreationFallback(EnumSet<StorageType> unavailables) {
+    return getFallback(unavailables, creationFallbacks);
+  }
+  
+  /** @return the fallback {@link StorageType} for replication. */
+  public StorageType getReplicationFallback(EnumSet<StorageType> unavailables) {
+    return getFallback(unavailables, replicationFallbacks);
+  }
+
+  @Override
+  public int hashCode() {
+    return Byte.valueOf(id).hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == this) {
+      return true;
+    } else if (obj == null || !(obj instanceof BlockStoragePolicy)) {
+      return false;
+    }
+    final BlockStoragePolicy that = (BlockStoragePolicy)obj;
+    return this.id == that.id;
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "{" + name + ":" + id
+        + ", storageTypes=" + Arrays.asList(storageTypes)
+        + ", creationFallbacks=" + Arrays.asList(creationFallbacks)
+        + ", replicationFallbacks=" + Arrays.asList(replicationFallbacks);
+  }
+
+  public byte getId() {
+    return id;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  private static StorageType getFallback(EnumSet<StorageType> unavailables,
+      StorageType[] fallbacks) {
+    for(StorageType fb : fallbacks) {
+      if (!unavailables.contains(fb)) {
+        return fb;
+      }
+    }
+    return null;
+  }
+  
+  private static byte parseID(String idString, String element, Configuration conf) {
+    byte id = 0;
+    try {
+      id = Byte.parseByte(idString);
+    } catch(NumberFormatException nfe) {
+      throwIllegalArgumentException("Failed to parse policy ID \"" + idString
+          + "\" to a " + ID_BIT_LENGTH + "-bit integer", conf);
+    }
+    if (id < 0) {
+      throwIllegalArgumentException("Invalid policy ID: id = " + id
+          + " < 1 in \"" + element + "\"", conf);
+    } else if (id == 0) {
+      throw new IllegalArgumentException("Policy ID 0 is reserved: " + element);
+    } else if (id > ID_MAX) {
+      throwIllegalArgumentException("Invalid policy ID: id = " + id
+          + " > MAX = " + ID_MAX + " in \"" + element + "\"", conf);
+    }
+    return id;
+  }
+
+  private static StorageType[] parseStorageTypes(String[] strings) {
+    if (strings == null || strings.length == 0) {
+      return StorageType.EMPTY_ARRAY;
+    }
+    final StorageType[] types = new StorageType[strings.length];
+    for(int i = 0; i < types.length; i++) {
+      types[i] = StorageType.valueOf(strings[i].trim().toUpperCase());
+    }
+    return types;
+  }
+  
+  private static StorageType[] readStorageTypes(byte id, String keyPrefix,
+      Configuration conf) {
+    final String key = keyPrefix + id;
+    final String[] values = conf.getStrings(key);
+    try {
+      return parseStorageTypes(values);
+    } catch(Exception e) {
+      throw new IllegalArgumentException("Failed to parse " + key
+          + " \"" + conf.get(key), e);
+    }
+  }
+
+  private static BlockStoragePolicy readBlockStoragePolicy(byte id, String name,
+      Configuration conf) {
+    final StorageType[] storageTypes = readStorageTypes(id, 
+        DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX, conf);
+    if (storageTypes.length == 0) {
+      throw new IllegalArgumentException(
+          DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX + id + " is missing or is empty.");
+    }
+    final StorageType[] creationFallbacks = readStorageTypes(id, 
+        DFS_BLOCK_STORAGE_POLICY_CREATION_FALLBACK_KEY_PREFIX, conf);
+    final StorageType[] replicationFallbacks = readStorageTypes(id, 
+        DFS_BLOCK_STORAGE_POLICY_REPLICATION_FALLBACK_KEY_PREFIX, conf);
+    return new BlockStoragePolicy(id, name, storageTypes, creationFallbacks,
+        replicationFallbacks);
+  }
+
+  /** Read {@link Suite} from conf. */
+  public static Suite readBlockStorageSuite(Configuration conf) {
+    final BlockStoragePolicy[] policies = new BlockStoragePolicy[1 << ID_BIT_LENGTH];
+    final String[] values = conf.getStrings(DFS_BLOCK_STORAGE_POLICIES_KEY);
+    if (values == null) {
+      // conf property is missing, use default suite.
+      return DEFAULT_SUITE;
+    }
+    byte firstID = -1;
+    for(String v : values) {
+      v = v.trim();
+      final int i = v.indexOf(':');
+      if (i < 0) {
+        throwIllegalArgumentException("Failed to parse element \"" + v
+            + "\" (expected format is NAME:ID)", conf);
+      } else if (i == 0) {
+        throwIllegalArgumentException("Policy name is missing in \"" + v + "\"", conf);
+      } else if (i == v.length() - 1) {
+        throwIllegalArgumentException("Policy ID is missing in \"" + v + "\"", conf);
+      }
+      final String name = v.substring(0, i).trim();
+      for(int j = 1; j < policies.length; j++) {
+        if (policies[j] != null && policies[j].name.equals(name)) {
+          throwIllegalArgumentException("Policy name duplication: \""
+              + name + "\" appears more than once", conf);
+        }
+      }
+      
+      final byte id = parseID(v.substring(i + 1).trim(), v, conf);
+      if (policies[id] != null) {
+        throwIllegalArgumentException("Policy duplication: ID " + id
+            + " appears more than once", conf);
+      }
+      policies[id] = readBlockStoragePolicy(id, name, conf);
+      String prefix = "";
+      if (firstID == -1) {
+        firstID = id;
+        prefix = "(default) ";
+      }
+      LOG.info(prefix + policies[id]);
+    }
+    if (firstID == -1) {
+      throwIllegalArgumentException("Empty list is not allowed", conf);
+    }
+    return new Suite(firstID, policies);
+  }
+
+  public static String buildXAttrName() {
+    return XAttrNS.toString().toLowerCase() + "." + STORAGE_POLICY_XATTR_NAME;
+  }
+
+  public static XAttr buildXAttr(byte policyId) {
+    final String name = buildXAttrName();
+    return XAttrHelper.buildXAttr(name, new byte[] { policyId });
+  }
+
+  public static boolean isStoragePolicyXAttr(XAttr xattr) {
+    return xattr != null && xattr.getNameSpace() == BlockStoragePolicy.XAttrNS
+        && xattr.getName().equals(BlockStoragePolicy.STORAGE_POLICY_XATTR_NAME);
+  }
+
+  private static void throwIllegalArgumentException(String message,
+      Configuration conf) {
+    throw new IllegalArgumentException(message + " in "
+        + DFS_BLOCK_STORAGE_POLICIES_KEY + " \""
+        + conf.get(DFS_BLOCK_STORAGE_POLICIES_KEY) + "\".");
+  }
+}

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -1763,6 +1763,25 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  /**
+   * Set storage policy for an existing file/directory
+   * @param src file/directory name
+   * @param policyName name of the storage policy
+   */
+  public void setStoragePolicy(String src, String policyName)
+      throws IOException {
+    try {
+      namenode.setStoragePolicy(src, policyName);
+    } catch (RemoteException e) {
+      throw e.unwrapRemoteException(AccessControlException.class,
+                                    FileNotFoundException.class,
+                                    SafeModeException.class,
+                                    NSQuotaExceededException.class,
+                                    UnresolvedPathException.class,
+                                    SnapshotAccessControlException.class);
+    }
+  }
+
   /**
    * Rename file or directory.
    * @see ClientProtocol#rename(String, String)

+ 15 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -232,9 +232,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_KEY = "dfs.namenode.min.supported.datanode.version";
   public static final String  DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_DEFAULT = "3.0.0-SNAPSHOT";
 
-  public static final String DFS_NAMENODE_RANDOMIZE_BLOCK_LOCATIONS_PER_BLOCK = "dfs.namenode.randomize-block-locations-per-block";
-  public static final boolean DFS_NAMENODE_RANDOMIZE_BLOCK_LOCATIONS_PER_BLOCK_DEFAULT = false;
-
   public static final String  DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY = "dfs.namenode.edits.dir.minimum";
   public static final int     DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT = 1;
 
@@ -386,6 +383,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_BALANCER_MOVERTHREADS_DEFAULT = 1000;
   public static final String  DFS_BALANCER_DISPATCHERTHREADS_KEY = "dfs.balancer.dispatcherThreads";
   public static final int     DFS_BALANCER_DISPATCHERTHREADS_DEFAULT = 200;
+
+  public static final String  DFS_MOVER_MOVEDWINWIDTH_KEY = "dfs.mover.movedWinWidth";
+  public static final long    DFS_MOVER_MOVEDWINWIDTH_DEFAULT = 5400*1000L;
+  public static final String  DFS_MOVER_MOVERTHREADS_KEY = "dfs.mover.moverThreads";
+  public static final int     DFS_MOVER_MOVERTHREADS_DEFAULT = 1000;
+
   public static final String  DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
   public static final int     DFS_DATANODE_DEFAULT_PORT = 50010;
   public static final String  DFS_DATANODE_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_DATANODE_DEFAULT_PORT;
@@ -449,6 +452,15 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final Class<BlockPlacementPolicyDefault> DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT = BlockPlacementPolicyDefault.class;
   public static final String  DFS_REPLICATION_MAX_KEY = "dfs.replication.max";
   public static final int     DFS_REPLICATION_MAX_DEFAULT = 512;
+  public static final String DFS_BLOCK_STORAGE_POLICIES_KEY
+        = BlockStoragePolicy.DFS_BLOCK_STORAGE_POLICIES_KEY;
+  public static final String DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX
+        = BlockStoragePolicy.DFS_BLOCK_STORAGE_POLICY_KEY_PREFIX;
+  public static final String DFS_BLOCK_STORAGE_POLICY_CREATION_FALLBACK_KEY_PREFIX
+        = BlockStoragePolicy.DFS_BLOCK_STORAGE_POLICY_CREATION_FALLBACK_KEY_PREFIX;
+  public static final String DFS_BLOCK_STORAGE_POLICY_REPLICATION_FALLBACK_KEY_PREFIX
+        = BlockStoragePolicy.DFS_BLOCK_STORAGE_POLICY_REPLICATION_FALLBACK_KEY_PREFIX;
+
   public static final String  DFS_DF_INTERVAL_KEY = "dfs.df.interval";
   public static final int     DFS_DF_INTERVAL_DEFAULT = 60000;
   public static final String  DFS_BLOCKREPORT_INTERVAL_MSEC_KEY = "dfs.blockreport.intervalMsec";

+ 48 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -41,6 +41,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.fs.CanSetDropBehind;
@@ -76,6 +77,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+import org.apache.hadoop.hdfs.server.namenode.RetryStartFileException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
@@ -126,6 +128,13 @@ import com.google.common.cache.RemovalNotification;
 public class DFSOutputStream extends FSOutputSummer
     implements Syncable, CanSetDropBehind {
   private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB
+  /**
+   * Number of times to retry creating a file when there are transient 
+   * errors (typically related to encryption zones and KeyProvider operations).
+   */
+  @VisibleForTesting
+  public static final int CREATE_RETRY_COUNT = 10;
+
   private final DFSClient dfsClient;
   private final long dfsclientSlowLogThresholdMs;
   private Socket s;
@@ -1651,23 +1660,46 @@ public class DFSOutputStream extends FSOutputSummer
       short replication, long blockSize, Progressable progress, int buffersize,
       DataChecksum checksum, String[] favoredNodes,
       List<CipherSuite> cipherSuites) throws IOException {
-    final HdfsFileStatus stat;
-    try {
-      stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
-          new EnumSetWritable<CreateFlag>(flag), createParent, replication,
-          blockSize, cipherSuites);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     DSQuotaExceededException.class,
-                                     FileAlreadyExistsException.class,
-                                     FileNotFoundException.class,
-                                     ParentNotDirectoryException.class,
-                                     NSQuotaExceededException.class,
-                                     SafeModeException.class,
-                                     UnresolvedPathException.class,
-                                     SnapshotAccessControlException.class,
-                                     UnknownCipherSuiteException.class);
+    HdfsFileStatus stat = null;
+
+    // Retry the create if we get a RetryStartFileException up to a maximum
+    // number of times
+    boolean shouldRetry = true;
+    int retryCount = CREATE_RETRY_COUNT;
+    while (shouldRetry) {
+      shouldRetry = false;
+      try {
+        stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
+            new EnumSetWritable<CreateFlag>(flag), createParent, replication,
+            blockSize, cipherSuites);
+        break;
+      } catch (RemoteException re) {
+        IOException e = re.unwrapRemoteException(
+            AccessControlException.class,
+            DSQuotaExceededException.class,
+            FileAlreadyExistsException.class,
+            FileNotFoundException.class,
+            ParentNotDirectoryException.class,
+            NSQuotaExceededException.class,
+            RetryStartFileException.class,
+            SafeModeException.class,
+            UnresolvedPathException.class,
+            SnapshotAccessControlException.class,
+            UnknownCipherSuiteException.class);
+        if (e instanceof RetryStartFileException) {
+          if (retryCount > 0) {
+            shouldRetry = true;
+            retryCount--;
+          } else {
+            throw new IOException("Too many retries because of encryption" +
+                " zone operations", e);
+          }
+        } else {
+          throw e;
+        }
+      }
     }
+    Preconditions.checkNotNull(stat, "HdfsFileStatus should not be null!");
     final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat,
         flag, progress, checksum, favoredNodes);
     out.start();

+ 33 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -471,7 +471,39 @@ public class DistributedFileSystem extends FileSystem {
       }
     }.resolve(this, absF);
   }
-  
+
+  /**
+   * Set the source path to the specified storage policy.
+   *
+   * @param src The source path referring to either a directory or a file.
+   * @param policyName The name of the storage policy.
+   */
+  public void setStoragePolicy(final Path src, final String policyName)
+      throws IOException {
+    statistics.incrementWriteOps(1);
+    Path absF = fixRelativePart(src);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(final Path p)
+          throws IOException, UnresolvedLinkException {
+        dfs.setStoragePolicy(getPathName(p), policyName);
+        return null;
+      }
+      @Override
+      public Void next(final FileSystem fs, final Path p)
+          throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          ((DistributedFileSystem) fs).setStoragePolicy(p, policyName);
+          return null;
+        } else {
+          throw new UnsupportedOperationException(
+              "Cannot perform setStoragePolicy on a non-DistributedFileSystem: "
+                  + src + " -> " + p);
+        }
+      }
+    }.resolve(this, absF);
+  }
+
   /**
    * Move blocks from srcs to trg and delete srcs afterwards.
    * The file block sizes must be the same.

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StorageType.java

@@ -33,9 +33,11 @@ import org.apache.hadoop.classification.InterfaceStability;
 public enum StorageType {
   DISK,
   SSD,
-  RAM_DISK;
+  ARCHIVE,
+  RAM_DISK;  
 
   public static final StorageType DEFAULT = DISK;
+  
   public static final StorageType[] EMPTY_ARRAY = {};
   
   private static final StorageType[] VALUES = values();
@@ -43,4 +45,4 @@ public enum StorageType {
   public static List<StorageType> asList() {
     return Arrays.asList(VALUES);
   }
-}
+}

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -259,6 +259,20 @@ public interface ClientProtocol {
       FileNotFoundException, SafeModeException, UnresolvedLinkException,
       SnapshotAccessControlException, IOException;
 
+  /**
+   * Set the storage policy for a file/directory
+   * @param src Path of an existing file/directory. 
+   * @param policyName The name of the storage policy
+   * @throws SnapshotAccessControlException If access is denied
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
+   * @throws FileNotFoundException If file/dir <code>src</code> is not found
+   * @throws QuotaExceededException If changes violate the quota restriction
+   */
+  @Idempotent
+  public void setStoragePolicy(String src, String policyName)
+      throws SnapshotAccessControlException, UnresolvedLinkException,
+      FileNotFoundException, QuotaExceededException, IOException;
+
   /**
    * Set permissions for an existing file/directory.
    * 

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java

@@ -160,5 +160,8 @@ public class HdfsConstants {
       = DFSUtil.string2Bytes(DOT_SNAPSHOT_DIR);
   
   public static final String SEPARATOR_DOT_SNAPSHOT_DIR
-      = Path.SEPARATOR + DOT_SNAPSHOT_DIR; 
+      = Path.SEPARATOR + DOT_SNAPSHOT_DIR;
+
+  public static final String SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR
+      = Path.SEPARATOR + DOT_SNAPSHOT_DIR + Path.SEPARATOR;
 }

+ 8 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java

@@ -51,6 +51,7 @@ public class HdfsFileStatus {
   
   // Used by dir, not including dot and dotdot. Always zero for a regular file.
   private final int childrenNum;
+  private final byte storagePolicy;
   
   public static final byte[] EMPTY_NAME = new byte[0];
 
@@ -73,7 +74,7 @@ public class HdfsFileStatus {
       long blocksize, boolean isLazyPersist, long modification_time,
       long access_time, FsPermission permission, String owner,
       String group, byte[] symlink, byte[] path, long fileId,
-      int childrenNum, FileEncryptionInfo feInfo) {
+      int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy) {
     this.length = length;
     this.isdir = isdir;
     this.block_replication = (short)block_replication;
@@ -93,6 +94,7 @@ public class HdfsFileStatus {
     this.fileId = fileId;
     this.childrenNum = childrenNum;
     this.feInfo = feInfo;
+    this.storagePolicy = storagePolicy;
   }
 
   /**
@@ -261,6 +263,11 @@ public class HdfsFileStatus {
     return childrenNum;
   }
 
+  /** @return the storage policy id */
+  public final byte getStoragePolicy() {
+    return storagePolicy;
+  }
+
   public final FileStatus makeQualified(URI defaultUri, Path path) {
     return new FileStatus(getLen(), isDir(), getReplication(),
         getBlockSize(), isLazyPersist(), getModificationTime(),

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java

@@ -35,7 +35,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 @InterfaceStability.Evolving
 public class HdfsLocatedFileStatus extends HdfsFileStatus {
   private final LocatedBlocks locations;
-  
+
   /**
    * Constructor
    * 
@@ -58,13 +58,14 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
       int block_replication, long blocksize, boolean isLazyPersist,
       long modification_time, long access_time, FsPermission permission,
       String owner, String group, byte[] symlink, byte[] path, long fileId,
-      LocatedBlocks locations, int childrenNum, FileEncryptionInfo feInfo) {
+      LocatedBlocks locations, int childrenNum, FileEncryptionInfo feInfo,
+      byte storagePolicy) {
     super(length, isdir, block_replication, blocksize, isLazyPersist,
         modification_time, access_time, permission, owner, group, symlink,
-        path, fileId, childrenNum, feInfo);
+        path, fileId, childrenNum, feInfo, storagePolicy);
     this.locations = locations;
   }
-	
+
   public LocatedBlocks getBlockLocations() {
     return locations;
   }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java

@@ -24,6 +24,7 @@ import java.util.Date;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSUtil;
 
 /**
@@ -61,7 +62,7 @@ public class SnapshottableDirectoryStatus {
       int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
     this.dirStatus = new HdfsFileStatus(0, true, 0, 0, false, modification_time,
         access_time, permission, owner, group, null, localName, inodeId,
-        childrenNum, null);
+        childrenNum, null, BlockStoragePolicy.ID_UNSPECIFIED);
     this.snapshotNumber = snapshotNumber;
     this.snapshotQuota = snapshotQuota;
     this.parentFullPath = parentFullPath;

+ 16 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -173,6 +173,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetRep
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
@@ -236,6 +238,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   static final GetSnapshottableDirListingResponseProto 
       NULL_GET_SNAPSHOTTABLE_DIR_LISTING_RESPONSE = 
       GetSnapshottableDirListingResponseProto.newBuilder().build();
+  static final SetStoragePolicyResponseProto VOID_SET_STORAGE_POLICY_RESPONSE =
+      SetStoragePolicyResponseProto.newBuilder().build();
 
   private static final CreateResponseProto VOID_CREATE_RESPONSE = 
   CreateResponseProto.newBuilder().build();
@@ -1413,6 +1417,18 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     return VOID_CHECKACCESS_RESPONSE;
   }
 
+  @Override
+  public SetStoragePolicyResponseProto setStoragePolicy(
+      RpcController controller, SetStoragePolicyRequestProto request)
+      throws ServiceException {
+    try {
+      server.setStoragePolicy(request.getSrc(), request.getPolicyName());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return VOID_SET_STORAGE_POLICY_RESPONSE;
+  }
+
   public GetCurrentEditLogTxidResponseProto getCurrentEditLogTxid(RpcController controller,
       GetCurrentEditLogTxidRequestProto req) throws ServiceException {
     try {

+ 16 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -64,7 +64,9 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
@@ -152,6 +154,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTim
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
@@ -1435,6 +1438,19 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
   }
 
+  @Override
+  public void setStoragePolicy(String src, String policyName)
+      throws SnapshotAccessControlException, UnresolvedLinkException,
+      FileNotFoundException, QuotaExceededException, IOException {
+    SetStoragePolicyRequestProto req = SetStoragePolicyRequestProto
+        .newBuilder().setSrc(src).setPolicyName(policyName).build();
+    try {
+      rpcProxy.setStoragePolicy(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
   public long getCurrentEditLogTxid() throws IOException {
     GetCurrentEditLogTxidRequestProto req = GetCurrentEditLogTxidRequestProto
         .getDefaultInstance();

+ 31 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.inotify.Event;
@@ -176,6 +177,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 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;
+import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
@@ -893,9 +895,25 @@ public class PBHelper {
     }
     builder.addAllTargets(convert(cmd.getTargets()))
            .addAllTargetStorageUuids(convert(cmd.getTargetStorageIDs()));
+    StorageType[][] types = cmd.getTargetStorageTypes();
+    if (types != null) {
+      builder.addAllTargetStorageTypes(convert(types));
+    }
     return builder.build();
   }
-  
+
+  private static List<StorageTypesProto> convert(StorageType[][] types) {
+    List<StorageTypesProto> list = Lists.newArrayList();
+    if (types != null) {
+      for (StorageType[] ts : types) {
+        StorageTypesProto.Builder builder = StorageTypesProto.newBuilder();
+        builder.addAllStorageTypes(convertStorageTypes(ts));
+        list.add(builder.build());
+      }
+    }
+    return list;
+  }
+
   public static BlockIdCommandProto convert(BlockIdCommand cmd) {
     BlockIdCommandProto.Builder builder = BlockIdCommandProto.newBuilder()
         .setBlockPoolId(cmd.getBlockPoolId());
@@ -1024,7 +1042,7 @@ public class PBHelper {
     } else {
       for(int i = 0; i < targetStorageTypes.length; i++) {
         List<StorageTypeProto> p = targetStorageTypesList.get(i).getStorageTypesList();
-        targetStorageTypes[i] = p.toArray(new StorageType[p.size()]);
+        targetStorageTypes[i] = convertStorageTypes(p, targets[i].length);
       }
     }
 
@@ -1338,8 +1356,9 @@ public class PBHelper {
         fs.hasFileId()? fs.getFileId(): INodeId.GRANDFATHER_INODE_ID,
         fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null,
         fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
-        fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) :
-            null);
+        fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
+        fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
+            : BlockStoragePolicy.ID_UNSPECIFIED);
   }
 
   public static SnapshottableDirectoryStatus convert(
@@ -1386,7 +1405,8 @@ public class PBHelper {
       setGroup(fs.getGroup()).
       setFileId(fs.getFileId()).
       setChildrenNum(fs.getChildrenNum()).
-      setPath(ByteString.copyFrom(fs.getLocalNameInBytes()));
+      setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
+      setStoragePolicy(fs.getStoragePolicy());
     if (fs.isSymlink())  {
       builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
     }
@@ -1394,7 +1414,8 @@ public class PBHelper {
       builder.setFileEncryptionInfo(convert(fs.getFileEncryptionInfo()));
     }
     if (fs instanceof HdfsLocatedFileStatus) {
-      LocatedBlocks locations = ((HdfsLocatedFileStatus)fs).getBlockLocations();
+      final HdfsLocatedFileStatus lfs = (HdfsLocatedFileStatus) fs;
+      LocatedBlocks locations = lfs.getBlockLocations();
       if (locations != null) {
         builder.setLocations(PBHelper.convert(locations));
       }
@@ -1707,6 +1728,8 @@ public class PBHelper {
       return StorageTypeProto.DISK;
     case SSD:
       return StorageTypeProto.SSD;
+    case ARCHIVE:
+      return StorageTypeProto.ARCHIVE;
     case RAM_DISK:
       return StorageTypeProto.RAM_DISK;
     default:
@@ -1737,6 +1760,8 @@ public class PBHelper {
         return StorageType.DISK;
       case SSD:
         return StorageType.SSD;
+      case ARCHIVE:
+        return StorageType.ARCHIVE;
       case RAM_DISK:
         return StorageType.RAM_DISK;
       default:

+ 7 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -23,7 +23,6 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.net.URI;
 import java.text.DateFormat;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -54,6 +53,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault
 import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
@@ -270,7 +270,7 @@ public class Balancer {
     //   over-utilized, above-average, below-average and under-utilized.
     long overLoadedBytes = 0L, underLoadedBytes = 0L;
     for(DatanodeStorageReport r : reports) {
-      final DDatanode dn = dispatcher.newDatanode(r);
+      final DDatanode dn = dispatcher.newDatanode(r.getDatanodeInfo());
       for(StorageType t : StorageType.asList()) {
         final Double utilization = policy.getUtilization(r, t);
         if (utilization == null) { // datanode does not have such storage type 
@@ -294,7 +294,7 @@ public class Balancer {
           }
           g = s;
         } else {
-          g = dn.addStorageGroup(t, maxSize2Move);
+          g = dn.addTarget(t, maxSize2Move);
           if (thresholdDiff <= 0) { // within threshold
             belowAvgUtilized.add(g);
           } else {
@@ -548,15 +548,10 @@ public class Balancer {
     final Formatter formatter = new Formatter(System.out);
     System.out.println("Time Stamp               Iteration#  Bytes Already Moved  Bytes Left To Move  Bytes Being Moved");
     
-    final List<NameNodeConnector> connectors
-        = new ArrayList<NameNodeConnector>(namenodes.size());
+    List<NameNodeConnector> connectors = Collections.emptyList();
     try {
-      for (URI uri : namenodes) {
-        final NameNodeConnector nnc = new NameNodeConnector(
-            Balancer.class.getSimpleName(), uri, BALANCER_ID_PATH, conf);
-        nnc.getKeyManager().startBlockKeyUpdater();
-        connectors.add(nnc);
-      }
+      connectors = NameNodeConnector.newNameNodeConnectors(namenodes, 
+            Balancer.class.getSimpleName(), BALANCER_ID_PATH, conf);
     
       boolean done = false;
       for(int iteration = 0; !done; iteration++) {
@@ -581,7 +576,7 @@ public class Balancer {
       }
     } finally {
       for(NameNodeConnector nnc : connectors) {
-        nnc.close();
+        IOUtils.cleanup(LOG, nnc);
       }
     }
     return ExitStatus.SUCCESS.getExitCode();

+ 168 - 73
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java

@@ -43,12 +43,15 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicLong;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -86,7 +89,11 @@ public class Dispatcher {
   private static final long MAX_BLOCKS_SIZE_TO_FETCH = 2 * GB;
 
   private static final int MAX_NO_PENDING_MOVE_ITERATIONS = 5;
-  private static final long DELAY_AFTER_ERROR = 10 * 1000L; // 10 seconds
+  /**
+   * the period of time to delay the usage of a DataNode after hitting
+   * errors when using it for migrating data
+   */
+  private static long delayAfterErrors = 10 * 1000;
 
   private final NameNodeConnector nnc;
   private final SaslDataTransferClient saslClient;
@@ -103,12 +110,14 @@ public class Dispatcher {
   private final MovedBlocks<StorageGroup> movedBlocks;
 
   /** Map (datanodeUuid,storageType -> StorageGroup) */
-  private final StorageGroupMap storageGroupMap = new StorageGroupMap();
+  private final StorageGroupMap<StorageGroup> storageGroupMap
+      = new StorageGroupMap<StorageGroup>();
 
   private NetworkTopology cluster;
 
   private final ExecutorService moveExecutor;
   private final ExecutorService dispatchExecutor;
+
   /** The maximum number of concurrent blocks moves at a datanode */
   private final int maxConcurrentMovesPerNode;
 
@@ -140,18 +149,18 @@ public class Dispatcher {
     }
   }
 
-  static class StorageGroupMap {
+  public static class StorageGroupMap<G extends StorageGroup> {
     private static String toKey(String datanodeUuid, StorageType storageType) {
       return datanodeUuid + ":" + storageType;
     }
 
-    private final Map<String, StorageGroup> map = new HashMap<String, StorageGroup>();
+    private final Map<String, G> map = new HashMap<String, G>();
 
-    StorageGroup get(String datanodeUuid, StorageType storageType) {
+    public G get(String datanodeUuid, StorageType storageType) {
       return map.get(toKey(datanodeUuid, storageType));
     }
 
-    void put(StorageGroup g) {
+    public void put(G g) {
       final String key = toKey(g.getDatanodeInfo().getDatanodeUuid(), g.storageType);
       final StorageGroup existing = map.put(key, g);
       Preconditions.checkState(existing == null);
@@ -164,24 +173,32 @@ public class Dispatcher {
     void clear() {
       map.clear();
     }
+
+    public Collection<G> values() {
+      return map.values();
+    }
   }
 
   /** This class keeps track of a scheduled block move */
-  private class PendingMove {
+  public class PendingMove {
     private DBlock block;
     private Source source;
     private DDatanode proxySource;
     private StorageGroup target;
 
-    private PendingMove() {
+    private PendingMove(Source source, StorageGroup target) {
+      this.source = source;
+      this.target = target;
     }
 
     @Override
     public String toString() {
-      final Block b = block.getBlock();
-      return b + " with size=" + b.getNumBytes() + " from "
-          + source.getDisplayName() + " to " + target.getDisplayName()
-          + " through " + proxySource.datanode;
+      final Block b = block != null ? block.getBlock() : null;
+      String bStr = b != null ? (b + " with size=" + b.getNumBytes() + " ")
+          : " ";
+      return bStr + "from " + source.getDisplayName() + " to " + target
+          .getDisplayName() + " through " + (proxySource != null ? proxySource
+          .datanode : "");
     }
 
     /**
@@ -191,9 +208,11 @@ public class Dispatcher {
      * @return true if a block and its proxy are chosen; false otherwise
      */
     private boolean chooseBlockAndProxy() {
+      // source and target must have the same storage type
+      final StorageType t = source.getStorageType();
       // iterate all source's blocks until find a good one
       for (Iterator<DBlock> i = source.getBlockIterator(); i.hasNext();) {
-        if (markMovedIfGoodBlock(i.next())) {
+        if (markMovedIfGoodBlock(i.next(), t)) {
           i.remove();
           return true;
         }
@@ -204,10 +223,10 @@ public class Dispatcher {
     /**
      * @return true if the given block is good for the tentative move.
      */
-    private boolean markMovedIfGoodBlock(DBlock block) {
+    private boolean markMovedIfGoodBlock(DBlock block, StorageType targetStorageType) {
       synchronized (block) {
         synchronized (movedBlocks) {
-          if (isGoodBlockCandidate(source, target, block)) {
+          if (isGoodBlockCandidate(source, target, targetStorageType, block)) {
             this.block = block;
             if (chooseProxySource()) {
               movedBlocks.put(block);
@@ -300,12 +319,13 @@ public class Dispatcher {
         LOG.info("Successfully moved " + this);
       } catch (IOException e) {
         LOG.warn("Failed to move " + this + ": " + e.getMessage());
+        target.getDDatanode().setHasFailure();
         // Proxy or target may have some issues, delay before using these nodes
         // further in order to avoid a potential storm of "threads quota
         // exceeded" warnings when the dispatcher gets out of sync with work
         // going on in datanodes.
-        proxySource.activateDelay(DELAY_AFTER_ERROR);
-        target.getDDatanode().activateDelay(DELAY_AFTER_ERROR);
+        proxySource.activateDelay(delayAfterErrors);
+        target.getDDatanode().activateDelay(delayAfterErrors);
       } finally {
         IOUtils.closeStream(out);
         IOUtils.closeStream(in);
@@ -356,10 +376,23 @@ public class Dispatcher {
   }
 
   /** A class for keeping track of block locations in the dispatcher. */
-  private static class DBlock extends MovedBlocks.Locations<StorageGroup> {
-    DBlock(Block block) {
+  public static class DBlock extends MovedBlocks.Locations<StorageGroup> {
+    public DBlock(Block block) {
       super(block);
     }
+
+    @Override
+    public synchronized boolean isLocatedOn(StorageGroup loc) {
+      // currently we only check if replicas are located on the same DataNodes
+      // since we do not have the capability to store two replicas in the same
+      // DataNode even though they are on two different storage types
+      for (StorageGroup existing : locations) {
+        if (existing.getDatanodeInfo().equals(loc.getDatanodeInfo())) {
+          return true;
+        }
+      }
+      return false;
+    }
   }
 
   /** The class represents a desired move. */
@@ -378,10 +411,10 @@ public class Dispatcher {
   }
 
   /** A class that keeps track of a datanode. */
-  static class DDatanode {
+  public static class DDatanode {
 
     /** A group of storages in a datanode with the same storage type. */
-    class StorageGroup {
+    public class StorageGroup {
       final StorageType storageType;
       final long maxSize2Move;
       private long scheduledSize = 0L;
@@ -390,18 +423,26 @@ public class Dispatcher {
         this.storageType = storageType;
         this.maxSize2Move = maxSize2Move;
       }
+      
+      public StorageType getStorageType() {
+        return storageType;
+      }
 
       private DDatanode getDDatanode() {
         return DDatanode.this;
       }
 
-      DatanodeInfo getDatanodeInfo() {
+      public DatanodeInfo getDatanodeInfo() {
         return DDatanode.this.datanode;
       }
 
       /** Decide if still need to move more bytes */
-      synchronized boolean hasSpaceForScheduling() {
-        return availableSizeToMove() > 0L;
+      boolean hasSpaceForScheduling() {
+        return hasSpaceForScheduling(0L);
+      }
+
+      synchronized boolean hasSpaceForScheduling(long size) {
+        return availableSizeToMove() > size;
       }
 
       /** @return the total number of bytes that need to be moved */
@@ -410,7 +451,7 @@ public class Dispatcher {
       }
 
       /** increment scheduled size */
-      synchronized void incScheduledSize(long size) {
+      public synchronized void incScheduledSize(long size) {
         scheduledSize += size;
       }
 
@@ -424,6 +465,18 @@ public class Dispatcher {
         scheduledSize = 0L;
       }
 
+      private PendingMove addPendingMove(DBlock block, final PendingMove pm) {
+        if (getDDatanode().addPendingBlock(pm)) {
+          if (pm.markMovedIfGoodBlock(block, getStorageType())) {
+            incScheduledSize(pm.block.getNumBytes());
+            return pm;
+          } else {
+            getDDatanode().removePendingBlock(pm);
+          }
+        }
+        return null;
+      }
+
       /** @return the name for display */
       String getDisplayName() {
         return datanode + ":" + storageType;
@@ -436,38 +489,46 @@ public class Dispatcher {
     }
 
     final DatanodeInfo datanode;
-    final EnumMap<StorageType, StorageGroup> storageMap
+    private final EnumMap<StorageType, Source> sourceMap
+        = new EnumMap<StorageType, Source>(StorageType.class);
+    private final EnumMap<StorageType, StorageGroup> targetMap
         = new EnumMap<StorageType, StorageGroup>(StorageType.class);
     protected long delayUntil = 0L;
     /** blocks being moved but not confirmed yet */
     private final List<PendingMove> pendings;
+    private volatile boolean hasFailure = false;
     private final int maxConcurrentMoves;
 
     @Override
     public String toString() {
-      return getClass().getSimpleName() + ":" + datanode + ":" + storageMap.values();
+      return getClass().getSimpleName() + ":" + datanode;
     }
 
-    private DDatanode(DatanodeStorageReport r, int maxConcurrentMoves) {
-      this.datanode = r.getDatanodeInfo();
+    private DDatanode(DatanodeInfo datanode, int maxConcurrentMoves) {
+      this.datanode = datanode;
       this.maxConcurrentMoves = maxConcurrentMoves;
       this.pendings = new ArrayList<PendingMove>(maxConcurrentMoves);
     }
 
-    private void put(StorageType storageType, StorageGroup g) {
-      final StorageGroup existing = storageMap.put(storageType, g);
+    public DatanodeInfo getDatanodeInfo() {
+      return datanode;
+    }
+
+    private static <G extends StorageGroup> void put(StorageType storageType,
+        G g, EnumMap<StorageType, G> map) {
+      final StorageGroup existing = map.put(storageType, g);
       Preconditions.checkState(existing == null);
     }
 
-    StorageGroup addStorageGroup(StorageType storageType, long maxSize2Move) {
+    public StorageGroup addTarget(StorageType storageType, long maxSize2Move) {
       final StorageGroup g = new StorageGroup(storageType, maxSize2Move);
-      put(storageType, g);
+      put(storageType, g, targetMap);
       return g;
     }
 
-    Source addSource(StorageType storageType, long maxSize2Move, Dispatcher d) {
+    public Source addSource(StorageType storageType, long maxSize2Move, Dispatcher d) {
       final Source s = d.new Source(storageType, maxSize2Move, this);
-      put(storageType, s);
+      put(storageType, s, sourceMap);
       return s;
     }
 
@@ -505,10 +566,14 @@ public class Dispatcher {
     synchronized boolean removePendingBlock(PendingMove pendingBlock) {
       return pendings.remove(pendingBlock);
     }
+
+    void setHasFailure() {
+      this.hasFailure = true;
+    }
   }
 
   /** A node that can be the sources of a block move */
-  class Source extends DDatanode.StorageGroup {
+  public class Source extends DDatanode.StorageGroup {
 
     private final List<Task> tasks = new ArrayList<Task>(2);
     private long blocksToReceive = 0L;
@@ -576,8 +641,11 @@ public class Dispatcher {
 
     /** Decide if the given block is a good candidate to move or not */
     private boolean isGoodBlockCandidate(DBlock block) {
+      // source and target must have the same storage type
+      final StorageType sourceStorageType = getStorageType();
       for (Task t : tasks) {
-        if (Dispatcher.this.isGoodBlockCandidate(this, t.target, block)) {
+        if (Dispatcher.this.isGoodBlockCandidate(this, t.target,
+            sourceStorageType, block)) {
           return true;
         }
       }
@@ -597,11 +665,9 @@ public class Dispatcher {
       for (Iterator<Task> i = tasks.iterator(); i.hasNext();) {
         final Task task = i.next();
         final DDatanode target = task.target.getDDatanode();
-        PendingMove pendingBlock = new PendingMove();
+        final PendingMove pendingBlock = new PendingMove(this, task.target);
         if (target.addPendingBlock(pendingBlock)) {
           // target is not busy, so do a tentative block allocation
-          pendingBlock.source = this;
-          pendingBlock.target = task.target;
           if (pendingBlock.chooseBlockAndProxy()) {
             long blockSize = pendingBlock.block.getNumBytes();
             incScheduledSize(-blockSize);
@@ -618,6 +684,11 @@ public class Dispatcher {
       }
       return null;
     }
+    
+    /** Add a pending move */
+    public PendingMove addPendingMove(DBlock block, StorageGroup target) {
+      return target.addPendingMove(block, new PendingMove(this, target));
+    }
 
     /** Iterate all source's blocks to remove moved ones */
     private void removeMovedBlocks() {
@@ -656,13 +727,7 @@ public class Dispatcher {
         if (p != null) {
           // Reset no pending move counter
           noPendingMoveIteration=0;
-          // move the block
-          moveExecutor.execute(new Runnable() {
-            @Override
-            public void run() {
-              p.dispatch();
-            }
-          });
+          executePendingMove(p);
           continue;
         }
 
@@ -718,7 +783,8 @@ public class Dispatcher {
     this.cluster = NetworkTopology.getInstance(conf);
 
     this.moveExecutor = Executors.newFixedThreadPool(moverThreads);
-    this.dispatchExecutor = Executors.newFixedThreadPool(dispatcherThreads);
+    this.dispatchExecutor = dispatcherThreads == 0? null
+        : Executors.newFixedThreadPool(dispatcherThreads);
     this.maxConcurrentMovesPerNode = maxConcurrentMovesPerNode;
 
     final boolean fallbackToSimpleAuthAllowed = conf.getBoolean(
@@ -729,11 +795,15 @@ public class Dispatcher {
         TrustedChannelResolver.getInstance(conf), fallbackToSimpleAuthAllowed);
   }
 
-  StorageGroupMap getStorageGroupMap() {
+  public DistributedFileSystem getDistributedFileSystem() {
+    return nnc.getDistributedFileSystem();
+  }
+
+  public StorageGroupMap<StorageGroup> getStorageGroupMap() {
     return storageGroupMap;
   }
 
-  NetworkTopology getCluster() {
+  public NetworkTopology getCluster() {
     return cluster;
   }
   
@@ -781,7 +851,7 @@ public class Dispatcher {
   }
 
   /** Get live datanode storage reports and then build the network topology. */
-  List<DatanodeStorageReport> init() throws IOException {
+  public List<DatanodeStorageReport> init() throws IOException {
     final DatanodeStorageReport[] reports = nnc.getLiveDatanodeStorageReport();
     final List<DatanodeStorageReport> trimmed = new ArrayList<DatanodeStorageReport>(); 
     // create network topology and classify utilization collections:
@@ -797,8 +867,18 @@ public class Dispatcher {
     return trimmed;
   }
 
-  public DDatanode newDatanode(DatanodeStorageReport r) {
-    return new DDatanode(r, maxConcurrentMovesPerNode);
+  public DDatanode newDatanode(DatanodeInfo datanode) {
+    return new DDatanode(datanode, maxConcurrentMovesPerNode);
+  }
+
+  public void executePendingMove(final PendingMove p) {
+    // move the block
+    moveExecutor.execute(new Runnable() {
+      @Override
+      public void run() {
+        p.dispatch();
+      }
+    });
   }
 
   public boolean dispatchAndCheckContinue() throws InterruptedException {
@@ -838,7 +918,7 @@ public class Dispatcher {
     }
 
     // wait for all block moving to be done
-    waitForMoveCompletion();
+    waitForMoveCompletion(targets);
 
     return bytesMoved.get() - bytesLastMoved;
   }
@@ -846,23 +926,25 @@ public class Dispatcher {
   /** The sleeping period before checking if block move is completed again */
   static private long blockMoveWaitTime = 30000L;
 
-  /** set the sleeping period for block move completion check */
-  static void setBlockMoveWaitTime(long time) {
-    blockMoveWaitTime = time;
-  }
-
-  /** Wait for all block move confirmations. */
-  private void waitForMoveCompletion() {
+  /**
+   * Wait for all block move confirmations.
+   * @return true if there is failed move execution
+   */
+  public static boolean waitForMoveCompletion(
+      Iterable<? extends StorageGroup> targets) {
+    boolean hasFailure = false;
     for(;;) {
       boolean empty = true;
       for (StorageGroup t : targets) {
         if (!t.getDDatanode().isPendingQEmpty()) {
           empty = false;
           break;
+        } else {
+          hasFailure |= t.getDDatanode().hasFailure;
         }
       }
       if (empty) {
-        return; //all pending queues are empty
+        return hasFailure; // all pending queues are empty
       }
       try {
         Thread.sleep(blockMoveWaitTime);
@@ -873,14 +955,14 @@ public class Dispatcher {
 
   /**
    * Decide if the block is a good candidate to be moved from source to target.
-   * A block is a good candidate if 
+   * A block is a good candidate if
    * 1. the block is not in the process of being moved/has not been moved;
    * 2. the block does not have a replica on the target;
    * 3. doing the move does not reduce the number of racks that the block has
    */
-  private boolean isGoodBlockCandidate(Source source, StorageGroup target,
-      DBlock block) {
-    if (source.storageType != target.storageType) {
+  private boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target,
+      StorageType targetStorageType, DBlock block) {
+    if (target.storageType != targetStorageType) {
       return false;
     }
     // check if the block is moved or not
@@ -891,7 +973,7 @@ public class Dispatcher {
       return false;
     }
     if (cluster.isNodeGroupAware()
-        && isOnSameNodeGroupWithReplicas(target, block, source)) {
+        && isOnSameNodeGroupWithReplicas(source, target, block)) {
       return false;
     }
     if (reduceNumOfRacks(source, target, block)) {
@@ -904,7 +986,7 @@ public class Dispatcher {
    * Determine whether moving the given block replica from source to target
    * would reduce the number of racks of the block replicas.
    */
-  private boolean reduceNumOfRacks(Source source, StorageGroup target,
+  private boolean reduceNumOfRacks(StorageGroup source, StorageGroup target,
       DBlock block) {
     final DatanodeInfo sourceDn = source.getDatanodeInfo();
     if (cluster.isOnSameRack(sourceDn, target.getDatanodeInfo())) {
@@ -937,12 +1019,12 @@ public class Dispatcher {
    * Check if there are any replica (other than source) on the same node group
    * with target. If true, then target is not a good candidate for placing
    * specific replica as we don't want 2 replicas under the same nodegroup.
-   * 
+   *
    * @return true if there are any replica (other than source) on the same node
    *         group with target
    */
-  private boolean isOnSameNodeGroupWithReplicas(
-      StorageGroup target, DBlock block, Source source) {
+  private boolean isOnSameNodeGroupWithReplicas(StorageGroup source,
+      StorageGroup target, DBlock block) {
     final DatanodeInfo targetDn = target.getDatanodeInfo();
     for (StorageGroup g : block.getLocations()) {
       if (g != source && cluster.isOnSameNodeGroup(g.getDatanodeInfo(), targetDn)) {
@@ -962,9 +1044,22 @@ public class Dispatcher {
     movedBlocks.cleanup();
   }
 
+  /** set the sleeping period for block move completion check */
+  @VisibleForTesting
+  public static void setBlockMoveWaitTime(long time) {
+    blockMoveWaitTime = time;
+  }
+
+  @VisibleForTesting
+  public static void setDelayAfterErrors(long time) {
+    delayAfterErrors = time;
+  }
+
   /** shutdown thread pools */
-  void shutdownNow() {
-    dispatchExecutor.shutdownNow();
+  public void shutdownNow() {
+    if (dispatchExecutor != null) {
+      dispatchExecutor.shutdownNow();
+    }
     moveExecutor.shutdownNow();
   }
 

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Matcher.java

@@ -31,6 +31,11 @@ public interface Matcher {
     public boolean match(NetworkTopology cluster, Node left, Node right) {
       return cluster.isOnSameNodeGroup(left, right);
     }
+
+    @Override
+    public String toString() {
+      return "SAME_NODE_GROUP";
+    }
   };
 
   /** Match datanodes in the same rack. */
@@ -39,6 +44,11 @@ public interface Matcher {
     public boolean match(NetworkTopology cluster, Node left, Node right) {
       return cluster.isOnSameRack(left, right);
     }
+
+    @Override
+    public String toString() {
+      return "SAME_RACK";
+    }
   };
 
   /** Match any datanode with any other datanode. */
@@ -47,5 +57,10 @@ public interface Matcher {
     public boolean match(NetworkTopology cluster, Node left, Node right) {
       return left != right;
     }
+
+    @Override
+    public String toString() {
+      return "ANY_OTHER";
+    }
   };
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/MovedBlocks.java

@@ -40,7 +40,7 @@ public class MovedBlocks<L> {
   public static class Locations<L> {
     private final Block block; // the block
     /** The locations of the replicas of the block. */
-    private final List<L> locations = new ArrayList<L>(3);
+    protected final List<L> locations = new ArrayList<L>(3);
     
     public Locations(Block block) {
       this.block = block;

+ 69 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java

@@ -18,19 +18,25 @@
 package org.apache.hadoop.hdfs.server.balancer;
 
 import java.io.Closeable;
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.net.InetAddress;
 import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
@@ -43,6 +49,8 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * The class provides utilities for accessing a NameNode.
  */
@@ -51,6 +59,41 @@ public class NameNodeConnector implements Closeable {
   private static final Log LOG = LogFactory.getLog(NameNodeConnector.class);
 
   private static final int MAX_NOT_CHANGED_ITERATIONS = 5;
+  private static boolean write2IdFile = true;
+  
+  /** Create {@link NameNodeConnector} for the given namenodes. */
+  public static List<NameNodeConnector> newNameNodeConnectors(
+      Collection<URI> namenodes, String name, Path idPath, Configuration conf)
+      throws IOException {
+    final List<NameNodeConnector> connectors = new ArrayList<NameNodeConnector>(
+        namenodes.size());
+    for (URI uri : namenodes) {
+      NameNodeConnector nnc = new NameNodeConnector(name, uri, idPath,
+          null, conf);
+      nnc.getKeyManager().startBlockKeyUpdater();
+      connectors.add(nnc);
+    }
+    return connectors;
+  }
+
+  public static List<NameNodeConnector> newNameNodeConnectors(
+      Map<URI, List<Path>> namenodes, String name, Path idPath,
+      Configuration conf) throws IOException {
+    final List<NameNodeConnector> connectors = new ArrayList<NameNodeConnector>(
+        namenodes.size());
+    for (Map.Entry<URI, List<Path>> entry : namenodes.entrySet()) {
+      NameNodeConnector nnc = new NameNodeConnector(name, entry.getKey(),
+          idPath, entry.getValue(), conf);
+      nnc.getKeyManager().startBlockKeyUpdater();
+      connectors.add(nnc);
+    }
+    return connectors;
+  }
+
+  @VisibleForTesting
+  public static void setWrite2IdFile(boolean write2IdFile) {
+    NameNodeConnector.write2IdFile = write2IdFile;
+  }
 
   private final URI nameNodeUri;
   private final String blockpoolID;
@@ -59,22 +102,26 @@ public class NameNodeConnector implements Closeable {
   private final ClientProtocol client;
   private final KeyManager keyManager;
 
-  private final FileSystem fs;
+  private final DistributedFileSystem fs;
   private final Path idPath;
   private final OutputStream out;
+  private final List<Path> targetPaths;
 
   private int notChangedIterations = 0;
 
   public NameNodeConnector(String name, URI nameNodeUri, Path idPath,
-      Configuration conf) throws IOException {
+                           List<Path> targetPaths, Configuration conf)
+      throws IOException {
     this.nameNodeUri = nameNodeUri;
     this.idPath = idPath;
-    
+    this.targetPaths = targetPaths == null || targetPaths.isEmpty() ? Arrays
+        .asList(new Path("/")) : targetPaths;
+
     this.namenode = NameNodeProxies.createProxy(conf, nameNodeUri,
         NamenodeProtocol.class).getProxy();
     this.client = NameNodeProxies.createProxy(conf, nameNodeUri,
         ClientProtocol.class).getProxy();
-    this.fs = FileSystem.get(nameNodeUri, conf);
+    this.fs = (DistributedFileSystem)FileSystem.get(nameNodeUri, conf);
 
     final NamespaceInfo namespaceinfo = namenode.versionRequest();
     this.blockpoolID = namespaceinfo.getBlockPoolID();
@@ -82,13 +129,18 @@ public class NameNodeConnector implements Closeable {
     final FsServerDefaults defaults = fs.getServerDefaults(new Path("/"));
     this.keyManager = new KeyManager(blockpoolID, namenode,
         defaults.getEncryptDataTransfer(), conf);
-    // Exit if there is another one running.
-    out = checkAndMarkRunning(); 
+    // if it is for test, we do not create the id file
+    out = checkAndMarkRunning();
     if (out == null) {
+      // Exit if there is another one running.
       throw new IOException("Another " + name + " is running.");
     }
   }
 
+  public DistributedFileSystem getDistributedFileSystem() {
+    return fs;
+  }
+
   /** @return the block pool ID */
   public String getBlockpoolID() {
     return blockpoolID;
@@ -111,6 +163,11 @@ public class NameNodeConnector implements Closeable {
     return keyManager;
   }
 
+  /** @return the list of paths to scan/migrate */
+  public List<Path> getTargetPaths() {
+    return targetPaths;
+  }
+
   /** Should the instance continue running? */
   public boolean shouldContinue(long dispatchBlockMoveBytes) {
     if (dispatchBlockMoveBytes > 0) {
@@ -144,9 +201,11 @@ public class NameNodeConnector implements Closeable {
    */
   private OutputStream checkAndMarkRunning() throws IOException {
     try {
-      final DataOutputStream out = fs.create(idPath);
-      out.writeBytes(InetAddress.getLocalHost().getHostName());
-      out.flush();
+      final FSDataOutputStream out = fs.create(idPath);
+      if (write2IdFile) {
+        out.writeBytes(InetAddress.getLocalHost().getHostName());
+        out.hflush();
+      }
       return out;
     } catch(RemoteException e) {
       if(AlreadyBeingCreatedException.class.getName().equals(e.getClassName())){

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java

@@ -66,6 +66,11 @@ public interface BlockCollection {
    */
   public short getBlockReplication();
 
+  /** 
+   * @return the storage policy ID.
+   */
+  public byte getStoragePolicyID();
+
   /**
    * Get the name of the collection.
    */

+ 72 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -42,6 +42,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -254,6 +255,7 @@ public class BlockManager {
 
   /** for block replicas placement */
   private BlockPlacementPolicy blockplacement;
+  private final BlockStoragePolicy.Suite storagePolicySuite;
 
   /** Check whether name system is running before terminating */
   private boolean checkNSRunning = true;
@@ -276,6 +278,7 @@ public class BlockManager {
     blockplacement = BlockPlacementPolicy.getInstance(
         conf, stats, datanodeManager.getNetworkTopology(), 
         datanodeManager.getHost2DatanodeMap());
+    storagePolicySuite = BlockStoragePolicy.readBlockStorageSuite(conf);
     pendingReplications = new PendingReplicationBlocks(conf.getInt(
       DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY,
       DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_DEFAULT) * 1000L);
@@ -394,7 +397,11 @@ public class BlockManager {
           lifetimeMin*60*1000L, 0, null, encryptionAlgorithm);
     }
   }
-  
+
+  public BlockStoragePolicy getStoragePolicy(final String policyName) {
+    return storagePolicySuite.getPolicy(policyName);
+  }
+
   public void setBlockPoolId(String blockPoolId) {
     if (isBlockTokenEnabled()) {
       blockTokenSecretManager.setBlockPoolId(blockPoolId);
@@ -445,7 +452,7 @@ public class BlockManager {
     return datanodeManager;
   }
 
-  /** @return the BlockPlacementPolicy */
+  @VisibleForTesting
   public BlockPlacementPolicy getBlockPlacementPolicy() {
     return blockplacement;
   }
@@ -1366,7 +1373,7 @@ public class BlockManager {
       // choose replication targets: NOT HOLDING THE GLOBAL LOCK
       // It is costly to extract the filename for which chooseTargets is called,
       // so for now we pass in the block collection itself.
-      rw.chooseTargets(blockplacement, excludedNodes);
+      rw.chooseTargets(blockplacement, storagePolicySuite, excludedNodes);
     }
 
     namesystem.writeLock();
@@ -1467,24 +1474,48 @@ public class BlockManager {
     return scheduledWork;
   }
 
+  /** Choose target for WebHDFS redirection. */
+  public DatanodeStorageInfo[] chooseTarget4WebHDFS(String src,
+      DatanodeDescriptor clientnode, Set<Node> excludes, long blocksize) {
+    return blockplacement.chooseTarget(src, 1, clientnode,
+        Collections.<DatanodeStorageInfo>emptyList(), false, excludes,
+        blocksize, storagePolicySuite.getDefaultPolicy());
+  }
+
+  /** Choose target for getting additional datanodes for an existing pipeline. */
+  public DatanodeStorageInfo[] chooseTarget4AdditionalDatanode(String src,
+      int numAdditionalNodes,
+      DatanodeDescriptor clientnode,
+      List<DatanodeStorageInfo> chosen,
+      Set<Node> excludes,
+      long blocksize,
+      byte storagePolicyID) {
+    
+    final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID);
+    return blockplacement.chooseTarget(src, numAdditionalNodes, clientnode,
+        chosen, true, excludes, blocksize, storagePolicy);
+  }
+
   /**
-   * Choose target datanodes according to the replication policy.
+   * Choose target datanodes for creating a new block.
    * 
    * @throws IOException
    *           if the number of targets < minimum replication.
    * @see BlockPlacementPolicy#chooseTarget(String, int, Node,
-   *      List, boolean, Set, long, StorageType)
+   *      Set, long, List, BlockStoragePolicy)
    */
-  public DatanodeStorageInfo[] chooseTarget(final String src,
+  public DatanodeStorageInfo[] chooseTarget4NewBlock(final String src,
       final int numOfReplicas, final DatanodeDescriptor client,
       final Set<Node> excludedNodes,
-      final long blocksize, List<String> favoredNodes) throws IOException {
+      final long blocksize,
+      final List<String> favoredNodes,
+      final byte storagePolicyID) throws IOException {
     List<DatanodeDescriptor> favoredDatanodeDescriptors = 
         getDatanodeDescriptors(favoredNodes);
+    final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID);
     final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src,
         numOfReplicas, client, excludedNodes, blocksize, 
-        // TODO: get storage type from file
-        favoredDatanodeDescriptors, StorageType.DEFAULT);
+        favoredDatanodeDescriptors, storagePolicy);
     if (targets.length < minReplication) {
       throw new IOException("File " + src + " could only be replicated to "
           + targets.length + " nodes instead of minReplication (="
@@ -2716,6 +2747,10 @@ public class BlockManager {
     assert namesystem.hasWriteLock();
     // first form a rack to datanodes map and
     BlockCollection bc = getBlockCollection(b);
+    final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(bc.getStoragePolicyID());
+    final List<StorageType> excessTypes = storagePolicy.chooseExcess(
+        replication, DatanodeStorageInfo.toStorageTypes(nonExcess));
+
 
     final Map<String, List<DatanodeStorageInfo>> rackMap
         = new HashMap<String, List<DatanodeStorageInfo>>();
@@ -2736,16 +2771,13 @@ public class BlockManager {
     final DatanodeStorageInfo addedNodeStorage
         = DatanodeStorageInfo.getDatanodeStorageInfo(nonExcess, addedNode);
     while (nonExcess.size() - replication > 0) {
-      // check if we can delete delNodeHint
       final DatanodeStorageInfo cur;
-      if (firstOne && delNodeHintStorage != null
-          && (moreThanOne.contains(delNodeHintStorage)
-              || (addedNodeStorage != null
-                  && !moreThanOne.contains(addedNodeStorage)))) {
+      if (useDelHint(firstOne, delNodeHintStorage, addedNodeStorage,
+          moreThanOne, excessTypes)) {
         cur = delNodeHintStorage;
       } else { // regular excessive replica removal
         cur = replicator.chooseReplicaToDelete(bc, b, replication,
-        		moreThanOne, exactlyOne);
+            moreThanOne, exactlyOne, excessTypes);
       }
       firstOne = false;
 
@@ -2771,6 +2803,27 @@ public class BlockManager {
     }
   }
 
+  /** Check if we can use delHint */
+  static boolean useDelHint(boolean isFirst, DatanodeStorageInfo delHint,
+      DatanodeStorageInfo added, List<DatanodeStorageInfo> moreThan1Racks,
+      List<StorageType> excessTypes) {
+    if (!isFirst) {
+      return false; // only consider delHint for the first case
+    } else if (delHint == null) {
+      return false; // no delHint
+    } else if (!excessTypes.contains(delHint.getStorageType())) {
+      return false; // delHint storage type is not an excess type
+    } else {
+      // check if removing delHint reduces the number of racks
+      if (moreThan1Racks.contains(delHint)) {
+        return true; // delHint and some other nodes are under the same rack 
+      } else if (added != null && !moreThan1Racks.contains(added)) {
+        return true; // the added node adds a new rack
+      }
+      return false; // removing delHint reduces the number of racks;
+    }
+  }
+
   private void addToExcessReplicate(DatanodeInfo dn, Block block) {
     assert namesystem.hasWriteLock();
     LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(dn.getDatanodeUuid());
@@ -2877,7 +2930,7 @@ public class BlockManager {
     // Decrement number of blocks scheduled to this datanode.
     // for a retry request (of DatanodeProtocol#blockReceivedAndDeleted with 
     // RECEIVED_BLOCK), we currently also decrease the approximate number. 
-    node.decrementBlocksScheduled();
+    node.decrementBlocksScheduled(storageInfo.getStorageType());
 
     // get the deletion hint node
     DatanodeDescriptor delHintNode = null;
@@ -3546,10 +3599,12 @@ public class BlockManager {
     }
     
     private void chooseTargets(BlockPlacementPolicy blockplacement,
+        BlockStoragePolicy.Suite storagePolicySuite,
         Set<Node> excludedNodes) {
       targets = blockplacement.chooseTarget(bc.getName(),
           additionalReplRequired, srcNode, liveReplicaStorages, false,
-          excludedNodes, block.getNumBytes(), StorageType.DEFAULT);
+          excludedNodes, block.getNumBytes(),
+          storagePolicySuite.getPolicy(bc.getStoragePolicyID()));
     }
   }
 

+ 13 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java

@@ -27,6 +27,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -75,7 +76,7 @@ public abstract class BlockPlacementPolicy {
                                              boolean returnChosenNodes,
                                              Set<Node> excludedNodes,
                                              long blocksize,
-                                             StorageType storageType);
+                                             BlockStoragePolicy storagePolicy);
   
   /**
    * Same as {@link #chooseTarget(String, int, Node, Set, long, List, StorageType)}
@@ -89,14 +90,14 @@ public abstract class BlockPlacementPolicy {
       Set<Node> excludedNodes,
       long blocksize,
       List<DatanodeDescriptor> favoredNodes,
-      StorageType storageType) {
+      BlockStoragePolicy storagePolicy) {
     // This class does not provide the functionality of placing
     // a block in favored datanodes. The implementations of this class
     // are expected to provide this functionality
 
     return chooseTarget(src, numOfReplicas, writer, 
         new ArrayList<DatanodeStorageInfo>(numOfReplicas), false,
-        excludedNodes, blocksize, storageType);
+        excludedNodes, blocksize, storagePolicy);
   }
 
   /**
@@ -118,18 +119,21 @@ public abstract class BlockPlacementPolicy {
    * @param srcBC block collection of file to which block-to-be-deleted belongs
    * @param block The block to be deleted
    * @param replicationFactor The required number of replicas for this block
-   * @param existingReplicas The replica locations of this block that are present
-                  on at least two unique racks. 
-   * @param moreExistingReplicas Replica locations of this block that are not
-                   listed in the previous parameter.
+   * @param moreThanOne The replica locations of this block that are present
+   *                    on more than one unique racks.
+   * @param exactlyOne Replica locations of this block that  are present
+   *                    on exactly one unique racks.
+   * @param excessTypes The excess {@link StorageType}s according to the
+   *                    {@link BlockStoragePolicy}.
    * @return the replica that is the best candidate for deletion
    */
   abstract public DatanodeStorageInfo chooseReplicaToDelete(
       BlockCollection srcBC,
       Block block, 
       short replicationFactor,
-      Collection<DatanodeStorageInfo> existingReplicas,
-      Collection<DatanodeStorageInfo> moreExistingReplicas);
+      Collection<DatanodeStorageInfo> moreThanOne,
+      Collection<DatanodeStorageInfo> exactlyOne,
+      List<StorageType> excessTypes);
 
   /**
    * Used to setup a BlockPlacementPolicy object. This should be defined by 

+ 250 - 127
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -19,15 +19,11 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import static org.apache.hadoop.util.Time.now;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
+import java.util.*;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.StorageType;
@@ -80,12 +76,6 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    */
   protected int tolerateHeartbeatMultiplier;
 
-  protected BlockPlacementPolicyDefault(Configuration conf, FSClusterStats stats,
-                           NetworkTopology clusterMap, 
-                           Host2NodesMap host2datanodeMap) {
-    initialize(conf, stats, clusterMap, host2datanodeMap);
-  }
-
   protected BlockPlacementPolicyDefault() {
   }
     
@@ -117,9 +107,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                                     boolean returnChosenNodes,
                                     Set<Node> excludedNodes,
                                     long blocksize,
-                                    StorageType storageType) {
+                                    final BlockStoragePolicy storagePolicy) {
     return chooseTarget(numOfReplicas, writer, chosenNodes, returnChosenNodes,
-        excludedNodes, blocksize, storageType);
+        excludedNodes, blocksize, storagePolicy);
   }
 
   @Override
@@ -129,17 +119,21 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       Set<Node> excludedNodes,
       long blocksize,
       List<DatanodeDescriptor> favoredNodes,
-      StorageType storageType) {
+      BlockStoragePolicy storagePolicy) {
     try {
       if (favoredNodes == null || favoredNodes.size() == 0) {
         // Favored nodes not specified, fall back to regular block placement.
         return chooseTarget(src, numOfReplicas, writer,
             new ArrayList<DatanodeStorageInfo>(numOfReplicas), false, 
-            excludedNodes, blocksize, storageType);
+            excludedNodes, blocksize, storagePolicy);
       }
 
       Set<Node> favoriteAndExcludedNodes = excludedNodes == null ?
           new HashSet<Node>() : new HashSet<Node>(excludedNodes);
+      final List<StorageType> requiredStorageTypes = storagePolicy
+          .chooseStorageTypes((short)numOfReplicas);
+      final EnumMap<StorageType, Integer> storageTypes =
+          getRequiredStorageTypes(requiredStorageTypes);
 
       // Choose favored nodes
       List<DatanodeStorageInfo> results = new ArrayList<DatanodeStorageInfo>();
@@ -152,7 +146,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         final DatanodeStorageInfo target = chooseLocalStorage(favoredNode,
             favoriteAndExcludedNodes, blocksize, 
             getMaxNodesPerRack(results.size(), numOfReplicas)[1],
-            results, avoidStaleNodes, storageType, false);
+            results, avoidStaleNodes, storageTypes, false);
         if (target == null) {
           LOG.warn("Could not find a target for file " + src
               + " with favored node " + favoredNode); 
@@ -166,7 +160,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         numOfReplicas -= results.size();
         DatanodeStorageInfo[] remainingTargets = 
             chooseTarget(src, numOfReplicas, writer, results,
-                false, favoriteAndExcludedNodes, blocksize, storageType);
+                false, favoriteAndExcludedNodes, blocksize, storagePolicy);
         for (int i = 0; i < remainingTargets.length; i++) {
           results.add(remainingTargets[i]);
         }
@@ -174,10 +168,14 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       return getPipeline(writer,
           results.toArray(new DatanodeStorageInfo[results.size()]));
     } catch (NotEnoughReplicasException nr) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Failed to choose with favored nodes (=" + favoredNodes
+            + "), disregard favored nodes hint and retry.", nr);
+      }
       // Fall back to regular block placement disregarding favored nodes hint
       return chooseTarget(src, numOfReplicas, writer, 
           new ArrayList<DatanodeStorageInfo>(numOfReplicas), false, 
-          excludedNodes, blocksize, storageType);
+          excludedNodes, blocksize, storagePolicy);
     }
   }
 
@@ -188,7 +186,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                                     boolean returnChosenNodes,
                                     Set<Node> excludedNodes,
                                     long blocksize,
-                                    StorageType storageType) {
+                                    final BlockStoragePolicy storagePolicy) {
     if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
       return DatanodeStorageInfo.EMPTY_ARRAY;
     }
@@ -213,8 +211,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       
     boolean avoidStaleNodes = (stats != null
         && stats.isAvoidingStaleDataNodesForWrite());
-    Node localNode = chooseTarget(numOfReplicas, writer,
-        excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
+    final Node localNode = chooseTarget(numOfReplicas, writer, excludedNodes,
+        blocksize, maxNodesPerRack, results, avoidStaleNodes, storagePolicy,
+        EnumSet.noneOf(StorageType.class), results.isEmpty());
     if (!returnChosenNodes) {  
       results.removeAll(chosenStorage);
     }
@@ -234,7 +233,22 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     int maxNodesPerRack = (totalNumOfReplicas-1)/clusterMap.getNumOfRacks()+2;
     return new int[] {numOfReplicas, maxNodesPerRack};
   }
-    
+
+  private EnumMap<StorageType, Integer> getRequiredStorageTypes(
+      List<StorageType> types) {
+    EnumMap<StorageType, Integer> map = new EnumMap<StorageType,
+        Integer>(StorageType.class);
+    for (StorageType type : types) {
+      if (!map.containsKey(type)) {
+        map.put(type, 1);
+      } else {
+        int num = map.get(type);
+        map.put(type, num + 1);
+      }
+    }
+    return map;
+  }
+
   /**
    * choose <i>numOfReplicas</i> from all data nodes
    * @param numOfReplicas additional number of replicas wanted
@@ -247,31 +261,49 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    * @return local node of writer (not chosen node)
    */
   private Node chooseTarget(int numOfReplicas,
-                                          Node writer,
-                                          Set<Node> excludedNodes,
-                                          long blocksize,
-                                          int maxNodesPerRack,
-                                          List<DatanodeStorageInfo> results,
-                                          final boolean avoidStaleNodes,
-                                          StorageType storageType) {
+                            Node writer,
+                            final Set<Node> excludedNodes,
+                            final long blocksize,
+                            final int maxNodesPerRack,
+                            final List<DatanodeStorageInfo> results,
+                            final boolean avoidStaleNodes,
+                            final BlockStoragePolicy storagePolicy,
+                            final EnumSet<StorageType> unavailableStorages,
+                            final boolean newBlock) {
     if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
       return writer;
     }
-    int totalReplicasExpected = numOfReplicas + results.size();
-      
-    int numOfResults = results.size();
-    boolean newBlock = (numOfResults==0);
+    final int numOfResults = results.size();
+    final int totalReplicasExpected = numOfReplicas + numOfResults;
     if ((writer == null || !(writer instanceof DatanodeDescriptor)) && !newBlock) {
       writer = results.get(0).getDatanodeDescriptor();
     }
 
     // Keep a copy of original excludedNodes
-    final Set<Node> oldExcludedNodes = avoidStaleNodes ? 
-        new HashSet<Node>(excludedNodes) : null;
+    final Set<Node> oldExcludedNodes = new HashSet<Node>(excludedNodes);
+
+    // choose storage types; use fallbacks for unavailable storages
+    final List<StorageType> requiredStorageTypes = storagePolicy
+        .chooseStorageTypes((short) totalReplicasExpected,
+            DatanodeStorageInfo.toStorageTypes(results),
+            unavailableStorages, newBlock);
+    final EnumMap<StorageType, Integer> storageTypes =
+        getRequiredStorageTypes(requiredStorageTypes);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("storageTypes=" + storageTypes);
+    }
+
     try {
+      if ((numOfReplicas = requiredStorageTypes.size()) == 0) {
+        throw new NotEnoughReplicasException(
+            "All required storage types are unavailable: "
+            + " unavailableStorages=" + unavailableStorages
+            + ", storagePolicy=" + storagePolicy);
+      }
+
       if (numOfResults == 0) {
         writer = chooseLocalStorage(writer, excludedNodes, blocksize,
-            maxNodesPerRack, results, avoidStaleNodes, storageType, true)
+            maxNodesPerRack, results, avoidStaleNodes, storageTypes, true)
                 .getDatanodeDescriptor();
         if (--numOfReplicas == 0) {
           return writer;
@@ -280,7 +312,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       final DatanodeDescriptor dn0 = results.get(0).getDatanodeDescriptor();
       if (numOfResults <= 1) {
         chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack,
-            results, avoidStaleNodes, storageType);
+            results, avoidStaleNodes, storageTypes);
         if (--numOfReplicas == 0) {
           return writer;
         }
@@ -289,24 +321,28 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         final DatanodeDescriptor dn1 = results.get(1).getDatanodeDescriptor();
         if (clusterMap.isOnSameRack(dn0, dn1)) {
           chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack,
-              results, avoidStaleNodes, storageType);
+              results, avoidStaleNodes, storageTypes);
         } else if (newBlock){
           chooseLocalRack(dn1, excludedNodes, blocksize, maxNodesPerRack,
-              results, avoidStaleNodes, storageType);
+              results, avoidStaleNodes, storageTypes);
         } else {
           chooseLocalRack(writer, excludedNodes, blocksize, maxNodesPerRack,
-              results, avoidStaleNodes, storageType);
+              results, avoidStaleNodes, storageTypes);
         }
         if (--numOfReplicas == 0) {
           return writer;
         }
       }
       chooseRandom(numOfReplicas, NodeBase.ROOT, excludedNodes, blocksize,
-          maxNodesPerRack, results, avoidStaleNodes, storageType);
+          maxNodesPerRack, results, avoidStaleNodes, storageTypes);
     } catch (NotEnoughReplicasException e) {
       final String message = "Failed to place enough replicas, still in need of "
           + (totalReplicasExpected - results.size()) + " to reach "
-          + totalReplicasExpected + ".";
+          + totalReplicasExpected
+          + " (unavailableStorages=" + unavailableStorages
+          + ", storagePolicy=" + storagePolicy
+          + ", newBlock=" + newBlock + ")";
+
       if (LOG.isTraceEnabled()) {
         LOG.trace(message, e);
       } else {
@@ -327,7 +363,28 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         // if the NotEnoughReplicasException was thrown in chooseRandom().
         numOfReplicas = totalReplicasExpected - results.size();
         return chooseTarget(numOfReplicas, writer, oldExcludedNodes, blocksize,
-            maxNodesPerRack, results, false, storageType);
+            maxNodesPerRack, results, false, storagePolicy, unavailableStorages,
+            newBlock);
+      }
+
+      boolean retry = false;
+      // simply add all the remaining types into unavailableStorages and give
+      // another try. No best effort is guaranteed here.
+      for (StorageType type : storageTypes.keySet()) {
+        if (!unavailableStorages.contains(type)) {
+          unavailableStorages.add(type);
+          retry = true;
+        }
+      }
+      if (retry) {
+        for (DatanodeStorageInfo resultStorage : results) {
+          addToExcludedNodes(resultStorage.getDatanodeDescriptor(),
+              oldExcludedNodes);
+        }
+        numOfReplicas = totalReplicasExpected - results.size();
+        return chooseTarget(numOfReplicas, writer, oldExcludedNodes, blocksize,
+            maxNodesPerRack, results, false, storagePolicy, unavailableStorages,
+            newBlock);
       }
     }
     return writer;
@@ -340,28 +397,35 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    * @return the chosen storage
    */
   protected DatanodeStorageInfo chooseLocalStorage(Node localMachine,
-                                             Set<Node> excludedNodes,
-                                             long blocksize,
-                                             int maxNodesPerRack,
-                                             List<DatanodeStorageInfo> results,
-                                             boolean avoidStaleNodes,
-                                             StorageType storageType,
-                                             boolean fallbackToLocalRack)
+      Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
+      List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
+      EnumMap<StorageType, Integer> storageTypes, boolean fallbackToLocalRack)
       throws NotEnoughReplicasException {
     // if no local machine, randomly choose one node
     if (localMachine == null) {
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-          maxNodesPerRack, results, avoidStaleNodes, storageType);
+          maxNodesPerRack, results, avoidStaleNodes, storageTypes);
     }
     if (preferLocalNode && localMachine instanceof DatanodeDescriptor) {
       DatanodeDescriptor localDatanode = (DatanodeDescriptor) localMachine;
       // otherwise try local machine first
       if (excludedNodes.add(localMachine)) { // was not in the excluded list
-        for(DatanodeStorageInfo localStorage : DFSUtil.shuffle(
-            localDatanode.getStorageInfos())) {
-          if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
-              maxNodesPerRack, false, results, avoidStaleNodes, storageType) >= 0) {
-            return localStorage;
+        for (Iterator<Map.Entry<StorageType, Integer>> iter = storageTypes
+            .entrySet().iterator(); iter.hasNext(); ) {
+          Map.Entry<StorageType, Integer> entry = iter.next();
+          for (DatanodeStorageInfo localStorage : DFSUtil.shuffle(
+              localDatanode.getStorageInfos())) {
+            StorageType type = entry.getKey();
+            if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
+                maxNodesPerRack, false, results, avoidStaleNodes, type) >= 0) {
+              int num = entry.getValue();
+              if (num == 1) {
+                iter.remove();
+              } else {
+                entry.setValue(num - 1);
+              }
+              return localStorage;
+            }
           }
         }
       } 
@@ -372,7 +436,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     }
     // try a node on local rack
     return chooseLocalRack(localMachine, excludedNodes, blocksize,
-        maxNodesPerRack, results, avoidStaleNodes, storageType);
+        maxNodesPerRack, results, avoidStaleNodes, storageTypes);
   }
   
   /**
@@ -395,50 +459,71 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    * @return the chosen node
    */
   protected DatanodeStorageInfo chooseLocalRack(Node localMachine,
-                                             Set<Node> excludedNodes,
-                                             long blocksize,
-                                             int maxNodesPerRack,
-                                             List<DatanodeStorageInfo> results,
-                                             boolean avoidStaleNodes,
-                                             StorageType storageType)
+                                                Set<Node> excludedNodes,
+                                                long blocksize,
+                                                int maxNodesPerRack,
+                                                List<DatanodeStorageInfo> results,
+                                                boolean avoidStaleNodes,
+                                                EnumMap<StorageType, Integer> storageTypes)
       throws NotEnoughReplicasException {
     // no local machine, so choose a random machine
     if (localMachine == null) {
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-          maxNodesPerRack, results, avoidStaleNodes, storageType);
+          maxNodesPerRack, results, avoidStaleNodes, storageTypes);
     }
+    final String localRack = localMachine.getNetworkLocation();
       
-    // choose one from the local rack
     try {
-      return chooseRandom(localMachine.getNetworkLocation(), excludedNodes,
-          blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
-    } catch (NotEnoughReplicasException e1) {
-      // find the second replica
-      DatanodeDescriptor newLocal=null;
+      // choose one from the local rack
+      return chooseRandom(localRack, excludedNodes,
+          blocksize, maxNodesPerRack, results, avoidStaleNodes, storageTypes);
+    } catch (NotEnoughReplicasException e) {
+      // find the next replica and retry with its rack
       for(DatanodeStorageInfo resultStorage : results) {
         DatanodeDescriptor nextNode = resultStorage.getDatanodeDescriptor();
         if (nextNode != localMachine) {
-          newLocal = nextNode;
-          break;
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Failed to choose from local rack (location = " + localRack
+                + "), retry with the rack of the next replica (location = "
+                + nextNode.getNetworkLocation() + ")", e);
+          }
+          return chooseFromNextRack(nextNode, excludedNodes, blocksize,
+              maxNodesPerRack, results, avoidStaleNodes, storageTypes);
         }
       }
-      if (newLocal != null) {
-        try {
-          return chooseRandom(newLocal.getNetworkLocation(), excludedNodes,
-              blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
-        } catch(NotEnoughReplicasException e2) {
-          //otherwise randomly choose one from the network
-          return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-              maxNodesPerRack, results, avoidStaleNodes, storageType);
-        }
-      } else {
-        //otherwise randomly choose one from the network
-        return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-            maxNodesPerRack, results, avoidStaleNodes, storageType);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Failed to choose from local rack (location = " + localRack
+            + "); the second replica is not found, retry choosing ramdomly", e);
       }
+      //the second replica is not found, randomly choose one from the network
+      return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
+          maxNodesPerRack, results, avoidStaleNodes, storageTypes);
     }
   }
-    
+
+  private DatanodeStorageInfo chooseFromNextRack(Node next,
+      Set<Node> excludedNodes,
+      long blocksize,
+      int maxNodesPerRack,
+      List<DatanodeStorageInfo> results,
+      boolean avoidStaleNodes,
+      EnumMap<StorageType, Integer> storageTypes) throws NotEnoughReplicasException {
+    final String nextRack = next.getNetworkLocation();
+    try {
+      return chooseRandom(nextRack, excludedNodes, blocksize, maxNodesPerRack,
+          results, avoidStaleNodes, storageTypes);
+    } catch(NotEnoughReplicasException e) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Failed to choose from the next rack (location = " + nextRack
+            + "), retry choosing ramdomly", e);
+      }
+      //otherwise randomly choose one from the network
+      return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
+          maxNodesPerRack, results, avoidStaleNodes, storageTypes);
+    }
+  }
+
   /** 
    * Choose <i>numOfReplicas</i> nodes from the racks 
    * that <i>localMachine</i> is NOT on.
@@ -453,18 +538,22 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                                 int maxReplicasPerRack,
                                 List<DatanodeStorageInfo> results,
                                 boolean avoidStaleNodes,
-                                StorageType storageType)
+                                EnumMap<StorageType, Integer> storageTypes)
                                     throws NotEnoughReplicasException {
     int oldNumOfReplicas = results.size();
     // randomly choose one node from remote racks
     try {
       chooseRandom(numOfReplicas, "~" + localMachine.getNetworkLocation(),
           excludedNodes, blocksize, maxReplicasPerRack, results,
-          avoidStaleNodes, storageType);
+          avoidStaleNodes, storageTypes);
     } catch (NotEnoughReplicasException e) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Failed to choose remote rack (location = ~"
+            + localMachine.getNetworkLocation() + "), fallback to local rack", e);
+      }
       chooseRandom(numOfReplicas-(results.size()-oldNumOfReplicas),
                    localMachine.getNetworkLocation(), excludedNodes, blocksize, 
-                   maxReplicasPerRack, results, avoidStaleNodes, storageType);
+                   maxReplicasPerRack, results, avoidStaleNodes, storageTypes);
     }
   }
 
@@ -478,10 +567,10 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       int maxNodesPerRack,
       List<DatanodeStorageInfo> results,
       boolean avoidStaleNodes,
-      StorageType storageType)
+      EnumMap<StorageType, Integer> storageTypes)
           throws NotEnoughReplicasException {
     return chooseRandom(1, scope, excludedNodes, blocksize, maxNodesPerRack,
-        results, avoidStaleNodes, storageType);
+        results, avoidStaleNodes, storageTypes);
   }
 
   /**
@@ -495,8 +584,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                             int maxNodesPerRack,
                             List<DatanodeStorageInfo> results,
                             boolean avoidStaleNodes,
-                            StorageType storageType)
-                                throws NotEnoughReplicasException {
+                            EnumMap<StorageType, Integer> storageTypes)
+                            throws NotEnoughReplicasException {
       
     int numOfAvailableNodes = clusterMap.countNumOfAvailableNodes(
         scope, excludedNodes);
@@ -512,24 +601,43 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       DatanodeDescriptor chosenNode = 
           (DatanodeDescriptor)clusterMap.chooseRandom(scope);
       if (excludedNodes.add(chosenNode)) { //was not in the excluded list
+        if (LOG.isDebugEnabled()) {
+          builder.append("\nNode ").append(NodeBase.getPath(chosenNode)).append(" [");
+        }
         numOfAvailableNodes--;
 
         final DatanodeStorageInfo[] storages = DFSUtil.shuffle(
             chosenNode.getStorageInfos());
-        int i;
-        for(i = 0; i < storages.length; i++) {
-          final int newExcludedNodes = addIfIsGoodTarget(storages[i],
-              excludedNodes, blocksize, maxNodesPerRack, considerLoad, results,
-              avoidStaleNodes, storageType);
-          if (newExcludedNodes >= 0) {
-            numOfReplicas--;
-            if (firstChosen == null) {
-              firstChosen = storages[i];
+        int i = 0;
+        boolean search = true;
+        for (Iterator<Map.Entry<StorageType, Integer>> iter = storageTypes
+            .entrySet().iterator(); search && iter.hasNext(); ) {
+          Map.Entry<StorageType, Integer> entry = iter.next();
+          for (i = 0; i < storages.length; i++) {
+            StorageType type = entry.getKey();
+            final int newExcludedNodes = addIfIsGoodTarget(storages[i],
+                excludedNodes, blocksize, maxNodesPerRack, considerLoad, results,
+                avoidStaleNodes, type);
+            if (newExcludedNodes >= 0) {
+              numOfReplicas--;
+              if (firstChosen == null) {
+                firstChosen = storages[i];
+              }
+              numOfAvailableNodes -= newExcludedNodes;
+              int num = entry.getValue();
+              if (num == 1) {
+                iter.remove();
+              } else {
+                entry.setValue(num - 1);
+              }
+              search = false;
+              break;
             }
-            numOfAvailableNodes -= newExcludedNodes;
-            break;
           }
         }
+        if (LOG.isDebugEnabled()) {
+          builder.append("\n]");
+        }
 
         // If no candidate storage was found on this DN then set badTarget.
         badTarget = (i == storages.length);
@@ -540,9 +648,11 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       String detail = enableDebugLogging;
       if (LOG.isDebugEnabled()) {
         if (badTarget && builder != null) {
-          detail = builder.append("]").toString();
+          detail = builder.toString();
           builder.setLength(0);
-        } else detail = "";
+        } else {
+          detail = "";
+        }
       }
       throw new NotEnoughReplicasException(detail);
     }
@@ -576,14 +686,10 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
 
   private static void logNodeIsNotChosen(DatanodeStorageInfo storage, String reason) {
     if (LOG.isDebugEnabled()) {
-      final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       // build the error message for later use.
       debugLoggingBuilder.get()
-          .append(node).append(": ")
-          .append("Storage ").append(storage)
-          .append("at node ").append(NodeBase.getPath(node))
-          .append(" is not chosen because ")
-          .append(reason);
+          .append("\n  Storage ").append(storage)
+          .append(" is not chosen since ").append(reason).append(".");
     }
   }
 
@@ -608,11 +714,10 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                                boolean considerLoad,
                                List<DatanodeStorageInfo> results,
                                boolean avoidStaleNodes,
-                               StorageType storageType) {
-    if (storage.getStorageType() != storageType) {
-      logNodeIsNotChosen(storage,
-          "storage types do not match, where the expected storage type is "
-              + storageType);
+                               StorageType requiredStorageType) {
+    if (storage.getStorageType() != requiredStorageType) {
+      logNodeIsNotChosen(storage, "storage types do not match,"
+          + " where the required storage type is " + requiredStorageType);
       return false;
     }
     if (storage.getState() == State.READ_ONLY_SHARED) {
@@ -634,9 +739,14 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     }
     
     final long requiredSize = blockSize * HdfsConstants.MIN_BLOCKS_FOR_WRITE;
-    final long scheduledSize = blockSize * node.getBlocksScheduled();
-    if (requiredSize > storage.getRemaining() - scheduledSize) {
-      logNodeIsNotChosen(storage, "the node does not have enough space ");
+    final long scheduledSize = blockSize * node.getBlocksScheduled(storage.getStorageType());
+    final long remaining = node.getRemaining(storage.getStorageType());
+    if (requiredSize > remaining - scheduledSize) {
+      logNodeIsNotChosen(storage, "the node does not have enough "
+          + storage.getStorageType() + " space"
+          + " (required=" + requiredSize
+          + ", scheduled=" + scheduledSize
+          + ", remaining=" + remaining + ")");
       return false;
     }
 
@@ -645,8 +755,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       final double maxLoad = 2.0 * stats.getInServiceXceiverAverage();
       final int nodeLoad = node.getXceiverCount();
       if (nodeLoad > maxLoad) {
-        logNodeIsNotChosen(storage,
-            "the node is too busy (load:"+nodeLoad+" > "+maxLoad+") ");
+        logNodeIsNotChosen(storage, "the node is too busy (load: " + nodeLoad
+            + " > " + maxLoad + ") ");
         return false;
       }
     }
@@ -666,7 +776,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     }
     return true;
   }
-    
+
   /**
    * Return a pipeline of nodes.
    * The pipeline is formed finding a shortest path that 
@@ -732,7 +842,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
   public DatanodeStorageInfo chooseReplicaToDelete(BlockCollection bc,
       Block block, short replicationFactor,
       Collection<DatanodeStorageInfo> first,
-      Collection<DatanodeStorageInfo> second) {
+      Collection<DatanodeStorageInfo> second,
+      final List<StorageType> excessTypes) {
     long oldestHeartbeat =
       now() - heartbeatInterval * tolerateHeartbeatMultiplier;
     DatanodeStorageInfo oldestHeartbeatStorage = null;
@@ -742,6 +853,10 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     // Pick the node with the oldest heartbeat or with the least free space,
     // if all hearbeats are within the tolerable heartbeat interval
     for(DatanodeStorageInfo storage : pickupReplicaSet(first, second)) {
+      if (!excessTypes.contains(storage.getStorageType())) {
+        continue;
+      }
+
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       long free = node.getRemaining();
       long lastHeartbeat = node.getLastUpdate();
@@ -755,8 +870,16 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       }
     }
 
-    return oldestHeartbeatStorage != null? oldestHeartbeatStorage
-        : minSpaceStorage;
+    final DatanodeStorageInfo storage;
+    if (oldestHeartbeatStorage != null) {
+      storage = oldestHeartbeatStorage;
+    } else if (minSpaceStorage != null) {
+      storage = minSpaceStorage;
+    } else {
+      return null;
+    }
+    excessTypes.remove(storage.getStorageType());
+    return storage;
   }
 
   /**

+ 41 - 32
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java

@@ -17,12 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -70,22 +65,33 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
   protected DatanodeStorageInfo chooseLocalStorage(Node localMachine,
       Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
       List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
-      StorageType storageType, boolean fallbackToLocalRack
-      ) throws NotEnoughReplicasException {
+      EnumMap<StorageType, Integer> storageTypes, boolean fallbackToLocalRack)
+      throws NotEnoughReplicasException {
     // if no local machine, randomly choose one node
     if (localMachine == null)
       return chooseRandom(NodeBase.ROOT, excludedNodes, 
-          blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
+          blocksize, maxNodesPerRack, results, avoidStaleNodes, storageTypes);
 
     // otherwise try local machine first
     if (localMachine instanceof DatanodeDescriptor) {
       DatanodeDescriptor localDataNode = (DatanodeDescriptor)localMachine;
       if (excludedNodes.add(localMachine)) { // was not in the excluded list
-        for(DatanodeStorageInfo localStorage : DFSUtil.shuffle(
-            localDataNode.getStorageInfos())) {
-          if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
-              maxNodesPerRack, false, results, avoidStaleNodes, storageType) >= 0) {
-            return localStorage;
+        for (Iterator<Map.Entry<StorageType, Integer>> iter = storageTypes
+            .entrySet().iterator(); iter.hasNext(); ) {
+          Map.Entry<StorageType, Integer> entry = iter.next();
+          for (DatanodeStorageInfo localStorage : DFSUtil.shuffle(
+              localDataNode.getStorageInfos())) {
+            StorageType type = entry.getKey();
+            if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
+                maxNodesPerRack, false, results, avoidStaleNodes, type) >= 0) {
+              int num = entry.getValue();
+              if (num == 1) {
+                iter.remove();
+              } else {
+                entry.setValue(num - 1);
+              }
+              return localStorage;
+            }
           }
         }
       }
@@ -94,7 +100,7 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
     // try a node on local node group
     DatanodeStorageInfo chosenStorage = chooseLocalNodeGroup(
         (NetworkTopologyWithNodeGroup)clusterMap, localMachine, excludedNodes, 
-        blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
+        blocksize, maxNodesPerRack, results, avoidStaleNodes, storageTypes);
     if (chosenStorage != null) {
       return chosenStorage;
     }
@@ -104,7 +110,7 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
     }
     // try a node on local rack
     return chooseLocalRack(localMachine, excludedNodes, 
-        blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
+        blocksize, maxNodesPerRack, results, avoidStaleNodes, storageTypes);
   }
 
   /** @return the node of the second replica */
@@ -124,18 +130,19 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
   protected DatanodeStorageInfo chooseLocalRack(Node localMachine,
       Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
       List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
-      StorageType storageType) throws NotEnoughReplicasException {
+      EnumMap<StorageType, Integer> storageTypes) throws
+      NotEnoughReplicasException {
     // no local machine, so choose a random machine
     if (localMachine == null) {
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-          maxNodesPerRack, results, avoidStaleNodes, storageType);
+          maxNodesPerRack, results, avoidStaleNodes, storageTypes);
     }
 
     // choose one from the local rack, but off-nodegroup
     try {
       final String scope = NetworkTopology.getFirstHalf(localMachine.getNetworkLocation());
       return chooseRandom(scope, excludedNodes, blocksize, maxNodesPerRack,
-          results, avoidStaleNodes, storageType);
+          results, avoidStaleNodes, storageTypes);
     } catch (NotEnoughReplicasException e1) {
       // find the second replica
       final DatanodeDescriptor newLocal = secondNode(localMachine, results);
@@ -143,16 +150,17 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
         try {
           return chooseRandom(
               clusterMap.getRack(newLocal.getNetworkLocation()), excludedNodes,
-              blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
+              blocksize, maxNodesPerRack, results, avoidStaleNodes,
+              storageTypes);
         } catch(NotEnoughReplicasException e2) {
           //otherwise randomly choose one from the network
           return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-              maxNodesPerRack, results, avoidStaleNodes, storageType);
+              maxNodesPerRack, results, avoidStaleNodes, storageTypes);
         }
       } else {
         //otherwise randomly choose one from the network
         return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-            maxNodesPerRack, results, avoidStaleNodes, storageType);
+            maxNodesPerRack, results, avoidStaleNodes, storageTypes);
       }
     }
   }
@@ -161,8 +169,8 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
   protected void chooseRemoteRack(int numOfReplicas,
       DatanodeDescriptor localMachine, Set<Node> excludedNodes,
       long blocksize, int maxReplicasPerRack, List<DatanodeStorageInfo> results,
-      boolean avoidStaleNodes, StorageType storageType)
-          throws NotEnoughReplicasException {
+      boolean avoidStaleNodes, EnumMap<StorageType, Integer> storageTypes)
+      throws NotEnoughReplicasException {
     int oldNumOfReplicas = results.size();
 
     final String rackLocation = NetworkTopology.getFirstHalf(
@@ -170,12 +178,12 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
     try {
       // randomly choose from remote racks
       chooseRandom(numOfReplicas, "~" + rackLocation, excludedNodes, blocksize,
-          maxReplicasPerRack, results, avoidStaleNodes, storageType);
+          maxReplicasPerRack, results, avoidStaleNodes, storageTypes);
     } catch (NotEnoughReplicasException e) {
       // fall back to the local rack
       chooseRandom(numOfReplicas - (results.size() - oldNumOfReplicas),
           rackLocation, excludedNodes, blocksize,
-          maxReplicasPerRack, results, avoidStaleNodes, storageType);
+          maxReplicasPerRack, results, avoidStaleNodes, storageTypes);
     }
   }
 
@@ -189,11 +197,12 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
       NetworkTopologyWithNodeGroup clusterMap, Node localMachine,
       Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
       List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
-      StorageType storageType) throws NotEnoughReplicasException {
+      EnumMap<StorageType, Integer> storageTypes) throws
+      NotEnoughReplicasException {
     // no local machine, so choose a random machine
     if (localMachine == null) {
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-          maxNodesPerRack, results, avoidStaleNodes, storageType);
+          maxNodesPerRack, results, avoidStaleNodes, storageTypes);
     }
 
     // choose one from the local node group
@@ -201,7 +210,7 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
       return chooseRandom(
           clusterMap.getNodeGroup(localMachine.getNetworkLocation()),
           excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes,
-          storageType);
+          storageTypes);
     } catch (NotEnoughReplicasException e1) {
       final DatanodeDescriptor newLocal = secondNode(localMachine, results);
       if (newLocal != null) {
@@ -209,16 +218,16 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
           return chooseRandom(
               clusterMap.getNodeGroup(newLocal.getNetworkLocation()),
               excludedNodes, blocksize, maxNodesPerRack, results,
-              avoidStaleNodes, storageType);
+              avoidStaleNodes, storageTypes);
         } catch(NotEnoughReplicasException e2) {
           //otherwise randomly choose one from the network
           return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-              maxNodesPerRack, results, avoidStaleNodes, storageType);
+              maxNodesPerRack, results, avoidStaleNodes, storageTypes);
         }
       } else {
         //otherwise randomly choose one from the network
         return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-            maxNodesPerRack, results, avoidStaleNodes, storageType);
+            maxNodesPerRack, results, avoidStaleNodes, storageTypes);
       }
     }
   }

+ 40 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -28,16 +28,19 @@ import java.util.Map;
 import java.util.Queue;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.hdfs.util.EnumCounters;
 import org.apache.hadoop.hdfs.util.LightWeightHashSet;
 import org.apache.hadoop.util.IntrusiveCollection;
 import org.apache.hadoop.util.Time;
@@ -202,8 +205,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * in case of errors (e.g. datanode does not report if an error occurs
    * while writing the block).
    */
-  private int currApproxBlocksScheduled = 0;
-  private int prevApproxBlocksScheduled = 0;
+  private EnumCounters<StorageType> currApproxBlocksScheduled
+      = new EnumCounters<StorageType>(StorageType.class);
+  private EnumCounters<StorageType> prevApproxBlocksScheduled
+      = new EnumCounters<StorageType>(StorageType.class);
   private long lastBlocksScheduledRollTime = 0;
   private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min
   private int volumeFailures = 0;
@@ -474,25 +479,48 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
+  /**
+   * @return Approximate number of blocks currently scheduled to be written 
+   */
+  public long getRemaining(StorageType t) {
+    long remaining = 0;
+    for(DatanodeStorageInfo s : getStorageInfos()) {
+      if (s.getStorageType() == t) {
+        remaining += s.getRemaining();
+      }
+    }
+    return remaining;    
+  }
+
+  /**
+   * @return Approximate number of blocks currently scheduled to be written 
+   * to the given storage type of this datanode.
+   */
+  public int getBlocksScheduled(StorageType t) {
+    return (int)(currApproxBlocksScheduled.get(t)
+        + prevApproxBlocksScheduled.get(t));
+  }
+
   /**
    * @return Approximate number of blocks currently scheduled to be written 
    * to this datanode.
    */
   public int getBlocksScheduled() {
-    return currApproxBlocksScheduled + prevApproxBlocksScheduled;
+    return (int)(currApproxBlocksScheduled.sum()
+        + prevApproxBlocksScheduled.sum());
   }
 
   /** Increment the number of blocks scheduled. */
-  void incrementBlocksScheduled() {
-    currApproxBlocksScheduled++;
+  void incrementBlocksScheduled(StorageType t) {
+    currApproxBlocksScheduled.add(t, 1);;
   }
   
   /** Decrement the number of blocks scheduled. */
-  void decrementBlocksScheduled() {
-    if (prevApproxBlocksScheduled > 0) {
-      prevApproxBlocksScheduled--;
-    } else if (currApproxBlocksScheduled > 0) {
-      currApproxBlocksScheduled--;
+  void decrementBlocksScheduled(StorageType t) {
+    if (prevApproxBlocksScheduled.get(t) > 0) {
+      prevApproxBlocksScheduled.subtract(t, 1);
+    } else if (currApproxBlocksScheduled.get(t) > 0) {
+      currApproxBlocksScheduled.subtract(t, 1);
     } 
     // its ok if both counters are zero.
   }
@@ -500,8 +528,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /** Adjusts curr and prev number of blocks scheduled every few minutes. */
   private void rollBlocksScheduled(long now) {
     if (now - lastBlocksScheduledRollTime > BLOCKS_SCHEDULED_ROLL_INTERVAL) {
-      prevApproxBlocksScheduled = currApproxBlocksScheduled;
-      currApproxBlocksScheduled = 0;
+      prevApproxBlocksScheduled.set(currApproxBlocksScheduled);
+      currApproxBlocksScheduled.reset();
       lastBlocksScheduledRollTime = now;
     }
   }

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

@@ -348,8 +348,7 @@ public class DatanodeManager {
   
   /** Sort the located blocks by the distance to the target host. */
   public void sortLocatedBlocks(final String targethost,
-      final List<LocatedBlock> locatedblocks,
-      boolean randomizeBlockLocationsPerBlock) {
+      final List<LocatedBlock> locatedblocks) {
     //sort the blocks
     // As it is possible for the separation of node manager and datanode, 
     // here we should get node but not datanode only .
@@ -376,8 +375,7 @@ public class DatanodeManager {
           --lastActiveIndex;
       }
       int activeLen = lastActiveIndex + 1;      
-      networktopology.sortByDistance(client, b.getLocations(), activeLen, b
-          .getBlock().getBlockId(), randomizeBlockLocationsPerBlock);
+      networktopology.sortByDistance(client, b.getLocations(), activeLen);
     }
   }
   

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

@@ -109,7 +109,7 @@ public class DatanodeStorageInfo {
 
   private long capacity;
   private long dfsUsed;
-  private long remaining;
+  private volatile long remaining;
   private long blockPoolUsed;
 
   private volatile BlockInfo blockList = null;
@@ -283,7 +283,7 @@ public class DatanodeStorageInfo {
   /** Increment the number of blocks scheduled for each given storage */ 
   public static void incrementBlocksScheduled(DatanodeStorageInfo... storages) {
     for (DatanodeStorageInfo s : storages) {
-      s.getDatanodeDescriptor().incrementBlocksScheduled();
+      s.getDatanodeDescriptor().incrementBlocksScheduled(s.getStorageType());
     }
   }
 
@@ -314,6 +314,26 @@ public class DatanodeStorageInfo {
         false, capacity, dfsUsed, remaining, blockPoolUsed);
   }
 
+  static Iterable<StorageType> toStorageTypes(
+      final Iterable<DatanodeStorageInfo> infos) {
+    return new Iterable<StorageType>() {
+        @Override
+        public Iterator<StorageType> iterator() {
+          return new Iterator<StorageType>() {
+            final Iterator<DatanodeStorageInfo> i = infos.iterator();
+            @Override
+            public boolean hasNext() {return i.hasNext();}
+            @Override
+            public StorageType next() {return i.next().getStorageType();}
+            @Override
+            public void remove() {
+              throw new UnsupportedOperationException();
+            }
+          };
+        }
+      };
+  }
+
   /** @return the first {@link DatanodeStorageInfo} corresponding to
    *          the given datanode
    */

+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java

@@ -94,7 +94,12 @@ public final class HdfsServerConstants {
     NONINTERACTIVE("-nonInteractive"),
     RENAMERESERVED("-renameReserved"),
     METADATAVERSION("-metadataVersion"),
-    UPGRADEONLY("-upgradeOnly");
+    UPGRADEONLY("-upgradeOnly"),
+    // The -hotswap constant should not be used as a startup option, it is
+    // only used for StorageDirectory.analyzeStorage() in hot swap drive scenario.
+    // TODO refactor StorageDirectory.analyzeStorage() so that we can do away with
+    // this in StartupOption.
+    HOTSWAP("-hotswap");
 
     private static final Pattern ENUM_WITH_ROLLING_UPGRADE_OPTION = Pattern.compile(
         "(\\w+)\\((\\w+)\\)");

+ 8 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -464,17 +464,20 @@ public abstract class Storage extends StorageInfo {
     public StorageState analyzeStorage(StartupOption startOpt, Storage storage)
         throws IOException {
       assert root != null : "root is null";
+      boolean hadMkdirs = false;
       String rootPath = root.getCanonicalPath();
       try { // check that storage exists
         if (!root.exists()) {
           // storage directory does not exist
-          if (startOpt != StartupOption.FORMAT) {
+          if (startOpt != StartupOption.FORMAT &&
+              startOpt != StartupOption.HOTSWAP) {
             LOG.warn("Storage directory " + rootPath + " does not exist");
             return StorageState.NON_EXISTENT;
           }
           LOG.info(rootPath + " does not exist. Creating ...");
           if (!root.mkdirs())
             throw new IOException("Cannot create directory " + rootPath);
+          hadMkdirs = true;
         }
         // or is inaccessible
         if (!root.isDirectory()) {
@@ -492,7 +495,10 @@ public abstract class Storage extends StorageInfo {
 
       this.lock(); // lock storage if it exists
 
-      if (startOpt == HdfsServerConstants.StartupOption.FORMAT)
+      // If startOpt is HOTSWAP, it returns NOT_FORMATTED for empty directory,
+      // while it also checks the layout version.
+      if (startOpt == HdfsServerConstants.StartupOption.FORMAT ||
+          (startOpt == StartupOption.HOTSWAP && hadMkdirs))
         return StorageState.NOT_FORMATTED;
 
       if (startOpt != HdfsServerConstants.StartupOption.IMPORT) {

+ 160 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -70,8 +70,10 @@ import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -80,11 +82,13 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.management.ObjectName;
 
+import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.conf.ReconfigurableBase;
+import org.apache.hadoop.conf.ReconfigurationException;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -137,6 +141,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
@@ -220,7 +225,7 @@ import com.google.protobuf.BlockingService;
  *
  **********************************************************/
 @InterfaceAudience.Private
-public class DataNode extends Configured 
+public class DataNode extends ReconfigurableBase
     implements InterDatanodeProtocol, ClientDatanodeProtocol,
     DataNodeMXBean {
   public static final Log LOG = LogFactory.getLog(DataNode.class);
@@ -305,6 +310,7 @@ public class DataNode extends Configured
   private JvmPauseMonitor pauseMonitor;
 
   private SecureResources secureResources = null;
+  // dataDirs must be accessed while holding the DataNode lock.
   private List<StorageLocation> dataDirs;
   private Configuration conf;
   private final String confVersion;
@@ -386,6 +392,149 @@ public class DataNode extends Configured
     }
   }
 
+  @Override
+  public void reconfigurePropertyImpl(String property, String newVal)
+      throws ReconfigurationException {
+    if (property.equals(DFS_DATANODE_DATA_DIR_KEY)) {
+      try {
+        LOG.info("Reconfiguring " + property + " to " + newVal);
+        this.refreshVolumes(newVal);
+      } catch (Exception e) {
+        throw new ReconfigurationException(property, newVal,
+            getConf().get(property), e);
+      }
+    } else {
+      throw new ReconfigurationException(
+          property, newVal, getConf().get(property));
+    }
+  }
+
+  /**
+   * Get a list of the keys of the re-configurable properties in configuration.
+   */
+  @Override
+  public Collection<String> getReconfigurableProperties() {
+    List<String> reconfigurable =
+        Collections.unmodifiableList(Arrays.asList(DFS_DATANODE_DATA_DIR_KEY));
+    return reconfigurable;
+  }
+
+  /**
+   * Contains the StorageLocations for changed data volumes.
+   */
+  @VisibleForTesting
+  static class ChangedVolumes {
+    List<StorageLocation> newLocations = Lists.newArrayList();
+    List<StorageLocation> deactivateLocations = Lists.newArrayList();
+  }
+
+  /**
+   * Parse the new DFS_DATANODE_DATA_DIR value in the configuration to detect
+   * changed volumes.
+   * @return changed volumes.
+   * @throws IOException if none of the directories are specified in the
+   * configuration.
+   */
+  @VisibleForTesting
+  ChangedVolumes parseChangedVolumes() throws IOException {
+    List<StorageLocation> locations = getStorageLocations(getConf());
+
+    if (locations.isEmpty()) {
+      throw new IOException("No directory is specified.");
+    }
+
+    ChangedVolumes results = new ChangedVolumes();
+    results.newLocations.addAll(locations);
+
+    for (Iterator<Storage.StorageDirectory> it = storage.dirIterator();
+         it.hasNext(); ) {
+      Storage.StorageDirectory dir = it.next();
+      boolean found = false;
+      for (Iterator<StorageLocation> sl = results.newLocations.iterator();
+           sl.hasNext(); ) {
+        if (sl.next().getFile().getCanonicalPath().equals(
+            dir.getRoot().getCanonicalPath())) {
+          sl.remove();
+          found = true;
+          break;
+        }
+      }
+
+      if (!found) {
+        results.deactivateLocations.add(
+            StorageLocation.parse(dir.getRoot().toString()));
+      }
+    }
+
+    return results;
+  }
+
+  /**
+   * Attempts to reload data volumes with new configuration.
+   * @param newVolumes a comma separated string that specifies the data volumes.
+   * @throws Exception
+   */
+  private synchronized void refreshVolumes(String newVolumes) throws Exception {
+    Configuration conf = getConf();
+    String oldVolumes = conf.get(DFS_DATANODE_DATA_DIR_KEY);
+    conf.set(DFS_DATANODE_DATA_DIR_KEY, newVolumes);
+    List<StorageLocation> locations = getStorageLocations(conf);
+
+    final int numOldDataDirs = dataDirs.size();
+    dataDirs = locations;
+    ChangedVolumes changedVolumes = parseChangedVolumes();
+
+    try {
+      if (numOldDataDirs + changedVolumes.newLocations.size() -
+          changedVolumes.deactivateLocations.size() <= 0) {
+        throw new IOException("Attempt to remove all volumes.");
+      }
+      if (!changedVolumes.newLocations.isEmpty()) {
+        LOG.info("Adding new volumes: " +
+            Joiner.on(",").join(changedVolumes.newLocations));
+
+        // Add volumes for each Namespace
+        for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) {
+          NamespaceInfo nsInfo = bpos.getNamespaceInfo();
+          LOG.info("Loading volumes for namesapce: " + nsInfo.getNamespaceID());
+          storage.addStorageLocations(
+              this, nsInfo, changedVolumes.newLocations, StartupOption.HOTSWAP);
+        }
+        List<String> bpids = Lists.newArrayList();
+        for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) {
+          bpids.add(bpos.getBlockPoolId());
+        }
+        List<StorageLocation> succeedVolumes =
+            data.addVolumes(changedVolumes.newLocations, bpids);
+
+        if (succeedVolumes.size() < changedVolumes.newLocations.size()) {
+          List<StorageLocation> failedVolumes = Lists.newArrayList();
+          // Clean all failed volumes.
+          for (StorageLocation location : changedVolumes.newLocations) {
+            if (!succeedVolumes.contains(location)) {
+              failedVolumes.add(location);
+            }
+          }
+          storage.removeVolumes(failedVolumes);
+          data.removeVolumes(failedVolumes);
+        }
+      }
+
+      if (!changedVolumes.deactivateLocations.isEmpty()) {
+        LOG.info("Deactivating volumes: " +
+            Joiner.on(",").join(changedVolumes.deactivateLocations));
+
+        data.removeVolumes(changedVolumes.deactivateLocations);
+        storage.removeVolumes(changedVolumes.deactivateLocations);
+      }
+    } catch (IOException e) {
+      LOG.warn("There is IOException when refreshing volumes! "
+          + "Recover configurations: " + DFS_DATANODE_DATA_DIR_KEY
+          + " = " + oldVolumes, e);
+      throw e;
+    }
+  }
+
   private synchronized void setClusterId(final String nsCid, final String bpid
       ) throws IOException {
     if(clusterId != null && !clusterId.equals(nsCid)) {
@@ -822,7 +971,9 @@ public class DataNode extends Configured
 
     // settings global for all BPs in the Data Node
     this.secureResources = resources;
-    this.dataDirs = dataDirs;
+    synchronized (this) {
+      this.dataDirs = dataDirs;
+    }
     this.conf = conf;
     this.dnConf = new DNConf(conf);
     this.spanReceiverHost = SpanReceiverHost.getInstance(conf);
@@ -1112,7 +1263,9 @@ public class DataNode extends Configured
       }
       final String bpid = nsInfo.getBlockPoolID();
       //read storage info, lock data dirs and transition fs state if necessary
-      storage.recoverTransitionRead(this, bpid, nsInfo, dataDirs, startOpt);
+      synchronized (this) {
+        storage.recoverTransitionRead(this, bpid, nsInfo, dataDirs, startOpt);
+      }
       final StorageInfo bpStorage = storage.getBPStorage(bpid);
       LOG.info("Setting up storage: nsid=" + bpStorage.getNamespaceID()
           + ";bpid=" + bpid + ";lv=" + storage.getLayoutVersion()
@@ -1744,7 +1897,9 @@ public class DataNode extends Configured
             + b + " (numBytes=" + b.getNumBytes() + ")"
             + ", stage=" + stage
             + ", clientname=" + clientname
-            + ", targets=" + Arrays.asList(targets));
+            + ", targets=" + Arrays.asList(targets)
+            + ", target storage types=" + (targetStorageTypes == null ? "[]" :
+            Arrays.asList(targetStorageTypes)));
       }
       this.targets = targets;
       this.targetStorageTypes = targetStorageTypes;

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

@@ -94,8 +94,8 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   public List<V> getVolumes();
 
   /** Add an array of StorageLocation to FsDataset. */
-  public void addVolumes(Collection<StorageLocation> volumes)
-      throws IOException;
+  public List<StorageLocation> addVolumes(List<StorageLocation> volumes,
+      final Collection<String> bpids);
 
   /** Removes a collection of volumes from FsDataset. */
   public void removeVolumes(Collection<StorageLocation> volumes);

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

@@ -28,13 +28,16 @@ import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Executor;
 
 import javax.management.NotCompliantMBeanException;
@@ -42,6 +45,7 @@ import javax.management.ObjectName;
 import javax.management.StandardMBean;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -88,6 +92,7 @@ import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlo
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.util.Daemon;
@@ -255,7 +260,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           + ", volume failures tolerated: " + volFailuresTolerated);
     }
 
-    storageMap = new HashMap<String, DatanodeStorage>();
+    storageMap = new ConcurrentHashMap<String, DatanodeStorage>();
     volumeMap = new ReplicaMap(this);
     lazyWriteReplicaTracker = new LazyWriteReplicaTracker(this);
 
@@ -291,45 +296,124 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     // storageMap and asyncDiskService, consistent.
     FsVolumeImpl fsVolume = FsVolumeImplAllocator.createVolume(
         this, sd.getStorageUuid(), dir, this.conf, storageType);
-    fsVolume.getVolumeMap(volumeMap, lazyWriteReplicaTracker);
+    ReplicaMap tempVolumeMap = new ReplicaMap(this);
+    fsVolume.getVolumeMap(tempVolumeMap, lazyWriteReplicaTracker);
 
+    volumeMap.addAll(tempVolumeMap);
     volumes.addVolume(fsVolume);
     storageMap.put(sd.getStorageUuid(),
         new DatanodeStorage(sd.getStorageUuid(),
-                            DatanodeStorage.State.NORMAL,
-                            storageType));
+            DatanodeStorage.State.NORMAL,
+            storageType));
     asyncDiskService.addVolume(sd.getCurrentDir());
 
     LOG.info("Added volume - " + dir + ", StorageType: " + storageType);
   }
 
+  private void addVolumeAndBlockPool(Collection<StorageLocation> dataLocations,
+      Storage.StorageDirectory sd, final Collection<String> bpids)
+      throws IOException {
+    final File dir = sd.getCurrentDir();
+    final StorageType storageType =
+        getStorageTypeFromLocations(dataLocations, sd.getRoot());
+
+    final FsVolumeImpl fsVolume = new FsVolumeImpl(
+        this, sd.getStorageUuid(), dir, this.conf, storageType);
+    final ReplicaMap tempVolumeMap = new ReplicaMap(fsVolume);
+
+    List<IOException> exceptions = Lists.newArrayList();
+    for (final String bpid : bpids) {
+      try {
+        fsVolume.addBlockPool(bpid, this.conf);
+        fsVolume.getVolumeMap(bpid, tempVolumeMap, lazyWriteReplicaTracker);
+      } catch (IOException e) {
+        LOG.warn("Caught exception when adding " + fsVolume +
+            ". Will throw later.", e);
+        exceptions.add(e);
+      }
+    }
+    if (!exceptions.isEmpty()) {
+      // The states of FsDatasteImpl are not modified, thus no need to rolled back.
+      throw MultipleIOException.createIOException(exceptions);
+    }
+
+    volumeMap.addAll(tempVolumeMap);
+    storageMap.put(sd.getStorageUuid(),
+        new DatanodeStorage(sd.getStorageUuid(),
+            DatanodeStorage.State.NORMAL,
+            storageType));
+    asyncDiskService.addVolume(sd.getCurrentDir());
+    volumes.addVolume(fsVolume);
+
+    LOG.info("Added volume - " + dir + ", StorageType: " + storageType);
+  }
+
   /**
    * Add an array of StorageLocation to FsDataset.
    *
    * @pre dataStorage must have these volumes.
-   * @param volumes
-   * @throws IOException
+   * @param volumes an array of storage locations for adding volumes.
+   * @param bpids block pool IDs.
+   * @return an array of successfully loaded volumes.
    */
   @Override
-  public synchronized void addVolumes(Collection<StorageLocation> volumes)
-      throws IOException {
+  public synchronized List<StorageLocation> addVolumes(
+      final List<StorageLocation> volumes, final Collection<String> bpids) {
     final Collection<StorageLocation> dataLocations =
         DataNode.getStorageLocations(this.conf);
-    Map<String, Storage.StorageDirectory> allStorageDirs =
+    final Map<String, Storage.StorageDirectory> allStorageDirs =
         new HashMap<String, Storage.StorageDirectory>();
-    for (int idx = 0; idx < dataStorage.getNumStorageDirs(); idx++) {
-      Storage.StorageDirectory sd = dataStorage.getStorageDir(idx);
-      allStorageDirs.put(sd.getRoot().getAbsolutePath(), sd);
+    List<StorageLocation> succeedVolumes = Lists.newArrayList();
+    try {
+      for (int idx = 0; idx < dataStorage.getNumStorageDirs(); idx++) {
+        Storage.StorageDirectory sd = dataStorage.getStorageDir(idx);
+        allStorageDirs.put(sd.getRoot().getCanonicalPath(), sd);
+      }
+    } catch (IOException ioe) {
+      LOG.warn("Caught exception when parsing storage URL.", ioe);
+      return succeedVolumes;
+    }
+
+    final boolean[] successFlags = new boolean[volumes.size()];
+    Arrays.fill(successFlags, false);
+    List<Thread> volumeAddingThreads = Lists.newArrayList();
+    for (int i = 0; i < volumes.size(); i++) {
+      final int idx = i;
+      Thread t = new Thread() {
+        public void run() {
+          StorageLocation vol = volumes.get(idx);
+          try {
+            String key = vol.getFile().getCanonicalPath();
+            if (!allStorageDirs.containsKey(key)) {
+              LOG.warn("Attempt to add an invalid volume: " + vol.getFile());
+            } else {
+              addVolumeAndBlockPool(dataLocations, allStorageDirs.get(key),
+                  bpids);
+              successFlags[idx] = true;
+            }
+          } catch (IOException e) {
+            LOG.warn("Caught exception when adding volume " + vol, e);
+          }
+        }
+      };
+      volumeAddingThreads.add(t);
+      t.start();
     }
 
-    for (StorageLocation vol : volumes) {
-      String key = vol.getFile().getAbsolutePath();
-      if (!allStorageDirs.containsKey(key)) {
-        LOG.warn("Attempt to add an invalid volume: " + vol.getFile());
-      } else {
-        addVolume(dataLocations, allStorageDirs.get(key));
+    for (Thread t : volumeAddingThreads) {
+      try {
+        t.join();
+      } catch (InterruptedException e) {
+        LOG.warn("Caught InterruptedException when adding volume.", e);
       }
     }
+
+    for (int i = 0; i < volumes.size(); i++) {
+      if (successFlags[i]) {
+        succeedVolumes.add(volumes.get(i));
+      }
+    }
+    return succeedVolumes;
   }
 
   /**
@@ -351,9 +435,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         String volume = sd.getRoot().toString();
         LOG.info("Removing " + volume + " from FsDataset.");
 
-        this.volumes.removeVolume(volume);
-        storageMap.remove(sd.getStorageUuid());
+        // Disable the volume from the service.
         asyncDiskService.removeVolume(sd.getCurrentDir());
+        this.volumes.removeVolume(volume);
 
         // Removed all replica information for the blocks on the volume. Unlike
         // updating the volumeMap in addVolume(), this operation does not scan
@@ -364,7 +448,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
               it.hasNext(); ) {
             ReplicaInfo block = it.next();
             if (block.getVolume().getBasePath().equals(volume)) {
-              invalidate(bpid, block.getBlockId());
+              invalidate(bpid, block);
               blocks.add(block);
               it.remove();
             }
@@ -373,6 +457,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           datanode.getBlockScanner().deleteBlocks(bpid,
               blocks.toArray(new Block[blocks.size()]));
         }
+
+        storageMap.remove(sd.getStorageUuid());
       }
     }
   }
@@ -1434,23 +1520,26 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   /**
    * Invalidate a block but does not delete the actual on-disk block file.
    *
-   * It should only be used for decommissioning disks.
+   * It should only be used when deactivating disks.
    *
    * @param bpid the block pool ID.
-   * @param blockId the ID of the block.
+   * @param block The block to be invalidated.
    */
-  public void invalidate(String bpid, long blockId) {
+  public void invalidate(String bpid, ReplicaInfo block) {
     // If a DFSClient has the replica in its cache of short-circuit file
     // descriptors (and the client is using ShortCircuitShm), invalidate it.
     // The short-circuit registry is null in the unit tests, because the
     // datanode is mock object.
     if (datanode.getShortCircuitRegistry() != null) {
       datanode.getShortCircuitRegistry().processBlockInvalidation(
-          new ExtendedBlockId(blockId, bpid));
+          new ExtendedBlockId(block.getBlockId(), bpid));
 
       // If the block is cached, start uncaching it.
-      cacheManager.uncacheBlock(bpid, blockId);
+      cacheManager.uncacheBlock(bpid, block.getBlockId());
     }
+
+    datanode.notifyNamenodeDeletedBlock(new ExtendedBlock(bpid, block),
+        block.getStorageUuid());
   }
 
   /**

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

@@ -0,0 +1,644 @@
+/**
+ * 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.mover;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.cli.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.*;
+import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.server.balancer.Dispatcher;
+import org.apache.hadoop.hdfs.server.balancer.Dispatcher.*;
+import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DDatanode.StorageGroup;
+import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
+import org.apache.hadoop.hdfs.server.balancer.Matcher;
+import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import java.io.BufferedReader;
+import java.io.FileReader;
+import java.io.IOException;
+import java.net.URI;
+import java.text.DateFormat;
+import java.util.*;
+
+@InterfaceAudience.Private
+public class Mover {
+  static final Log LOG = LogFactory.getLog(Mover.class);
+
+  static final Path MOVER_ID_PATH = new Path("/system/mover.id");
+
+  private static class StorageMap {
+    private final StorageGroupMap<Source> sources
+        = new StorageGroupMap<Source>();
+    private final StorageGroupMap<StorageGroup> targets
+        = new StorageGroupMap<StorageGroup>();
+    private final EnumMap<StorageType, List<StorageGroup>> targetStorageTypeMap
+        = new EnumMap<StorageType, List<StorageGroup>>(StorageType.class);
+    
+    private StorageMap() {
+      for(StorageType t : StorageType.asList()) {
+        targetStorageTypeMap.put(t, new LinkedList<StorageGroup>());
+      }
+    }
+    
+    private void add(Source source, StorageGroup target) {
+      sources.put(source);
+      targets.put(target);
+      getTargetStorages(target.getStorageType()).add(target);
+    }
+    
+    private Source getSource(MLocation ml) {
+      return get(sources, ml);
+    }
+
+    private StorageGroup getTarget(MLocation ml) {
+      return get(targets, ml);
+    }
+
+    private static <G extends StorageGroup> G get(StorageGroupMap<G> map, MLocation ml) {
+      return map.get(ml.datanode.getDatanodeUuid(), ml.storageType);
+    }
+    
+    private List<StorageGroup> getTargetStorages(StorageType t) {
+      return targetStorageTypeMap.get(t);
+    }
+  }
+
+  private final Dispatcher dispatcher;
+  private final StorageMap storages;
+  private final List<Path> targetPaths;
+
+  private final BlockStoragePolicy.Suite blockStoragePolicies;
+
+  Mover(NameNodeConnector nnc, Configuration conf) {
+    final long movedWinWidth = conf.getLong(
+        DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_KEY,
+        DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_DEFAULT);
+    final int moverThreads = conf.getInt(
+        DFSConfigKeys.DFS_MOVER_MOVERTHREADS_KEY,
+        DFSConfigKeys.DFS_MOVER_MOVERTHREADS_DEFAULT);
+    final int maxConcurrentMovesPerNode = conf.getInt(
+        DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY,
+        DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT);
+
+    this.dispatcher = new Dispatcher(nnc, Collections.<String> emptySet(),
+        Collections.<String> emptySet(), movedWinWidth, moverThreads, 0,
+        maxConcurrentMovesPerNode, conf);
+    this.storages = new StorageMap();
+    this.blockStoragePolicies = BlockStoragePolicy.readBlockStorageSuite(conf);
+    this.targetPaths = nnc.getTargetPaths();
+  }
+
+  void init() throws IOException {
+    final List<DatanodeStorageReport> reports = dispatcher.init();
+    for(DatanodeStorageReport r : reports) {
+      final DDatanode dn = dispatcher.newDatanode(r.getDatanodeInfo());
+      for(StorageType t : StorageType.asList()) {
+        final long maxRemaining = getMaxRemaining(r, t);
+        if (maxRemaining > 0L) {
+          final Source source = dn.addSource(t, Long.MAX_VALUE, dispatcher); 
+          final StorageGroup target = dn.addTarget(t, maxRemaining);
+          storages.add(source, target);
+        }
+      }
+    }
+  }
+
+  private ExitStatus run() {
+    try {
+      init();
+      boolean hasRemaining = new Processor().processNamespace();
+      return hasRemaining ? ExitStatus.IN_PROGRESS : ExitStatus.SUCCESS;
+    } catch (IllegalArgumentException e) {
+      System.out.println(e + ".  Exiting ...");
+      return ExitStatus.ILLEGAL_ARGUMENTS;
+    } catch (IOException e) {
+      System.out.println(e + ".  Exiting ...");
+      return ExitStatus.IO_EXCEPTION;
+    } finally {
+      dispatcher.shutdownNow();
+    }
+  }
+
+  DBlock newDBlock(Block block, List<MLocation> locations) {
+    final DBlock db = new DBlock(block);
+    for(MLocation ml : locations) {
+      db.addLocation(storages.getTarget(ml));
+    }
+    return db;
+  }
+
+  private static long getMaxRemaining(DatanodeStorageReport report, StorageType t) {
+    long max = 0L;
+    for(StorageReport r : report.getStorageReports()) {
+      if (r.getStorage().getStorageType() == t) {
+        if (r.getRemaining() > max) {
+          max = r.getRemaining();
+        }
+      }
+    }
+    return max;
+  }
+
+  /**
+   * convert a snapshot path to non-snapshot path. E.g.,
+   * /foo/.snapshot/snapshot-name/bar --> /foo/bar
+   */
+  private static String convertSnapshotPath(String[] pathComponents) {
+    StringBuilder sb = new StringBuilder(Path.SEPARATOR);
+    for (int i = 0; i < pathComponents.length; i++) {
+      if (pathComponents[i].equals(HdfsConstants.DOT_SNAPSHOT_DIR)) {
+        i++;
+      } else {
+        sb.append(pathComponents[i]);
+      }
+    }
+    return sb.toString();
+  }
+
+  class Processor {
+    private final DFSClient dfs;
+    private final List<String> snapshottableDirs = new ArrayList<String>();
+
+    Processor() {
+      dfs = dispatcher.getDistributedFileSystem().getClient();
+    }
+
+    private void getSnapshottableDirs() {
+      SnapshottableDirectoryStatus[] dirs = null;
+      try {
+        dirs = dfs.getSnapshottableDirListing();
+      } catch (IOException e) {
+        LOG.warn("Failed to get snapshottable directories."
+            + " Ignore and continue.", e);
+      }
+      if (dirs != null) {
+        for (SnapshottableDirectoryStatus dir : dirs) {
+          snapshottableDirs.add(dir.getFullPath().toString());
+        }
+      }
+    }
+
+    /**
+     * @return true if the given path is a snapshot path and the corresponding
+     * INode is still in the current fsdirectory.
+     */
+    private boolean isSnapshotPathInCurrent(String path) throws IOException {
+      // if the parent path contains "/.snapshot/", this is a snapshot path
+      if (path.contains(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR)) {
+        String[] pathComponents = INode.getPathNames(path);
+        if (HdfsConstants.DOT_SNAPSHOT_DIR
+            .equals(pathComponents[pathComponents.length - 2])) {
+          // this is a path for a specific snapshot (e.g., /foo/.snapshot/s1)
+          return false;
+        }
+        String nonSnapshotPath = convertSnapshotPath(pathComponents);
+        return dfs.getFileInfo(nonSnapshotPath) != null;
+      } else {
+        return false;
+      }
+    }
+
+    /**
+     * @return whether there is still remaining migration work for the next
+     *         round
+     */
+    private boolean processNamespace() {
+      getSnapshottableDirs();
+      boolean hasRemaining = true;
+      try {
+        for (Path target : targetPaths) {
+          hasRemaining = processDirRecursively("", dfs.getFileInfo(target
+              .toUri().getPath()));
+        }
+      } catch (IOException e) {
+        LOG.warn("Failed to get root directory status. Ignore and continue.", e);
+      }
+      // wait for pending move to finish and retry the failed migration
+      hasRemaining |= Dispatcher.waitForMoveCompletion(storages.targets.values());
+      return hasRemaining;
+    }
+
+    /**
+     * @return whether there is still remaing migration work for the next
+     *         round
+     */
+    private boolean processChildrenList(String fullPath) {
+      boolean hasRemaining = false;
+      for (byte[] lastReturnedName = HdfsFileStatus.EMPTY_NAME;;) {
+        final DirectoryListing children;
+        try {
+          children = dfs.listPaths(fullPath, lastReturnedName, true);
+        } catch(IOException e) {
+          LOG.warn("Failed to list directory " + fullPath
+              + ". Ignore the directory and continue.", e);
+          return hasRemaining;
+        }
+        if (children == null) {
+          return hasRemaining;
+        }
+        for (HdfsFileStatus child : children.getPartialListing()) {
+          hasRemaining |= processDirRecursively(fullPath, child);
+        }
+        if (children.hasMore()) {
+          lastReturnedName = children.getLastName();
+        } else {
+          return hasRemaining;
+        }
+      }
+    }
+
+    /** @return whether the migration requires next round */
+    private boolean processDirRecursively(String parent,
+                                          HdfsFileStatus status) {
+      String fullPath = status.getFullName(parent);
+      boolean hasRemaining = false;
+      if (status.isDir()) {
+        if (!fullPath.endsWith(Path.SEPARATOR)) {
+          fullPath = fullPath + Path.SEPARATOR;
+        }
+
+        hasRemaining = processChildrenList(fullPath);
+        // process snapshots if this is a snapshottable directory
+        if (snapshottableDirs.contains(fullPath)) {
+          final String dirSnapshot = fullPath + HdfsConstants.DOT_SNAPSHOT_DIR;
+          hasRemaining |= processChildrenList(dirSnapshot);
+        }
+      } else if (!status.isSymlink()) { // file
+        try {
+          if (!isSnapshotPathInCurrent(fullPath)) {
+            // the full path is a snapshot path but it is also included in the
+            // current directory tree, thus ignore it.
+            hasRemaining = processFile((HdfsLocatedFileStatus)status);
+          }
+        } catch (IOException e) {
+          LOG.warn("Failed to check the status of " + parent
+              + ". Ignore it and continue.", e);
+          return false;
+        }
+      }
+      return hasRemaining;
+    }
+
+    /** @return true if it is necessary to run another round of migration */
+    private boolean processFile(HdfsLocatedFileStatus status) {
+      final BlockStoragePolicy policy = blockStoragePolicies.getPolicy(
+          status.getStoragePolicy());
+      final List<StorageType> types = policy.chooseStorageTypes(
+          status.getReplication());
+
+      final LocatedBlocks locatedBlocks = status.getBlockLocations();
+      boolean hasRemaining = false;
+      final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete();
+      List<LocatedBlock> lbs = locatedBlocks.getLocatedBlocks();
+      for(int i = 0; i < lbs.size(); i++) {
+        if (i == lbs.size() - 1 && !lastBlkComplete) {
+          // last block is incomplete, skip it
+          continue;
+        }
+        LocatedBlock lb = lbs.get(i);
+        final StorageTypeDiff diff = new StorageTypeDiff(types,
+            lb.getStorageTypes());
+        if (!diff.removeOverlap()) {
+          if (scheduleMoves4Block(diff, lb)) {
+            hasRemaining |= (diff.existing.size() > 1 &&
+                diff.expected.size() > 1);
+          }
+        }
+      }
+      return hasRemaining;
+    }
+
+    boolean scheduleMoves4Block(StorageTypeDiff diff, LocatedBlock lb) {
+      final List<MLocation> locations = MLocation.toLocations(lb);
+      Collections.shuffle(locations);
+      final DBlock db = newDBlock(lb.getBlock().getLocalBlock(), locations);
+
+      for (final StorageType t : diff.existing) {
+        for (final MLocation ml : locations) {
+          final Source source = storages.getSource(ml);
+          if (ml.storageType == t) {
+            // try to schedule one replica move.
+            if (scheduleMoveReplica(db, source, diff.expected)) {
+              return true;
+            }
+          }
+        }
+      }
+      return false;
+    }
+
+    @VisibleForTesting
+    boolean scheduleMoveReplica(DBlock db, MLocation ml,
+                                List<StorageType> targetTypes) {
+      return scheduleMoveReplica(db, storages.getSource(ml), targetTypes);
+    }
+
+    boolean scheduleMoveReplica(DBlock db, Source source,
+        List<StorageType> targetTypes) {
+      if (dispatcher.getCluster().isNodeGroupAware()) {
+        if (chooseTarget(db, source, targetTypes, Matcher.SAME_NODE_GROUP)) {
+          return true;
+        }
+      }
+      
+      // Then, match nodes on the same rack
+      if (chooseTarget(db, source, targetTypes, Matcher.SAME_RACK)) {
+        return true;
+      }
+      // At last, match all remaining nodes
+      return chooseTarget(db, source, targetTypes, Matcher.ANY_OTHER);
+    }
+
+    boolean chooseTarget(DBlock db, Source source,
+        List<StorageType> targetTypes, Matcher matcher) {
+      final NetworkTopology cluster = dispatcher.getCluster(); 
+      for (StorageType t : targetTypes) {
+        for(StorageGroup target : storages.getTargetStorages(t)) {
+          if (matcher.match(cluster, source.getDatanodeInfo(),
+              target.getDatanodeInfo())) {
+            final PendingMove pm = source.addPendingMove(db, target);
+            if (pm != null) {
+              dispatcher.executePendingMove(pm);
+              return true;
+            }
+          }
+        }
+      }
+      return false;
+    }
+  }
+
+  static class MLocation {
+    final DatanodeInfo datanode;
+    final StorageType storageType;
+    final long size;
+    
+    MLocation(DatanodeInfo datanode, StorageType storageType, long size) {
+      this.datanode = datanode;
+      this.storageType = storageType;
+      this.size = size;
+    }
+    
+    static List<MLocation> toLocations(LocatedBlock lb) {
+      final DatanodeInfo[] datanodeInfos = lb.getLocations();
+      final StorageType[] storageTypes = lb.getStorageTypes();
+      final long size = lb.getBlockSize();
+      final List<MLocation> locations = new LinkedList<MLocation>();
+      for(int i = 0; i < datanodeInfos.length; i++) {
+        locations.add(new MLocation(datanodeInfos[i], storageTypes[i], size));
+      }
+      return locations;
+    }
+  }
+
+  @VisibleForTesting
+  static class StorageTypeDiff {
+    final List<StorageType> expected;
+    final List<StorageType> existing;
+
+    StorageTypeDiff(List<StorageType> expected, StorageType[] existing) {
+      this.expected = new LinkedList<StorageType>(expected);
+      this.existing = new LinkedList<StorageType>(Arrays.asList(existing));
+    }
+    
+    /**
+     * Remove the overlap between the expected types and the existing types.
+     * @return if the existing types or the expected types is empty after
+     *         removing the overlap.
+     */
+    boolean removeOverlap() { 
+      for(Iterator<StorageType> i = existing.iterator(); i.hasNext(); ) {
+        final StorageType t = i.next();
+        if (expected.remove(t)) {
+          i.remove();
+        }
+      }
+      return expected.isEmpty() || existing.isEmpty();
+    }
+    
+    @Override
+    public String toString() {
+      return getClass().getSimpleName() + "{expected=" + expected
+          + ", existing=" + existing + "}";
+    }
+  }
+
+  static int run(Map<URI, List<Path>> namenodes, Configuration conf)
+      throws IOException, InterruptedException {
+    final long sleeptime =
+        conf.getLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
+            DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT) * 2000 +
+        conf.getLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
+            DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000;
+    LOG.info("namenodes = " + namenodes);
+    
+    List<NameNodeConnector> connectors = Collections.emptyList();
+    try {
+      connectors = NameNodeConnector.newNameNodeConnectors(namenodes,
+            Mover.class.getSimpleName(), MOVER_ID_PATH, conf);
+
+      while (connectors.size() > 0) {
+        Collections.shuffle(connectors);
+        Iterator<NameNodeConnector> iter = connectors.iterator();
+        while (iter.hasNext()) {
+          NameNodeConnector nnc = iter.next();
+          final Mover m = new Mover(nnc, conf);
+          final ExitStatus r = m.run();
+
+          if (r == ExitStatus.SUCCESS) {
+            IOUtils.cleanup(LOG, nnc);
+            iter.remove();
+          } else if (r != ExitStatus.IN_PROGRESS) {
+            // must be an error statue, return
+            return r.getExitCode();
+          }
+        }
+        Thread.sleep(sleeptime);
+      }
+      return ExitStatus.SUCCESS.getExitCode();
+    } finally {
+      for (NameNodeConnector nnc : connectors) {
+        IOUtils.cleanup(LOG, nnc);
+      }
+    }
+  }
+
+  static class Cli extends Configured implements Tool {
+    private static final String USAGE = "Usage: java "
+        + Mover.class.getSimpleName() + " [-p <files/dirs> | -f <local file>]"
+        + "\n\t-p <files/dirs>\ta space separated list of HDFS files/dirs to migrate."
+        + "\n\t-f <local file>\ta local file containing a list of HDFS files/dirs to migrate.";
+
+    private static Options buildCliOptions() {
+      Options opts = new Options();
+      Option file = OptionBuilder.withArgName("pathsFile").hasArg()
+          .withDescription("a local file containing files/dirs to migrate")
+          .create("f");
+      Option paths = OptionBuilder.withArgName("paths").hasArgs()
+          .withDescription("specify space separated files/dirs to migrate")
+          .create("p");
+      OptionGroup group = new OptionGroup();
+      group.addOption(file);
+      group.addOption(paths);
+      opts.addOptionGroup(group);
+      return opts;
+    }
+
+    private static String[] readPathFile(String file) throws IOException {
+      List<String> list = Lists.newArrayList();
+      BufferedReader reader = new BufferedReader(new FileReader(file));
+      try {
+        String line;
+        while ((line = reader.readLine()) != null) {
+          if (!line.trim().isEmpty()) {
+            list.add(line);
+          }
+        }
+      } finally {
+        IOUtils.cleanup(LOG, reader);
+      }
+      return list.toArray(new String[list.size()]);
+    }
+
+    private static Map<URI, List<Path>> getNameNodePaths(CommandLine line,
+        Configuration conf) throws Exception {
+      Map<URI, List<Path>> map = Maps.newHashMap();
+      String[] paths = null;
+      if (line.hasOption("f")) {
+        paths = readPathFile(line.getOptionValue("f"));
+      } else if (line.hasOption("p")) {
+        paths = line.getOptionValues("p");
+      }
+      Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
+      if (paths == null || paths.length == 0) {
+        for (URI namenode : namenodes) {
+          map.put(namenode, null);
+        }
+        return map;
+      }
+      final URI singleNs = namenodes.size() == 1 ?
+          namenodes.iterator().next() : null;
+      for (String path : paths) {
+        Path target = new Path(path);
+        if (!target.isUriPathAbsolute()) {
+          throw new IllegalArgumentException("The path " + target
+              + " is not absolute");
+        }
+        URI targetUri = target.toUri();
+        if ((targetUri.getAuthority() == null || targetUri.getScheme() ==
+            null) && singleNs == null) {
+          // each path must contains both scheme and authority information
+          // unless there is only one name service specified in the
+          // configuration
+          throw new IllegalArgumentException("The path " + target
+              + " does not contain scheme and authority thus cannot identify"
+              + " its name service");
+        }
+        URI key = singleNs;
+        if (singleNs == null) {
+          key = new URI(targetUri.getScheme(), targetUri.getAuthority(),
+              null, null, null);
+          if (!namenodes.contains(key)) {
+            throw new IllegalArgumentException("Cannot resolve the path " +
+                target + ". The namenode services specified in the " +
+                "configuration: " + namenodes);
+          }
+        }
+        List<Path> targets = map.get(key);
+        if (targets == null) {
+          targets = Lists.newArrayList();
+          map.put(key, targets);
+        }
+        targets.add(Path.getPathWithoutSchemeAndAuthority(target));
+      }
+      return map;
+    }
+
+    @VisibleForTesting
+    static Map<URI, List<Path>> getNameNodePathsToMove(Configuration conf,
+        String... args) throws Exception {
+      final Options opts = buildCliOptions();
+      CommandLineParser parser = new GnuParser();
+      CommandLine commandLine = parser.parse(opts, args, true);
+      return getNameNodePaths(commandLine, conf);
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+      final long startTime = Time.monotonicNow();
+      final Configuration conf = getConf();
+
+      try {
+        final Map<URI, List<Path>> map = getNameNodePathsToMove(conf, args);
+        return Mover.run(map, conf);
+      } catch (IOException e) {
+        System.out.println(e + ".  Exiting ...");
+        return ExitStatus.IO_EXCEPTION.getExitCode();
+      } catch (InterruptedException e) {
+        System.out.println(e + ".  Exiting ...");
+        return ExitStatus.INTERRUPTED.getExitCode();
+      } catch (ParseException e) {
+        System.out.println(e + ".  Exiting ...");
+        return ExitStatus.ILLEGAL_ARGUMENTS.getExitCode();
+      } catch (IllegalArgumentException e) {
+        System.out.println(e + ".  Exiting ...");
+        return ExitStatus.ILLEGAL_ARGUMENTS.getExitCode();
+      } finally {
+        System.out.format("%-24s ", DateFormat.getDateTimeInstance().format(new Date()));
+        System.out.println("Mover took " + StringUtils.formatTime(Time.monotonicNow()-startTime));
+      }
+    }
+  }
+
+  /**
+   * Run a Mover in command line.
+   *
+   * @param args Command line arguments
+   */
+  public static void main(String[] args) {
+    if (DFSUtil.parseHelpArgument(args, Cli.USAGE, System.out, true)) {
+      System.exit(0);
+    }
+
+    try {
+      System.exit(ToolRunner.run(new HdfsConfiguration(), new Cli(), args));
+    } catch (Throwable e) {
+      LOG.error("Exiting " + Mover.class.getSimpleName()
+          + " due to an exception", e);
+      System.exit(-1);
+    }
+  }
+}

+ 93 - 29
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -52,6 +52,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -273,6 +274,14 @@ public class FSDirectory implements Closeable {
     skipQuotaCheck = true;
   }
 
+  private static INodeFile newINodeFile(long id, PermissionStatus permissions,
+      long mtime, long atime, short replication, long preferredBlockSize,
+      boolean isLazyPersist) {
+    return new INodeFile(id, null, permissions, mtime, atime,
+        BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize,
+        isLazyPersist, (byte) 0);
+  }
+
   /**
    * Add the given filename to the fs.
    * @throws FileAlreadyExistsException
@@ -288,9 +297,9 @@ public class FSDirectory implements Closeable {
       UnresolvedLinkException, SnapshotAccessControlException, AclException {
 
     long modTime = now();
-    INodeFile newNode = new INodeFile(namesystem.allocateNewInodeId(), null,
-        permissions, modTime, modTime, BlockInfo.EMPTY_ARRAY, replication,
-        preferredBlockSize, isLazyPersist);
+    INodeFile newNode = newINodeFile(namesystem.allocateNewInodeId(),
+        permissions, modTime, modTime, replication, preferredBlockSize,
+        isLazyPersist);
     newNode.toUnderConstruction(clientName, clientMachine);
 
     boolean added = false;
@@ -327,14 +336,13 @@ public class FSDirectory implements Closeable {
     final INodeFile newNode;
     assert hasWriteLock();
     if (underConstruction) {
-      newNode = new INodeFile(id, null, permissions, modificationTime,
-          modificationTime, BlockInfo.EMPTY_ARRAY, replication,
-          preferredBlockSize, isLazyPersist);
+      newNode = newINodeFile(id, permissions, modificationTime,
+          modificationTime, replication, preferredBlockSize, isLazyPersist);
       newNode.toUnderConstruction(clientName, clientMachine);
 
     } else {
-      newNode = new INodeFile(id, null, permissions, modificationTime, atime,
-          BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize, isLazyPersist);
+      newNode = newINodeFile(id, permissions, modificationTime, atime,
+          replication, preferredBlockSize, isLazyPersist);
     }
 
     try {
@@ -1001,6 +1009,44 @@ public class FSDirectory implements Closeable {
     return file.getBlocks();
   }
 
+  /** Set block storage policy for a directory */
+  void setStoragePolicy(String src, byte policyId)
+      throws IOException {
+    writeLock();
+    try {
+      unprotectedSetStoragePolicy(src, policyId);
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  void unprotectedSetStoragePolicy(String src, byte policyId)
+      throws IOException {
+    assert hasWriteLock();
+    final INodesInPath iip = getINodesInPath4Write(src, true);
+    final INode inode = iip.getLastINode();
+    if (inode == null) {
+      throw new FileNotFoundException("File/Directory does not exist: " + src);
+    }
+    final int snapshotId = iip.getLatestSnapshotId();
+    if (inode.isFile()) {
+      inode.asFile().setStoragePolicyID(policyId, snapshotId);
+    } else if (inode.isDirectory()) {
+      setDirStoragePolicy(inode.asDirectory(), policyId, snapshotId);  
+    } else {
+      throw new FileNotFoundException(src + " is not a file or directory");
+    }
+  }
+
+  private void setDirStoragePolicy(INodeDirectory inode, byte policyId,
+      int latestSnapshotId) throws IOException {
+    List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
+    XAttr xAttr = BlockStoragePolicy.buildXAttr(policyId);
+    List<XAttr> newXAttrs = setINodeXAttrs(existingXAttrs, Arrays.asList(xAttr),
+        EnumSet.of(XAttrSetFlag.CREATE, XAttrSetFlag.REPLACE));
+    XAttrStorage.updateINodeXAttrs(inode, newXAttrs, latestSnapshotId);
+  }
+
   /**
    * @param path the file path
    * @return the block size of the file. 
@@ -1332,6 +1378,11 @@ public class FSDirectory implements Closeable {
     }
   }
 
+  private byte getStoragePolicyID(byte inodePolicy, byte parentPolicy) {
+    return inodePolicy != BlockStoragePolicy.ID_UNSPECIFIED ? inodePolicy :
+        parentPolicy;
+  }
+
   /**
    * Get a partial listing of the indicated directory
    *
@@ -1346,7 +1397,8 @@ public class FSDirectory implements Closeable {
    * @return a partial listing starting after startAfter
    */
   DirectoryListing getListing(String src, byte[] startAfter,
-      boolean needLocation) throws UnresolvedLinkException, IOException {
+      boolean needLocation, boolean isSuperUser)
+      throws UnresolvedLinkException, IOException {
     String srcs = normalizePath(src);
     final boolean isRawPath = isReservedRawName(src);
 
@@ -1355,16 +1407,19 @@ public class FSDirectory implements Closeable {
       if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
         return getSnapshotsListing(srcs, startAfter);
       }
-      final INodesInPath inodesInPath = getLastINodeInPath(srcs, true);
+      final INodesInPath inodesInPath = getLastINodeInPath(srcs);
       final int snapshot = inodesInPath.getPathSnapshotId();
-      final INode targetNode = inodesInPath.getINode(0);
+      final INode targetNode = inodesInPath.getLastINode();
       if (targetNode == null)
         return null;
+      byte parentStoragePolicy = isSuperUser ?
+          targetNode.getStoragePolicyID() : BlockStoragePolicy.ID_UNSPECIFIED;
       
       if (!targetNode.isDirectory()) {
         return new DirectoryListing(
             new HdfsFileStatus[]{createFileStatus(HdfsFileStatus.EMPTY_NAME,
-                targetNode, needLocation, snapshot, isRawPath)}, 0);
+                targetNode, needLocation, parentStoragePolicy, snapshot,
+                isRawPath)}, 0);
       }
 
       final INodeDirectory dirInode = targetNode.asDirectory();
@@ -1377,8 +1432,11 @@ public class FSDirectory implements Closeable {
       HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
       for (int i=0; i<numOfListing && locationBudget>0; i++) {
         INode cur = contents.get(startChild+i);
-        listing[i] = createFileStatus(cur.getLocalNameBytes(), cur,
-            needLocation, snapshot, isRawPath);
+        byte curPolicy = isSuperUser && !cur.isSymlink()?
+            cur.getLocalStoragePolicyID(): BlockStoragePolicy.ID_UNSPECIFIED;
+        listing[i] = createFileStatus(cur.getLocalNameBytes(), cur, needLocation,
+            getStoragePolicyID(curPolicy, parentStoragePolicy), snapshot,
+            isRawPath);
         listingCnt++;
         if (needLocation) {
             // Once we  hit lsLimit locations, stop.
@@ -1429,7 +1487,7 @@ public class FSDirectory implements Closeable {
     for (int i = 0; i < numOfListing; i++) {
       Root sRoot = snapshots.get(i + skipSize).getRoot();
       listing[i] = createFileStatus(sRoot.getLocalNameBytes(), sRoot,
-          Snapshot.CURRENT_STATE_ID, false);
+          BlockStoragePolicy.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID, false);
     }
     return new DirectoryListing(
         listing, snapshots.size() - skipSize - numOfListing);
@@ -1439,10 +1497,12 @@ public class FSDirectory implements Closeable {
    * @param src The string representation of the path to the file
    * @param resolveLink whether to throw UnresolvedLinkException
    * @param isRawPath true if a /.reserved/raw pathname was passed by the user
+   * @param includeStoragePolicy whether to include storage policy
    * @return object containing information regarding the file
    *         or null if file not found
    */
-  HdfsFileStatus getFileInfo(String src, boolean resolveLink, boolean isRawPath)
+  HdfsFileStatus getFileInfo(String src, boolean resolveLink,
+      boolean isRawPath, boolean includeStoragePolicy)
     throws IOException {
     String srcs = normalizePath(src);
     readLock();
@@ -1452,9 +1512,10 @@ public class FSDirectory implements Closeable {
       }
       final INodesInPath inodesInPath = getLastINodeInPath(srcs, resolveLink);
       final INode i = inodesInPath.getINode(0);
-
-      return i == null? null: createFileStatus(HdfsFileStatus.EMPTY_NAME, i,
-          inodesInPath.getPathSnapshotId(), isRawPath);
+      byte policyId = includeStoragePolicy && i != null && !i.isSymlink() ?
+          i.getStoragePolicyID() : BlockStoragePolicy.ID_UNSPECIFIED;
+      return i == null ? null : createFileStatus(HdfsFileStatus.EMPTY_NAME, i,
+          policyId, inodesInPath.getPathSnapshotId(), isRawPath);
     } finally {
       readUnlock();
     }
@@ -1471,7 +1532,8 @@ public class FSDirectory implements Closeable {
       throws UnresolvedLinkException {
     if (getINode4DotSnapshot(src) != null) {
       return new HdfsFileStatus(0, true, 0, 0, false, 0, 0, null, null, null, null,
-          HdfsFileStatus.EMPTY_NAME, -1L, 0, null);
+          HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
+          BlockStoragePolicy.ID_UNSPECIFIED);
     }
     return null;
   }
@@ -2299,19 +2361,20 @@ public class FSDirectory implements Closeable {
    * @throws IOException if any error occurs
    */
   private HdfsFileStatus createFileStatus(byte[] path, INode node,
-      boolean needLocation, int snapshot, boolean isRawPath)
+      boolean needLocation, byte storagePolicy, int snapshot, boolean isRawPath)
       throws IOException {
     if (needLocation) {
-      return createLocatedFileStatus(path, node, snapshot, isRawPath);
+      return createLocatedFileStatus(path, node, storagePolicy, snapshot, isRawPath);
     } else {
-      return createFileStatus(path, node, snapshot, isRawPath);
+      return createFileStatus(path, node, storagePolicy, snapshot, isRawPath);
     }
   }
+
   /**
    * Create FileStatus by file INode 
    */
-   HdfsFileStatus createFileStatus(byte[] path, INode node,
-       int snapshot, boolean isRawPath) throws IOException {
+  HdfsFileStatus createFileStatus(byte[] path, INode node, byte storagePolicy,
+      int snapshot, boolean isRawPath) throws IOException {
      long size = 0;     // length is zero for directories
      short replication = 0;
      long blocksize = 0;
@@ -2351,14 +2414,15 @@ public class FSDirectory implements Closeable {
         path,
         node.getId(),
         childrenNum,
-        feInfo);
+        feInfo,
+        storagePolicy);
   }
 
   /**
    * Create FileStatus with location info by file INode
    */
-  private HdfsLocatedFileStatus createLocatedFileStatus(byte[] path,
-      INode node, int snapshot, boolean isRawPath) throws IOException {
+  private HdfsLocatedFileStatus createLocatedFileStatus(byte[] path, INode node,
+      byte storagePolicy, int snapshot, boolean isRawPath) throws IOException {
     assert hasReadLock();
     long size = 0; // length is zero for directories
     short replication = 0;
@@ -2400,7 +2464,7 @@ public class FSDirectory implements Closeable {
           getPermissionForFileStatus(node, snapshot, isEncrypted),
           node.getUserName(snapshot), node.getGroupName(snapshot),
           node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
-          node.getId(), loc, childrenNum, feInfo);
+          node.getId(), loc, childrenNum, feInfo, storagePolicy);
     // Set caching information for the located blocks.
     if (loc != null) {
       CacheManager cacheManager = namesystem.getCacheManager();

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

@@ -82,6 +82,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetStoragePolicyOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetXAttrOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
@@ -831,7 +832,16 @@ public class FSEditLog implements LogsPurgeable {
       .setReplication(replication);
     logEdit(op);
   }
-  
+
+  /** 
+   * Add set storage policy id record to edit log
+   */
+  void logSetStoragePolicy(String src, byte policyId) {
+    SetStoragePolicyOp op = SetStoragePolicyOp.getInstance(cache.get())
+        .setPath(src).setPolicyId(policyId);
+    logEdit(op);
+  }
+
   /** Add set namespace quota record to edit log
    * 
    * @param src the string representation of the path to a directory

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

@@ -33,6 +33,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -78,6 +79,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetStoragePolicyOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetXAttrOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveXAttrOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
@@ -369,7 +371,8 @@ public class FSEditLogLoader {
         // add the op into retry cache if necessary
         if (toAddRetryCache) {
           HdfsFileStatus stat = fsNamesys.dir.createFileStatus(
-              HdfsFileStatus.EMPTY_NAME, newFile, Snapshot.CURRENT_STATE_ID,
+              HdfsFileStatus.EMPTY_NAME, newFile,
+              BlockStoragePolicy.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
               false);
           fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
               addCloseOp.rpcCallId, stat);
@@ -833,6 +836,13 @@ public class FSEditLogLoader {
       }
       break;
     }
+    case OP_SET_STORAGE_POLICY: {
+      SetStoragePolicyOp setStoragePolicyOp = (SetStoragePolicyOp) op;
+      fsDir.unprotectedSetStoragePolicy(
+          renameReservedPathsOnUpgrade(setStoragePolicyOp.path, logVersion),
+          setStoragePolicyOp.policyId);
+      break;
+    }
     default:
       throw new IOException("Invalid operation read " + op.opCode);
     }

+ 67 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -61,6 +61,7 @@ import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SYMLINK
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_TIMES;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_UPDATE_BLOCKS;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_UPDATE_MASTER_KEY;
+import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_STORAGE_POLICY;
 
 import java.io.DataInput;
 import java.io.DataInputStream;
@@ -193,6 +194,7 @@ public abstract class FSEditLogOp {
           OP_ROLLING_UPGRADE_FINALIZE, "finalize"));
       inst.put(OP_SET_XATTR, new SetXAttrOp());
       inst.put(OP_REMOVE_XATTR, new RemoveXAttrOp());
+      inst.put(OP_SET_STORAGE_POLICY, new SetStoragePolicyOp());
     }
     
     public FSEditLogOp get(FSEditLogOpCodes opcode) {
@@ -3821,6 +3823,71 @@ public abstract class FSEditLogOp {
     }
   }
 
+  /** {@literal @Idempotent} for {@link ClientProtocol#setStoragePolicy} */
+  static class SetStoragePolicyOp extends FSEditLogOp {
+    String path;
+    byte policyId;
+
+    private SetStoragePolicyOp() {
+      super(OP_SET_STORAGE_POLICY);
+    }
+
+    static SetStoragePolicyOp getInstance(OpInstanceCache cache) {
+      return (SetStoragePolicyOp) cache.get(OP_SET_STORAGE_POLICY);
+    }
+
+    SetStoragePolicyOp setPath(String path) {
+      this.path = path;
+      return this;
+    }
+
+    SetStoragePolicyOp setPolicyId(byte policyId) {
+      this.policyId = policyId;
+      return this;
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeString(path, out);
+      out.writeByte(policyId);
+    }
+
+    @Override
+    void readFields(DataInputStream in, int logVersion)
+        throws IOException {
+      this.path = FSImageSerialization.readString(in);
+      this.policyId = in.readByte();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("SetStoragePolicyOp [path=");
+      builder.append(path);
+      builder.append(", policyId=");
+      builder.append(policyId);
+      builder.append(", opCode=");
+      builder.append(opCode);
+      builder.append(", txid=");
+      builder.append(txid);
+      builder.append("]");
+      return builder.toString();
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "PATH", path);
+      XMLUtils.addSaxString(contentHandler, "POLICYID",
+          Byte.valueOf(policyId).toString());
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      this.path = st.getValue("PATH");
+      this.policyId = Byte.valueOf(st.getValue("POLICYID"));
+    }
+  }  
+
   /**
    * Class for writing editlog ops
    */

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

@@ -72,6 +72,7 @@ public enum FSEditLogOpCodes {
   OP_ROLLING_UPGRADE_FINALIZE   ((byte) 42),
   OP_SET_XATTR                  ((byte) 43),
   OP_REMOVE_XATTR               ((byte) 44),
+  OP_SET_STORAGE_POLICY         ((byte) 45),
 
   // Note that the current range of the valid OP code is 0~127
   OP_INVALID                    ((byte) -1);

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

@@ -890,7 +890,7 @@ public class FSImageFormat {
       // LazyPersist flag will not be present in old image formats and hence
       // can be safely set to false always.
       return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime,
-          accessTime, replication, preferredBlockSize, false, null);
+          accessTime, replication, preferredBlockSize, false, (byte) 0, null);
     }
 
     public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in)

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

@@ -291,7 +291,8 @@ public final class FSImageFormatPBINode {
       final INodeFile file = new INodeFile(n.getId(),
           n.getName().toByteArray(), permissions, f.getModificationTime(),
           f.getAccessTime(), blocks, replication, f.getPreferredBlockSize(),
-          f.hasIsLazyPersist() ? f.getIsLazyPersist() : false);
+          f.hasIsLazyPersist() ? f.getIsLazyPersist() : false,
+          (byte)f.getStoragePolicyID());
 
       if (f.hasAcl()) {
         file.addAclFeature(new AclFeature(loadAclEntries(f.getAcl(),
@@ -403,7 +404,8 @@ public final class FSImageFormatPBINode {
           .setPermission(buildPermissionStatus(file, state.getStringMap()))
           .setPreferredBlockSize(file.getPreferredBlockSize())
           .setReplication(file.getFileReplication())
-          .setIsLazyPersist(file.getLazyPersistFlag());
+          .setIsLazyPersist(file.getLazyPersistFlag())
+          .setStoragePolicyID(file.getLocalStoragePolicyID());
 
       AclFeature f = file.getAclFeature();
       if (f != null) {

+ 124 - 101
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -67,8 +67,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FIL
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RANDOMIZE_BLOCK_LOCATIONS_PER_BLOCK;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RANDOMIZE_BLOCK_LOCATIONS_PER_BLOCK_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REPL_QUEUE_THRESHOLD_PCT_KEY;
@@ -164,6 +162,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -338,7 +337,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private HdfsFileStatus getAuditFileInfo(String path, boolean resolveSymlink)
       throws IOException {
     return (isAuditEnabled() && isExternalInvocation())
-        ? dir.getFileInfo(path, resolveSymlink, false) : null;
+        ? dir.getFileInfo(path, resolveSymlink, false, false) : null;
   }
   
   private void logAuditEvent(boolean succeeded, String cmd, String src)
@@ -559,8 +558,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   private final FSImage fsImage;
 
-  private boolean randomizeBlockLocationsPerBlock;
-
   /**
    * Notify that loading of this FSDirectory is complete, and
    * it is imageLoaded for use
@@ -886,10 +883,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY,
           DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT);
       
-      this.randomizeBlockLocationsPerBlock = conf.getBoolean(
-          DFS_NAMENODE_RANDOMIZE_BLOCK_LOCATIONS_PER_BLOCK,
-          DFS_NAMENODE_RANDOMIZE_BLOCK_LOCATIONS_PER_BLOCK_DEFAULT);
-
       this.dtSecretManager = createDelegationTokenSecretManager(conf);
       this.dir = new FSDirectory(this, conf);
       this.snapshotManager = new SnapshotManager(dir);
@@ -1770,7 +1763,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         true);
     if (blocks != null) {
       blockManager.getDatanodeManager().sortLocatedBlocks(clientMachine,
-          blocks.getLocatedBlocks(), randomizeBlockLocationsPerBlock);
+          blocks.getLocatedBlocks());
 
       // lastBlock is not part of getLocatedBlocks(), might need to sort it too
       LocatedBlock lastBlock = blocks.getLastLocatedBlock();
@@ -1779,7 +1772,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
             Lists.newArrayListWithCapacity(1);
         lastBlockList.add(lastBlock);
         blockManager.getDatanodeManager().sortLocatedBlocks(clientMachine,
-            lastBlockList, randomizeBlockLocationsPerBlock);
+            lastBlockList);
       }
     }
     return blocks;
@@ -2292,6 +2285,52 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return isFile;
   }
 
+  /**
+   * Set the storage policy for a file or a directory.
+   *
+   * @param src file/directory path
+   * @param policyName storage policy name
+   */
+  void setStoragePolicy(String src, final String policyName)
+      throws IOException {
+    try {
+      setStoragePolicyInt(src, policyName);
+    } catch (AccessControlException e) {
+      logAuditEvent(false, "setStoragePolicy", src);
+      throw e;
+    }
+  }
+
+  private void setStoragePolicyInt(String src, final String policyName)
+      throws IOException {
+    checkSuperuserPrivilege();
+    checkOperation(OperationCategory.WRITE);
+    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    waitForLoadingFSImage();
+    HdfsFileStatus fileStat;
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      checkNameNodeSafeMode("Cannot set storage policy for " + src);
+      src = FSDirectory.resolvePath(src, pathComponents, dir);
+
+      // get the corresponding policy and make sure the policy name is valid
+      BlockStoragePolicy policy = blockManager.getStoragePolicy(policyName);
+      if (policy == null) {
+        throw new HadoopIllegalArgumentException(
+            "Cannot find a block policy with the name " + policyName);
+      }
+      dir.setStoragePolicy(src, policy.getId());
+      getEditLog().logSetStoragePolicy(src, policy.getId());
+      fileStat = getAuditFileInfo(src, false);
+    } finally {
+      writeUnlock();
+    }
+
+    getEditLog().logSync();
+    logAuditEvent(true, "setStoragePolicy", src, null, fileStat);
+  }
+
   long getPreferredBlockSize(String filename) 
       throws IOException, UnresolvedLinkException {
     FSPermissionChecker pc = getPermissionChecker();
@@ -2476,84 +2515,66 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
     waitForLoadingFSImage();
 
-    /*
-     * We want to avoid holding any locks while doing KeyProvider operations,
-     * since they can be very slow. Since the path can
-     * flip flop between being in an encryption zone and not in the meantime,
-     * we need to recheck the preconditions and redo KeyProvider operations
-     * in some situations.
-     *
-     * A special RetryStartFileException is used to indicate that we should
-     * retry creation of a FileEncryptionInfo.
+    /**
+     * If the file is in an encryption zone, we optimistically create an
+     * EDEK for the file by calling out to the configured KeyProvider.
+     * Since this typically involves doing an RPC, we take the readLock
+     * initially, then drop it to do the RPC.
+     * 
+     * Since the path can flip-flop between being in an encryption zone and not
+     * in the meantime, we need to recheck the preconditions when we retake the
+     * lock to do the create. If the preconditions are not met, we throw a
+     * special RetryStartFileException to ask the DFSClient to try the create
+     * again later.
      */
-    BlocksMapUpdateInfo toRemoveBlocks = null;
+    CipherSuite suite = null;
+    String ezKeyName = null;
+    readLock();
     try {
-      boolean shouldContinue = true;
-      int iters = 0;
-      while (shouldContinue) {
-        skipSync = false;
-        if (iters >= 10) {
-          throw new IOException("Too many retries because of encryption zone " +
-              "operations, something might be broken!");
-        }
-        shouldContinue = false;
-        iters++;
-
-        // Optimistically determine CipherSuite and ezKeyName if the path is
-        // currently within an encryption zone
-        CipherSuite suite = null;
-        String ezKeyName = null;
-        readLock();
-        try {
-          src = resolvePath(src, pathComponents);
-          INodesInPath iip = dir.getINodesInPath4Write(src);
-          // Nothing to do if the path is not within an EZ
-          if (dir.isInAnEZ(iip)) {
-            suite = chooseCipherSuite(iip, cipherSuites);
-            if (suite != null) {
-              Preconditions.checkArgument(!suite.equals(CipherSuite.UNKNOWN),
-                  "Chose an UNKNOWN CipherSuite!");
-            }
-            ezKeyName = dir.getKeyName(iip);
-            Preconditions.checkState(ezKeyName != null);
-          }
-        } finally {
-          readUnlock();
+      src = resolvePath(src, pathComponents);
+      INodesInPath iip = dir.getINodesInPath4Write(src);
+      // Nothing to do if the path is not within an EZ
+      if (dir.isInAnEZ(iip)) {
+        suite = chooseCipherSuite(iip, cipherSuites);
+        if (suite != null) {
+          Preconditions.checkArgument(!suite.equals(CipherSuite.UNKNOWN),
+              "Chose an UNKNOWN CipherSuite!");
         }
+        ezKeyName = dir.getKeyName(iip);
+        Preconditions.checkState(ezKeyName != null);
+      }
+    } finally {
+      readUnlock();
+    }
 
-        Preconditions.checkState(
-            (suite == null && ezKeyName == null) ||
+    Preconditions.checkState(
+        (suite == null && ezKeyName == null) ||
             (suite != null && ezKeyName != null),
-            "Both suite and ezKeyName should both be null or not null");
-        // Generate EDEK if necessary while not holding the lock
-        EncryptedKeyVersion edek =
-            generateEncryptedDataEncryptionKey(ezKeyName);
-        EncryptionFaultInjector.getInstance().startFileAfterGenerateKey();
-        // Try to create the file with the computed cipher suite and EDEK
-        writeLock();
-        try {
-          checkOperation(OperationCategory.WRITE);
-          checkNameNodeSafeMode("Cannot create file" + src);
-          src = resolvePath(src, pathComponents);
-          toRemoveBlocks = startFileInternal(pc, src, permissions, holder, 
-              clientMachine, create, overwrite, createParent, replication, 
-              blockSize, isLazyPersist, suite, edek, logRetryCache);
-          stat = dir.getFileInfo(src, false,
-              FSDirectory.isReservedRawName(srcArg));
-        } catch (StandbyException se) {
-          skipSync = true;
-          throw se;
-        } catch (RetryStartFileException e) {
-          shouldContinue = true;
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("Preconditions failed, retrying creation of " +
-                    "FileEncryptionInfo", e);
-          }
-        } finally {
-          writeUnlock();
-        }
-      }
+        "Both suite and ezKeyName should both be null or not null");
+
+    // Generate EDEK if necessary while not holding the lock
+    EncryptedKeyVersion edek =
+        generateEncryptedDataEncryptionKey(ezKeyName);
+    EncryptionFaultInjector.getInstance().startFileAfterGenerateKey();
+
+    // Proceed with the create, using the computed cipher suite and 
+    // generated EDEK
+    BlocksMapUpdateInfo toRemoveBlocks = null;
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      checkNameNodeSafeMode("Cannot create file" + src);
+      src = resolvePath(src, pathComponents);
+      toRemoveBlocks = startFileInternal(pc, src, permissions, holder, 
+          clientMachine, create, overwrite, createParent, replication, 
+          blockSize, isLazyPersist, suite, edek, logRetryCache);
+      stat = dir.getFileInfo(src, false,
+          FSDirectory.isReservedRawName(srcArg), false);
+    } catch (StandbyException se) {
+      skipSync = true;
+      throw se;
     } finally {
+      writeUnlock();
       // There might be transactions logged while trying to recover the lease.
       // They need to be sync'ed even when an exception was thrown.
       if (!skipSync) {
@@ -3020,8 +3041,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throws LeaseExpiredException, NotReplicatedYetException,
       QuotaExceededException, SafeModeException, UnresolvedLinkException,
       IOException {
-    long blockSize;
-    int replication;
+    final long blockSize;
+    final int replication;
+    final byte storagePolicyID;
     DatanodeDescriptor clientNode = null;
 
     if(NameNode.stateChangeLog.isDebugEnabled()) {
@@ -3056,13 +3078,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       clientNode = blockManager.getDatanodeManager().getDatanodeByHost(
               pendingFile.getFileUnderConstructionFeature().getClientMachine());
       replication = pendingFile.getFileReplication();
+      storagePolicyID = pendingFile.getStoragePolicyID();
     } finally {
       readUnlock();
     }
 
     // choose targets for the new block to be allocated.
-    final DatanodeStorageInfo targets[] = getBlockManager().chooseTarget( 
-        src, replication, clientNode, excludedNodes, blockSize, favoredNodes);
+    final DatanodeStorageInfo targets[] = getBlockManager().chooseTarget4NewBlock( 
+        src, replication, clientNode, excludedNodes, blockSize, favoredNodes,
+        storagePolicyID);
 
     // Part II.
     // Allocate a new block, add it to the INode and the BlocksMap. 
@@ -3250,6 +3274,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
     final DatanodeDescriptor clientnode;
     final long preferredblocksize;
+    final byte storagePolicyID;
     final List<DatanodeStorageInfo> chosen;
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
@@ -3276,6 +3301,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
               .getClientMachine();
       clientnode = blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
       preferredblocksize = file.getPreferredBlockSize();
+      storagePolicyID = file.getStoragePolicyID();
 
       //find datanode storages
       final DatanodeManager dm = blockManager.getDatanodeManager();
@@ -3285,10 +3311,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
 
     // choose new datanodes.
-    final DatanodeStorageInfo[] targets = blockManager.getBlockPlacementPolicy(
-        ).chooseTarget(src, numAdditionalNodes, clientnode, chosen, true,
-            // TODO: get storage type from the file
-        excludes, preferredblocksize, StorageType.DEFAULT);
+    final DatanodeStorageInfo[] targets = blockManager.chooseTarget4AdditionalDatanode(
+        src, numAdditionalNodes, clientnode, chosen, 
+        excludes, preferredblocksize, storagePolicyID);
     final LocatedBlock lb = new LocatedBlock(blk, targets);
     blockManager.setBlockToken(lb, AccessMode.COPY);
     return lb;
@@ -3975,12 +4000,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       checkOperation(OperationCategory.READ);
       src = resolvePath(src, pathComponents);
+      boolean isSuperUser = true;
       if (isPermissionEnabled) {
         checkPermission(pc, src, false, null, null, null, null, false,
             resolveLink);
+        isSuperUser = pc.isSuperUser();
       }
       stat = dir.getFileInfo(src, resolveLink,
-          FSDirectory.isReservedRawName(srcArg));
+          FSDirectory.isReservedRawName(srcArg), isSuperUser);
     } catch (AccessControlException e) {
       logAuditEvent(false, "getfileinfo", srcArg);
       throw e;
@@ -4209,7 +4236,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /**
    * Get the content summary for a specific file/dir.
    *
-   * @param src The string representation of the path to the file
+   * @param srcArg The string representation of the path to the file
    *
    * @throws AccessControlException if access is denied
    * @throws UnresolvedLinkException if a symlink is encountered.
@@ -4785,16 +4812,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
               "Can't find startAfter " + startAfterString);
         }
       }
-      
+
+      boolean isSuperUser = true;
       if (isPermissionEnabled) {
         if (dir.isDir(src)) {
           checkPathAccess(pc, src, FsAction.READ_EXECUTE);
         } else {
           checkTraverse(pc, src);
         }
+        isSuperUser = pc.isSuperUser();
       }
       logAuditEvent(true, "listStatus", srcArg);
-      dl = dir.getListing(src, startAfter, needLocation);
+      dl = dir.getListing(src, startAfter, needLocation, isSuperUser);
     } finally {
       readUnlock();
     }
@@ -4944,12 +4973,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /**
    * Add the given symbolic link to the fs. Record it in the edits log.
-   * @param path
-   * @param target
-   * @param dirPerms
-   * @param createParent
-   * @param logRetryCache
-   * @param dir
    */
   private INodeSymlink addSymlink(String path, String target,
                                   PermissionStatus dirPerms,

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

@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
@@ -684,6 +685,20 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     return this;
   }
 
+  /**
+   * @return the latest block storage policy id of the INode. Specifically,
+   * if a storage policy is directly specified on the INode then return the ID
+   * of that policy. Otherwise follow the latest parental path and return the
+   * ID of the first specified storage policy.
+   */
+  public abstract byte getStoragePolicyID();
+
+  /**
+   * @return the storage policy directly specified on the INode. Return
+   * {@link BlockStoragePolicy#ID_UNSPECIFIED} if no policy has
+   * been specified.
+   */
+  public abstract byte getLocalStoragePolicyID();
 
   /**
    * Breaks {@code path} into components.
@@ -711,7 +726,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * @throws AssertionError if the given path is invalid.
    * @return array of path components.
    */
-  static String[] getPathNames(String path) {
+  public static String[] getPathNames(String path) {
     if (path == null || !path.startsWith(Path.SEPARATOR)) {
       throw new AssertionError("Absolute path required");
     }

+ 27 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -26,7 +26,9 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
+import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
@@ -40,6 +42,7 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
 
 /**
  * Directory INode class.
@@ -103,6 +106,30 @@ public class INodeDirectory extends INodeWithAdditionalFields
     return this;
   }
 
+  @Override
+  public byte getLocalStoragePolicyID() {
+    XAttrFeature f = getXAttrFeature();
+    ImmutableList<XAttr> xattrs = f == null ? ImmutableList.<XAttr> of() : f
+        .getXAttrs();
+    for (XAttr xattr : xattrs) {
+      if (BlockStoragePolicy.isStoragePolicyXAttr(xattr)) {
+        return (xattr.getValue())[0];
+      }
+    }
+    return BlockStoragePolicy.ID_UNSPECIFIED;
+  }
+
+  @Override
+  public byte getStoragePolicyID() {
+    byte id = getLocalStoragePolicyID();
+    if (id != BlockStoragePolicy.ID_UNSPECIFIED) {
+      return id;
+    }
+    // if it is unspecified, check its parent
+    return getParent() != null ? getParent().getStoragePolicyID() :
+        BlockStoragePolicy.ID_UNSPECIFIED;
+  }
+
   void setQuota(long nsQuota, long dsQuota) {
     DirectoryWithQuotaFeature quota = getDirectoryWithQuotaFeature();
     if (quota != null) {

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

@@ -18,10 +18,12 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.server.namenode.XAttrFeature;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
 
 /**
  * The attributes of an inode.

+ 45 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -28,6 +28,7 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
@@ -71,12 +72,15 @@ public class INodeFile extends INodeWithAdditionalFields
     return inode.asFile();
   }
 
-  /** Format: [16 bits for replication][48 bits for PreferredBlockSize] */
+  /** 
+   * Bit format:
+   * [4-bit storagePolicyID][12-bit replication][48-bit preferredBlockSize]
+   */
   static enum HeaderFormat {
     PREFERRED_BLOCK_SIZE(null, 48, 1),
-    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 1),
-    LAZY_PERSIST(REPLICATION.BITS, 4, 0);
-
+    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 11, 1),
+    STORAGE_POLICY_ID(REPLICATION.BITS, BlockStoragePolicy.ID_BIT_LENGTH, 0),
+    LAZY_PERSIST(STORAGE_POLICY_ID.BITS, 1, 0);
 
     private final LongBitFormat BITS;
 
@@ -96,10 +100,16 @@ public class INodeFile extends INodeWithAdditionalFields
       return LAZY_PERSIST.BITS.retrieve(header) == 0 ? false : true;
     }
 
-    static long toLong(long preferredBlockSize, short replication, boolean isLazyPersist) {
+    static byte getStoragePolicyID(long header) {
+      return (byte)STORAGE_POLICY_ID.BITS.retrieve(header);
+    }
+
+    static long toLong(long preferredBlockSize, short replication,
+        boolean isLazyPersist, byte storagePolicyID) {
       long h = 0;
       h = PREFERRED_BLOCK_SIZE.BITS.combine(preferredBlockSize, h);
       h = REPLICATION.BITS.combine(replication, h);
+      h = STORAGE_POLICY_ID.BITS.combine(storagePolicyID, h);
       h = LAZY_PERSIST.BITS.combine(isLazyPersist ? 1 : 0, h);
       return h;
     }
@@ -114,14 +124,15 @@ public class INodeFile extends INodeWithAdditionalFields
             long atime, BlockInfo[] blklist, short replication,
             long preferredBlockSize) {
     this(id, name, permissions, mtime, atime, blklist, replication,
-         preferredBlockSize, false);
+         preferredBlockSize, false, (byte) 0);
   }
 
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
       long atime, BlockInfo[] blklist, short replication,
-      long preferredBlockSize, boolean isLazyPersist) {
+      long preferredBlockSize, boolean isLazyPersist, byte storagePolicyID) {
     super(id, name, permissions, mtime, atime);
-    header = HeaderFormat.toLong(preferredBlockSize, replication, isLazyPersist);
+    header = HeaderFormat.toLong(preferredBlockSize, replication,
+        isLazyPersist, storagePolicyID);
     this.blocks = blklist;
   }
   
@@ -374,6 +385,32 @@ public class INodeFile extends INodeWithAdditionalFields
     return HeaderFormat.getLazyPersistFlag(header);
   }
 
+  @Override
+  public byte getLocalStoragePolicyID() {
+    return HeaderFormat.getStoragePolicyID(header);
+  }
+
+  @Override
+  public byte getStoragePolicyID() {
+    byte id = getLocalStoragePolicyID();
+    if (id == BlockStoragePolicy.ID_UNSPECIFIED) {
+      return this.getParent() != null ?
+          this.getParent().getStoragePolicyID() : id;
+    }
+    return id;
+  }
+
+  private void setStoragePolicyID(byte storagePolicyId) {
+    header = HeaderFormat.STORAGE_POLICY_ID.BITS.combine(storagePolicyId,
+        header);
+  }
+
+  public final void setStoragePolicyID(byte storagePolicyId,
+      int latestSnapshotId) throws QuotaExceededException {
+    recordModification(latestSnapshotId);
+    setStoragePolicyID(storagePolicyId);
+  }
+
   @Override
   public long getHeaderLong() {
     return header;

+ 11 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java

@@ -39,6 +39,8 @@ public interface INodeFileAttributes extends INodeAttributes {
 
   public boolean metadataEquals(INodeFileAttributes other);
 
+  public byte getLocalStoragePolicyID();
+
   /** A copy of the inode file attributes */
   public static class SnapshotCopy extends INodeAttributes.SnapshotCopy
       implements INodeFileAttributes {
@@ -46,11 +48,12 @@ public interface INodeFileAttributes extends INodeAttributes {
 
     public SnapshotCopy(byte[] name, PermissionStatus permissions,
         AclFeature aclFeature, long modificationTime, long accessTime,
-        short replication, long preferredBlockSize,
-        boolean isTransient, XAttrFeature xAttrsFeature) {
+        short replication, long preferredBlockSize, boolean isLazyPersist,
+        byte storagePolicyID, XAttrFeature xAttrsFeature) {
       super(name, permissions, aclFeature, modificationTime, accessTime, 
           xAttrsFeature);
-      header = HeaderFormat.toLong(preferredBlockSize, replication, isTransient);
+      header = HeaderFormat.toLong(preferredBlockSize, replication,
+          isLazyPersist, storagePolicyID);
     }
 
     public SnapshotCopy(INodeFile file) {
@@ -71,6 +74,11 @@ public interface INodeFileAttributes extends INodeAttributes {
     @Override
     public boolean getLazyPersistFlag() { return HeaderFormat.getLazyPersistFlag(header); }
 
+    @Override
+    public byte getLocalStoragePolicyID() {
+      return HeaderFormat.getStoragePolicyID(header);
+    }
+
     @Override
     public long getHeaderLong() {
       return header;

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java

@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.Quota.Counts;
 import org.apache.hadoop.util.GSet;
@@ -121,6 +122,16 @@ public class INodeMap {
           boolean countDiffChange) throws QuotaExceededException {
         return null;
       }
+
+      @Override
+      public byte getStoragePolicyID(){
+        return BlockStoragePolicy.ID_UNSPECIFIED;
+      }
+
+      @Override
+      public byte getLocalStoragePolicyID() {
+        return BlockStoragePolicy.ID_UNSPECIFIED;
+      }
     };
       
     return map.get(inode);

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java

@@ -285,6 +285,16 @@ public abstract class INodeReference extends INode {
     referred.setAccessTime(accessTime);
   }
 
+  @Override
+  public final byte getStoragePolicyID() {
+    return referred.getStoragePolicyID();
+  }
+
+  @Override
+  public final byte getLocalStoragePolicyID() {
+    return referred.getLocalStoragePolicyID();
+  }
+
   @Override
   final void recordModification(int latestSnapshotId)
       throws QuotaExceededException {

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

@@ -145,4 +145,16 @@ public class INodeSymlink extends INodeWithAdditionalFields {
   public void addXAttrFeature(XAttrFeature f) {
     throw new UnsupportedOperationException("XAttrs are not supported on symlinks");
   }
+
+  @Override
+  public byte getStoragePolicyID() {
+    throw new UnsupportedOperationException(
+        "Storage policy are not supported on symlinks");
+  }
+
+  @Override
+  public byte getLocalStoragePolicyID() {
+    throw new UnsupportedOperationException(
+        "Storage policy are not supported on symlinks");
+  }
 }

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java

@@ -67,11 +67,12 @@ public class NameNodeLayoutVersion {
     EDITLOG_LENGTH(-56, "Add length field to every edit log op"),
     XATTRS(-57, "Extended attributes"),
     CREATE_OVERWRITE(-58, "Use single editlog record for " +
-        "creating file with overwrite"),
+      "creating file with overwrite"),
     XATTRS_NAMESPACE_EXT(-59, "Increase number of xattr namespaces"),
+    BLOCK_STORAGE_POLICY(-60, "Block Storage policy"),
     LAZY_PERSIST_FILES(-60, "Support for optional lazy persistence of " + 
-        " files with reduced durability guarantees");
-
+        " files with reduced durability guarantees");    
+    
     private final FeatureInfo info;
 
     /**

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

@@ -585,7 +585,13 @@ class NameNodeRpcServer implements NamenodeProtocols {
     throws IOException {  
     return namesystem.setReplication(src, replication);
   }
-    
+
+  @Override
+  public void setStoragePolicy(String src, String policyName)
+      throws IOException {
+    namesystem.setStoragePolicy(src, policyName);
+  }
+
   @Override // ClientProtocol
   public void setPermission(String src, FsPermission permissions)
       throws IOException {

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

@@ -17,5 +17,20 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-public class RetryStartFileException extends Exception {
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class RetryStartFileException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public RetryStartFileException() {
+    super("Preconditions for creating a file failed because of a " +
+        "transient error, retry create later.");
+  }
+
+  public RetryStartFileException(String s) {
+    super(s);
+  }
 }

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

@@ -222,7 +222,7 @@ public class FSImageFormatPBSnapshot {
               fileInPb.getAccessTime(), (short) fileInPb.getReplication(),
               fileInPb.getPreferredBlockSize(),
               fileInPb.hasIsLazyPersist() ? fileInPb.getIsLazyPersist() : false,
-              xAttrs);
+              (byte)fileInPb.getStoragePolicyID(), xAttrs);
         }
 
         FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,

+ 2 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -26,7 +26,6 @@ import java.net.InetAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
@@ -223,11 +222,8 @@ public class NamenodeWebHdfsMethods {
       final DatanodeDescriptor clientNode = bm.getDatanodeManager(
           ).getDatanodeByHost(getRemoteAddress());
       if (clientNode != null) {
-        final DatanodeStorageInfo[] storages = bm.getBlockPlacementPolicy()
-            .chooseTarget(path, 1, clientNode,
-                new ArrayList<DatanodeStorageInfo>(), false, excludes, blocksize,
-                // TODO: get storage type from the file
-                StorageType.DEFAULT);
+        final DatanodeStorageInfo[] storages = bm.chooseTarget4WebHDFS(
+            path, clientNode, excludes, blocksize);
         if (storages.length > 0) {
           return storages[0].getDatanodeDescriptor();
         }

+ 69 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.tools;
 
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.net.InetSocketAddress;
@@ -43,6 +44,7 @@ import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.shell.Command;
 import org.apache.hadoop.fs.shell.CommandFormat;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -58,23 +60,24 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.GenericRefreshProtocol;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RefreshCallQueueProtocol;
-import org.apache.hadoop.ipc.GenericRefreshProtocol;
 import org.apache.hadoop.ipc.RefreshResponse;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.protocolPB.GenericRefreshProtocolClientSideTranslatorPB;
+import org.apache.hadoop.ipc.protocolPB.GenericRefreshProtocolPB;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
-import org.apache.hadoop.ipc.protocolPB.GenericRefreshProtocolClientSideTranslatorPB;
-import org.apache.hadoop.ipc.protocolPB.GenericRefreshProtocolPB;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -384,6 +387,8 @@ public class DFSAdmin extends FsShell {
     "\t[-shutdownDatanode <datanode_host:ipc_port> [upgrade]]\n" +
     "\t[-getDatanodeInfo <datanode_host:ipc_port>]\n" +
     "\t[-metasave filename]\n" +
+    "\t[-setStoragePolicy path policyName]\n" +
+    "\t[-getStoragePolicy path]\n" +
     "\t[-help [cmd]]\n";
 
   /**
@@ -589,6 +594,32 @@ public class DFSAdmin extends FsShell {
     return inSafeMode;
   }
 
+  public int setStoragePolicy(String[] argv) throws IOException {
+    DistributedFileSystem dfs = getDFS();
+    dfs.setStoragePolicy(new Path(argv[1]), argv[2]);
+    System.out.println("Set storage policy " + argv[2] + " on " + argv[1]);
+    return 0;
+  }
+
+  public int getStoragePolicy(String[] argv) throws IOException {
+    DistributedFileSystem dfs = getDFS();
+    HdfsFileStatus status = dfs.getClient().getFileInfo(argv[1]);
+    if (status == null) {
+      throw new FileNotFoundException("File/Directory does not exist: "
+          + argv[1]);
+    }
+    byte storagePolicyId = status.getStoragePolicy();
+    BlockStoragePolicy.Suite suite = BlockStoragePolicy
+        .readBlockStorageSuite(getConf());
+    BlockStoragePolicy policy = suite.getPolicy(storagePolicyId);
+    if (policy != null) {
+      System.out.println("The storage policy of " + argv[1] + ":\n" + policy);
+      return 0;
+    } else {
+      throw new IOException("Cannot identify the storage policy for " + argv[1]);
+    }
+  }
+
   /**
    * Allow snapshot on a directory.
    * Usage: java DFSAdmin -allowSnapshot snapshotDir
@@ -930,7 +961,13 @@ public class DFSAdmin extends FsShell {
     String getDatanodeInfo = "-getDatanodeInfo <datanode_host:ipc_port>\n"
         + "\tGet the information about the given datanode. This command can\n"
         + "\tbe used for checking if a datanode is alive.\n";
-    
+
+    String setStoragePolicy = "-setStoragePolicy path policyName\n"
+        + "\tSet the storage policy for a file/directory.\n";
+
+    String getStoragePolicy = "-getStoragePolicy path\n"
+        + "\tGet the storage policy for a file/directory.\n";
+
     String help = "-help [cmd]: \tDisplays help for the given command or all commands if none\n" +
       "\t\tis specified.\n";
 
@@ -988,6 +1025,10 @@ public class DFSAdmin extends FsShell {
       System.out.println(shutdownDatanode);
     } else if ("getDatanodeInfo".equalsIgnoreCase(cmd)) {
       System.out.println(getDatanodeInfo);
+    } else if ("setStoragePolicy".equalsIgnoreCase(cmd))  {
+      System.out.println(setStoragePolicy);
+    } else if ("getStoragePolicy".equalsIgnoreCase(cmd))  {
+      System.out.println(getStoragePolicy);
     } else if ("help".equals(cmd)) {
       System.out.println(help);
     } else {
@@ -1019,6 +1060,8 @@ public class DFSAdmin extends FsShell {
       System.out.println(disallowSnapshot);
       System.out.println(shutdownDatanode);
       System.out.println(getDatanodeInfo);
+      System.out.println(setStoragePolicy);
+      System.out.println(getStoragePolicy);
       System.out.println(help);
       System.out.println();
       ToolRunner.printGenericCommandUsage(System.out);
@@ -1378,6 +1421,12 @@ public class DFSAdmin extends FsShell {
     } else if ("-safemode".equals(cmd)) {
       System.err.println("Usage: hdfs dfsadmin"
           + " [-safemode enter | leave | get | wait]");
+    } else if ("-setStoragePolicy".equals(cmd)) {
+      System.err.println("Usage: java DFSAdmin"
+          + " [-setStoragePolicy path policyName]");
+    } else if ("-getStoragePolicy".equals(cmd)) {
+      System.err.println("Usage: java DFSAdmin"
+          + " [-getStoragePolicy path]");
     } else if ("-allowSnapshot".equalsIgnoreCase(cmd)) {
       System.err.println("Usage: hdfs dfsadmin"
           + " [-allowSnapshot <snapshotDir>]");
@@ -1586,6 +1635,16 @@ public class DFSAdmin extends FsShell {
         printUsage(cmd);
         return exitCode;
       }
+    } else if ("-setStoragePolicy".equals(cmd)) {
+      if (argv.length != 3) {
+        printUsage(cmd);
+        return exitCode;
+      }
+    } else if ("-getStoragePolicy".equals(cmd)) {
+      if (argv.length != 2) {
+        printUsage(cmd);
+        return exitCode;
+      }
     }
     
     // initialize DFSAdmin
@@ -1657,6 +1716,10 @@ public class DFSAdmin extends FsShell {
         exitCode = shutdownDatanode(argv, i);
       } else if ("-getDatanodeInfo".equals(cmd)) {
         exitCode = getDatanodeInfo(argv, i);
+      } else if ("-setStoragePolicy".equals(cmd)) {
+        exitCode = setStoragePolicy(argv);
+      } else if ("-getStoragePolicy".equals(cmd)) {
+        exitCode = getStoragePolicy(argv);
       } else if ("-help".equals(cmd)) {
         if (i < argv.length) {
           printHelp(argv[i]);

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/EnumCounters.java

@@ -105,6 +105,15 @@ public class EnumCounters<E extends Enum<E>> {
       this.counters[i] -= that.counters[i];
     }
   }
+  
+  /** @return the sum of all counters. */
+  public final long sum() {
+    long sum = 0;
+    for(int i = 0; i < counters.length; i++) {
+      sum += counters[i];
+    }
+    return sum;
+  }
 
   @Override
   public boolean equals(Object obj) {

+ 7 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -21,6 +21,7 @@ import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.*;
@@ -230,6 +231,7 @@ public class JsonUtil {
     m.put("replication", status.getReplication());
     m.put("fileId", status.getFileId());
     m.put("childrenNum", status.getChildrenNum());
+    m.put("storagePolicy", status.getStoragePolicy());
     return includeType ? toJsonString(FileStatus.class, m): JSON.toString(m);
   }
 
@@ -262,9 +264,13 @@ public class JsonUtil {
     Long childrenNumLong = (Long) m.get("childrenNum");
     final int childrenNum = (childrenNumLong == null) ? -1
             : childrenNumLong.intValue();
+    final byte storagePolicy = m.containsKey("storagePolicy") ?
+        (byte) (long) (Long) m.get("storagePolicy") :
+          BlockStoragePolicy.ID_UNSPECIFIED;
     return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication,
         blockSize, isLazyPersist, mTime, aTime, permission, owner, group,
-        symlink, DFSUtil.string2Bytes(localName), fileId, childrenNum, null);
+        symlink, DFSUtil.string2Bytes(localName), fileId, childrenNum, null,
+        storagePolicy);
   }
 
   /** Convert an ExtendedBlock to a Json map. */

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -101,6 +101,14 @@ message SetReplicationResponseProto {
   required bool result = 1;
 }
 
+message SetStoragePolicyRequestProto {
+  required string src = 1;
+  required string policyName = 2;
+}
+
+message SetStoragePolicyResponseProto { // void response
+}
+
 message SetPermissionRequestProto {
   required string src = 1;
   required FsPermissionProto permission = 2;
@@ -690,6 +698,8 @@ service ClientNamenodeProtocol {
   rpc append(AppendRequestProto) returns(AppendResponseProto);
   rpc setReplication(SetReplicationRequestProto)
       returns(SetReplicationResponseProto);
+  rpc setStoragePolicy(SetStoragePolicyRequestProto)
+      returns(SetStoragePolicyResponseProto);
   rpc setPermission(SetPermissionRequestProto)
       returns(SetPermissionResponseProto);
   rpc setOwner(SetOwnerRequestProto) returns(SetOwnerResponseProto);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto

@@ -138,7 +138,8 @@ message INodeSection {
     optional FileUnderConstructionFeature fileUC = 7;
     optional AclFeatureProto acl = 8;
     optional XAttrFeatureProto xAttrs = 9;
-    optional bool isLazyPersist = 10 [default = false];
+    optional uint32 storagePolicyID = 10;
+    optional bool isLazyPersist = 11 [default = false];
   }
 
   message INodeDirectory {

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

@@ -158,7 +158,8 @@ message FsPermissionProto {
 enum StorageTypeProto {
   DISK = 1;
   SSD = 2;
-  RAM_DISK = 3;
+  ARCHIVE = 3;
+  RAM_DISK = 4;
 }
 
 /**
@@ -263,7 +264,9 @@ message HdfsFileStatusProto {
   optional int32 childrenNum = 14 [default = -1];
   // Optional field for file encryption
   optional FileEncryptionInfoProto fileEncryptionInfo = 15;
-  optional bool isLazyPersist = 16 [default = false];
+
+  optional uint32 storagePolicy = 16 [default = 0]; // block storage policy id
+  optional bool isLazyPersist = 17 [default = false];
 } 
 
 /**

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

@@ -0,0 +1,118 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!--
+   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.
+-->
+
+<!-- Do not modify this file directly.  Instead, copy entries that you wish -->
+<!-- to modify from this file into blockStoragePolicy-site.xml and change   -->
+<!-- there.  If blockStoragePolicy-site.xml does not exist, create it.      -->
+
+<configuration>
+<property>
+  <name>dfs.block.storage.policies</name>
+  <value>HOT:12, WARM:8, COLD:4</value>
+  <description>
+    A list of block storage policy names and IDs.  The syntax is
+
+      NAME_1:ID_1, NAME_2:ID_2, ..., NAME_n:ID_n
+
+    where ID is an integer in the range [1,15] and NAME is case insensitive.
+    The first element is the default policy.  Empty list is not allowed.
+  </description>
+</property>
+
+<!-- Block Storage Policy HOT:12 -->
+<property>
+  <name>dfs.block.storage.policy.12</name>
+  <value>DISK</value>
+  <description>
+    A list of storage types for storing the block replicas such as
+
+      STORAGE_TYPE_1, STORAGE_TYPE_2, ..., STORAGE_TYPE_n
+  
+    When creating a block, the i-th replica is stored using i-th storage type
+    for i less than or equal to n, and
+    the j-th replica is stored using n-th storage type for j greater than n.
+
+    Empty list is not allowed.
+
+    Examples:
+    DISK          : all replicas stored using DISK.
+    DISK, ARCHIVE : the first replica is stored using DISK and all the
+                    remaining replicas are stored using ARCHIVE.
+  </description>
+</property>
+
+<property>
+  <name>dfs.block.storage.policy.creation-fallback.12</name>
+  <value></value>
+  <description>
+    A list of storage types for creation fallback storage.
+
+      STORAGE_TYPE_1, STORAGE_TYPE_2, ..., STORAGE_TYPE_n
+  
+    When creating a block, if a particular storage type specified in the policy
+    is unavailable, the fallback STORAGE_TYPE_1 is used.  Further, if
+    STORAGE_TYPE_i is also unavailable, the fallback STORAGE_TYPE_(i+1) is used.
+    In case that all fallback storages are unavailabe, the block will be created
+    with number of replicas less than the specified replication factor.
+
+    An empty list indicates that there is no fallback storage.
+  </description>
+</property>
+
+<property>
+  <name>dfs.block.storage.policy.replication-fallback.12</name>
+  <value>ARCHIVE</value>
+  <description>
+    Similar to dfs.block.storage.policy.creation-fallback.x but for replication.
+  </description>
+</property>
+
+<!-- Block Storage Policy WARM:8 -->
+<property>
+  <name>dfs.block.storage.policy.8</name>
+  <value>DISK, ARCHIVE</value>
+</property>
+
+<property>
+  <name>dfs.block.storage.policy.creation-fallback.8</name>
+  <value>DISK, ARCHIVE</value>
+</property>
+
+<property>
+  <name>dfs.block.storage.policy.replication-fallback.8</name>
+  <value>DISK, ARCHIVE</value>
+</property>
+
+<!-- Block Storage Policy COLD:4 -->
+<property>
+  <name>dfs.block.storage.policy.4</name>
+  <value>ARCHIVE</value>
+</property>
+
+<property>
+  <name>dfs.block.storage.policy.creation-fallback.4</name>
+  <value></value>
+</property>
+
+<property>
+  <name>dfs.block.storage.policy.replication-fallback.4</name>
+  <value></value>
+</property>
+</configuration>

+ 2 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -22,7 +22,8 @@
 <!-- wish to modify from this file into hdfs-site.xml and change them -->
 <!-- there.  If hdfs-site.xml does not already exist, create it.      -->
 
-<configuration>
+<configuration xmlns:xi="http://www.w3.org/2001/XInclude"> 
+  <xi:include href="blockStoragePolicy-default.xml" />
 
 <property>
   <name>hadoop.hdfs.configuration.version</name>
@@ -2078,19 +2079,6 @@
   </description>
 </property>
 
-<property>
-  <name>dfs.namenode.randomize-block-locations-per-block</name>
-  <value>false</value>
-  <description>When fetching replica locations of a block, the replicas
-   are sorted based on network distance. This configuration parameter
-   determines whether the replicas at the same network distance are randomly
-   shuffled. By default, this is false, such that repeated requests for a block's
-   replicas always result in the same order. This potentially improves page cache
-   behavior. However, for some network topologies, it is desirable to shuffle this
-   order for better load balancing.
-  </description>
-</property>
-
 <property>
   <name>dfs.datanode.block.id.layout.upgrade.threads</name>
   <value>12</value>

+ 302 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/ArchivalStorage.apt.vm

@@ -0,0 +1,302 @@
+~~ 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 Archival Storage
+  ---
+  ---
+  ${maven.build.timestamp}
+
+HDFS Archival Storage
+
+%{toc|section=1|fromDepth=0}
+
+* {Introduction}
+
+  <Archival Storage> is a solution to decouple growing storage capacity from compute capacity.
+  Nodes with higher density and less expensive storage with low compute power are becoming available
+  and can be used as cold storage in the clusters.
+  Based on policy the data from hot can be moved to the cold.
+  Adding more nodes to the cold storage can grow the storage independent of the compute capacity
+  in the cluster.
+
+* {Storage Types and Storage Policies}
+
+** {Storage Types: DISK, SSD and ARCHIVE}
+
+  The first phase of
+  {{{https://issues.apache.org/jira/browse/HDFS-2832}Heterogeneous Storage (HDFS-2832)}}
+  changed datanode storage model from a single storage,
+  which may correspond to multiple physical storage medias,
+  to a collection of storages with each storage corresponding to a physical storage media.
+  It also added the notion of storage types, DISK and SSD,
+  where DISK is the default storage type.
+
+  A new storage type <ARCHIVE>,
+  which has high storage density (petabyte of storage) but little compute power,
+  is added for supporting archival storage.
+
+** {Storage Policies: Hot, Warm and Cold}
+
+  A new concept of storage policies is introduced in order to allow files to be stored
+  in different storage types according to the storage policy.
+
+  We have the following storage policies:
+
+  * <<Hot>> - for both storage and compute.
+              The data that is popular and still being used for processing will stay in this policy.
+              When a block is hot, all replicas are stored in DISK.
+
+  * <<Cold>> - only for storage with limited compute.
+               The data that is no longer being used, or data that needs to be archived is moved
+               from hot storage to cold storage.
+               When a block is cold, all replicas are stored in ARCHIVE.
+
+  * <<Warm>> - partially hot and partially cold.
+               When a block is warm, some of its replicas are stored in DISK
+               and the remaining replicas are stored in ARCHIVE.
+
+  []
+
+  More formally, a storage policy consists of the following fields:
+
+  [[1]] Policy ID
+
+  [[2]] Policy name
+
+  [[3]] A list of storage types for block placement
+
+  [[4]] A list of fallback storage types for file creation
+
+  [[5]] A list of fallback storage types for replication
+
+  []
+
+  When there is enough space,
+  block replicas are stored according to the storage type list specified in #3.
+  When some of the storage types in list #3 are running out of space,
+  the fallback storage type lists specified in #4 and #5 are used
+  to replace the out-of-space storage types for file creation and replication, respectively.
+
+  The following is a typical storage policy table.
+
+*--------+---------------+-------------------------+-----------------------+-----------------------+
+| <<Policy>> | <<Policy>>| <<Block Placement>>     | <<Fallback storages>> | <<Fallback storages>> |
+| <<ID>>     | <<Name>>  | <<(n\ replicas)>>      | <<for creation>>      | <<for replication>>   |
+*--------+---------------+-------------------------+-----------------------+-----------------------+
+| 12     | Hot (default) | DISK: <n>               | \<none\>              | ARCHIVE               |
+*--------+---------------+-------------------------+-----------------------+-----------------------+
+| 8      | Warm          | DISK: 1, ARCHIVE: <n>-1 | ARCHIVE, DISK         | ARCHIVE, DISK         |
+*--------+---------------+-------------------------+-----------------------+-----------------------+
+| 4      | Cold          | ARCHIVE: <n>            | \<none\>              | \<none\>              |
+*--------+---------------+-------------------------+-----------------------+-----------------------+
+
+  Note that cluster administrators may change the storage policy table
+  according to the characteristic of the cluster.
+  For example, in order to prevent losing archival data,
+  administrators may want to use DISK as fallback storage for replication in the Cold policy.
+  A drawback of such setting is that the DISK storages could be filled up with archival data.
+  As a result, the entire cluster may become full and cannot serve hot data anymore.
+
+** {Configurations}
+
+*** {Setting The List of All Storage Policies}
+
+  * <<dfs.block.storage.policies>>
+    - a list of block storage policy names and IDs.
+    The syntax is
+
+      NAME_1:ID_1, NAME_2:ID_2, ..., NAME_<n>:ID_<n>
+
+    where ID is an integer in the closed range [1,15] and NAME is case insensitive.
+    The first element is the <default policy>.  Empty list is not allowed.
+
+    The default value is shown below.
+
++------------------------------------------+
+<property>
+  <name>dfs.block.storage.policies</name>
+  <value>HOT:12, WARM:8, COLD:4</value>
+</property>
++------------------------------------------+
+
+  []
+
+*** {Setting Storage Policy Details}
+
+  The following configuration properties are for setting the details of each storage policy,
+  where <<<\<ID\>>>> is the actual policy ID.
+
+  * <<dfs.block.storage.policy.\<ID\>>>
+    - a list of storage types for storing the block replicas.
+    The syntax is
+
+      STORAGE_TYPE_1, STORAGE_TYPE_2, ..., STORAGE_TYPE_<n>
+  
+    When creating a block, the <i>-th replica is stored using <i>-th storage type
+    for <i> less than or equal to <n>, and
+    the <j>-th replica is stored using <n>-th storage type for <j> greater than <n>.
+
+    Empty list is not allowed.
+
+    Examples:
+
++------------------------------------------+
+DISK          : all replicas stored using DISK.
+DISK, ARCHIVE : the first replica is stored using DISK and all the
+                remaining replicas are stored using ARCHIVE.
++------------------------------------------+
+
+  * <<dfs.block.storage.policy.creation-fallback.\<ID\>>>
+    - a list of storage types for creation fallback storage.
+    The syntax is
+
+      STORAGE_TYPE_1, STORAGE_TYPE_2, ..., STORAGE_TYPE_n
+  
+    When creating a block, if a particular storage type specified in the policy
+    is unavailable, the fallback STORAGE_TYPE_1 is used.  Further, if
+    STORAGE_TYPE_<i> is also unavailable, the fallback STORAGE_TYPE_<(i+1)> is used.
+    In case all fallback storages are unavailable, the block will be created
+    with number of replicas less than the specified replication factor.
+
+    An empty list indicates that there is no fallback storage.
+
+  * <<dfs.block.storage.policy.replication-fallback.\<ID\>>>
+    - a list of storage types for replication fallback storage.
+    The usage of this configuration property is similar to
+    <<<dfs.block.storage.policy.creation-fallback.\<ID\>>>>
+    except that it takes effect on replication but not block creation.
+
+  []
+
+  The following are the default configuration values for Hot, Warm and Cold storage policies.
+
+  * Block Storage Policy <<HOT:12>>
+
++------------------------------------------+
+<property>
+  <name>dfs.block.storage.policy.12</name>
+  <value>DISK</value>
+</property>
+<property>
+  <name>dfs.block.storage.policy.creation-fallback.12</name>
+  <value></value>
+</property>
+<property>
+  <name>dfs.block.storage.policy.replication-fallback.12</name>
+  <value>ARCHIVE</value>
+</property>
++------------------------------------------+
+
+  * Block Storage Policy <<WARM:8>>
+
++------------------------------------------+
+<property>
+  <name>dfs.block.storage.policy.8</name>
+  <value>DISK, ARCHIVE</value>
+</property>
+<property>
+  <name>dfs.block.storage.policy.creation-fallback.8</name>
+  <value>DISK, ARCHIVE</value>
+</property>
+<property>
+  <name>dfs.block.storage.policy.replication-fallback.8</name>
+  <value>DISK, ARCHIVE</value>
+</property>
++------------------------------------------+
+
+  * Block Storage Policy <<COLD:4>>
+
++------------------------------------------+
+<property>
+  <name>dfs.block.storage.policy.4</name>
+  <value>ARCHIVE</value>
+</property>
+<property>
+  <name>dfs.block.storage.policy.creation-fallback.4</name>
+  <value></value>
+</property>
+<property>
+  <name>dfs.block.storage.policy.replication-fallback.4</name>
+  <value></value>
+</property>
++------------------------------------------+
+
+  []
+
+* {Mover - A New Data Migration Tool}
+
+  A new data migration tool is added for archiving data.
+  The tool is similar to Balancer.
+  It periodically scans the files in HDFS to check if the block placement satisfies the storage policy.
+  For the blocks violating the storage policy,
+  it moves the replicas to a different storage type
+  in order to fulfill the storage policy requirement.
+
+  * Command:
+
++------------------------------------------+
+hdfs mover [-p <files/dirs> | -f <local file name>]
++------------------------------------------+
+
+  * Arguments:
+
+*-------------------------+--------------------------------------------------------+
+| <<<-p \<files/dirs\>>>> | Specify a space separated list of HDFS files/dirs to migrate.
+*-------------------------+--------------------------------------------------------+
+| <<<-f \<local file\>>>> | Specify a local file containing a list of HDFS files/dirs to migrate.
+*-------------------------+--------------------------------------------------------+
+
+  Note that, when both -p and -f options are omitted, the default path is the root directory.
+
+  []
+
+
+* {<<<DFSAdmin>>> Commands}
+
+** {Set Storage Policy}
+
+  Set a storage policy to a file or a directory.
+
+  * Command:
+
++------------------------------------------+
+hdfs dfsadmin -setStoragePolicy <path> <policyName>
++------------------------------------------+
+
+  * Arguments:
+
+*----------------------+-----------------------------------------------------+
+| <<<\<path\>>>>       | The path referring to either a directory or a file. |
+*----------------------+-----------------------------------------------------+
+| <<<\<policyName\>>>> | The name of the storage policy.                     |
+*----------------------+-----------------------------------------------------+
+
+  []
+
+** {Get Storage Policy}
+
+  Get the storage policy of a file or a directory.
+
+  * Command:
+
++------------------------------------------+
+hdfs dfsadmin -getStoragePolicy <path>
++------------------------------------------+
+
+  * Arguments:
+
+*----------------------+-----------------------------------------------------+
+| <<<\<path\>>>>       | The path referring to either a directory or a file. |
+*----------------------+-----------------------------------------------------+
+
+  []

+ 35 - 8
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSCommands.apt.vm

@@ -147,18 +147,19 @@ HDFS Commands Guide
 *-----------------+-----------------------------------------------------------+
 | -regular        | Normal datanode startup (default).
 *-----------------+-----------------------------------------------------------+
-| -rollback       | Rollsback the datanode to the previous version. This should
+| -rollback       | Rollback the datanode to the previous version. This should
 |                 | be used after stopping the datanode and distributing the
 |                 | old hadoop version.
 *-----------------+-----------------------------------------------------------+
-| -rollingupgrade rollback | Rollsback a rolling upgrade operation.
+| -rollingupgrade rollback | Rollback a rolling upgrade operation.
 *-----------------+-----------------------------------------------------------+
 
 ** <<<dfsadmin>>>
 
    Runs a HDFS dfsadmin client.
 
-   Usage: <<<hdfs dfsadmin [GENERIC_OPTIONS]
++------------------------------------------+
+   Usage: hdfs dfsadmin [GENERIC_OPTIONS]
           [-report [-live] [-dead] [-decommissioning]]
           [-safemode enter | leave | get | wait]
           [-saveNamespace]
@@ -169,6 +170,8 @@ HDFS Commands Guide
           [-clrQuota <dirname>...<dirname>]
           [-setSpaceQuota <quota> <dirname>...<dirname>]
           [-clrSpaceQuota <dirname>...<dirname>]
+          [-setStoragePolicy <path> <policyName>]
+          [-getStoragePolicy <path>]
           [-finalizeUpgrade]
           [-rollingUpgrade [<query>|<prepare>|<finalize>]]
           [-metasave filename]
@@ -186,7 +189,8 @@ HDFS Commands Guide
           [-fetchImage <local directory>]
           [-shutdownDatanode <datanode_host:ipc_port> [upgrade]]
           [-getDatanodeInfo <datanode_host:ipc_port>]
-          [-help [cmd]]>>>
+          [-help [cmd]]
++------------------------------------------+
 
 *-----------------+-----------------------------------------------------------+
 || COMMAND_OPTION || Description
@@ -236,6 +240,10 @@ HDFS Commands Guide
                   | {{{../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands}HDFS Quotas Guide}}
                   | for the detail.
 *-----------------+-----------------------------------------------------------+
+| -setStoragePolicy \<path\> \<policyName\> | Set a storage policy to a file or a directory.
+*-----------------+-----------------------------------------------------------+
+| -getStoragePolicy \<path\> | Get the storage policy of a file or a directory.
+*-----------------+-----------------------------------------------------------+
 | -finalizeUpgrade| Finalize upgrade of HDFS. Datanodes delete their previous
                   | version working directories, followed by Namenode doing the
                   | same. This completes the upgrade process.
@@ -250,7 +258,7 @@ HDFS Commands Guide
                   | <filename> will contain one line for each of the following\
                   | 1. Datanodes heart beating with Namenode\
                   | 2. Blocks waiting to be replicated\
-                  | 3. Blocks currrently being replicated\
+                  | 3. Blocks currently being replicated\
                   | 4. Blocks waiting to be deleted
 *-----------------+-----------------------------------------------------------+
 | -refreshServiceAcl | Reload the service-level authorization policy file.
@@ -312,12 +320,30 @@ HDFS Commands Guide
                   | is specified.
 *-----------------+-----------------------------------------------------------+
 
+** <<<mover>>>
+
+   Runs the data migration utility.
+   See {{{./ArchivalStorage.html#Mover_-_A_New_Data_Migration_Tool}Mover}} for more details.
+
+   Usage: <<<hdfs mover [-p <files/dirs> | -f <local file name>]>>>
+
+*--------------------+--------------------------------------------------------+
+|| COMMAND_OPTION    || Description
+*--------------------+--------------------------------------------------------+
+| -p \<files/dirs\>  | Specify a space separated list of HDFS files/dirs to migrate.
+*--------------------+--------------------------------------------------------+
+| -f \<local file\>  | Specify a local file containing a list of HDFS files/dirs to migrate.
+*--------------------+--------------------------------------------------------+
+
+  Note that, when both -p and -f options are omitted, the default path is the root directory.
+
 ** <<<namenode>>>
 
    Runs the namenode. More info about the upgrade, rollback and finalize is at
    {{{./HdfsUserGuide.html#Upgrade_and_Rollback}Upgrade Rollback}}.
 
-   Usage: <<<hdfs namenode [-backup] |
++------------------------------------------+
+   Usage: hdfs namenode [-backup] |
           [-checkpoint] |
           [-format [-clusterid cid ] [-force] [-nonInteractive] ] |
           [-upgrade [-clusterid cid] [-renameReserved<k-v pairs>] ] |
@@ -329,7 +355,8 @@ HDFS Commands Guide
           [-initializeSharedEdits] |
           [-bootstrapStandby] |
           [-recover [-force] ] |
-          [-metadataVersion ]>>>
+          [-metadataVersion ]
++------------------------------------------+
 
 *--------------------+--------------------------------------------------------+
 || COMMAND_OPTION    || Description
@@ -351,7 +378,7 @@ HDFS Commands Guide
 | -upgradeOnly [-clusterid cid] [-renameReserved\<k-v pairs\>] | Upgrade the
                      | specified NameNode and then shutdown it.
 *--------------------+--------------------------------------------------------+
-| -rollback          | Rollsback the NameNode to the previous version. This
+| -rollback          | Rollback the NameNode to the previous version. This
                      | should be used after stopping the cluster and
                      | distributing the old Hadoop version.
 *--------------------+--------------------------------------------------------+

+ 114 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -24,6 +24,8 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
 import com.google.common.collect.Lists;
 
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -66,6 +68,7 @@ import org.apache.hadoop.hdfs.server.namenode.ha
         .ConfiguredFailoverProxyProvider;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.net.NetUtils;
@@ -75,6 +78,8 @@ import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.VersionInfo;
 import org.junit.Assume;
 
@@ -88,11 +93,8 @@ import java.security.PrivilegedExceptionAction;
 import java.util.*;
 import java.util.concurrent.TimeoutException;
 
-import static org.apache.hadoop.fs.CreateFlag.CREATE;
-import static org.apache.hadoop.fs.CreateFlag.LAZY_PERSIST;
-import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
+import static org.apache.hadoop.fs.CreateFlag.*;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -891,6 +893,37 @@ public class DFSTestUtil {
     conf.set(DFSConfigKeys.DFS_NAMESERVICES, Joiner.on(",")
         .join(nameservices));
   }
+
+  public static void setFederatedHAConfiguration(MiniDFSCluster cluster,
+      Configuration conf) {
+    Map<String, List<String>> nameservices = Maps.newHashMap();
+    for (NameNodeInfo info : cluster.getNameNodeInfos()) {
+      Preconditions.checkState(info.nameserviceId != null);
+      List<String> nns = nameservices.get(info.nameserviceId);
+      if (nns == null) {
+        nns = Lists.newArrayList();
+        nameservices.put(info.nameserviceId, nns);
+      }
+      nns.add(info.nnId);
+
+      conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
+          info.nameserviceId, info.nnId),
+          DFSUtil.createUri(HdfsConstants.HDFS_URI_SCHEME,
+          info.nameNode.getNameNodeAddress()).toString());
+      conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
+          info.nameserviceId, info.nnId),
+          DFSUtil.createUri(HdfsConstants.HDFS_URI_SCHEME,
+          info.nameNode.getNameNodeAddress()).toString());
+    }
+    for (Map.Entry<String, List<String>> entry : nameservices.entrySet()) {
+      conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX,
+          entry.getKey()), Joiner.on(",").join(entry.getValue()));
+      conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + entry
+          .getKey(), ConfiguredFailoverProxyProvider.class.getName());
+    }
+    conf.set(DFSConfigKeys.DFS_NAMESERVICES, Joiner.on(",")
+        .join(nameservices.keySet()));
+  }
   
   private static DatanodeID getDatanodeID(String ipAddr) {
     return new DatanodeID(ipAddr, "localhost",
@@ -974,9 +1007,14 @@ public class DFSTestUtil {
   public static DatanodeStorageInfo[] createDatanodeStorageInfos(int n) {
     return createDatanodeStorageInfos(n, null, null);
   }
-    
+
   public static DatanodeStorageInfo[] createDatanodeStorageInfos(
       int n, String[] racks, String[] hostnames) {
+    return createDatanodeStorageInfos(n, racks, hostnames, null);
+  }
+
+  public static DatanodeStorageInfo[] createDatanodeStorageInfos(
+      int n, String[] racks, String[] hostnames, StorageType[] types) {
     DatanodeStorageInfo[] storages = new DatanodeStorageInfo[n];
     for(int i = storages.length; i > 0; ) {
       final String storageID = "s" + i;
@@ -984,16 +1022,30 @@ public class DFSTestUtil {
       i--;
       final String rack = (racks!=null && i < racks.length)? racks[i]: "defaultRack";
       final String hostname = (hostnames!=null && i < hostnames.length)? hostnames[i]: "host";
-      storages[i] = createDatanodeStorageInfo(storageID, ip, rack, hostname);
+      final StorageType type = (types != null && i < types.length) ? types[i]
+          : StorageType.DEFAULT;
+      storages[i] = createDatanodeStorageInfo(storageID, ip, rack, hostname,
+          type);
     }
     return storages;
   }
+
   public static DatanodeStorageInfo createDatanodeStorageInfo(
       String storageID, String ip, String rack, String hostname) {
-    final DatanodeStorage storage = new DatanodeStorage(storageID);
-    final DatanodeDescriptor dn = BlockManagerTestUtil.getDatanodeDescriptor(ip, rack, storage, hostname);
+    return createDatanodeStorageInfo(storageID, ip, rack, hostname,
+        StorageType.DEFAULT);
+  }
+
+  public static DatanodeStorageInfo createDatanodeStorageInfo(
+      String storageID, String ip, String rack, String hostname,
+      StorageType type) {
+    final DatanodeStorage storage = new DatanodeStorage(storageID,
+        DatanodeStorage.State.NORMAL, type);
+    final DatanodeDescriptor dn = BlockManagerTestUtil.getDatanodeDescriptor(
+        ip, rack, storage, hostname);
     return BlockManagerTestUtil.newDatanodeStorageInfo(dn, storage);
   }
+
   public static DatanodeDescriptor[] toDatanodeDescriptor(
       DatanodeStorageInfo[] storages) {
     DatanodeDescriptor[] datanodes = new DatanodeDescriptor[storages.length];
@@ -1080,6 +1132,8 @@ public class DFSTestUtil {
     FSDataOutputStream s = filesystem.create(pathFileCreate);
     // OP_CLOSE 9
     s.close();
+    // OP_SET_STORAGE_POLICY 45
+    filesystem.setStoragePolicy(pathFileCreate, "HOT");
     // OP_RENAME_OLD 1
     final Path pathFileMoved = new Path("/file_moved");
     filesystem.rename(pathFileCreate, pathFileMoved);
@@ -1440,6 +1494,57 @@ public class DFSTestUtil {
     return expectedPrimary.getDatanodeDescriptor();
   }
 
+  public static void toolRun(Tool tool, String cmd, int retcode, String contain)
+      throws Exception {
+    String [] cmds = StringUtils.split(cmd, ' ');
+    System.out.flush();
+    System.err.flush();
+    PrintStream origOut = System.out;
+    PrintStream origErr = System.err;
+    String output = null;
+    int ret = 0;
+    try {
+      ByteArrayOutputStream bs = new ByteArrayOutputStream(1024);
+      PrintStream out = new PrintStream(bs);
+      System.setOut(out);
+      System.setErr(out);
+      ret = tool.run(cmds);
+      System.out.flush();
+      System.err.flush();
+      out.close();
+      output = bs.toString();
+    } finally {
+      System.setOut(origOut);
+      System.setErr(origErr);
+    }
+    System.out.println("Output for command: " + cmd + " retcode: " + ret);
+    if (output != null) {
+      System.out.println(output);
+    }
+    assertEquals(retcode, ret);
+    if (contain != null) {
+      assertTrue("The real output is: " + output + ".\n It should contain: "
+          + contain, output.contains(contain));
+    }
+  }
+
+  public static void FsShellRun(String cmd, int retcode, String contain,
+      Configuration conf) throws Exception {
+    FsShell shell = new FsShell(new Configuration(conf));
+    toolRun(shell, cmd, retcode, contain);
+  }  
+
+  public static void DFSAdminRun(String cmd, int retcode, String contain,
+      Configuration conf) throws Exception {
+    DFSAdmin admin = new DFSAdmin(new Configuration(conf));
+    toolRun(admin, cmd, retcode, contain);
+  }
+
+  public static void FsShellRun(String cmd, Configuration conf)
+      throws Exception {
+    FsShellRun(cmd, 0, null, conf);
+  }
+
   public static void addDataNodeLayoutVersion(final int lv, final String description)
       throws NoSuchFieldException, IllegalAccessException {
     Preconditions.checkState(lv < DataNodeLayoutVersion.CURRENT_LAYOUT_VERSION);

+ 8 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -1475,19 +1475,21 @@ public class MiniDFSCluster {
           secureResources, dn.getIpcPort()));
       dns[i - curDatanodesNum] = dn;
     }
-    curDatanodesNum += numDataNodes;
     this.numDataNodes += numDataNodes;
     waitActive();
-
+    
     if (storageCapacities != null) {
-      for (int i = curDatanodesNumSaved; i < curDatanodesNumSaved+numDataNodes; ++i) {
-        List<? extends FsVolumeSpi> volumes = dns[i].getFSDataset().getVolumes();
-        assert storageCapacities[i].length == storagesPerDatanode;
+      for (int i = curDatanodesNum; i < curDatanodesNum+numDataNodes; ++i) {
+        final int index = i - curDatanodesNum;
+        List<? extends FsVolumeSpi> volumes = dns[index].getFSDataset().getVolumes();
+        assert storageCapacities[index].length == storagesPerDatanode;
         assert volumes.size() == storagesPerDatanode;
 
         for (int j = 0; j < volumes.size(); ++j) {
           FsVolumeImpl volume = (FsVolumeImpl) volumes.get(j);
-          volume.setCapacityForTesting(storageCapacities[i][j]);
+          LOG.info("setCapacityForTesting "  + storageCapacities[index][j]
+              + " for [" + volume.getStorageType() + "]" + volume.getStorageID());
+          volume.setCapacityForTesting(storageCapacities[index][j]);
         }
       }
     }

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

@@ -0,0 +1,1075 @@
+/**
+ * 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;
+
+import static org.apache.hadoop.hdfs.BlockStoragePolicy.ID_UNSPECIFIED;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.util.*;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.blockmanagement.*;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.net.Node;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+/** Test {@link BlockStoragePolicy} */
+public class TestBlockStoragePolicy {
+  public static final BlockStoragePolicy.Suite POLICY_SUITE;
+  public static final BlockStoragePolicy DEFAULT_STORAGE_POLICY;
+  public static final Configuration conf;
+
+  static {
+    conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
+    POLICY_SUITE = BlockStoragePolicy.readBlockStorageSuite(conf);
+    DEFAULT_STORAGE_POLICY = POLICY_SUITE.getDefaultPolicy();
+  }
+
+  static final EnumSet<StorageType> none = EnumSet.noneOf(StorageType.class);
+  static final EnumSet<StorageType> archive = EnumSet.of(StorageType.ARCHIVE);
+  static final EnumSet<StorageType> disk = EnumSet.of(StorageType.DISK);
+  static final EnumSet<StorageType> both = EnumSet.of(StorageType.DISK, StorageType.ARCHIVE);
+
+  static final long FILE_LEN = 1024;
+  static final short REPLICATION = 3;
+
+  static final byte COLD = (byte) 4;
+  static final byte WARM = (byte) 8;
+  static final byte HOT  = (byte) 12;
+
+  @Test
+  public void testDefaultPolicies() {
+    final Map<Byte, String> expectedPolicyStrings = new HashMap<Byte, String>();
+    expectedPolicyStrings.put(COLD,
+        "BlockStoragePolicy{COLD:4, storageTypes=[ARCHIVE], creationFallbacks=[], replicationFallbacks=[]");
+    expectedPolicyStrings.put(WARM,
+        "BlockStoragePolicy{WARM:8, storageTypes=[DISK, ARCHIVE], creationFallbacks=[DISK, ARCHIVE], replicationFallbacks=[DISK, ARCHIVE]");
+    expectedPolicyStrings.put(HOT,
+        "BlockStoragePolicy{HOT:12, storageTypes=[DISK], creationFallbacks=[], replicationFallbacks=[ARCHIVE]");
+
+    for(byte i = 1; i < 16; i++) {
+      final BlockStoragePolicy policy = POLICY_SUITE.getPolicy(i); 
+      if (policy != null) {
+        final String s = policy.toString();
+        Assert.assertEquals(expectedPolicyStrings.get(i), s);
+      }
+    }
+    Assert.assertEquals(POLICY_SUITE.getPolicy(HOT), POLICY_SUITE.getDefaultPolicy());
+    
+    { // check Cold policy
+      final BlockStoragePolicy cold = POLICY_SUITE.getPolicy(COLD);
+      for(short replication = 1; replication < 6; replication++) {
+        final List<StorageType> computed = cold.chooseStorageTypes(replication);
+        assertStorageType(computed, replication, StorageType.ARCHIVE);
+      }
+      assertCreationFallback(cold, null, null, null);
+      assertReplicationFallback(cold, null, null, null);
+    }
+    
+    { // check Warm policy
+      final BlockStoragePolicy warm = POLICY_SUITE.getPolicy(WARM);
+      for(short replication = 1; replication < 6; replication++) {
+        final List<StorageType> computed = warm.chooseStorageTypes(replication);
+        assertStorageType(computed, replication, StorageType.DISK, StorageType.ARCHIVE);
+      }
+      assertCreationFallback(warm, StorageType.DISK, StorageType.DISK, StorageType.ARCHIVE);
+      assertReplicationFallback(warm, StorageType.DISK, StorageType.DISK, StorageType.ARCHIVE);
+    }
+
+    { // check Hot policy
+      final BlockStoragePolicy hot = POLICY_SUITE.getPolicy(HOT);
+      for(short replication = 1; replication < 6; replication++) {
+        final List<StorageType> computed = hot.chooseStorageTypes(replication);
+        assertStorageType(computed, replication, StorageType.DISK);
+      }
+      assertCreationFallback(hot, null, null, null);
+      assertReplicationFallback(hot, StorageType.ARCHIVE, null, StorageType.ARCHIVE);
+    }
+  }
+
+  static StorageType[] newStorageTypes(int nDisk, int nArchive) {
+    final StorageType[] t = new StorageType[nDisk + nArchive];
+    Arrays.fill(t, 0, nDisk, StorageType.DISK);
+    Arrays.fill(t, nDisk, t.length, StorageType.ARCHIVE);
+    return t;
+  }
+
+  static List<StorageType> asList(int nDisk, int nArchive) {
+    return Arrays.asList(newStorageTypes(nDisk, nArchive));
+  }
+
+  static void assertStorageType(List<StorageType> computed, short replication,
+      StorageType... answers) {
+    Assert.assertEquals(replication, computed.size());
+    final StorageType last = answers[answers.length - 1];
+    for(int i = 0; i < computed.size(); i++) {
+      final StorageType expected = i < answers.length? answers[i]: last;
+      Assert.assertEquals(expected, computed.get(i));
+    }
+  }
+
+  static void assertCreationFallback(BlockStoragePolicy policy, StorageType noneExpected,
+      StorageType archiveExpected, StorageType diskExpected) {
+    Assert.assertEquals(noneExpected, policy.getCreationFallback(none));
+    Assert.assertEquals(archiveExpected, policy.getCreationFallback(archive));
+    Assert.assertEquals(diskExpected, policy.getCreationFallback(disk));
+    Assert.assertEquals(null, policy.getCreationFallback(both));
+  }
+
+  static void assertReplicationFallback(BlockStoragePolicy policy, StorageType noneExpected,
+      StorageType archiveExpected, StorageType diskExpected) {
+    Assert.assertEquals(noneExpected, policy.getReplicationFallback(none));
+    Assert.assertEquals(archiveExpected, policy.getReplicationFallback(archive));
+    Assert.assertEquals(diskExpected, policy.getReplicationFallback(disk));
+    Assert.assertEquals(null, policy.getReplicationFallback(both));
+  }
+
+  private static interface CheckChooseStorageTypes {
+    public void checkChooseStorageTypes(BlockStoragePolicy p, short replication,
+        List<StorageType> chosen, StorageType... expected);
+
+    /** Basic case: pass only replication and chosen */
+    static final CheckChooseStorageTypes Basic = new CheckChooseStorageTypes() {
+      @Override
+      public void checkChooseStorageTypes(BlockStoragePolicy p, short replication,
+          List<StorageType> chosen, StorageType... expected) {
+        final List<StorageType> types = p.chooseStorageTypes(replication, chosen);
+        assertStorageTypes(types, expected);
+      }
+    };
+    
+    /** With empty unavailables and isNewBlock=true */
+    static final CheckChooseStorageTypes EmptyUnavailablesAndNewBlock
+        = new CheckChooseStorageTypes() {
+      @Override
+      public void checkChooseStorageTypes(BlockStoragePolicy p,
+          short replication, List<StorageType> chosen, StorageType... expected) {
+        final List<StorageType> types = p.chooseStorageTypes(replication,
+            chosen, none, true);
+        assertStorageTypes(types, expected);
+      }
+    };
+
+    /** With empty unavailables and isNewBlock=false */
+    static final CheckChooseStorageTypes EmptyUnavailablesAndNonNewBlock
+        = new CheckChooseStorageTypes() {
+      @Override
+      public void checkChooseStorageTypes(BlockStoragePolicy p,
+          short replication, List<StorageType> chosen, StorageType... expected) {
+        final List<StorageType> types = p.chooseStorageTypes(replication,
+            chosen, none, false);
+        assertStorageTypes(types, expected);
+      }
+    };
+    
+    /** With both DISK and ARCHIVE unavailables and isNewBlock=true */
+    static final CheckChooseStorageTypes BothUnavailableAndNewBlock
+        = new CheckChooseStorageTypes() {
+      @Override
+      public void checkChooseStorageTypes(BlockStoragePolicy p,
+          short replication, List<StorageType> chosen, StorageType... expected) {
+        final List<StorageType> types = p.chooseStorageTypes(replication,
+            chosen, both, true);
+        assertStorageTypes(types, expected);
+      }
+    };
+
+    /** With both DISK and ARCHIVE unavailable and isNewBlock=false */
+    static final CheckChooseStorageTypes BothUnavailableAndNonNewBlock
+        = new CheckChooseStorageTypes() {
+      @Override
+      public void checkChooseStorageTypes(BlockStoragePolicy p,
+          short replication, List<StorageType> chosen, StorageType... expected) {
+        final List<StorageType> types = p.chooseStorageTypes(replication,
+            chosen, both, false);
+        assertStorageTypes(types, expected);
+      }
+    };
+
+    /** With ARCHIVE unavailable and isNewBlock=true */
+    static final CheckChooseStorageTypes ArchivalUnavailableAndNewBlock
+        = new CheckChooseStorageTypes() {
+      @Override
+      public void checkChooseStorageTypes(BlockStoragePolicy p,
+          short replication, List<StorageType> chosen, StorageType... expected) {
+        final List<StorageType> types = p.chooseStorageTypes(replication,
+            chosen, archive, true);
+        assertStorageTypes(types, expected);
+      }
+    };
+
+    /** With ARCHIVE unavailable and isNewBlock=true */
+    static final CheckChooseStorageTypes ArchivalUnavailableAndNonNewBlock
+        = new CheckChooseStorageTypes() {
+      @Override
+      public void checkChooseStorageTypes(BlockStoragePolicy p,
+          short replication, List<StorageType> chosen, StorageType... expected) {
+        final List<StorageType> types = p.chooseStorageTypes(replication,
+            chosen, archive, false);
+        assertStorageTypes(types, expected);
+      }
+    };
+  }
+
+  @Test
+  public void testChooseStorageTypes() {
+    run(CheckChooseStorageTypes.Basic);
+    run(CheckChooseStorageTypes.EmptyUnavailablesAndNewBlock);
+    run(CheckChooseStorageTypes.EmptyUnavailablesAndNonNewBlock);
+  }
+
+  private static void run(CheckChooseStorageTypes method) {
+    final BlockStoragePolicy hot = POLICY_SUITE.getPolicy(HOT);
+    final BlockStoragePolicy warm = POLICY_SUITE.getPolicy(WARM);
+    final BlockStoragePolicy cold = POLICY_SUITE.getPolicy(COLD);
+
+    final short replication = 3;
+    {
+      final List<StorageType> chosen = Lists.newArrayList();
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK, StorageType.DISK, StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.DISK, StorageType.ARCHIVE, StorageType.ARCHIVE);
+      method.checkChooseStorageTypes(cold, replication, chosen,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(StorageType.DISK); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK, StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+      method.checkChooseStorageTypes(cold, replication, chosen,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(StorageType.ARCHIVE); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK, StorageType.DISK, StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.DISK, StorageType.ARCHIVE);
+      method.checkChooseStorageTypes(cold, replication, chosen,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.DISK); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+      method.checkChooseStorageTypes(cold, replication, chosen,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.ARCHIVE); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK, StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.ARCHIVE);
+      method.checkChooseStorageTypes(cold, replication, chosen,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.ARCHIVE, StorageType.ARCHIVE); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK, StorageType.DISK, StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.DISK);
+      method.checkChooseStorageTypes(cold, replication, chosen,
+          StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.DISK, StorageType.DISK); 
+      method.checkChooseStorageTypes(hot, replication, chosen);
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+      method.checkChooseStorageTypes(cold, replication, chosen,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.DISK, StorageType.ARCHIVE); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.ARCHIVE);
+      method.checkChooseStorageTypes(cold, replication, chosen,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.ARCHIVE, StorageType.ARCHIVE); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK, StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen);
+      method.checkChooseStorageTypes(cold, replication, chosen,
+          StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK, StorageType.DISK, StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.DISK);
+      method.checkChooseStorageTypes(cold, replication, chosen);
+    }
+  }
+
+  @Test
+  public void testChooseStorageTypesWithBothUnavailable() {
+    runWithBothUnavailable(CheckChooseStorageTypes.BothUnavailableAndNewBlock);
+    runWithBothUnavailable(CheckChooseStorageTypes.BothUnavailableAndNonNewBlock);
+  }
+
+  private static void runWithBothUnavailable(CheckChooseStorageTypes method) {
+    final BlockStoragePolicy hot = POLICY_SUITE.getPolicy(HOT);
+    final BlockStoragePolicy warm = POLICY_SUITE.getPolicy(WARM);
+    final BlockStoragePolicy cold = POLICY_SUITE.getPolicy(COLD);
+
+    final short replication = 3;
+    for(int n = 0; n <= 3; n++) {
+      for(int d = 0; d <= n; d++) {
+        final int a = n - d;
+        final List<StorageType> chosen = asList(d, a);
+        method.checkChooseStorageTypes(hot, replication, chosen);
+        method.checkChooseStorageTypes(warm, replication, chosen);
+        method.checkChooseStorageTypes(cold, replication, chosen);
+      }
+    }
+  }
+
+  @Test
+  public void testChooseStorageTypesWithDiskUnavailableAndNewBlock() {
+    final BlockStoragePolicy hot = POLICY_SUITE.getPolicy(HOT);
+    final BlockStoragePolicy warm = POLICY_SUITE.getPolicy(WARM);
+    final BlockStoragePolicy cold = POLICY_SUITE.getPolicy(COLD);
+
+    final short replication = 3;
+    final EnumSet<StorageType> unavailables = disk;
+    final boolean isNewBlock = true;
+    {
+      final List<StorageType> chosen = Lists.newArrayList();
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(StorageType.DISK); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(StorageType.ARCHIVE); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.DISK); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.ARCHIVE); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.ARCHIVE, StorageType.ARCHIVE); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.DISK, StorageType.DISK); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.DISK, StorageType.ARCHIVE); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.ARCHIVE, StorageType.ARCHIVE); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock);
+    }
+  }
+
+  @Test
+  public void testChooseStorageTypesWithArchiveUnavailable() {
+    runWithArchiveUnavailable(CheckChooseStorageTypes.ArchivalUnavailableAndNewBlock);
+    runWithArchiveUnavailable(CheckChooseStorageTypes.ArchivalUnavailableAndNonNewBlock);
+  }
+
+  private static void runWithArchiveUnavailable(CheckChooseStorageTypes method) {
+    final BlockStoragePolicy hot = POLICY_SUITE.getPolicy(HOT);
+    final BlockStoragePolicy warm = POLICY_SUITE.getPolicy(WARM);
+    final BlockStoragePolicy cold = POLICY_SUITE.getPolicy(COLD);
+
+    final short replication = 3;
+    {
+      final List<StorageType> chosen = Lists.newArrayList();
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK, StorageType.DISK, StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.DISK, StorageType.DISK, StorageType.DISK);
+      method.checkChooseStorageTypes(cold, replication, chosen);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(StorageType.DISK); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK, StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.DISK, StorageType.DISK);
+      method.checkChooseStorageTypes(cold, replication, chosen);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(StorageType.ARCHIVE); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK, StorageType.DISK, StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.DISK, StorageType.DISK);
+      method.checkChooseStorageTypes(cold, replication, chosen);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.DISK); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.DISK);
+      method.checkChooseStorageTypes(cold, replication, chosen);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.ARCHIVE); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK, StorageType.DISK); 
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.DISK);
+      method.checkChooseStorageTypes(cold, replication, chosen);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.ARCHIVE, StorageType.ARCHIVE); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK, StorageType.DISK, StorageType.DISK); 
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.DISK);
+      method.checkChooseStorageTypes(cold, replication, chosen);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.DISK, StorageType.DISK); 
+      method.checkChooseStorageTypes(hot, replication, chosen);
+      method.checkChooseStorageTypes(warm, replication, chosen);
+      method.checkChooseStorageTypes(cold, replication, chosen);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.DISK, StorageType.ARCHIVE); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen);
+      method.checkChooseStorageTypes(cold, replication, chosen);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.ARCHIVE, StorageType.ARCHIVE); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK, StorageType.DISK); 
+      method.checkChooseStorageTypes(warm, replication, chosen);
+      method.checkChooseStorageTypes(cold, replication, chosen);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE); 
+      method.checkChooseStorageTypes(hot, replication, chosen,
+          StorageType.DISK, StorageType.DISK, StorageType.DISK);
+      method.checkChooseStorageTypes(warm, replication, chosen,
+          StorageType.DISK);
+      method.checkChooseStorageTypes(cold, replication, chosen);
+    }
+  }
+
+  @Test
+  public void testChooseStorageTypesWithDiskUnavailableAndNonNewBlock() {
+    final BlockStoragePolicy hot = POLICY_SUITE.getPolicy(HOT);
+    final BlockStoragePolicy warm = POLICY_SUITE.getPolicy(WARM);
+    final BlockStoragePolicy cold = POLICY_SUITE.getPolicy(COLD);
+
+    final short replication = 3;
+    final EnumSet<StorageType> unavailables = disk;
+    final boolean isNewBlock = false;
+    {
+      final List<StorageType> chosen = Lists.newArrayList();
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(StorageType.DISK); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(StorageType.ARCHIVE); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.DISK); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.ARCHIVE); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.ARCHIVE, StorageType.ARCHIVE); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.DISK, StorageType.DISK); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.DISK, StorageType.ARCHIVE); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE, StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.DISK, StorageType.ARCHIVE, StorageType.ARCHIVE); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock,
+          StorageType.ARCHIVE);
+    }
+
+    {
+      final List<StorageType> chosen = Arrays.asList(
+          StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE); 
+      checkChooseStorageTypes(hot, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(warm, replication, chosen, unavailables, isNewBlock);
+      checkChooseStorageTypes(cold, replication, chosen, unavailables, isNewBlock);
+    }
+  }
+
+  static void checkChooseStorageTypes(BlockStoragePolicy p, short replication,
+      List<StorageType> chosen, EnumSet<StorageType> unavailables,
+      boolean isNewBlock, StorageType... expected) {
+    final List<StorageType> types = p.chooseStorageTypes(replication, chosen,
+        unavailables, isNewBlock);
+    assertStorageTypes(types, expected);
+  }
+
+  static void assertStorageTypes(List<StorageType> computed, StorageType... expected) {
+    assertStorageTypes(computed.toArray(StorageType.EMPTY_ARRAY), expected);
+  }
+
+  static void assertStorageTypes(StorageType[] computed, StorageType... expected) {
+    Arrays.sort(expected);
+    Arrays.sort(computed);
+    Assert.assertArrayEquals(expected, computed);
+  }
+
+  @Test
+  public void testChooseExcess() {
+    final BlockStoragePolicy hot = POLICY_SUITE.getPolicy(HOT);
+    final BlockStoragePolicy warm = POLICY_SUITE.getPolicy(WARM);
+    final BlockStoragePolicy cold = POLICY_SUITE.getPolicy(COLD);
+
+    final short replication = 3;
+    for(int n = 0; n <= 6; n++) {
+      for(int d = 0; d <= n; d++) {
+        final int a = n - d;
+        final List<StorageType> chosen = asList(d, a);
+        {
+          final int nDisk = Math.max(0, d - replication); 
+          final int nArchive = a;
+          final StorageType[] expected = newStorageTypes(nDisk, nArchive);
+          checkChooseExcess(hot, replication, chosen, expected);
+        }
+
+        {
+          final int nDisk = Math.max(0, d - 1); 
+          final int nArchive = Math.max(0, a - replication + 1);
+          final StorageType[] expected = newStorageTypes(nDisk, nArchive);
+          checkChooseExcess(warm, replication, chosen, expected);
+        }
+
+        {
+          final int nDisk = d; 
+          final int nArchive = Math.max(0, a - replication );
+          final StorageType[] expected = newStorageTypes(nDisk, nArchive);
+          checkChooseExcess(cold, replication, chosen, expected);
+        }
+      }
+    }
+  }
+
+  static void checkChooseExcess(BlockStoragePolicy p, short replication,
+      List<StorageType> chosen, StorageType... expected) {
+    final List<StorageType> types = p.chooseExcess(replication, chosen);
+    assertStorageTypes(types, expected);
+  }
+
+  private void checkDirectoryListing(HdfsFileStatus[] stats, byte... policies) {
+    Assert.assertEquals(stats.length, policies.length);
+    for (int i = 0; i < stats.length; i++) {
+      Assert.assertEquals(stats[i].getStoragePolicy(), policies[i]);
+    }
+  }
+
+  @Test
+  public void testSetStoragePolicy() throws Exception {
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    final DistributedFileSystem fs = cluster.getFileSystem();
+    try {
+      final Path dir = new Path("/testSetStoragePolicy");
+      final Path fooFile = new Path(dir, "foo");
+      final Path barDir = new Path(dir, "bar");
+      final Path barFile1= new Path(barDir, "f1");
+      final Path barFile2= new Path(barDir, "f2");
+      DFSTestUtil.createFile(fs, fooFile, FILE_LEN, REPLICATION, 0L);
+      DFSTestUtil.createFile(fs, barFile1, FILE_LEN, REPLICATION, 0L);
+      DFSTestUtil.createFile(fs, barFile2, FILE_LEN, REPLICATION, 0L);
+
+      final String invalidPolicyName = "INVALID-POLICY";
+      try {
+        fs.setStoragePolicy(fooFile, invalidPolicyName);
+        Assert.fail("Should throw a HadoopIllegalArgumentException");
+      } catch (RemoteException e) {
+        GenericTestUtils.assertExceptionContains(invalidPolicyName, e);
+      }
+
+      // check storage policy
+      HdfsFileStatus[] dirList = fs.getClient().listPaths(dir.toString(),
+          HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
+      HdfsFileStatus[] barList = fs.getClient().listPaths(barDir.toString(),
+          HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
+      checkDirectoryListing(dirList, ID_UNSPECIFIED, ID_UNSPECIFIED);
+      checkDirectoryListing(barList, ID_UNSPECIFIED, ID_UNSPECIFIED);
+
+      final Path invalidPath = new Path("/invalidPath");
+      try {
+        fs.setStoragePolicy(invalidPath, "WARM");
+        Assert.fail("Should throw a FileNotFoundException");
+      } catch (FileNotFoundException e) {
+        GenericTestUtils.assertExceptionContains(invalidPath.toString(), e);
+      }
+
+      fs.setStoragePolicy(fooFile, "COLD");
+      fs.setStoragePolicy(barDir, "WARM");
+      fs.setStoragePolicy(barFile2, "HOT");
+
+      dirList = fs.getClient().listPaths(dir.toString(),
+          HdfsFileStatus.EMPTY_NAME).getPartialListing();
+      barList = fs.getClient().listPaths(barDir.toString(),
+          HdfsFileStatus.EMPTY_NAME).getPartialListing();
+      checkDirectoryListing(dirList, WARM, COLD); // bar is warm, foo is cold
+      checkDirectoryListing(barList, WARM, HOT);
+
+      // restart namenode to make sure the editlog is correct
+      cluster.restartNameNode(true);
+      dirList = fs.getClient().listPaths(dir.toString(),
+          HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
+      barList = fs.getClient().listPaths(barDir.toString(),
+          HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
+      checkDirectoryListing(dirList, WARM, COLD); // bar is warm, foo is cold
+      checkDirectoryListing(barList, WARM, HOT);
+
+      // restart namenode with checkpoint to make sure the fsimage is correct
+      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.saveNamespace();
+      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      cluster.restartNameNode(true);
+      dirList = fs.getClient().listPaths(dir.toString(),
+          HdfsFileStatus.EMPTY_NAME).getPartialListing();
+      barList = fs.getClient().listPaths(barDir.toString(),
+          HdfsFileStatus.EMPTY_NAME).getPartialListing();
+      checkDirectoryListing(dirList, WARM, COLD); // bar is warm, foo is cold
+      checkDirectoryListing(barList, WARM, HOT);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testSetStoragePolicyWithSnapshot() throws Exception {
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    final DistributedFileSystem fs = cluster.getFileSystem();
+    try {
+      final Path dir = new Path("/testSetStoragePolicyWithSnapshot");
+      final Path fooDir = new Path(dir, "foo");
+      final Path fooFile1= new Path(fooDir, "f1");
+      final Path fooFile2= new Path(fooDir, "f2");
+      DFSTestUtil.createFile(fs, fooFile1, FILE_LEN, REPLICATION, 0L);
+      DFSTestUtil.createFile(fs, fooFile2, FILE_LEN, REPLICATION, 0L);
+
+      fs.setStoragePolicy(fooDir, "WARM");
+
+      HdfsFileStatus[] dirList = fs.getClient().listPaths(dir.toString(),
+          HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
+      checkDirectoryListing(dirList, WARM);
+      HdfsFileStatus[] fooList = fs.getClient().listPaths(fooDir.toString(),
+          HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
+      checkDirectoryListing(fooList, WARM, WARM);
+
+      // take snapshot
+      SnapshotTestHelper.createSnapshot(fs, dir, "s1");
+      // change the storage policy of fooFile1
+      fs.setStoragePolicy(fooFile1, "COLD");
+
+      fooList = fs.getClient().listPaths(fooDir.toString(),
+          HdfsFileStatus.EMPTY_NAME).getPartialListing();
+      checkDirectoryListing(fooList, COLD, WARM);
+
+      // check the policy for /dir/.snapshot/s1/foo/f1. Note we always return
+      // the latest storage policy for a file/directory.
+      Path s1f1 = SnapshotTestHelper.getSnapshotPath(dir, "s1", "foo/f1");
+      DirectoryListing f1Listing = fs.getClient().listPaths(s1f1.toString(),
+          HdfsFileStatus.EMPTY_NAME);
+      checkDirectoryListing(f1Listing.getPartialListing(), COLD);
+
+      // delete f1
+      fs.delete(fooFile1, true);
+      fooList = fs.getClient().listPaths(fooDir.toString(),
+          HdfsFileStatus.EMPTY_NAME).getPartialListing();
+      checkDirectoryListing(fooList, WARM);
+      // check the policy for /dir/.snapshot/s1/foo/f1 again after the deletion
+      checkDirectoryListing(fs.getClient().listPaths(s1f1.toString(),
+          HdfsFileStatus.EMPTY_NAME).getPartialListing(), COLD);
+
+      // change the storage policy of foo dir
+      fs.setStoragePolicy(fooDir, "HOT");
+      // /dir/foo is now hot
+      dirList = fs.getClient().listPaths(dir.toString(),
+          HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
+      checkDirectoryListing(dirList, HOT);
+      // /dir/foo/f2 is hot
+      fooList = fs.getClient().listPaths(fooDir.toString(),
+          HdfsFileStatus.EMPTY_NAME).getPartialListing();
+      checkDirectoryListing(fooList, HOT);
+
+      // check storage policy of snapshot path
+      Path s1 = SnapshotTestHelper.getSnapshotRoot(dir, "s1");
+      Path s1foo = SnapshotTestHelper.getSnapshotPath(dir, "s1", "foo");
+      checkDirectoryListing(fs.getClient().listPaths(s1.toString(),
+          HdfsFileStatus.EMPTY_NAME).getPartialListing(), HOT);
+      // /dir/.snapshot/.s1/foo/f1 and /dir/.snapshot/.s1/foo/f2 should still
+      // follow the latest
+      checkDirectoryListing(fs.getClient().listPaths(s1foo.toString(),
+          HdfsFileStatus.EMPTY_NAME).getPartialListing(), COLD, HOT);
+
+      // delete foo
+      fs.delete(fooDir, true);
+      checkDirectoryListing(fs.getClient().listPaths(s1.toString(),
+          HdfsFileStatus.EMPTY_NAME).getPartialListing(), HOT);
+      checkDirectoryListing(fs.getClient().listPaths(s1foo.toString(),
+          HdfsFileStatus.EMPTY_NAME).getPartialListing(), COLD, HOT);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  private static StorageType[][] genStorageTypes(int numDataNodes) {
+    StorageType[][] types = new StorageType[numDataNodes][];
+    for (int i = 0; i < types.length; i++) {
+      types[i] = new StorageType[]{StorageType.DISK, StorageType.ARCHIVE};
+    }
+    return types;
+  }
+
+  private void checkLocatedBlocks(HdfsLocatedFileStatus status, int blockNum,
+                                  int replicaNum, StorageType... types) {
+    List<StorageType> typeList = Lists.newArrayList();
+    Collections.addAll(typeList, types);
+    LocatedBlocks lbs = status.getBlockLocations();
+    Assert.assertEquals(blockNum, lbs.getLocatedBlocks().size());
+    for (LocatedBlock lb : lbs.getLocatedBlocks()) {
+      Assert.assertEquals(replicaNum, lb.getStorageTypes().length);
+      for (StorageType type : lb.getStorageTypes()) {
+        Assert.assertTrue(typeList.remove(type));
+      }
+    }
+    Assert.assertTrue(typeList.isEmpty());
+  }
+
+  private void testIncreaseFileRep(String policyName, byte policyId,
+                                   StorageType[] before,
+                                   StorageType[] after) throws Exception {
+    final int numDataNodes = 5;
+    final StorageType[][] types = genStorageTypes(numDataNodes);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numDataNodes).storageTypes(types).build();
+    cluster.waitActive();
+    final DistributedFileSystem fs = cluster.getFileSystem();
+    try {
+      final Path dir = new Path("/test");
+      fs.mkdirs(dir);
+      fs.setStoragePolicy(dir, policyName);
+
+      final Path foo = new Path(dir, "foo");
+      DFSTestUtil.createFile(fs, foo, FILE_LEN, REPLICATION, 0L);
+
+      // the storage policy of foo should be WARM, and the replicas
+      // should be stored in DISK and ARCHIE
+      HdfsFileStatus[] status = fs.getClient().listPaths(foo.toString(),
+          HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
+      checkDirectoryListing(status, policyId);
+      HdfsLocatedFileStatus fooStatus = (HdfsLocatedFileStatus) status[0];
+      checkLocatedBlocks(fooStatus, 1, 3, before);
+
+      // change the replication factor to 5
+      fs.setReplication(foo, (short) numDataNodes);
+      Thread.sleep(1000);
+      for (DataNode dn : cluster.getDataNodes()) {
+        DataNodeTestUtils.triggerHeartbeat(dn);
+      }
+      Thread.sleep(1000);
+      status = fs.getClient().listPaths(foo.toString(),
+          HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
+      checkDirectoryListing(status, policyId);
+      fooStatus = (HdfsLocatedFileStatus) status[0];
+      checkLocatedBlocks(fooStatus, 1, 5, after);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Consider a File with Hot storage policy. Increase replication factor of
+   * that file from 3 to 5. Make sure all replications are created in DISKS.
+   */
+  @Test
+  public void testIncreaseHotFileRep() throws Exception {
+    testIncreaseFileRep("HOT", HOT, new StorageType[]{StorageType.DISK,
+            StorageType.DISK, StorageType.DISK},
+        new StorageType[]{StorageType.DISK, StorageType.DISK,
+            StorageType.DISK, StorageType.DISK, StorageType.DISK});
+  }
+
+  /**
+   * Consider a File with Warm temperature. Increase replication factor of
+   * that file from 3 to 5. Make sure all replicas are created in DISKS
+   * and ARCHIVE.
+   */
+  @Test
+  public void testIncreaseWarmRep() throws Exception {
+    testIncreaseFileRep("WARM", WARM, new StorageType[]{StorageType.DISK,
+        StorageType.ARCHIVE, StorageType.ARCHIVE},
+        new StorageType[]{StorageType.DISK, StorageType.ARCHIVE,
+            StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE});
+  }
+
+  /**
+   * Consider a File with Cold temperature. Increase replication factor of
+   * that file from 3 to 5. Make sure all replicas are created in ARCHIVE.
+   */
+  @Test
+  public void testIncreaseColdRep() throws Exception {
+    testIncreaseFileRep("COLD", COLD, new StorageType[]{StorageType.ARCHIVE,
+            StorageType.ARCHIVE, StorageType.ARCHIVE},
+        new StorageType[]{StorageType.ARCHIVE, StorageType.ARCHIVE,
+            StorageType.ARCHIVE, StorageType.ARCHIVE, StorageType.ARCHIVE});
+  }
+
+  @Test
+  public void testChooseTargetWithTopology() throws Exception {
+    BlockStoragePolicy policy1 = new BlockStoragePolicy((byte) 9, "TEST1",
+        new StorageType[]{StorageType.SSD, StorageType.DISK,
+            StorageType.ARCHIVE}, new StorageType[]{}, new StorageType[]{});
+    BlockStoragePolicy policy2 = new BlockStoragePolicy((byte) 11, "TEST2",
+        new StorageType[]{StorageType.DISK, StorageType.SSD,
+            StorageType.ARCHIVE}, new StorageType[]{}, new StorageType[]{});
+
+    final String[] racks = {"/d1/r1", "/d1/r2", "/d1/r2"};
+    final String[] hosts = {"host1", "host2", "host3"};
+    final StorageType[] types = {StorageType.DISK, StorageType.SSD,
+        StorageType.ARCHIVE};
+
+    final DatanodeStorageInfo[] storages = DFSTestUtil
+        .createDatanodeStorageInfos(3, racks, hosts, types);
+    final DatanodeDescriptor[] dataNodes = DFSTestUtil
+        .toDatanodeDescriptor(storages);
+
+    FileSystem.setDefaultUri(conf, "hdfs://localhost:0");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+    File baseDir = PathUtils.getTestDir(TestReplicationPolicy.class);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        new File(baseDir, "name").getPath());
+    DFSTestUtil.formatNameNode(conf);
+    NameNode namenode = new NameNode(conf);
+
+    final BlockManager bm = namenode.getNamesystem().getBlockManager();
+    BlockPlacementPolicy replicator = bm.getBlockPlacementPolicy();
+    NetworkTopology cluster = bm.getDatanodeManager().getNetworkTopology();
+    for (DatanodeDescriptor datanode : dataNodes) {
+      cluster.add(datanode);
+    }
+
+    DatanodeStorageInfo[] targets = replicator.chooseTarget("/foo", 3,
+        dataNodes[0], Collections.<DatanodeStorageInfo>emptyList(), false,
+        new HashSet<Node>(), 0, policy1);
+    System.out.println(Arrays.asList(targets));
+    Assert.assertEquals(3, targets.length);
+    targets = replicator.chooseTarget("/foo", 3,
+        dataNodes[0], Collections.<DatanodeStorageInfo>emptyList(), false,
+        new HashSet<Node>(), 0, policy2);
+    System.out.println(Arrays.asList(targets));
+    Assert.assertEquals(3, targets.length);
+  }
+}

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

@@ -255,12 +255,12 @@ public class TestDFSClientRetries {
     Mockito.doReturn(
             new HdfsFileStatus(0, false, 1, 1024, false, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010, 0, null)).when(mockNN).getFileInfo(anyString());
+                1010, 0, null, (byte) 0)).when(mockNN).getFileInfo(anyString());
     
     Mockito.doReturn(
             new HdfsFileStatus(0, false, 1, 1024, false, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010, 0, null))
+                1010, 0, null, (byte) 0))
         .when(mockNN)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),

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

@@ -64,7 +64,7 @@ public class TestDFSInotifyEventInputStream {
    */
   @Test
   public void testOpcodeCount() {
-    Assert.assertTrue(FSEditLogOpCodes.values().length == 46);
+    Assert.assertTrue(FSEditLogOpCodes.values().length == 47);
   }
 
 

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

@@ -940,7 +940,7 @@ public class TestEncryptionZones {
     Future<?> future = executor.submit(new CreateFileTask(fsWrapper, file));
 
     // Flip-flop between two EZs to repeatedly fail
-    for (int i=0; i<10; i++) {
+    for (int i=0; i<DFSOutputStream.CREATE_RETRY_COUNT+1; i++) {
       injector.ready.await();
       fsWrapper.delete(zone1, true);
       fsWrapper.mkdir(zone1, FsPermission.getDirDefault(), true);

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java

@@ -17,12 +17,14 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyList;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Matchers.anyShort;
 import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyObject;
+import static org.mockito.Matchers.anyShort;
+import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
@@ -38,7 +40,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
@@ -342,12 +343,12 @@ public class TestLease {
     Mockito.doReturn(
         new HdfsFileStatus(0, false, 1, 1024, false, 0, 0, new FsPermission(
             (short) 777), "owner", "group", new byte[0], new byte[0],
-            1010, 0, null)).when(mcp).getFileInfo(anyString());
+            1010, 0, null, (byte) 0)).when(mcp).getFileInfo(anyString());
     Mockito
         .doReturn(
             new HdfsFileStatus(0, false, 1, 1024, false, 0, 0, new FsPermission(
                 (short) 777), "owner", "group", new byte[0], new byte[0],
-                1010, 0, null))
+                1010, 0, null, (byte) 0))
         .when(mcp)
         .create(anyString(), (FsPermission) anyObject(), anyString(),
             (EnumSetWritable<CreateFlag>) anyObject(), anyBoolean(),

+ 46 - 106
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java

@@ -18,22 +18,11 @@
 
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.*;
-
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.PrintStream;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.tools.DFSAdmin;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.Tool;
-
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -89,136 +78,87 @@ public class TestSnapshotCommands {
     }
   }
 
-  private void toolRun(Tool tool, String cmd, int retcode, String contain)
-      throws Exception {
-    String [] cmds = StringUtils.split(cmd, ' ');
-    System.out.flush();
-    System.err.flush();
-    PrintStream origOut = System.out;
-    PrintStream origErr = System.err;
-    String output = null;
-    int ret = 0;
-    try {
-      ByteArrayOutputStream bs = new ByteArrayOutputStream(1024);
-      PrintStream out = new PrintStream(bs);
-      System.setOut(out);
-      System.setErr(out);
-      ret = tool.run(cmds);
-      System.out.flush();
-      System.err.flush();
-      out.close();
-      output = bs.toString();
-    } finally {
-      System.setOut(origOut);
-      System.setErr(origErr);
-    }
-    System.out.println("Output for command: " + cmd + " retcode: " + ret);
-    if (output != null) {
-      System.out.println(output);
-    }
-    assertEquals(retcode, ret);
-    if (contain != null) {
-      assertTrue(output.contains(contain));
-    }
-  }
-
-  private void FsShellRun(String cmd, int retcode, String contain)
-      throws Exception {
-    FsShell shell = new FsShell(new Configuration(conf));
-    toolRun(shell, cmd, retcode, contain);
-  }  
-
-  private void DFSAdminRun(String cmd, int retcode, String contain)
-      throws Exception {
-    DFSAdmin admin = new DFSAdmin(new Configuration(conf));
-    toolRun(admin, cmd, retcode, contain);
-  }
-
-  private void FsShellRun(String cmd) throws Exception {
-    FsShellRun(cmd, 0, null);
-  }
-
   @Test
   public void testAllowSnapshot() throws Exception {
     // Idempotent test
-    DFSAdminRun("-allowSnapshot /sub1", 0, "Allowing snaphot on /sub1 succeeded");
+    DFSTestUtil.DFSAdminRun("-allowSnapshot /sub1", 0, "Allowing snaphot on /sub1 succeeded", conf);
     // allow normal dir success 
-    FsShellRun("-mkdir /sub2");
-    DFSAdminRun("-allowSnapshot /sub2", 0, "Allowing snaphot on /sub2 succeeded");
+    DFSTestUtil.FsShellRun("-mkdir /sub2", conf);
+    DFSTestUtil.DFSAdminRun("-allowSnapshot /sub2", 0, "Allowing snaphot on /sub2 succeeded", conf);
     // allow non-exists dir failed
-    DFSAdminRun("-allowSnapshot /sub3", -1, null);
+    DFSTestUtil.DFSAdminRun("-allowSnapshot /sub3", -1, null, conf);
   }
 
   @Test
   public void testCreateSnapshot() throws Exception {
     // test createSnapshot
-    FsShellRun("-createSnapshot /sub1 sn0", 0, "Created snapshot /sub1/.snapshot/sn0");
-    FsShellRun("-createSnapshot /sub1 sn0", 1, "there is already a snapshot with the same name \"sn0\"");
-    FsShellRun("-rmr /sub1/sub1sub2");
-    FsShellRun("-mkdir /sub1/sub1sub3");
-    FsShellRun("-createSnapshot /sub1 sn1", 0, "Created snapshot /sub1/.snapshot/sn1");
+    DFSTestUtil.FsShellRun("-createSnapshot /sub1 sn0", 0, "Created snapshot /sub1/.snapshot/sn0", conf);
+    DFSTestUtil.FsShellRun("-createSnapshot /sub1 sn0", 1, "there is already a snapshot with the same name \"sn0\"", conf);
+    DFSTestUtil.FsShellRun("-rmr /sub1/sub1sub2", conf);
+    DFSTestUtil.FsShellRun("-mkdir /sub1/sub1sub3", conf);
+    DFSTestUtil.FsShellRun("-createSnapshot /sub1 sn1", 0, "Created snapshot /sub1/.snapshot/sn1", conf);
     // check snapshot contents
-    FsShellRun("-ls /sub1", 0, "/sub1/sub1sub1");
-    FsShellRun("-ls /sub1", 0, "/sub1/sub1sub3");
-    FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn0");
-    FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn1");
-    FsShellRun("-ls /sub1/.snapshot/sn0", 0, "/sub1/.snapshot/sn0/sub1sub1");
-    FsShellRun("-ls /sub1/.snapshot/sn0", 0, "/sub1/.snapshot/sn0/sub1sub2");
-    FsShellRun("-ls /sub1/.snapshot/sn1", 0, "/sub1/.snapshot/sn1/sub1sub1");
-    FsShellRun("-ls /sub1/.snapshot/sn1", 0, "/sub1/.snapshot/sn1/sub1sub3");
+    DFSTestUtil.FsShellRun("-ls /sub1", 0, "/sub1/sub1sub1", conf);
+    DFSTestUtil.FsShellRun("-ls /sub1", 0, "/sub1/sub1sub3", conf);
+    DFSTestUtil.FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn0", conf);
+    DFSTestUtil.FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn1", conf);
+    DFSTestUtil.FsShellRun("-ls /sub1/.snapshot/sn0", 0, "/sub1/.snapshot/sn0/sub1sub1", conf);
+    DFSTestUtil.FsShellRun("-ls /sub1/.snapshot/sn0", 0, "/sub1/.snapshot/sn0/sub1sub2", conf);
+    DFSTestUtil.FsShellRun("-ls /sub1/.snapshot/sn1", 0, "/sub1/.snapshot/sn1/sub1sub1", conf);
+    DFSTestUtil.FsShellRun("-ls /sub1/.snapshot/sn1", 0, "/sub1/.snapshot/sn1/sub1sub3", conf);
   }
 
   @Test
   public void testMkdirUsingReservedName() throws Exception {
     // test can not create dir with reserved name: .snapshot
-    FsShellRun("-ls /");
-    FsShellRun("-mkdir /.snapshot", 1, "File exists");
-    FsShellRun("-mkdir /sub1/.snapshot", 1, "File exists");
+    DFSTestUtil.FsShellRun("-ls /", conf);
+    DFSTestUtil.FsShellRun("-mkdir /.snapshot", 1, "File exists", conf);
+    DFSTestUtil.FsShellRun("-mkdir /sub1/.snapshot", 1, "File exists", conf);
     // mkdir -p ignore reserved name check if dir already exists
-    FsShellRun("-mkdir -p /sub1/.snapshot");
-    FsShellRun("-mkdir -p /sub1/sub1sub1/.snapshot", 1, "mkdir: \".snapshot\" is a reserved name.");
+    DFSTestUtil.FsShellRun("-mkdir -p /sub1/.snapshot", conf);
+    DFSTestUtil.FsShellRun("-mkdir -p /sub1/sub1sub1/.snapshot", 1, "mkdir: \".snapshot\" is a reserved name.", conf);
   }
 
   @Test
   public void testRenameSnapshot() throws Exception {
-    FsShellRun("-createSnapshot /sub1 sn.orig");
-    FsShellRun("-renameSnapshot /sub1 sn.orig sn.rename");
-    FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn.rename");
-    FsShellRun("-ls /sub1/.snapshot/sn.rename", 0, "/sub1/.snapshot/sn.rename/sub1sub1");
-    FsShellRun("-ls /sub1/.snapshot/sn.rename", 0, "/sub1/.snapshot/sn.rename/sub1sub2");
+    DFSTestUtil.FsShellRun("-createSnapshot /sub1 sn.orig", conf);
+    DFSTestUtil.FsShellRun("-renameSnapshot /sub1 sn.orig sn.rename", conf);
+    DFSTestUtil.FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn.rename", conf);
+    DFSTestUtil.FsShellRun("-ls /sub1/.snapshot/sn.rename", 0, "/sub1/.snapshot/sn.rename/sub1sub1", conf);
+    DFSTestUtil.FsShellRun("-ls /sub1/.snapshot/sn.rename", 0, "/sub1/.snapshot/sn.rename/sub1sub2", conf);
 
     //try renaming from a non-existing snapshot
-    FsShellRun("-renameSnapshot /sub1 sn.nonexist sn.rename", 1,
-        "renameSnapshot: The snapshot sn.nonexist does not exist for directory /sub1");
+    DFSTestUtil.FsShellRun("-renameSnapshot /sub1 sn.nonexist sn.rename", 1,
+        "renameSnapshot: The snapshot sn.nonexist does not exist for directory /sub1", conf);
 
     //try renaming to existing snapshots
-    FsShellRun("-createSnapshot /sub1 sn.new");
-    FsShellRun("-renameSnapshot /sub1 sn.new sn.rename", 1,
-        "renameSnapshot: The snapshot sn.rename already exists for directory /sub1");
-    FsShellRun("-renameSnapshot /sub1 sn.rename sn.new", 1,
-        "renameSnapshot: The snapshot sn.new already exists for directory /sub1");
+    DFSTestUtil.FsShellRun("-createSnapshot /sub1 sn.new", conf);
+    DFSTestUtil.FsShellRun("-renameSnapshot /sub1 sn.new sn.rename", 1,
+        "renameSnapshot: The snapshot sn.rename already exists for directory /sub1", conf);
+    DFSTestUtil.FsShellRun("-renameSnapshot /sub1 sn.rename sn.new", 1,
+        "renameSnapshot: The snapshot sn.new already exists for directory /sub1", conf);
   }
 
   @Test
   public void testDeleteSnapshot() throws Exception {
-    FsShellRun("-createSnapshot /sub1 sn1");
-    FsShellRun("-deleteSnapshot /sub1 sn1");
-    FsShellRun("-deleteSnapshot /sub1 sn1", 1,
-        "deleteSnapshot: Cannot delete snapshot sn1 from path /sub1: the snapshot does not exist.");
+    DFSTestUtil.FsShellRun("-createSnapshot /sub1 sn1", conf);
+    DFSTestUtil.FsShellRun("-deleteSnapshot /sub1 sn1", conf);
+    DFSTestUtil.FsShellRun("-deleteSnapshot /sub1 sn1", 1,
+        "deleteSnapshot: Cannot delete snapshot sn1 from path /sub1: the snapshot does not exist.", conf);
   }
 
   @Test
   public void testDisallowSnapshot() throws Exception {
-    FsShellRun("-createSnapshot /sub1 sn1");
+    DFSTestUtil.FsShellRun("-createSnapshot /sub1 sn1", conf);
     // cannot delete snapshotable dir
-    FsShellRun("-rmr /sub1", 1, "The directory /sub1 cannot be deleted since /sub1 is snapshottable and already has snapshots");
-    DFSAdminRun("-disallowSnapshot /sub1", -1,
-        "disallowSnapshot: The directory /sub1 has snapshot(s). Please redo the operation after removing all the snapshots.");
-    FsShellRun("-deleteSnapshot /sub1 sn1");
-    DFSAdminRun("-disallowSnapshot /sub1", 0, "Disallowing snaphot on /sub1 succeeded");
+    DFSTestUtil.FsShellRun("-rmr /sub1", 1, "The directory /sub1 cannot be deleted since /sub1 is snapshottable and already has snapshots", conf);
+    DFSTestUtil.DFSAdminRun("-disallowSnapshot /sub1", -1,
+        "disallowSnapshot: The directory /sub1 has snapshot(s). Please redo the operation after removing all the snapshots.", conf);
+    DFSTestUtil.FsShellRun("-deleteSnapshot /sub1 sn1", conf);
+    DFSTestUtil.DFSAdminRun("-disallowSnapshot /sub1", 0, "Disallowing snaphot on /sub1 succeeded", conf);
     // Idempotent test
-    DFSAdminRun("-disallowSnapshot /sub1", 0, "Disallowing snaphot on /sub1 succeeded");
+    DFSTestUtil.DFSAdminRun("-disallowSnapshot /sub1", 0, "Disallowing snaphot on /sub1 succeeded", conf);
     // now it can be deleted
-    FsShellRun("-rmr /sub1");
+    DFSTestUtil.FsShellRun("-rmr /sub1", conf);
   }
 }

+ 81 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestStoragePolicyCommands.java

@@ -0,0 +1,81 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test storage policy related DFSAdmin commands
+ */
+public class TestStoragePolicyCommands {
+  private static final short REPL = 1;
+  private static final int SIZE = 128;
+
+  private static Configuration conf;
+  private static MiniDFSCluster cluster;
+  private static DistributedFileSystem fs;
+  
+  @Before
+  public void clusterSetUp() throws IOException {
+    conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL).build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+  }
+
+  @After
+  public void clusterShutdown() throws IOException{
+    if(fs != null){
+      fs.close();
+    }
+    if(cluster != null){
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testSetAndGetStoragePolicy() throws Exception {
+    final Path foo = new Path("/foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(fs, bar, SIZE, REPL, 0);
+
+    DFSTestUtil.DFSAdminRun("-setStoragePolicy /foo WARM", 0,
+        "Set storage policy WARM on " + foo.toString(), conf);
+    DFSTestUtil.DFSAdminRun("-setStoragePolicy /foo/bar COLD", 0,
+        "Set storage policy COLD on " + bar.toString(), conf);
+    DFSTestUtil.DFSAdminRun("-setStoragePolicy /fooz WARM", -1,
+        "File/Directory does not exist: /fooz", conf);
+
+    final BlockStoragePolicy.Suite suite = BlockStoragePolicy
+        .readBlockStorageSuite(conf);
+    final BlockStoragePolicy warm = suite.getPolicy("WARM");
+    final BlockStoragePolicy cold = suite.getPolicy("COLD");
+    DFSTestUtil.DFSAdminRun("-getStoragePolicy /foo", 0,
+        "The storage policy of " + foo.toString() + ":\n" + warm, conf);
+    DFSTestUtil.DFSAdminRun("-getStoragePolicy /foo/bar", 0,
+        "The storage policy of " + bar.toString() + ":\n" + cold, conf);
+    DFSTestUtil.DFSAdminRun("-getStoragePolicy /fooz", -1,
+        "File/Directory does not exist: /fooz", conf);
+  }
+}

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java

@@ -89,7 +89,14 @@ public class TestBalancer {
   private static final Random r = new Random();
 
   static {
+    initTestSetup();
+  }
+
+  public static void initTestSetup() {
     Dispatcher.setBlockMoveWaitTime(1000L) ;
+
+    // do not create id file since it occupies the disk space
+    NameNodeConnector.setWrite2IdFile(false);
   }
 
   static void initConf(Configuration conf) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java

@@ -44,7 +44,7 @@ public class TestBalancerWithHANameNodes {
   ClientProtocol client;
 
   static {
-    Dispatcher.setBlockMoveWaitTime(1000L);
+    TestBalancer.initTestSetup();
   }
 
   /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java

@@ -73,7 +73,7 @@ public class TestBalancerWithMultipleNameNodes {
   private static final Random RANDOM = new Random();
 
   static {
-    Dispatcher.setBlockMoveWaitTime(1000L) ;
+    TestBalancer.initTestSetup();
   }
 
   /** Common objects used in various methods. */

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithNodeGroup.java

@@ -75,7 +75,7 @@ public class TestBalancerWithNodeGroup {
   static final int DEFAULT_BLOCK_SIZE = 100;
 
   static {
-    Dispatcher.setBlockMoveWaitTime(1000L) ;
+    TestBalancer.initTestSetup();
   }
 
   static Configuration createConf() {

+ 18 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
@@ -52,6 +53,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetworkTopology;
+import org.junit.Assert;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Before;
 import org.junit.Test;
@@ -607,7 +609,6 @@ public class TestBlockManager {
     assertEquals(1, ds.getBlockReportCount());
   }
   
-  
   /**
    * Tests that a namenode doesn't choose a datanode with full disks to 
    * store blocks.
@@ -654,5 +655,20 @@ public class TestBlockManager {
       cluster.shutdown();
     }
   }
-}
 
+  @Test
+  public void testUseDelHint() {
+    DatanodeStorageInfo delHint = new DatanodeStorageInfo(
+        DFSTestUtil.getLocalDatanodeDescriptor(), new DatanodeStorage("id"));
+    List<DatanodeStorageInfo> moreThan1Racks = Arrays.asList(delHint);
+    List<StorageType> excessTypes = new ArrayList<StorageType>();
+
+    excessTypes.add(StorageType.DEFAULT);
+    Assert.assertTrue(BlockManager.useDelHint(true, delHint, null,
+        moreThan1Racks, excessTypes));
+    excessTypes.remove(0);
+    excessTypes.add(StorageType.SSD);
+    Assert.assertFalse(BlockManager.useDelHint(true, delHint, null,
+        moreThan1Racks, excessTypes));
+  }
+}

+ 19 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
@@ -47,13 +48,14 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.LogVerificationAppender;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager.StatefulBlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager.StatefulBlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
@@ -228,7 +230,7 @@ public class TestReplicationPolicy {
       List<DatanodeStorageInfo> chosenNodes,
       Set<Node> excludedNodes) {
     return replicator.chooseTarget(filename, numOfReplicas, writer, chosenNodes,
-        false, excludedNodes, BLOCK_SIZE, StorageType.DEFAULT);
+        false, excludedNodes, BLOCK_SIZE, TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
   }
 
   /**
@@ -295,7 +297,7 @@ public class TestReplicationPolicy {
     excludedNodes.add(dataNodes[1]); 
     chosenNodes.add(storages[2]);
     targets = replicator.chooseTarget(filename, 1, dataNodes[0], chosenNodes, true,
-        excludedNodes, BLOCK_SIZE, StorageType.DEFAULT);
+        excludedNodes, BLOCK_SIZE, TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
     System.out.println("targets=" + Arrays.asList(targets));
     assertEquals(2, targets.length);
     //make sure that the chosen node is in the target.
@@ -630,7 +632,7 @@ public class TestReplicationPolicy {
           .getNamesystem().getBlockManager().getBlockPlacementPolicy();
       DatanodeStorageInfo[] targets = replicator.chooseTarget(filename, 3,
           staleNodeInfo, new ArrayList<DatanodeStorageInfo>(), false, null,
-          BLOCK_SIZE, StorageType.DEFAULT);
+          BLOCK_SIZE, TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
 
       assertEquals(targets.length, 3);
       assertFalse(isOnSameRack(targets[0], staleNodeInfo));
@@ -656,7 +658,7 @@ public class TestReplicationPolicy {
       // Call chooseTarget
       targets = replicator.chooseTarget(filename, 3, staleNodeInfo,
           new ArrayList<DatanodeStorageInfo>(), false, null, BLOCK_SIZE,
-          StorageType.DEFAULT);
+          TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
       assertEquals(targets.length, 3);
       assertTrue(isOnSameRack(targets[0], staleNodeInfo));
       
@@ -933,8 +935,16 @@ public class TestReplicationPolicy {
     // replica nodes, while storages[2] and dataNodes[5] are in second set.
     assertEquals(2, first.size());
     assertEquals(2, second.size());
+    List<StorageType> excessTypes = new ArrayList<StorageType>();
+    {
+      // test returning null
+      excessTypes.add(StorageType.SSD);
+      assertNull(replicator.chooseReplicaToDelete(
+          null, null, (short)3, first, second, excessTypes));
+    }
+    excessTypes.add(StorageType.DEFAULT);
     DatanodeStorageInfo chosen = replicator.chooseReplicaToDelete(
-        null, null, (short)3, first, second);
+        null, null, (short)3, first, second, excessTypes);
     // Within first set, storages[1] with less free space
     assertEquals(chosen, storages[1]);
 
@@ -942,11 +952,12 @@ public class TestReplicationPolicy {
     assertEquals(0, first.size());
     assertEquals(3, second.size());
     // Within second set, storages[5] with less free space
+    excessTypes.add(StorageType.DEFAULT);
     chosen = replicator.chooseReplicaToDelete(
-        null, null, (short)2, first, second);
+        null, null, (short)2, first, second, excessTypes);
     assertEquals(chosen, storages[5]);
   }
-  
+
   /**
    * This testcase tests whether the default value returned by
    * DFSUtil.getInvalidateWorkPctPerIteration() is positive, 

+ 14 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyConsiderLoad.java

@@ -17,17 +17,28 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
-import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -37,17 +48,6 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 public class TestReplicationPolicyConsiderLoad {
 
   private static NameNode namenode;
@@ -146,7 +146,7 @@ public class TestReplicationPolicyConsiderLoad {
       DatanodeStorageInfo[] targets = namenode.getNamesystem().getBlockManager()
           .getBlockPlacementPolicy().chooseTarget("testFile.txt", 3,
               dataNodes[0], new ArrayList<DatanodeStorageInfo>(), false, null,
-              1024, StorageType.DEFAULT);
+              1024, TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
 
       assertEquals(3, targets.length);
       Set<DatanodeStorageInfo> targetSet = new HashSet<DatanodeStorageInfo>(

+ 11 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.net.NetworkTopology;
@@ -258,7 +259,7 @@ public class TestReplicationPolicyWithNodeGroup {
       List<DatanodeStorageInfo> chosenNodes,
       Set<Node> excludedNodes) {
     return replicator.chooseTarget(filename, numOfReplicas, writer, chosenNodes,
-        false, excludedNodes, BLOCK_SIZE, StorageType.DEFAULT);
+        false, excludedNodes, BLOCK_SIZE, TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
   }
 
   /**
@@ -340,7 +341,7 @@ public class TestReplicationPolicyWithNodeGroup {
     Set<Node> excludedNodes = new HashSet<Node>();
     excludedNodes.add(dataNodes[1]); 
     targets = repl.chooseTarget(filename, 4, dataNodes[0], chosenNodes, false, 
-        excludedNodes, BLOCK_SIZE, StorageType.DEFAULT);
+        excludedNodes, BLOCK_SIZE, TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
     assertEquals(targets.length, 4);
     assertEquals(storages[0], targets[0]);
 
@@ -358,7 +359,7 @@ public class TestReplicationPolicyWithNodeGroup {
     excludedNodes.add(dataNodes[1]); 
     chosenNodes.add(storages[2]);
     targets = repl.chooseTarget(filename, 1, dataNodes[0], chosenNodes, true,
-        excludedNodes, BLOCK_SIZE, StorageType.DEFAULT);
+        excludedNodes, BLOCK_SIZE, TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);
     System.out.println("targets=" + Arrays.asList(targets));
     assertEquals(2, targets.length);
     //make sure that the chosen node is in the target.
@@ -612,8 +613,10 @@ public class TestReplicationPolicyWithNodeGroup {
         replicaList, rackMap, first, second);
     assertEquals(3, first.size());
     assertEquals(1, second.size());
+    List<StorageType> excessTypes = new ArrayList<StorageType>();
+    excessTypes.add(StorageType.DEFAULT);
     DatanodeStorageInfo chosen = replicator.chooseReplicaToDelete(
-        null, null, (short)3, first, second);
+        null, null, (short)3, first, second, excessTypes);
     // Within first set {dataNodes[0], dataNodes[1], dataNodes[2]}, 
     // dataNodes[0] and dataNodes[1] are in the same nodegroup, 
     // but dataNodes[1] is chosen as less free space
@@ -624,16 +627,18 @@ public class TestReplicationPolicyWithNodeGroup {
     assertEquals(1, second.size());
     // Within first set {dataNodes[0], dataNodes[2]}, dataNodes[2] is chosen
     // as less free space
+    excessTypes.add(StorageType.DEFAULT);
     chosen = replicator.chooseReplicaToDelete(
-        null, null, (short)2, first, second);
+        null, null, (short)2, first, second, excessTypes);
     assertEquals(chosen, storages[2]);
 
     replicator.adjustSetsWithChosenReplica(rackMap, first, second, chosen);
     assertEquals(0, first.size());
     assertEquals(2, second.size());
     // Within second set, dataNodes[5] with less free space
+    excessTypes.add(StorageType.DEFAULT);
     chosen = replicator.chooseReplicaToDelete(
-        null, null, (short)1, first, second);
+        null, null, (short)1, first, second, excessTypes);
     assertEquals(chosen, storages[5]);
   }
   

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

@@ -1099,7 +1099,8 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   @Override
-  public void addVolumes(Collection<StorageLocation> volumes) {
+  public List<StorageLocation> addVolumes(List<StorageLocation> volumes,
+      final Collection<String> bpids) {
     throw new UnsupportedOperationException();
   }
 

+ 423 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java

@@ -0,0 +1,423 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.ReconfigurationException;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.BlockMissingException;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestDataNodeHotSwapVolumes {
+  private static final int BLOCK_SIZE = 512;
+  private MiniDFSCluster cluster;
+
+  @After
+  public void tearDown() {
+    shutdown();
+  }
+
+  private void startDFSCluster(int numNameNodes, int numDataNodes)
+      throws IOException {
+    shutdown();
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+
+    /*
+     * Lower the DN heartbeat, DF rate, and recheck interval to one second
+     * so state about failures and datanode death propagates faster.
+     */
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_DF_INTERVAL_KEY, 1000);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
+        1000);
+
+    MiniDFSNNTopology nnTopology =
+        MiniDFSNNTopology.simpleFederatedTopology(numNameNodes);
+
+    cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(nnTopology)
+        .numDataNodes(numDataNodes)
+        .build();
+    cluster.waitActive();
+  }
+
+  private void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+
+  private void createFile(Path path, int numBlocks)
+      throws IOException, InterruptedException, TimeoutException {
+    final short replicateFactor = 1;
+    createFile(path, numBlocks, replicateFactor);
+  }
+
+  private void createFile(Path path, int numBlocks, short replicateFactor)
+      throws IOException, InterruptedException, TimeoutException {
+    createFile(0, path, numBlocks, replicateFactor);
+  }
+
+  private void createFile(int fsIdx, Path path, int numBlocks)
+      throws IOException, InterruptedException, TimeoutException {
+    final short replicateFactor = 1;
+    createFile(fsIdx, path, numBlocks, replicateFactor);
+  }
+
+  private void createFile(int fsIdx, Path path, int numBlocks,
+      short replicateFactor)
+      throws IOException, TimeoutException, InterruptedException {
+    final int seed = 0;
+    final DistributedFileSystem fs = cluster.getFileSystem(fsIdx);
+    DFSTestUtil.createFile(fs, path, BLOCK_SIZE * numBlocks,
+        replicateFactor, seed);
+    DFSTestUtil.waitReplication(fs, path, replicateFactor);
+  }
+
+  /**
+   * Verify whether a file has enough content.
+   */
+  private static void verifyFileLength(FileSystem fs, Path path, int numBlocks)
+      throws IOException {
+    FileStatus status = fs.getFileStatus(path);
+    assertEquals(numBlocks * BLOCK_SIZE, status.getLen());
+  }
+
+  /** Return the number of replicas for a given block in the file. */
+  private static int getNumReplicas(FileSystem fs, Path file,
+      int blockIdx) throws IOException {
+    BlockLocation locs[] = fs.getFileBlockLocations(file, 0, Long.MAX_VALUE);
+    return locs.length < blockIdx + 1 ? 0 : locs[blockIdx].getNames().length;
+  }
+
+  /**
+   * Wait the block to have the exact number of replicas as expected.
+   */
+  private static void waitReplication(FileSystem fs, Path file, int blockIdx,
+      int numReplicas)
+      throws IOException, TimeoutException, InterruptedException {
+    int attempts = 50;  // Wait 5 seconds.
+    while (attempts > 0) {
+      if (getNumReplicas(fs, file, blockIdx) == numReplicas) {
+        return;
+      }
+      Thread.sleep(100);
+      attempts--;
+    }
+    throw new TimeoutException("Timed out waiting the " + blockIdx + "-th block"
+        + " of " + file + " to have " + numReplicas + " replicas.");
+  }
+
+  /** Parses data dirs from DataNode's configuration. */
+  private static Collection<String> getDataDirs(DataNode datanode) {
+    return datanode.getConf().getTrimmedStringCollection(
+        DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
+  }
+
+  @Test
+  public void testParseChangedVolumes() throws IOException {
+    startDFSCluster(1, 1);
+    DataNode dn = cluster.getDataNodes().get(0);
+    Configuration conf = dn.getConf();
+
+    String oldPaths = conf.get(DFS_DATANODE_DATA_DIR_KEY);
+    List<StorageLocation> oldLocations = new ArrayList<StorageLocation>();
+    for (String path : oldPaths.split(",")) {
+      oldLocations.add(StorageLocation.parse(path));
+    }
+    assertFalse(oldLocations.isEmpty());
+
+    String newPaths = "/foo/path1,/foo/path2";
+    conf.set(DFS_DATANODE_DATA_DIR_KEY, newPaths);
+
+    DataNode.ChangedVolumes changedVolumes =dn.parseChangedVolumes();
+    List<StorageLocation> newVolumes = changedVolumes.newLocations;
+    assertEquals(2, newVolumes.size());
+    assertEquals("/foo/path1", newVolumes.get(0).getFile().getAbsolutePath());
+    assertEquals("/foo/path2", newVolumes.get(1).getFile().getAbsolutePath());
+
+    List<StorageLocation> removedVolumes = changedVolumes.deactivateLocations;
+    assertEquals(oldLocations.size(), removedVolumes.size());
+    for (int i = 0; i < removedVolumes.size(); i++) {
+      assertEquals(oldLocations.get(i).getFile(),
+          removedVolumes.get(i).getFile());
+    }
+  }
+
+  @Test
+  public void testParseChangedVolumesFailures() throws IOException {
+    startDFSCluster(1, 1);
+    DataNode dn = cluster.getDataNodes().get(0);
+    Configuration conf = dn.getConf();
+    try {
+      conf.set(DFS_DATANODE_DATA_DIR_KEY, "");
+      dn.parseChangedVolumes();
+      fail("Should throw IOException: empty inputs.");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("No directory is specified.", e);
+    }
+  }
+
+  /** Add volumes to the first DataNode. */
+  private void addVolumes(int numNewVolumes) throws ReconfigurationException {
+    File dataDir = new File(cluster.getDataDirectory());
+    DataNode dn = cluster.getDataNodes().get(0);  // First DataNode.
+    Configuration conf = dn.getConf();
+    String oldDataDir = conf.get(DFS_DATANODE_DATA_DIR_KEY);
+
+    List<File> newVolumeDirs = new ArrayList<File>();
+    StringBuilder newDataDirBuf = new StringBuilder(oldDataDir);
+    int startIdx = oldDataDir.split(",").length + 1;
+    // Find the first available (non-taken) directory name for data volume.
+    while (true) {
+      File volumeDir = new File(dataDir, "data" + startIdx);
+      if (!volumeDir.exists()) {
+        break;
+      }
+      startIdx++;
+    }
+    for (int i = startIdx; i < startIdx + numNewVolumes; i++) {
+      File volumeDir = new File(dataDir, "data" + String.valueOf(i));
+      newVolumeDirs.add(volumeDir);
+      volumeDir.mkdirs();
+      newDataDirBuf.append(",");
+      newDataDirBuf.append(volumeDir.toURI());
+    }
+
+    String newDataDir = newDataDirBuf.toString();
+    dn.reconfigurePropertyImpl(DFS_DATANODE_DATA_DIR_KEY, newDataDir);
+    assertEquals(newDataDir, conf.get(DFS_DATANODE_DATA_DIR_KEY));
+
+    // Check that all newly created volumes are appropriately formatted.
+    for (File volumeDir : newVolumeDirs) {
+      File curDir = new File(volumeDir, "current");
+      assertTrue(curDir.exists());
+      assertTrue(curDir.isDirectory());
+    }
+  }
+
+  private List<List<Integer>> getNumBlocksReport(int namesystemIdx) {
+    List<List<Integer>> results = new ArrayList<List<Integer>>();
+    final String bpid = cluster.getNamesystem(namesystemIdx).getBlockPoolId();
+    List<Map<DatanodeStorage, BlockListAsLongs>> blockReports =
+        cluster.getAllBlockReports(bpid);
+    for (Map<DatanodeStorage, BlockListAsLongs> datanodeReport : blockReports) {
+      List<Integer> numBlocksPerDN = new ArrayList<Integer>();
+      for (BlockListAsLongs blocks : datanodeReport.values()) {
+        numBlocksPerDN.add(blocks.getNumberOfBlocks());
+      }
+      results.add(numBlocksPerDN);
+    }
+    return results;
+  }
+
+  /**
+   * Test adding one volume on a running MiniDFSCluster with only one NameNode.
+   */
+  @Test
+  public void testAddOneNewVolume()
+      throws IOException, ReconfigurationException,
+      InterruptedException, TimeoutException {
+    startDFSCluster(1, 1);
+    String bpid = cluster.getNamesystem().getBlockPoolId();
+    final int numBlocks = 10;
+
+    addVolumes(1);
+
+    Path testFile = new Path("/test");
+    createFile(testFile, numBlocks);
+
+    List<Map<DatanodeStorage, BlockListAsLongs>> blockReports =
+        cluster.getAllBlockReports(bpid);
+    assertEquals(1, blockReports.size());  // 1 DataNode
+    assertEquals(3, blockReports.get(0).size());  // 3 volumes
+
+    // FSVolumeList uses Round-Robin block chooser by default. Thus the new
+    // blocks should be evenly located in all volumes.
+    int minNumBlocks = Integer.MAX_VALUE;
+    int maxNumBlocks = Integer.MIN_VALUE;
+    for (BlockListAsLongs blockList : blockReports.get(0).values()) {
+      minNumBlocks = Math.min(minNumBlocks, blockList.getNumberOfBlocks());
+      maxNumBlocks = Math.max(maxNumBlocks, blockList.getNumberOfBlocks());
+    }
+    assertTrue(Math.abs(maxNumBlocks - maxNumBlocks) <= 1);
+    verifyFileLength(cluster.getFileSystem(), testFile, numBlocks);
+  }
+
+  @Test(timeout = 60000)
+  public void testAddVolumesDuringWrite()
+      throws IOException, InterruptedException, TimeoutException,
+      ReconfigurationException {
+    startDFSCluster(1, 1);
+    String bpid = cluster.getNamesystem().getBlockPoolId();
+    Path testFile = new Path("/test");
+    createFile(testFile, 4);  // Each volume has 2 blocks.
+
+    addVolumes(2);
+
+    // Continue to write the same file, thus the new volumes will have blocks.
+    DFSTestUtil.appendFile(cluster.getFileSystem(), testFile, BLOCK_SIZE * 8);
+    verifyFileLength(cluster.getFileSystem(), testFile, 8 + 4);
+    // After appending data, there should be [2, 2, 4, 4] blocks in each volume
+    // respectively.
+    List<Integer> expectedNumBlocks = Arrays.asList(2, 2, 4, 4);
+
+    List<Map<DatanodeStorage, BlockListAsLongs>> blockReports =
+        cluster.getAllBlockReports(bpid);
+    assertEquals(1, blockReports.size());  // 1 DataNode
+    assertEquals(4, blockReports.get(0).size());  // 4 volumes
+    Map<DatanodeStorage, BlockListAsLongs> dnReport =
+        blockReports.get(0);
+    List<Integer> actualNumBlocks = new ArrayList<Integer>();
+    for (BlockListAsLongs blockList : dnReport.values()) {
+      actualNumBlocks.add(blockList.getNumberOfBlocks());
+    }
+    Collections.sort(actualNumBlocks);
+    assertEquals(expectedNumBlocks, actualNumBlocks);
+  }
+
+  @Test
+  public void testAddVolumesToFederationNN()
+      throws IOException, TimeoutException, InterruptedException,
+      ReconfigurationException {
+    // Starts a Cluster with 2 NameNode and 3 DataNodes. Each DataNode has 2
+    // volumes.
+    final int numNameNodes = 2;
+    final int numDataNodes = 1;
+    startDFSCluster(numNameNodes, numDataNodes);
+    Path testFile = new Path("/test");
+    // Create a file on the first namespace with 4 blocks.
+    createFile(0, testFile, 4);
+    // Create a file on the second namespace with 4 blocks.
+    createFile(1, testFile, 4);
+
+    // Add 2 volumes to the first DataNode.
+    final int numNewVolumes = 2;
+    addVolumes(numNewVolumes);
+
+    // Append to the file on the first namespace.
+    DFSTestUtil.appendFile(cluster.getFileSystem(0), testFile, BLOCK_SIZE * 8);
+
+    List<List<Integer>> actualNumBlocks = getNumBlocksReport(0);
+    assertEquals(cluster.getDataNodes().size(), actualNumBlocks.size());
+    List<Integer> blocksOnFirstDN = actualNumBlocks.get(0);
+    Collections.sort(blocksOnFirstDN);
+    assertEquals(Arrays.asList(2, 2, 4, 4), blocksOnFirstDN);
+
+    // Verify the second namespace also has the new volumes and they are empty.
+    actualNumBlocks = getNumBlocksReport(1);
+    assertEquals(4, actualNumBlocks.get(0).size());
+    assertEquals(numNewVolumes,
+        Collections.frequency(actualNumBlocks.get(0), 0));
+  }
+
+  @Test
+  public void testRemoveOneVolume()
+      throws ReconfigurationException, InterruptedException, TimeoutException,
+      IOException {
+    startDFSCluster(1, 1);
+    final short replFactor = 1;
+    Path testFile = new Path("/test");
+    createFile(testFile, 10, replFactor);
+
+    DataNode dn = cluster.getDataNodes().get(0);
+    Collection<String> oldDirs = getDataDirs(dn);
+    String newDirs = oldDirs.iterator().next();  // Keep the first volume.
+    dn.reconfigurePropertyImpl(
+        DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
+    dn.scheduleAllBlockReport(0);
+
+    try {
+      DFSTestUtil.readFile(cluster.getFileSystem(), testFile);
+      fail("Expect to throw BlockMissingException.");
+    } catch (BlockMissingException e) {
+      GenericTestUtils.assertExceptionContains("Could not obtain block", e);
+    }
+
+    Path newFile = new Path("/newFile");
+    createFile(newFile, 6);
+
+    String bpid = cluster.getNamesystem().getBlockPoolId();
+    List<Map<DatanodeStorage, BlockListAsLongs>> blockReports =
+        cluster.getAllBlockReports(bpid);
+    assertEquals((int)replFactor, blockReports.size());
+
+    BlockListAsLongs blocksForVolume1 =
+        blockReports.get(0).values().iterator().next();
+    // The first volume has half of the testFile and full of newFile.
+    assertEquals(10 / 2 + 6, blocksForVolume1.getNumberOfBlocks());
+  }
+
+  @Test
+  public void testReplicatingAfterRemoveVolume()
+      throws InterruptedException, TimeoutException, IOException,
+      ReconfigurationException {
+    startDFSCluster(1, 2);
+    final DistributedFileSystem fs = cluster.getFileSystem();
+    final short replFactor = 2;
+    Path testFile = new Path("/test");
+    createFile(testFile, 4, replFactor);
+
+    DataNode dn = cluster.getDataNodes().get(0);
+    Collection<String> oldDirs = getDataDirs(dn);
+    String newDirs = oldDirs.iterator().next();  // Keep the first volume.
+    dn.reconfigurePropertyImpl(
+        DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
+
+    // Force DataNode to report missing blocks.
+    dn.scheduleAllBlockReport(0);
+    DataNodeTestUtils.triggerDeletionReport(dn);
+
+    // The 2nd block only has 1 replica due to the removed data volume.
+    waitReplication(fs, testFile, 1, 1);
+
+    // Wait NameNode to replica missing blocks.
+    DFSTestUtil.waitReplication(fs, testFile, replFactor);
+  }
+}

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

@@ -40,7 +40,10 @@ import org.mockito.Mockito;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
@@ -117,6 +120,7 @@ public class TestFsDatasetImpl {
     final int numExistingVolumes = dataset.getVolumes().size();
     final int totalVolumes = numNewVolumes + numExistingVolumes;
     List<StorageLocation> newLocations = new ArrayList<StorageLocation>();
+    Set<String> expectedVolumes = new HashSet<String>();
     for (int i = 0; i < numNewVolumes; i++) {
       String path = BASE_DIR + "/newData" + i;
       newLocations.add(StorageLocation.parse(path));
@@ -125,13 +129,15 @@ public class TestFsDatasetImpl {
     }
     when(storage.getNumStorageDirs()).thenReturn(totalVolumes);
 
-    dataset.addVolumes(newLocations);
+    dataset.addVolumes(newLocations, Arrays.asList(BLOCK_POOL_IDS));
     assertEquals(totalVolumes, dataset.getVolumes().size());
     assertEquals(totalVolumes, dataset.storageMap.size());
+
+    Set<String> actualVolumes = new HashSet<String>();
     for (int i = 0; i < numNewVolumes; i++) {
-      assertEquals(newLocations.get(i).getFile().getPath(),
-          dataset.getVolumes().get(numExistingVolumes + i).getBasePath());
+      dataset.getVolumes().get(numExistingVolumes + i).getBasePath();
     }
+    assertEquals(actualVolumes, expectedVolumes);
   }
 
   @Test

+ 222 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.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.mover;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.*;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DBlock;
+import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
+import org.apache.hadoop.hdfs.server.mover.Mover.MLocation;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.GenericOptionsParser;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestMover {
+  static Mover newMover(Configuration conf) throws IOException {
+    final Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
+    Assert.assertEquals(1, namenodes.size());
+
+    final List<NameNodeConnector> nncs = NameNodeConnector.newNameNodeConnectors(
+        namenodes, Mover.class.getSimpleName(), Mover.MOVER_ID_PATH, conf);
+    return new Mover(nncs.get(0), conf);
+  }
+
+  @Test
+  public void testScheduleSameBlock() throws IOException {
+    final Configuration conf = new HdfsConfiguration();
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(4).build();
+    try {
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final String file = "/testScheduleSameBlock/file";
+      
+      {
+        final FSDataOutputStream out = dfs.create(new Path(file));
+        out.writeChars("testScheduleSameBlock");
+        out.close();
+      }
+
+      final Mover mover = newMover(conf);
+      mover.init();
+      final Mover.Processor processor = mover.new Processor();
+
+      final LocatedBlock lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
+      final List<MLocation> locations = MLocation.toLocations(lb);
+      final MLocation ml = locations.get(0);
+      final DBlock db = mover.newDBlock(lb.getBlock().getLocalBlock(), locations);
+
+      final List<StorageType> storageTypes = new ArrayList<StorageType>(
+          Arrays.asList(StorageType.DEFAULT, StorageType.DEFAULT));
+      Assert.assertTrue(processor.scheduleMoveReplica(db, ml, storageTypes));
+      Assert.assertFalse(processor.scheduleMoveReplica(db, ml, storageTypes));
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  private void checkMovePaths(List<Path> actual, Path... expected) {
+    Assert.assertEquals(expected.length, actual.size());
+    for (Path p : expected) {
+      Assert.assertTrue(actual.contains(p));
+    }
+  }
+
+  /**
+   * Test Mover Cli by specifying a list of files/directories using option "-p".
+   * There is only one namenode (and hence name service) specified in the conf.
+   */
+  @Test
+  public void testMoverCli() throws Exception {
+    final MiniDFSCluster cluster = new MiniDFSCluster
+        .Builder(new HdfsConfiguration()).numDataNodes(0).build();
+    try {
+      final Configuration conf = cluster.getConfiguration(0);
+      try {
+        Mover.Cli.getNameNodePathsToMove(conf, "-p", "/foo", "bar");
+        Assert.fail("Expected exception for illegal path bar");
+      } catch (IllegalArgumentException e) {
+        GenericTestUtils.assertExceptionContains("bar is not absolute", e);
+      }
+
+      Map<URI, List<Path>> movePaths = Mover.Cli.getNameNodePathsToMove(conf);
+      Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
+      Assert.assertEquals(1, namenodes.size());
+      Assert.assertEquals(1, movePaths.size());
+      URI nn = namenodes.iterator().next();
+      Assert.assertTrue(movePaths.containsKey(nn));
+      Assert.assertNull(movePaths.get(nn));
+
+      movePaths = Mover.Cli.getNameNodePathsToMove(conf, "-p", "/foo", "/bar");
+      namenodes = DFSUtil.getNsServiceRpcUris(conf);
+      Assert.assertEquals(1, movePaths.size());
+      nn = namenodes.iterator().next();
+      Assert.assertTrue(movePaths.containsKey(nn));
+      checkMovePaths(movePaths.get(nn), new Path("/foo"), new Path("/bar"));
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testMoverCliWithHAConf() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    final MiniDFSCluster cluster = new MiniDFSCluster
+        .Builder(new HdfsConfiguration())
+        .nnTopology(MiniDFSNNTopology.simpleHATopology())
+        .numDataNodes(0).build();
+    HATestUtil.setFailoverConfigurations(cluster, conf, "MyCluster");
+    try {
+      Map<URI, List<Path>> movePaths = Mover.Cli.getNameNodePathsToMove(conf,
+          "-p", "/foo", "/bar");
+      Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
+      Assert.assertEquals(1, namenodes.size());
+      Assert.assertEquals(1, movePaths.size());
+      URI nn = namenodes.iterator().next();
+      Assert.assertEquals(new URI("hdfs://MyCluster"), nn);
+      Assert.assertTrue(movePaths.containsKey(nn));
+      checkMovePaths(movePaths.get(nn), new Path("/foo"), new Path("/bar"));
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testMoverCliWithFederation() throws Exception {
+    final MiniDFSCluster cluster = new MiniDFSCluster
+        .Builder(new HdfsConfiguration())
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(3))
+        .numDataNodes(0).build();
+    final Configuration conf = new HdfsConfiguration();
+    DFSTestUtil.setFederatedConfiguration(cluster, conf);
+    try {
+      Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
+      Assert.assertEquals(3, namenodes.size());
+
+      try {
+        Mover.Cli.getNameNodePathsToMove(conf, "-p", "/foo");
+        Assert.fail("Expect exception for missing authority information");
+      } catch (IllegalArgumentException e) {
+        GenericTestUtils.assertExceptionContains(
+            "does not contain scheme and authority", e);
+      }
+
+      try {
+        Mover.Cli.getNameNodePathsToMove(conf, "-p", "hdfs:///foo");
+        Assert.fail("Expect exception for missing authority information");
+      } catch (IllegalArgumentException e) {
+        GenericTestUtils.assertExceptionContains(
+            "does not contain scheme and authority", e);
+      }
+
+      try {
+        Mover.Cli.getNameNodePathsToMove(conf, "-p", "wrong-hdfs://ns1/foo");
+        Assert.fail("Expect exception for wrong scheme");
+      } catch (IllegalArgumentException e) {
+        GenericTestUtils.assertExceptionContains("Cannot resolve the path", e);
+      }
+
+      Iterator<URI> iter = namenodes.iterator();
+      URI nn1 = iter.next();
+      URI nn2 = iter.next();
+      Map<URI, List<Path>> movePaths = Mover.Cli.getNameNodePathsToMove(conf,
+          "-p", nn1 + "/foo", nn1 + "/bar", nn2 + "/foo/bar");
+      Assert.assertEquals(2, movePaths.size());
+      checkMovePaths(movePaths.get(nn1), new Path("/foo"), new Path("/bar"));
+      checkMovePaths(movePaths.get(nn2), new Path("/foo/bar"));
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testMoverCliWithFederationHA() throws Exception {
+    final MiniDFSCluster cluster = new MiniDFSCluster
+        .Builder(new HdfsConfiguration())
+        .nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(3))
+        .numDataNodes(0).build();
+    final Configuration conf = new HdfsConfiguration();
+    DFSTestUtil.setFederatedHAConfiguration(cluster, conf);
+    try {
+      Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
+      Assert.assertEquals(3, namenodes.size());
+
+      Iterator<URI> iter = namenodes.iterator();
+      URI nn1 = iter.next();
+      URI nn2 = iter.next();
+      URI nn3 = iter.next();
+      Map<URI, List<Path>> movePaths = Mover.Cli.getNameNodePathsToMove(conf,
+          "-p", nn1 + "/foo", nn1 + "/bar", nn2 + "/foo/bar", nn3 + "/foobar");
+      Assert.assertEquals(3, movePaths.size());
+      checkMovePaths(movePaths.get(nn1), new Path("/foo"), new Path("/bar"));
+      checkMovePaths(movePaths.get(nn2), new Path("/foo/bar"));
+      checkMovePaths(movePaths.get(nn3), new Path("/foobar"));
+    } finally {
+       cluster.shutdown();
+    }
+  }
+}

+ 766 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java

@@ -0,0 +1,766 @@
+/**
+ * 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.mover;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSOutputStream;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+import org.apache.hadoop.hdfs.server.balancer.Dispatcher;
+import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
+import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+
+/**
+ * Test the data migration tool (for Archival Storage)
+ */
+public class TestStorageMover {
+  static final Log LOG = LogFactory.getLog(TestStorageMover.class);
+  static {
+    ((Log4JLogger)LogFactory.getLog(BlockPlacementPolicy.class)
+        ).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)LogFactory.getLog(Dispatcher.class)
+        ).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)LogFactory.getLog(DataTransferProtocol.class)).getLogger()
+        .setLevel(Level.ALL);
+  }
+
+  private static final int BLOCK_SIZE = 1024;
+  private static final short REPL = 3;
+  private static final int NUM_DATANODES = 6;
+  private static final Configuration DEFAULT_CONF = new HdfsConfiguration();
+  private static final BlockStoragePolicy.Suite DEFAULT_POLICIES;
+  private static final BlockStoragePolicy HOT;
+  private static final BlockStoragePolicy WARM;
+  private static final BlockStoragePolicy COLD;
+
+  static {
+    DEFAULT_CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    DEFAULT_CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+    DEFAULT_CONF.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
+        2L);
+    DEFAULT_CONF.setLong(DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_KEY, 2000L);
+
+    DEFAULT_POLICIES = BlockStoragePolicy.readBlockStorageSuite(DEFAULT_CONF);
+    HOT = DEFAULT_POLICIES.getPolicy("HOT");
+    WARM = DEFAULT_POLICIES.getPolicy("WARM");
+    COLD = DEFAULT_POLICIES.getPolicy("COLD");
+    TestBalancer.initTestSetup();
+    Dispatcher.setDelayAfterErrors(1000L);
+  }
+
+  /**
+   * This scheme defines files/directories and their block storage policies. It
+   * also defines snapshots.
+   */
+  static class NamespaceScheme {
+    final List<Path> dirs;
+    final List<Path> files;
+    final long fileSize;
+    final Map<Path, List<String>> snapshotMap;
+    final Map<Path, BlockStoragePolicy> policyMap;
+
+    NamespaceScheme(List<Path> dirs, List<Path> files, long fileSize, 
+                    Map<Path,List<String>> snapshotMap,
+                    Map<Path, BlockStoragePolicy> policyMap) {
+      this.dirs = dirs == null? Collections.<Path>emptyList(): dirs;
+      this.files = files == null? Collections.<Path>emptyList(): files;
+      this.fileSize = fileSize;
+      this.snapshotMap = snapshotMap == null ?
+          Collections.<Path, List<String>>emptyMap() : snapshotMap;
+      this.policyMap = policyMap;
+    }
+
+    /**
+     * Create files/directories/snapshots.
+     */
+    void prepare(DistributedFileSystem dfs, short repl) throws Exception {
+      for (Path d : dirs) {
+        dfs.mkdirs(d);
+      }
+      for (Path file : files) {
+        DFSTestUtil.createFile(dfs, file, fileSize, repl, 0L);
+      }
+      for (Map.Entry<Path, List<String>> entry : snapshotMap.entrySet()) {
+        for (String snapshot : entry.getValue()) {
+          SnapshotTestHelper.createSnapshot(dfs, entry.getKey(), snapshot);
+        }
+      }
+    }
+
+    /**
+     * Set storage policies according to the corresponding scheme.
+     */
+    void setStoragePolicy(DistributedFileSystem dfs) throws Exception {
+      for (Map.Entry<Path, BlockStoragePolicy> entry : policyMap.entrySet()) {
+        dfs.setStoragePolicy(entry.getKey(), entry.getValue().getName());
+      }
+    }
+  }
+
+  /**
+   * This scheme defines DataNodes and their storage, including storage types
+   * and remaining capacities.
+   */
+  static class ClusterScheme {
+    final Configuration conf;
+    final int numDataNodes;
+    final short repl;
+    final StorageType[][] storageTypes;
+    final long[][] storageCapacities;
+
+    ClusterScheme() {
+      this(DEFAULT_CONF, NUM_DATANODES, REPL,
+          genStorageTypes(NUM_DATANODES), null);
+    }
+
+    ClusterScheme(Configuration conf, int numDataNodes, short repl,
+        StorageType[][] types, long[][] capacities) {
+      Preconditions.checkArgument(types == null || types.length == numDataNodes);
+      Preconditions.checkArgument(capacities == null || capacities.length ==
+          numDataNodes);
+      this.conf = conf;
+      this.numDataNodes = numDataNodes;
+      this.repl = repl;
+      this.storageTypes = types;
+      this.storageCapacities = capacities;
+    }
+  }
+
+  class MigrationTest {
+    private final ClusterScheme clusterScheme;
+    private final NamespaceScheme nsScheme;
+    private final Configuration conf;
+
+    private MiniDFSCluster cluster;
+    private DistributedFileSystem dfs;
+    private final BlockStoragePolicy.Suite policies;
+
+    MigrationTest(ClusterScheme cScheme, NamespaceScheme nsScheme) {
+      this.clusterScheme = cScheme;
+      this.nsScheme = nsScheme;
+      this.conf = clusterScheme.conf;
+      this.policies = BlockStoragePolicy.readBlockStorageSuite(conf);
+    }
+
+    /**
+     * Set up the cluster and start NameNode and DataNodes according to the
+     * corresponding scheme.
+     */
+    void setupCluster() throws Exception {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(clusterScheme
+          .numDataNodes).storageTypes(clusterScheme.storageTypes)
+          .storageCapacities(clusterScheme.storageCapacities).build();
+      cluster.waitActive();
+      dfs = cluster.getFileSystem();
+    }
+
+    private void runBasicTest(boolean shutdown) throws Exception {
+      setupCluster();
+      try {
+        prepareNamespace();
+        verify(true);
+
+        setStoragePolicy();
+        migrate();
+        verify(true);
+      } finally {
+        if (shutdown) {
+          shutdownCluster();
+        }
+      }
+    }
+
+    void shutdownCluster() throws Exception {
+      IOUtils.cleanup(null, dfs);
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+
+    /**
+     * Create files/directories and set their storage policies according to the
+     * corresponding scheme.
+     */
+    void prepareNamespace() throws Exception {
+      nsScheme.prepare(dfs, clusterScheme.repl);
+    }
+
+    void setStoragePolicy() throws Exception {
+      nsScheme.setStoragePolicy(dfs);
+    }
+
+    /**
+     * Run the migration tool.
+     */
+    void migrate() throws Exception {
+      runMover();
+      Thread.sleep(5000); // let the NN finish deletion
+    }
+
+    /**
+     * Verify block locations after running the migration tool.
+     */
+    void verify(boolean verifyAll) throws Exception {
+      for (DataNode dn : cluster.getDataNodes()) {
+        DataNodeTestUtils.triggerBlockReport(dn);
+      }
+      if (verifyAll) {
+        verifyNamespace();
+      } else {
+        // TODO verify according to the given path list
+
+      }
+    }
+
+    private void runMover() throws Exception {
+      Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
+      Map<URI, List<Path>> nnMap = Maps.newHashMap();
+      for (URI nn : namenodes) {
+        nnMap.put(nn, null);
+      }
+      int result = Mover.run(nnMap, conf);
+      Assert.assertEquals(ExitStatus.SUCCESS.getExitCode(), result);
+    }
+
+    private void verifyNamespace() throws Exception {
+      HdfsFileStatus status = dfs.getClient().getFileInfo("/");
+      verifyRecursively(null, status);
+    }
+
+    private void verifyRecursively(final Path parent,
+        final HdfsFileStatus status) throws Exception {
+      if (status.isDir()) {
+        Path fullPath = parent == null ?
+            new Path("/") : status.getFullPath(parent);
+        DirectoryListing children = dfs.getClient().listPaths(
+            fullPath.toString(), HdfsFileStatus.EMPTY_NAME, true);
+        for (HdfsFileStatus child : children.getPartialListing()) {
+          verifyRecursively(fullPath, child);
+        }
+      } else if (!status.isSymlink()) { // is file
+        verifyFile(parent, status, null);
+      }
+    }
+
+    void verifyFile(final Path file, final Byte expectedPolicyId)
+        throws Exception {
+      final Path parent = file.getParent();
+      DirectoryListing children = dfs.getClient().listPaths(
+          parent.toString(), HdfsFileStatus.EMPTY_NAME, true);
+      for (HdfsFileStatus child : children.getPartialListing()) {
+        if (child.getLocalName().equals(file.getName())) {
+          verifyFile(parent,  child, expectedPolicyId);
+          return;
+        }
+      }
+      Assert.fail("File " + file + " not found.");
+    }
+
+    private void verifyFile(final Path parent, final HdfsFileStatus status,
+        final Byte expectedPolicyId) throws Exception {
+      HdfsLocatedFileStatus fileStatus = (HdfsLocatedFileStatus) status;
+      byte policyId = fileStatus.getStoragePolicy();
+      BlockStoragePolicy policy = policies.getPolicy(policyId);
+      if (expectedPolicyId != null) {
+        Assert.assertEquals((byte)expectedPolicyId, policy.getId());
+      }
+      final List<StorageType> types = policy.chooseStorageTypes(
+          status.getReplication());
+      for(LocatedBlock lb : fileStatus.getBlockLocations().getLocatedBlocks()) {
+        final Mover.StorageTypeDiff diff = new Mover.StorageTypeDiff(types,
+            lb.getStorageTypes());
+        Assert.assertTrue(fileStatus.getFullName(parent.toString())
+            + " with policy " + policy + " has non-empty overlap: " + diff
+            + ", the corresponding block is " + lb.getBlock().getLocalBlock(),
+            diff.removeOverlap());
+      }
+    }
+    
+    Replication getReplication(Path file) throws IOException {
+      return getOrVerifyReplication(file, null);
+    }
+
+    Replication verifyReplication(Path file, int expectedDiskCount,
+        int expectedArchiveCount) throws IOException {
+      final Replication r = new Replication();
+      r.disk = expectedDiskCount;
+      r.archive = expectedArchiveCount;
+      return getOrVerifyReplication(file, r);
+    }
+
+    private Replication getOrVerifyReplication(Path file, Replication expected)
+        throws IOException {
+      final List<LocatedBlock> lbs = dfs.getClient().getLocatedBlocks(
+          file.toString(), 0).getLocatedBlocks();
+      Assert.assertEquals(1, lbs.size());
+
+      LocatedBlock lb = lbs.get(0);
+      StringBuilder types = new StringBuilder(); 
+      final Replication r = new Replication();
+      for(StorageType t : lb.getStorageTypes()) {
+        types.append(t).append(", ");
+        if (t == StorageType.DISK) {
+          r.disk++;
+        } else if (t == StorageType.ARCHIVE) {
+          r.archive++;
+        } else {
+          Assert.fail("Unexpected storage type " + t);
+        }
+      }
+
+      if (expected != null) {
+        final String s = "file = " + file + "\n  types = [" + types + "]";
+        Assert.assertEquals(s, expected, r);
+      }
+      return r;
+    }
+  }
+
+  static class Replication {
+    int disk;
+    int archive;
+    
+    @Override
+    public int hashCode() {
+      return disk ^ archive;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == this) {
+        return true;
+      } else if (obj == null || !(obj instanceof Replication)) {
+        return false;
+      }
+      final Replication that = (Replication)obj;
+      return this.disk == that.disk && this.archive == that.archive;
+    }
+    
+    @Override
+    public String toString() {
+      return "[disk=" + disk + ", archive=" + archive + "]";
+    }
+  }
+
+  private static StorageType[][] genStorageTypes(int numDataNodes) {
+    return genStorageTypes(numDataNodes, 0, 0);
+  }
+
+  private static StorageType[][] genStorageTypes(int numDataNodes,
+      int numAllDisk, int numAllArchive) {
+    StorageType[][] types = new StorageType[numDataNodes][];
+    int i = 0;
+    for (; i < numAllDisk; i++) {
+      types[i] = new StorageType[]{StorageType.DISK, StorageType.DISK};
+    }
+    for (; i < numAllDisk + numAllArchive; i++) {
+      types[i] = new StorageType[]{StorageType.ARCHIVE, StorageType.ARCHIVE};
+    }
+    for (; i < types.length; i++) {
+      types[i] = new StorageType[]{StorageType.DISK, StorageType.ARCHIVE};
+    }
+    return types;
+  }
+  
+  private static long[][] genCapacities(int nDatanodes, int numAllDisk,
+      int numAllArchive, long diskCapacity, long archiveCapacity) {
+    final long[][] capacities = new long[nDatanodes][];
+    int i = 0;
+    for (; i < numAllDisk; i++) {
+      capacities[i] = new long[]{diskCapacity, diskCapacity};
+    }
+    for (; i < numAllDisk + numAllArchive; i++) {
+      capacities[i] = new long[]{archiveCapacity, archiveCapacity};
+    }
+    for(; i < capacities.length; i++) {
+      capacities[i] = new long[]{diskCapacity, archiveCapacity};
+    }
+    return capacities;
+  }
+
+  private static class PathPolicyMap {
+    final Map<Path, BlockStoragePolicy> map = Maps.newHashMap();
+    final Path hot = new Path("/hot");
+    final Path warm = new Path("/warm");
+    final Path cold = new Path("/cold");
+    final List<Path> files;
+
+    PathPolicyMap(int filesPerDir){
+      map.put(hot, HOT);
+      map.put(warm, WARM);
+      map.put(cold, COLD);
+      files = new ArrayList<Path>();
+      for(Path dir : map.keySet()) {
+        for(int i = 0; i < filesPerDir; i++) {
+          files.add(new Path(dir, "file" + i));
+        }
+      }
+    }
+
+    NamespaceScheme newNamespaceScheme() {
+      return new NamespaceScheme(Arrays.asList(hot, warm, cold),
+          files, BLOCK_SIZE/2, null, map);
+    }
+
+    /**
+     * Move hot files to warm and cold, warm files to hot and cold,
+     * and cold files to hot and warm.
+     */
+    void moveAround(DistributedFileSystem dfs) throws Exception {
+      for(Path srcDir : map.keySet()) {
+        int i = 0;
+        for(Path dstDir : map.keySet()) {
+          if (!srcDir.equals(dstDir)) {
+            final Path src = new Path(srcDir, "file" + i++);
+            final Path dst = new Path(dstDir, srcDir.getName() + "2" + dstDir.getName());
+            LOG.info("rename " + src + " to " + dst);
+            dfs.rename(src, dst);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * A normal case for Mover: move a file into archival storage
+   */
+  @Test
+  public void testMigrateFileToArchival() throws Exception {
+    LOG.info("testMigrateFileToArchival");
+    final Path foo = new Path("/foo");
+    Map<Path, BlockStoragePolicy> policyMap = Maps.newHashMap();
+    policyMap.put(foo, COLD);
+    NamespaceScheme nsScheme = new NamespaceScheme(null, Arrays.asList(foo),
+        2*BLOCK_SIZE, null, policyMap);
+    ClusterScheme clusterScheme = new ClusterScheme(DEFAULT_CONF,
+        NUM_DATANODES, REPL, genStorageTypes(NUM_DATANODES), null);
+    new MigrationTest(clusterScheme, nsScheme).runBasicTest(true);
+  }
+
+  /**
+   * Print a big banner in the test log to make debug easier.
+   */
+  static void banner(String string) {
+    LOG.info("\n\n\n\n================================================\n" +
+        string + "\n" +
+        "==================================================\n\n");
+  }
+
+  /**
+   * Move an open file into archival storage
+   */
+  @Test
+  public void testMigrateOpenFileToArchival() throws Exception {
+    LOG.info("testMigrateOpenFileToArchival");
+    final Path fooDir = new Path("/foo");
+    Map<Path, BlockStoragePolicy> policyMap = Maps.newHashMap();
+    policyMap.put(fooDir, COLD);
+    NamespaceScheme nsScheme = new NamespaceScheme(Arrays.asList(fooDir), null,
+        BLOCK_SIZE, null, policyMap);
+    ClusterScheme clusterScheme = new ClusterScheme(DEFAULT_CONF,
+        NUM_DATANODES, REPL, genStorageTypes(NUM_DATANODES), null);
+    MigrationTest test = new MigrationTest(clusterScheme, nsScheme);
+    test.setupCluster();
+
+    // create an open file
+    banner("writing to file /foo/bar");
+    final Path barFile = new Path(fooDir, "bar");
+    DFSTestUtil.createFile(test.dfs, barFile, BLOCK_SIZE, (short) 1, 0L);
+    FSDataOutputStream out = test.dfs.append(barFile);
+    out.writeBytes("hello, ");
+    ((DFSOutputStream) out.getWrappedStream()).hsync();
+
+    try {
+      banner("start data migration");
+      test.setStoragePolicy(); // set /foo to COLD
+      test.migrate();
+
+      // make sure the under construction block has not been migrated
+      LocatedBlocks lbs = test.dfs.getClient().getLocatedBlocks(
+          barFile.toString(), BLOCK_SIZE);
+      LOG.info("Locations: " + lbs);
+      List<LocatedBlock> blks = lbs.getLocatedBlocks();
+      Assert.assertEquals(1, blks.size());
+      Assert.assertEquals(1, blks.get(0).getLocations().length);
+
+      banner("finish the migration, continue writing");
+      // make sure the writing can continue
+      out.writeBytes("world!");
+      ((DFSOutputStream) out.getWrappedStream()).hsync();
+      IOUtils.cleanup(LOG, out);
+
+      lbs = test.dfs.getClient().getLocatedBlocks(
+          barFile.toString(), BLOCK_SIZE);
+      LOG.info("Locations: " + lbs);
+      blks = lbs.getLocatedBlocks();
+      Assert.assertEquals(1, blks.size());
+      Assert.assertEquals(1, blks.get(0).getLocations().length);
+
+      banner("finish writing, starting reading");
+      // check the content of /foo/bar
+      FSDataInputStream in = test.dfs.open(barFile);
+      byte[] buf = new byte[13];
+      // read from offset 1024
+      in.readFully(BLOCK_SIZE, buf, 0, buf.length);
+      IOUtils.cleanup(LOG, in);
+      Assert.assertEquals("hello, world!", new String(buf));
+    } finally {
+      test.shutdownCluster();
+    }
+  }
+
+  /**
+   * Test directories with Hot, Warm and Cold polices.
+   */
+  @Test
+  public void testHotWarmColdDirs() throws Exception {
+    LOG.info("testHotWarmColdDirs");
+    PathPolicyMap pathPolicyMap = new PathPolicyMap(3);
+    NamespaceScheme nsScheme = pathPolicyMap.newNamespaceScheme();
+    ClusterScheme clusterScheme = new ClusterScheme();
+    MigrationTest test = new MigrationTest(clusterScheme, nsScheme);
+
+    try {
+      test.runBasicTest(false);
+      pathPolicyMap.moveAround(test.dfs);
+      test.migrate();
+
+      test.verify(true);
+    } finally {
+      test.shutdownCluster();
+    }
+  }
+
+  private void waitForAllReplicas(int expectedReplicaNum, Path file,
+      DistributedFileSystem dfs) throws Exception {
+    for (int i = 0; i < 5; i++) {
+      LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(file.toString(), 0,
+          BLOCK_SIZE);
+      LocatedBlock lb = lbs.get(0);
+      if (lb.getLocations().length >= expectedReplicaNum) {
+        return;
+      } else {
+        Thread.sleep(1000);
+      }
+    }
+  }
+
+  /**
+   * Test DISK is running out of spaces.
+   */
+  @Test
+  public void testNoSpaceDisk() throws Exception {
+    LOG.info("testNoSpaceDisk");
+    final PathPolicyMap pathPolicyMap = new PathPolicyMap(0);
+    final NamespaceScheme nsScheme = pathPolicyMap.newNamespaceScheme();
+
+    final long diskCapacity = (6 + HdfsConstants.MIN_BLOCKS_FOR_WRITE)
+        * BLOCK_SIZE;
+    final long archiveCapacity = 100 * BLOCK_SIZE;
+    final long[][] capacities = genCapacities(NUM_DATANODES, 1, 1,
+        diskCapacity, archiveCapacity);
+    Configuration conf = new Configuration(DEFAULT_CONF);
+    final ClusterScheme clusterScheme = new ClusterScheme(conf,
+        NUM_DATANODES, REPL, genStorageTypes(NUM_DATANODES, 1, 1), capacities);
+    final MigrationTest test = new MigrationTest(clusterScheme, nsScheme);
+
+    try {
+      test.runBasicTest(false);
+
+      // create hot files with replication 3 until not more spaces.
+      final short replication = 3;
+      {
+        int hotFileCount = 0;
+        try {
+          for (; ; hotFileCount++) {
+            final Path p = new Path(pathPolicyMap.hot, "file" + hotFileCount);
+            DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
+            waitForAllReplicas(replication, p, test.dfs);
+          }
+        } catch (IOException e) {
+          LOG.info("Expected: hotFileCount=" + hotFileCount, e);
+        }
+        Assert.assertTrue(hotFileCount >= 1);
+      }
+
+      // create hot files with replication 1 to use up all remaining spaces.
+      {
+        int hotFileCount_r1 = 0;
+        try {
+          for (; ; hotFileCount_r1++) {
+            final Path p = new Path(pathPolicyMap.hot, "file_r1_" + hotFileCount_r1);
+            DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, (short) 1, 0L);
+            waitForAllReplicas(1, p, test.dfs);
+          }
+        } catch (IOException e) {
+          LOG.info("Expected: hotFileCount_r1=" + hotFileCount_r1, e);
+        }
+      }
+
+      { // test increasing replication.  Since DISK is full,
+        // new replicas should be stored in ARCHIVE as a fallback storage.
+        final Path file0 = new Path(pathPolicyMap.hot, "file0");
+        final Replication r = test.getReplication(file0);
+        final short newReplication = (short) 5;
+        test.dfs.setReplication(file0, newReplication);
+        Thread.sleep(10000);
+        test.verifyReplication(file0, r.disk, newReplication - r.disk);
+      }
+
+      { // test creating a cold file and then increase replication
+        final Path p = new Path(pathPolicyMap.cold, "foo");
+        DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
+        test.verifyReplication(p, 0, replication);
+
+        final short newReplication = 5;
+        test.dfs.setReplication(p, newReplication);
+        Thread.sleep(10000);
+        test.verifyReplication(p, 0, newReplication);
+      }
+
+      { //test move a hot file to warm
+        final Path file1 = new Path(pathPolicyMap.hot, "file1");
+        test.dfs.rename(file1, pathPolicyMap.warm);
+        test.migrate();
+        test.verifyFile(new Path(pathPolicyMap.warm, "file1"), WARM.getId());
+      }
+    } finally {
+      test.shutdownCluster();
+    }
+  }
+
+  /**
+   * Test ARCHIVE is running out of spaces.
+   */
+  @Test
+  public void testNoSpaceArchive() throws Exception {
+    LOG.info("testNoSpaceArchive");
+    final PathPolicyMap pathPolicyMap = new PathPolicyMap(0);
+    final NamespaceScheme nsScheme = pathPolicyMap.newNamespaceScheme();
+
+    final long diskCapacity = 100 * BLOCK_SIZE;
+    final long archiveCapacity = (6 + HdfsConstants.MIN_BLOCKS_FOR_WRITE)
+        * BLOCK_SIZE;
+    final long[][] capacities = genCapacities(NUM_DATANODES, 1, 1,
+        diskCapacity, archiveCapacity);
+    final ClusterScheme clusterScheme = new ClusterScheme(DEFAULT_CONF,
+        NUM_DATANODES, REPL, genStorageTypes(NUM_DATANODES, 1, 1), capacities);
+    final MigrationTest test = new MigrationTest(clusterScheme, nsScheme);
+
+    try {
+      test.runBasicTest(false);
+
+      // create cold files with replication 3 until not more spaces.
+      final short replication = 3;
+      {
+        int coldFileCount = 0;
+        try {
+          for (; ; coldFileCount++) {
+            final Path p = new Path(pathPolicyMap.cold, "file" + coldFileCount);
+            DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, replication, 0L);
+            waitForAllReplicas(replication, p, test.dfs);
+          }
+        } catch (IOException e) {
+          LOG.info("Expected: coldFileCount=" + coldFileCount, e);
+        }
+        Assert.assertTrue(coldFileCount >= 1);
+      }
+
+      // create cold files with replication 1 to use up all remaining spaces.
+      {
+        int coldFileCount_r1 = 0;
+        try {
+          for (; ; coldFileCount_r1++) {
+            final Path p = new Path(pathPolicyMap.cold, "file_r1_" + coldFileCount_r1);
+            DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, (short) 1, 0L);
+            waitForAllReplicas(1, p, test.dfs);
+          }
+        } catch (IOException e) {
+          LOG.info("Expected: coldFileCount_r1=" + coldFileCount_r1, e);
+        }
+      }
+
+      { // test increasing replication but new replicas cannot be created
+        // since no more ARCHIVE space.
+        final Path file0 = new Path(pathPolicyMap.cold, "file0");
+        final Replication r = test.getReplication(file0);
+        LOG.info("XXX " + file0 + ": replication=" + r);
+        Assert.assertEquals(0, r.disk);
+
+        final short newReplication = (short) 5;
+        test.dfs.setReplication(file0, newReplication);
+        Thread.sleep(10000);
+
+        test.verifyReplication(file0, 0, r.archive);
+      }
+
+      { // test creating a hot file
+        final Path p = new Path(pathPolicyMap.hot, "foo");
+        DFSTestUtil.createFile(test.dfs, p, BLOCK_SIZE, (short) 3, 0L);
+      }
+
+      { //test move a cold file to warm
+        final Path file1 = new Path(pathPolicyMap.cold, "file1");
+        test.dfs.rename(file1, pathPolicyMap.warm);
+        test.migrate();
+        test.verify(true);
+      }
+    } finally {
+      test.shutdownCluster();
+    }
+  }
+}

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java

@@ -121,9 +121,9 @@ public class TestAddBlockRetry {
         }
         return ret;
       }
-    }).when(spyBM).chooseTarget(Mockito.anyString(), Mockito.anyInt(),
+    }).when(spyBM).chooseTarget4NewBlock(Mockito.anyString(), Mockito.anyInt(),
         Mockito.<DatanodeDescriptor>any(), Mockito.<HashSet<Node>>any(),
-        Mockito.anyLong(), Mockito.<List<String>>any());
+        Mockito.anyLong(), Mockito.<List<String>>any(), Mockito.anyByte());
 
     // create file
     nn.create(src, FsPermission.getFileDefault(),

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java

@@ -29,13 +29,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.AppendTestUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
@@ -121,10 +121,10 @@ public class TestDeleteRace {
                                       boolean returnChosenNodes,
                                       Set<Node> excludedNodes,
                                       long blocksize,
-                                      StorageType storageType) {
+                                      final BlockStoragePolicy storagePolicy) {
       DatanodeStorageInfo[] results = super.chooseTarget(srcPath,
           numOfReplicas, writer, chosenNodes, returnChosenNodes, excludedNodes,
-          blocksize, storageType);
+          blocksize, storagePolicy);
       try {
         Thread.sleep(3000);
       } catch (InterruptedException e) {}

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