瀏覽代碼

Merge trunk to HDFS-4685.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4685@1550774 13f79535-47bb-0310-9956-ffa450edef68
Chris Nauroth 11 年之前
父節點
當前提交
f24e0c1563
共有 100 個文件被更改,包括 2957 次插入1622 次删除
  1. 8 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 4 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
  3. 14 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcConstants.java
  4. 105 22
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  5. 41 6
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/hadoop_user_info.c
  6. 176 27
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  7. 1 6
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
  8. 32 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  9. 27 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  10. 17 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  11. 35 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StorageType.java
  12. 19 22
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
  13. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  14. 37 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
  15. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  16. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
  17. 39 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
  18. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnregisteredNodeException.java
  19. 12 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  20. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  21. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
  22. 3 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
  23. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java
  24. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java
  25. 128 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  26. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java
  27. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
  28. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
  29. 2 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
  30. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java
  31. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
  32. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
  33. 2 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
  34. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
  35. 41 63
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  36. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
  37. 91 41
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  38. 33 20
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
  39. 185 165
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  40. 12 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
  41. 186 139
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  42. 59 56
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java
  43. 15 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
  44. 147 185
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  45. 53 60
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  46. 288 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
  47. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
  48. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
  49. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/MutableBlockCollection.java
  50. 9 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java
  51. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
  52. 34 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
  53. 204 74
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
  54. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
  55. 13 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  56. 171 123
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  57. 69 42
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  58. 11 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  59. 10 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  60. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/Replica.java
  61. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
  62. 101 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
  63. 14 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
  64. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
  65. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
  66. 103 61
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  67. 18 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
  68. 17 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
  69. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java
  70. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
  71. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DirectoryWithQuotaFeature.java
  72. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  73. 6 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  74. 61 68
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  75. 5 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
  76. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
  77. 1 42
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  78. 2 27
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  79. 5 35
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  80. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java
  81. 6 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
  82. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LogsPurgeable.java
  83. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
  84. 26 22
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  85. 6 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  86. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  87. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
  88. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
  89. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
  90. 9 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  91. 19 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java
  92. 28 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java
  93. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
  94. 45 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
  95. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java
  96. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  97. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  98. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  99. 6 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
  100. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto

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

@@ -280,6 +280,11 @@ Trunk (Unreleased)
     HDFS-5471. CacheAdmin -listPools fails when user lacks permissions to view
     HDFS-5471. CacheAdmin -listPools fails when user lacks permissions to view
     all pools (Andrew Wang via Colin Patrick McCabe)
     all pools (Andrew Wang via Colin Patrick McCabe)
 
 
+    HADOOP-10044 Improve the javadoc of rpc code (sanjay Radia)
+
+    HADOOP-8753. LocalDirAllocator throws "ArithmeticException: / by zero" when
+    there is no available space on configured local dir. (Benoy Antony via hitesh)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -538,6 +543,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-10081. Client.setupIOStreams can leak socket resources on exception
     HADOOP-10081. Client.setupIOStreams can leak socket resources on exception
     or error (Tsuyoshi OZAWA via jlowe)
     or error (Tsuyoshi OZAWA via jlowe)
 
 
+    HADOOP-10087. UserGroupInformation.getGroupNames() fails to return primary
+    group first when JniBasedUnixGroupsMappingWithFallback is used (cmccabe)
+
 Release 2.2.0 - 2013-10-13
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 4 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java

@@ -365,6 +365,10 @@ public class LocalDirAllocator {
           totalAvailable += availableOnDisk[i];
           totalAvailable += availableOnDisk[i];
         }
         }
 
 
+        if (totalAvailable == 0){
+          throw new DiskErrorException("No space available in any of the local directories.");
+        }
+
         // Keep rolling the wheel till we get a valid path
         // Keep rolling the wheel till we get a valid path
         Random r = new java.util.Random();
         Random r = new java.util.Random();
         while (numDirsSearched < numDirs && returnPath == null) {
         while (numDirsSearched < numDirs && returnPath == null) {

+ 14 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcConstants.java

@@ -37,10 +37,24 @@ public class RpcConstants {
   
   
   public static final int INVALID_RETRY_COUNT = -1;
   public static final int INVALID_RETRY_COUNT = -1;
   
   
+ /**
+  * The Rpc-connection header is as follows 
+  * +----------------------------------+
+  * |  "hrpc" 4 bytes                  |      
+  * +----------------------------------+
+  * |  Version (1 byte)                |
+  * +----------------------------------+
+  * |  Service Class (1 byte)          |
+  * +----------------------------------+
+  * |  AuthProtocol (1 byte)           |      
+  * +----------------------------------+
+  */
+  
   /**
   /**
    * The first four bytes of Hadoop RPC connections
    * The first four bytes of Hadoop RPC connections
    */
    */
   public static final ByteBuffer HEADER = ByteBuffer.wrap("hrpc".getBytes());
   public static final ByteBuffer HEADER = ByteBuffer.wrap("hrpc".getBytes());
+  public static final int HEADER_LEN_AFTER_HRPC_PART = 3; // 3 bytes that follow
   
   
   // 1 : Introduce ping and server does not throw away RPCs
   // 1 : Introduce ping and server does not throw away RPCs
   // 3 : Introduce the protocol into the RPC connection header
   // 3 : Introduce the protocol into the RPC connection header

+ 105 - 22
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -1105,6 +1105,9 @@ public abstract class Server {
       this.channel = channel;
       this.channel = channel;
       this.lastContact = lastContact;
       this.lastContact = lastContact;
       this.data = null;
       this.data = null;
+      
+      // the buffer is initialized to read the "hrpc" and after that to read
+      // the length of the Rpc-packet (i.e 4 bytes)
       this.dataLengthBuffer = ByteBuffer.allocate(4);
       this.dataLengthBuffer = ByteBuffer.allocate(4);
       this.unwrappedData = null;
       this.unwrappedData = null;
       this.unwrappedDataLengthBuffer = ByteBuffer.allocate(4);
       this.unwrappedDataLengthBuffer = ByteBuffer.allocate(4);
@@ -1200,7 +1203,16 @@ public abstract class Server {
       }
       }
     }
     }
 
 
-    private Throwable getCauseForInvalidToken(IOException e) {
+    /**
+     * Some exceptions ({@link RetriableException} and {@link StandbyException})
+     * that are wrapped as a cause of parameter e are unwrapped so that they can
+     * be sent as the true cause to the client side. In case of
+     * {@link InvalidToken} we go one level deeper to get the true cause.
+     * 
+     * @param e the exception that may have a cause we want to unwrap.
+     * @return the true cause for some exceptions.
+     */
+    private Throwable getTrueCause(IOException e) {
       Throwable cause = e;
       Throwable cause = e;
       while (cause != null) {
       while (cause != null) {
         if (cause instanceof RetriableException) {
         if (cause instanceof RetriableException) {
@@ -1223,6 +1235,18 @@ public abstract class Server {
       return e;
       return e;
     }
     }
     
     
+    /**
+     * Process saslMessage and send saslResponse back
+     * @param saslMessage received SASL message
+     * @throws WrappedRpcServerException setup failed due to SASL negotiation 
+     *         failure, premature or invalid connection context, or other state 
+     *         errors. This exception needs to be sent to the client. This 
+     *         exception will wrap {@link RetriableException}, 
+     *         {@link InvalidToken}, {@link StandbyException} or 
+     *         {@link SaslException}.
+     * @throws IOException if sending reply fails
+     * @throws InterruptedException
+     */
     private void saslProcess(RpcSaslProto saslMessage)
     private void saslProcess(RpcSaslProto saslMessage)
         throws WrappedRpcServerException, IOException, InterruptedException {
         throws WrappedRpcServerException, IOException, InterruptedException {
       if (saslContextEstablished) {
       if (saslContextEstablished) {
@@ -1239,7 +1263,7 @@ public abstract class Server {
           // attempting user could be null
           // attempting user could be null
           AUDITLOG.warn(AUTH_FAILED_FOR + this.toString() + ":"
           AUDITLOG.warn(AUTH_FAILED_FOR + this.toString() + ":"
               + attemptingUser + " (" + e.getLocalizedMessage() + ")");
               + attemptingUser + " (" + e.getLocalizedMessage() + ")");
-          throw (IOException) getCauseForInvalidToken(e);
+          throw (IOException) getTrueCause(e);
         }
         }
         
         
         if (saslServer != null && saslServer.isComplete()) {
         if (saslServer != null && saslServer.isComplete()) {
@@ -1274,13 +1298,26 @@ public abstract class Server {
       }
       }
     }
     }
     
     
+    /**
+     * Process a saslMessge.
+     * @param saslMessage received SASL message
+     * @return the sasl response to send back to client
+     * @throws SaslException if authentication or generating response fails, 
+     *                       or SASL protocol mixup
+     * @throws IOException if a SaslServer cannot be created
+     * @throws AccessControlException if the requested authentication type 
+     *         is not supported or trying to re-attempt negotiation.
+     * @throws InterruptedException
+     */
     private RpcSaslProto processSaslMessage(RpcSaslProto saslMessage)
     private RpcSaslProto processSaslMessage(RpcSaslProto saslMessage)
-        throws IOException, InterruptedException {
+        throws SaslException, IOException, AccessControlException,
+        InterruptedException {
       RpcSaslProto saslResponse = null;
       RpcSaslProto saslResponse = null;
       final SaslState state = saslMessage.getState(); // required      
       final SaslState state = saslMessage.getState(); // required      
       switch (state) {
       switch (state) {
         case NEGOTIATE: {
         case NEGOTIATE: {
           if (sentNegotiate) {
           if (sentNegotiate) {
+            // FIXME shouldn't this be SaslException?
             throw new AccessControlException(
             throw new AccessControlException(
                 "Client already attempted negotiation");
                 "Client already attempted negotiation");
           }
           }
@@ -1402,12 +1439,30 @@ public abstract class Server {
       }
       }
     }
     }
 
 
+    /**
+     * This method reads in a non-blocking fashion from the channel: 
+     * this method is called repeatedly when data is present in the channel; 
+     * when it has enough data to process one rpc it processes that rpc.
+     * 
+     * On the first pass, it processes the connectionHeader, 
+     * connectionContext (an outOfBand RPC) and at most one RPC request that 
+     * follows that. On future passes it will process at most one RPC request.
+     *  
+     * Quirky things: dataLengthBuffer (4 bytes) is used to read "hrpc" OR 
+     * rpc request length.
+     *    
+     * @return -1 in case of error, else num bytes read so far
+     * @throws WrappedRpcServerException - an exception that has already been 
+     *         sent back to the client that does not require verbose logging
+     *         by the Listener thread
+     * @throws IOException - internal error that should not be returned to
+     *         client, typically failure to respond to client
+     * @throws InterruptedException
+     */
     public int readAndProcess()
     public int readAndProcess()
         throws WrappedRpcServerException, IOException, InterruptedException {
         throws WrappedRpcServerException, IOException, InterruptedException {
       while (true) {
       while (true) {
-        /* Read at most one RPC. If the header is not read completely yet
-         * then iterate until we read first RPC or until there is no data left.
-         */    
+        // dataLengthBuffer is used to read "hrpc" or the rpc-packet length
         int count = -1;
         int count = -1;
         if (dataLengthBuffer.remaining() > 0) {
         if (dataLengthBuffer.remaining() > 0) {
           count = channelRead(channel, dataLengthBuffer);       
           count = channelRead(channel, dataLengthBuffer);       
@@ -1416,9 +1471,11 @@ public abstract class Server {
         }
         }
         
         
         if (!connectionHeaderRead) {
         if (!connectionHeaderRead) {
-          //Every connection is expected to send the header.
+          // Every connection is expected to send the header;
+          // so far we read "hrpc" of the connection header.
           if (connectionHeaderBuf == null) {
           if (connectionHeaderBuf == null) {
-            connectionHeaderBuf = ByteBuffer.allocate(3);
+            // for the bytes that follow "hrpc", in the connection header
+            connectionHeaderBuf = ByteBuffer.allocate(HEADER_LEN_AFTER_HRPC_PART);
           }
           }
           count = channelRead(channel, connectionHeaderBuf);
           count = channelRead(channel, connectionHeaderBuf);
           if (count < 0 || connectionHeaderBuf.remaining() > 0) {
           if (count < 0 || connectionHeaderBuf.remaining() > 0) {
@@ -1451,27 +1508,30 @@ public abstract class Server {
           // this may switch us into SIMPLE
           // this may switch us into SIMPLE
           authProtocol = initializeAuthContext(connectionHeaderBuf.get(2));          
           authProtocol = initializeAuthContext(connectionHeaderBuf.get(2));          
           
           
-          dataLengthBuffer.clear();
+          dataLengthBuffer.clear(); // clear to next read rpc packet len
           connectionHeaderBuf = null;
           connectionHeaderBuf = null;
           connectionHeaderRead = true;
           connectionHeaderRead = true;
-          continue;
+          continue; // connection header read, now read  4 bytes rpc packet len
         }
         }
         
         
-        if (data == null) {
+        if (data == null) { // just read 4 bytes -  length of RPC packet
           dataLengthBuffer.flip();
           dataLengthBuffer.flip();
           dataLength = dataLengthBuffer.getInt();
           dataLength = dataLengthBuffer.getInt();
           checkDataLength(dataLength);
           checkDataLength(dataLength);
+          // Set buffer for reading EXACTLY the RPC-packet length and no more.
           data = ByteBuffer.allocate(dataLength);
           data = ByteBuffer.allocate(dataLength);
         }
         }
-        
+        // Now read the RPC packet
         count = channelRead(channel, data);
         count = channelRead(channel, data);
         
         
         if (data.remaining() == 0) {
         if (data.remaining() == 0) {
-          dataLengthBuffer.clear();
+          dataLengthBuffer.clear(); // to read length of future rpc packets
           data.flip();
           data.flip();
           boolean isHeaderRead = connectionContextRead;
           boolean isHeaderRead = connectionContextRead;
           processOneRpc(data.array());
           processOneRpc(data.array());
           data = null;
           data = null;
+          // the last rpc-request we processed could have simply been the
+          // connectionContext; if so continue to read the first RPC.
           if (!isHeaderRead) {
           if (!isHeaderRead) {
             continue;
             continue;
           }
           }
@@ -1508,8 +1568,16 @@ public abstract class Server {
       return authProtocol;
       return authProtocol;
     }
     }
 
 
+    /**
+     * Process the Sasl's Negotiate request, including the optimization of 
+     * accelerating token negotiation.
+     * @return the response to Negotiate request - the list of enabled 
+     *         authMethods and challenge if the TOKENS are supported. 
+     * @throws SaslException - if attempt to generate challenge fails.
+     * @throws IOException - if it fails to create the SASL server for Tokens
+     */
     private RpcSaslProto buildSaslNegotiateResponse()
     private RpcSaslProto buildSaslNegotiateResponse()
-        throws IOException, InterruptedException {
+        throws InterruptedException, SaslException, IOException {
       RpcSaslProto negotiateMessage = negotiateResponse;
       RpcSaslProto negotiateMessage = negotiateResponse;
       // accelerate token negotiation by sending initial challenge
       // accelerate token negotiation by sending initial challenge
       // in the negotiation response
       // in the negotiation response
@@ -1635,8 +1703,11 @@ public abstract class Server {
     /**
     /**
      * Process a wrapped RPC Request - unwrap the SASL packet and process
      * Process a wrapped RPC Request - unwrap the SASL packet and process
      * each embedded RPC request 
      * each embedded RPC request 
-     * @param buf - SASL wrapped request of one or more RPCs
+     * @param inBuf - SASL wrapped request of one or more RPCs
      * @throws IOException - SASL packet cannot be unwrapped
      * @throws IOException - SASL packet cannot be unwrapped
+     * @throws WrappedRpcServerException - an exception that has already been 
+     *         sent back to the client that does not require verbose logging
+     *         by the Listener thread
      * @throws InterruptedException
      * @throws InterruptedException
      */    
      */    
     private void unwrapPacketAndProcessRpcs(byte[] inBuf)
     private void unwrapPacketAndProcessRpcs(byte[] inBuf)
@@ -1677,13 +1748,21 @@ public abstract class Server {
     }
     }
     
     
     /**
     /**
-     * Process an RPC Request - handle connection setup and decoding of
-     * request into a Call
+     * Process one RPC Request from buffer read from socket stream 
+     *  - decode rpc in a rpc-Call
+     *  - handle out-of-band RPC requests such as the initial connectionContext
+     *  - A successfully decoded RpcCall will be deposited in RPC-Q and
+     *    its response will be sent later when the request is processed.
+     * 
+     * Prior to this call the connectionHeader ("hrpc...") has been handled and
+     * if SASL then SASL has been established and the buf we are passed
+     * has been unwrapped from SASL.
+     * 
      * @param buf - contains the RPC request header and the rpc request
      * @param buf - contains the RPC request header and the rpc request
      * @throws IOException - internal error that should not be returned to
      * @throws IOException - internal error that should not be returned to
      *         client, typically failure to respond to client
      *         client, typically failure to respond to client
-     * @throws WrappedRpcServerException - an exception to be sent back to
-     *         the client that does not require verbose logging by the
+     * @throws WrappedRpcServerException - an exception that is sent back to the
+     *         client in this method and does not require verbose logging by the
      *         Listener thread
      *         Listener thread
      * @throws InterruptedException
      * @throws InterruptedException
      */    
      */    
@@ -1753,8 +1832,11 @@ public abstract class Server {
     }
     }
 
 
     /**
     /**
-     * Process an RPC Request - the connection headers and context must
-     * have been already read
+     * Process an RPC Request 
+     *   - the connection headers and context must have been already read.
+     *   - Based on the rpcKind, decode the rpcRequest.
+     *   - A successfully decoded RpcCall will be deposited in RPC-Q and
+     *     its response will be sent later when the request is processed.
      * @param header - RPC request header
      * @param header - RPC request header
      * @param dis - stream to request payload
      * @param dis - stream to request payload
      * @throws WrappedRpcServerException - due to fatal rpc layer issues such
      * @throws WrappedRpcServerException - due to fatal rpc layer issues such
@@ -1803,7 +1885,8 @@ public abstract class Server {
      * @param dis - stream to request payload
      * @param dis - stream to request payload
      * @throws WrappedRpcServerException - setup failed due to SASL
      * @throws WrappedRpcServerException - setup failed due to SASL
      *         negotiation failure, premature or invalid connection context,
      *         negotiation failure, premature or invalid connection context,
-     *         or other state errors 
+     *         or other state errors. This exception needs to be sent to the 
+     *         client.
      * @throws IOException - failed to send a response back to the client
      * @throws IOException - failed to send a response back to the client
      * @throws InterruptedException
      * @throws InterruptedException
      */
      */

+ 41 - 6
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/hadoop_user_info.c

@@ -122,13 +122,43 @@ int hadoop_user_info_fetch(struct hadoop_user_info *uinfo,
   }
   }
 }
 }
 
 
+static int put_primary_gid_first(struct hadoop_user_info *uinfo)
+{
+  int i, num_gids = uinfo->num_gids;
+  gid_t first_gid;
+  gid_t gid;
+  gid_t primary = uinfo->pwd.pw_gid;
+
+  if (num_gids < 1) {
+    // There are no gids, but we expected at least one.
+    return EINVAL;
+  }
+  first_gid = uinfo->gids[0];
+  if (first_gid == primary) {
+    // First gid is already the primary.
+    return 0;
+  }
+  for (i = 1; i < num_gids; i++) {
+    gid = uinfo->gids[i];
+    if (gid == primary) {
+      // swap first gid and this gid.
+      uinfo->gids[0] = gid;
+      uinfo->gids[i] = first_gid;
+      return 0;
+    }
+  }
+  // Did not find the primary gid in the list.
+  return EINVAL;
+}
+
 int hadoop_user_info_getgroups(struct hadoop_user_info *uinfo)
 int hadoop_user_info_getgroups(struct hadoop_user_info *uinfo)
 {
 {
   int ret, ngroups;
   int ret, ngroups;
   gid_t *ngids;
   gid_t *ngids;
 
 
   if (!uinfo->pwd.pw_name) {
   if (!uinfo->pwd.pw_name) {
-    return EINVAL; // invalid user info
+    // invalid user info
+    return EINVAL;
   }
   }
   uinfo->num_gids = 0;
   uinfo->num_gids = 0;
   if (!uinfo->gids) {
   if (!uinfo->gids) {
@@ -141,8 +171,12 @@ int hadoop_user_info_getgroups(struct hadoop_user_info *uinfo)
   ngroups = uinfo->gids_size;
   ngroups = uinfo->gids_size;
   ret = getgrouplist(uinfo->pwd.pw_name, uinfo->pwd.pw_gid, 
   ret = getgrouplist(uinfo->pwd.pw_name, uinfo->pwd.pw_gid, 
                          uinfo->gids, &ngroups);
                          uinfo->gids, &ngroups);
-  if (ret != -1) {
+  if (ret > 0) {
     uinfo->num_gids = ngroups;
     uinfo->num_gids = ngroups;
+    ret = put_primary_gid_first(uinfo);
+    if (ret) {
+      return ret;
+    }
     return 0;
     return 0;
   }
   }
   ngids = realloc(uinfo->gids, sizeof(uinfo->gids[0]) * ngroups);
   ngids = realloc(uinfo->gids, sizeof(uinfo->gids[0]) * ngroups);
@@ -153,11 +187,12 @@ int hadoop_user_info_getgroups(struct hadoop_user_info *uinfo)
   uinfo->gids_size = ngroups;
   uinfo->gids_size = ngroups;
   ret = getgrouplist(uinfo->pwd.pw_name, uinfo->pwd.pw_gid, 
   ret = getgrouplist(uinfo->pwd.pw_name, uinfo->pwd.pw_gid, 
                          uinfo->gids, &ngroups);
                          uinfo->gids, &ngroups);
-  if (ret != -1) {
-    uinfo->num_gids = ngroups;
-    return 0;
+  if (ret < 0) {
+    return EIO;
   }
   }
-  return EIO;
+  uinfo->num_gids = ngroups;
+  ret = put_primary_gid_first(uinfo);
+  return ret;
 }
 }
 
 
 #ifdef USER_TESTING
 #ifdef USER_TESTING

+ 176 - 27
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -13,6 +13,10 @@ Trunk (Unreleased)
 
 
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
 
 
+    HDFS-2832. Heterogeneous Storages support in HDFS phase 1 - treat DataNode
+    as a collection of storages (see breakdown of tasks below for features and
+    contributors).
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
@@ -242,6 +246,9 @@ Trunk (Unreleased)
     HDFS-5629. Support HTTPS in JournalNode and SecondaryNameNode. 
     HDFS-5629. Support HTTPS in JournalNode and SecondaryNameNode. 
     (Haohui Mai via jing9)
     (Haohui Mai via jing9)
 
 
+    HDFS-5647. Merge INodeDirectory.Feature and INodeFile.Feature. (Haohui Mai
+    via jing9)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 
 
@@ -440,6 +447,131 @@ Trunk (Unreleased)
 
 
     HDFS-5626. dfsadmin -report shows incorrect cache values. (cmccabe)
     HDFS-5626. dfsadmin -report shows incorrect cache values. (cmccabe)
 
 
+  BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
+
+    HDFS-4985. Add storage type to the protocol and expose it in block report
+    and block locations. (Arpit Agarwal)
+
+    HDFS-5115. Make StorageID a UUID. (Arpit Agarwal)
+
+    HDFS-5000. DataNode configuration should allow specifying storage type.
+    (Arpit Agarwal)
+
+    HDFS-4987. Namenode changes to track multiple storages per datanode.
+    (szetszwo)
+
+    HDFS-5154. Fix TestBlockManager and TestDatanodeDescriptor after HDFS-4987.
+    (Junping Du via szetszwo)
+
+    HDFS-5009. Include storage information in the LocatedBlock.  (szetszwo)
+
+    HDFS-5134. Move blockContentsStale, heartbeatedSinceFailover and
+    firstBlockReport from DatanodeDescriptor to DatanodeStorageInfo; and
+    fix a synchronization problem in DatanodeStorageInfo.  (szetszwo)
+
+    HDFS-5157. Add StorageType to FsVolume.  (Junping Du via szetszwo)
+
+    HDFS-4990. Change BlockPlacementPolicy to choose storages instead of
+    datanodes.  (szetszwo)
+
+    HDFS-5232. Protocol changes to transmit StorageUuid. (Arpit Agarwal)
+
+    HDFS-5233. Use Datanode UUID to identify Datanodes. (Arpit Agarwal)
+
+    HDFS-5222. Move block schedule information from DatanodeDescriptor to
+    DatanodeStorageInfo.  (szetszwo)
+
+    HDFS-4988. Datanode must support all the volumes as individual storages.
+    (Arpit Agarwal)
+
+    HDFS-5377. Heartbeats from Datandode should include one storage report
+    per storage directory. (Arpit Agarwal)
+
+    HDFS-5398. NameNode changes to process storage reports per storage
+    directory. (Arpit Agarwal)
+
+    HDFS-5390. Send one incremental block report per storage directory.
+    (Arpit Agarwal)
+
+    HDFS-5401. Fix NPE in Directory Scanner. (Arpit Agarwal)
+
+    HDFS-5417. Fix storage IDs in PBHelper and UpgradeUtilities.  (szetszwo)
+
+    HDFS-5214. Fix NPEs in BlockManager and DirectoryScanner. (Arpit Agarwal)
+
+    HDFS-5435. File append fails to initialize storageIDs. (Junping Du via
+    Arpit Agarwal)
+
+    HDFS-5437. Fix TestBlockReport and TestBPOfferService failures. (Arpit
+    Agarwal)
+
+    HDFS-5447. Fix TestJspHelper. (Arpit Agarwal)
+
+    HDFS-5452. Fix TestReplicationPolicy and TestBlocksScheduledCounter.
+
+    HDFS-5448. Datanode should generate its ID on first registration. (Arpit
+    Agarwal)
+
+    HDFS-5448. Fix break caused by previous checkin for HDFS-5448. (Arpit
+    Agarwal)
+
+    HDFS-5455. NN should update storageMap on first heartbeat. (Arpit Agarwal)
+
+    HDFS-5457. Fix TestDatanodeRegistration, TestFsck and TestAddBlockRetry.
+    (Contributed by szetszwo)
+
+    HDFS-5466. Update storage IDs when the pipeline is updated. (Contributed
+    by szetszwo)
+
+    HDFS-5439. Fix TestPendingReplication. (Contributed by Junping Du, Arpit
+    Agarwal)
+
+    HDFS-5470. Add back trunk's reportDiff algorithm to the branch.
+    (Contributed by szetszwo)
+
+    HDFS-5472. Fix TestDatanodeManager, TestSafeMode and
+    TestNNThroughputBenchmark (Contributed by szetszwo)
+
+    HDFS-5475. NN incorrectly tracks more than one replica per DN. (Arpit
+    Agarwal)
+
+    HDFS-5481. Fix TestDataNodeVolumeFailure in branch HDFS-2832. (Contributed
+    by Junping Du)
+
+    HDFS-5480. Update Balancer for HDFS-2832. (Contributed by szetszwo)
+
+    HDFS-5486. Fix TestNameNodeMetrics for HDFS-2832. (Arpit Agarwal)
+
+    HDFS-5491. Update editsStored for HDFS-2832. (Arpit Agarwal)
+
+    HDFS-5494. Fix findbugs warnings for HDFS-2832. (Arpit Agarwal)
+
+    HDFS-5508. Fix compilation error after merge. (Contributed by szetszwo)
+
+    HDFS-5501. Fix pendingReceivedRequests tracking in BPServiceActor. (Arpit
+    Agarwal)
+
+    HDFS-5510. Fix a findbug warning in DataStorage.java on HDFS-2832 branch.
+    (Junping Du via Arpit Agarwal)
+ 
+    HDFS-5515. Fix TestDFSStartupVersions for HDFS-2832. (Arpit Agarwal)
+
+    HDFS-5527. Fix TestUnderReplicatedBlocks on branch HDFS-2832. (Arpit
+    Agarwal)
+
+    HDFS-5547. Fix build break after merge from trunk to HDFS-2832. (Arpit
+    Agarwal)
+
+    HDFS-5542. Fix TODO and clean up the code in HDFS-2832. (Contributed by
+    szetszwo)
+
+    HDFS-5559. Fix TestDatanodeConfig in HDFS-2832. (Contributed by szetszwo)
+
+    HDFS-5484. StorageType and State in DatanodeStorageInfo in NameNode is
+    not accurate. (Eric Sirianni via Arpit Agarwal)
+
+    HDFS-5648. Get rid of FsDatasetImpl#perVolumeReplicaMap. (Arpit Agarwal)
+
 Release 2.4.0 - UNRELEASED
 Release 2.4.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -587,6 +719,15 @@ Release 2.4.0 - UNRELEASED
 
 
     HDFS-5633. Improve OfflineImageViewer to use less memory. (jing9)
     HDFS-5633. Improve OfflineImageViewer to use less memory. (jing9)
 
 
+    HDFS-5023. TestSnapshotPathINodes.testAllowSnapshot is failing with jdk7
+    (Mit Desai via jeagles)
+
+    HDFS-5637. Try to refeatchToken while local read InvalidToken occurred.
+    (Liang Xie via junping_du)
+
+    HDFS-5652. Refactor invalid block token exception handling in DFSInputStream.
+    (Liang Xie via junping_du)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
@@ -616,9 +757,6 @@ Release 2.4.0 - UNRELEASED
     HDFS-5352. Server#initLog() doesn't close InputStream in httpfs. (Ted Yu via
     HDFS-5352. Server#initLog() doesn't close InputStream in httpfs. (Ted Yu via
     jing9)
     jing9)
 
 
-    HDFS-5283. Under construction blocks only inside snapshots should not be
-    counted in safemode threshhold.  (Vinay via szetszwo)
-
     HDFS-4376. Fix race conditions in Balancer.  (Junping Du via szetszwo)
     HDFS-4376. Fix race conditions in Balancer.  (Junping Du via szetszwo)
 
 
     HDFS-5375. hdfs.cmd does not expose several snapshot commands. (cnauroth)
     HDFS-5375. hdfs.cmd does not expose several snapshot commands. (cnauroth)
@@ -629,43 +767,22 @@ Release 2.4.0 - UNRELEASED
     HDFS-5400.  DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT constant is set
     HDFS-5400.  DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT constant is set
     to the wrong value.  (Colin Patrick McCabe)
     to the wrong value.  (Colin Patrick McCabe)
 
 
-    HDFS-5257. addBlock() retry should return LocatedBlock with locations else client 
-    will get AIOBE. (Vinay via jing9)
-
-    HDFS-5427. Not able to read deleted files from snapshot directly under 
-    snapshottable dir after checkpoint and NN restart. (Vinay via jing9)
-
-    HDFS-5443. Delete 0-sized block when deleting an under-construction file that 
-    is included in snapshot. (jing9)
-
-    HDFS-5476. Snapshot: clean the blocks/files/directories under a renamed 
-    file/directory while deletion. (jing9)
-
-    HDFS-5425. Renaming underconstruction file with snapshots can make NN failure on 
-    restart. (jing9 and Vinay)
-
-    HDFS-5474. Deletesnapshot can make Namenode in safemode on NN restarts. 
-    (Sathish via jing9)
-
     HDFS-5075. httpfs-config.sh calls out incorrect env script name
     HDFS-5075. httpfs-config.sh calls out incorrect env script name
     (Timothy St. Clair via stevel)
     (Timothy St. Clair via stevel)
 
 
-    HDFS-5504. In HA mode, OP_DELETE_SNAPSHOT is not decrementing the safemode threshold, 
-    leads to NN safemode. (Vinay via jing9)
-
     HDFS-5438. Flaws in block report processing can cause data loss. (kihwal)
     HDFS-5438. Flaws in block report processing can cause data loss. (kihwal)
 
 
     HDFS-5502. Fix HTTPS support in HsftpFileSystem. (Haohui Mai via jing9)
     HDFS-5502. Fix HTTPS support in HsftpFileSystem. (Haohui Mai via jing9)
 
 
-    HDFS-5428. Under construction files deletion after snapshot+checkpoint+nn restart 
-    leads nn safemode. (jing9)
-
     HDFS-5552. Fix wrong information of "Cluster summay" in dfshealth.html.
     HDFS-5552. Fix wrong information of "Cluster summay" in dfshealth.html.
     (Haohui Mai via jing9)
     (Haohui Mai via jing9)
 
 
     HDFS-5533. Symlink delete/create should be treated as DELETE/CREATE in snapshot diff 
     HDFS-5533. Symlink delete/create should be treated as DELETE/CREATE in snapshot diff 
     report. (Binglin Chang via jing9)
     report. (Binglin Chang via jing9)
 
 
+    HDFS-5580. Fix infinite loop in Balancer.waitForMoveCompletion.
+    (Binglin Chang via junping_du)
+
 Release 2.3.0 - UNRELEASED
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -804,6 +921,38 @@ Release 2.3.0 - UNRELEASED
     HDFS-5353. Short circuit reads fail when dfs.encrypt.data.transfer is 
     HDFS-5353. Short circuit reads fail when dfs.encrypt.data.transfer is 
     enabled. (Colin Patrick McCabe via jing9)
     enabled. (Colin Patrick McCabe via jing9)
 
 
+    HDFS-5283. Under construction blocks only inside snapshots should not be
+    counted in safemode threshhold.  (Vinay via szetszwo)
+
+    HDFS-5257. addBlock() retry should return LocatedBlock with locations else client 
+    will get AIOBE. (Vinay via jing9)
+
+    HDFS-5427. Not able to read deleted files from snapshot directly under 
+    snapshottable dir after checkpoint and NN restart. (Vinay via jing9)
+
+    HDFS-5443. Delete 0-sized block when deleting an under-construction file that 
+    is included in snapshot. (jing9)
+
+    HDFS-5476. Snapshot: clean the blocks/files/directories under a renamed 
+    file/directory while deletion. (jing9)
+
+    HDFS-5425. Renaming underconstruction file with snapshots can make NN failure on 
+    restart. (jing9 and Vinay)
+
+    HDFS-5474. Deletesnapshot can make Namenode in safemode on NN restarts. 
+    (Sathish via jing9)
+
+    HDFS-5504. In HA mode, OP_DELETE_SNAPSHOT is not decrementing the safemode threshold, 
+    leads to NN safemode. (Vinay via jing9)
+
+    HDFS-5428. Under construction files deletion after snapshot+checkpoint+nn restart 
+    leads nn safemode. (jing9)
+
+    HDFS-5074. Allow starting up from an fsimage checkpoint in the middle of a
+    segment. (Todd Lipcon via atm)
+
+    HDFS-4201. NPE in BPServiceActor#sendHeartBeat. (jxiang via cmccabe)
+
 Release 2.2.0 - 2013-10-13
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 1 - 6
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java

@@ -500,14 +500,9 @@ public class BookKeeperJournalManager implements JournalManager {
     } 
     } 
   }
   }
 
 
-  public void selectInputStreams(Collection<EditLogInputStream> streams,
-      long fromTxId, boolean inProgressOk) throws IOException {
-    selectInputStreams(streams, fromTxId, inProgressOk, true);
-  }
-
   @Override
   @Override
   public void selectInputStreams(Collection<EditLogInputStream> streams,
   public void selectInputStreams(Collection<EditLogInputStream> streams,
-      long fromTxId, boolean inProgressOk, boolean forReading)
+      long fromTxId, boolean inProgressOk)
       throws IOException {
       throws IOException {
     List<EditLogLedgerMetadata> currentLedgerList = getLedgerList(fromTxId,
     List<EditLogLedgerMetadata> currentLedgerList = getLedgerList(fromTxId,
         inProgressOk);
         inProgressOk);

+ 32 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -65,6 +65,7 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.IdentityHashStore;
 import org.apache.hadoop.util.IdentityHashStore;
 
 
@@ -590,20 +591,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           // The encryption key used is invalid.
           // The encryption key used is invalid.
           refetchEncryptionKey--;
           refetchEncryptionKey--;
           dfsClient.clearDataEncryptionKey();
           dfsClient.clearDataEncryptionKey();
-        } else if (ex instanceof InvalidBlockTokenException && refetchToken > 0) {
-          DFSClient.LOG.info("Will fetch a new access token and retry, " 
-              + "access token was invalid when connecting to " + targetAddr
-              + " : " + ex);
-          /*
-           * Get a new access token and retry. Retry is needed in 2 cases. 1)
-           * When both NN and DN re-started while DFSClient holding a cached
-           * access token. 2) In the case that NN fails to update its
-           * access key at pre-set interval (by a wide margin) and
-           * subsequently restarts. In this case, DN re-registers itself with
-           * NN and receives a new access key, but DN will delete the old
-           * access key from its memory since it's considered expired based on
-           * the estimated expiration date.
-           */
+        } else if (refetchToken > 0 && tokenRefetchNeeded(ex, targetAddr)) {
           refetchToken--;
           refetchToken--;
           fetchBlockAt(target);
           fetchBlockAt(target);
         } else {
         } else {
@@ -976,10 +964,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           // The encryption key used is invalid.
           // The encryption key used is invalid.
           refetchEncryptionKey--;
           refetchEncryptionKey--;
           dfsClient.clearDataEncryptionKey();
           dfsClient.clearDataEncryptionKey();
-        } else if (e instanceof InvalidBlockTokenException && refetchToken > 0) {
-          DFSClient.LOG.info("Will get a new access token and retry, "
-              + "access token was invalid when connecting to " + targetAddr
-              + " : " + e);
+        } else if (refetchToken > 0 && tokenRefetchNeeded(e, targetAddr)) {
           refetchToken--;
           refetchToken--;
           fetchBlockAt(block.getStartOffset());
           fetchBlockAt(block.getStartOffset());
           continue;
           continue;
@@ -1000,6 +985,34 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
     }
   }
   }
 
 
+  /**
+   * Should the block access token be refetched on an exception
+   * 
+   * @param ex Exception received
+   * @param targetAddr Target datanode address from where exception was received
+   * @return true if block access token has expired or invalid and it should be
+   *         refetched
+   */
+  private static boolean tokenRefetchNeeded(IOException ex,
+      InetSocketAddress targetAddr) {
+    /*
+     * Get a new access token and retry. Retry is needed in 2 cases. 1)
+     * When both NN and DN re-started while DFSClient holding a cached
+     * access token. 2) In the case that NN fails to update its
+     * access key at pre-set interval (by a wide margin) and
+     * subsequently restarts. In this case, DN re-registers itself with
+     * NN and receives a new access key, but DN will delete the old
+     * access key from its memory since it's considered expired based on
+     * the estimated expiration date.
+     */
+    if (ex instanceof InvalidBlockTokenException || ex instanceof InvalidToken) {
+      DFSClient.LOG.info("Access token was invalid when connecting to "
+          + targetAddr + " : " + ex);
+      return true;
+    }
+    return false;
+  }
+
   private Peer newTcpPeer(InetSocketAddress addr) throws IOException {
   private Peer newTcpPeer(InetSocketAddress addr) throws IOException {
     Peer peer = null;
     Peer peer = null;
     boolean success = false;
     boolean success = false;
@@ -1344,7 +1357,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
        * deadNodes and added currentNode again. Thats ok. */
        * deadNodes and added currentNode again. Thats ok. */
       deadNodes.remove(oldNode);
       deadNodes.remove(oldNode);
     }
     }
-    if (!oldNode.getStorageID().equals(newNode.getStorageID())) {
+    if (!oldNode.getDatanodeUuid().equals(newNode.getDatanodeUuid())) {
       currentNode = newNode;
       currentNode = newNode;
       return true;
       return true;
     } else {
     } else {

+ 27 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -312,6 +312,7 @@ public class DFSOutputStream extends FSOutputSummer
     private DataInputStream blockReplyStream;
     private DataInputStream blockReplyStream;
     private ResponseProcessor response = null;
     private ResponseProcessor response = null;
     private volatile DatanodeInfo[] nodes = null; // list of targets for current block
     private volatile DatanodeInfo[] nodes = null; // list of targets for current block
+    private volatile String[] storageIDs = null;
     private LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes =
     private LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes =
         CacheBuilder.newBuilder()
         CacheBuilder.newBuilder()
         .expireAfterWrite(
         .expireAfterWrite(
@@ -402,7 +403,7 @@ public class DFSOutputStream extends FSOutputSummer
       }
       }
 
 
       // setup pipeline to append to the last block XXX retries??
       // setup pipeline to append to the last block XXX retries??
-      nodes = lastBlock.getLocations();
+      setPipeline(lastBlock);
       errorIndex = -1;   // no errors yet.
       errorIndex = -1;   // no errors yet.
       if (nodes.length < 1) {
       if (nodes.length < 1) {
         throw new IOException("Unable to retrieve blocks locations " +
         throw new IOException("Unable to retrieve blocks locations " +
@@ -411,6 +412,14 @@ public class DFSOutputStream extends FSOutputSummer
 
 
       }
       }
     }
     }
+    
+    private void setPipeline(LocatedBlock lb) {
+      setPipeline(lb.getLocations(), lb.getStorageIDs());
+    }
+    private void setPipeline(DatanodeInfo[] nodes, String[] storageIDs) {
+      this.nodes = nodes;
+      this.storageIDs = storageIDs;
+    }
 
 
     private void setFavoredNodes(String[] favoredNodes) {
     private void setFavoredNodes(String[] favoredNodes) {
       this.favoredNodes = favoredNodes;
       this.favoredNodes = favoredNodes;
@@ -434,7 +443,7 @@ public class DFSOutputStream extends FSOutputSummer
       this.setName("DataStreamer for file " + src);
       this.setName("DataStreamer for file " + src);
       closeResponder();
       closeResponder();
       closeStream();
       closeStream();
-      nodes = null;
+      setPipeline(null, null);
       stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
       stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
     }
     }
     
     
@@ -503,7 +512,7 @@ public class DFSOutputStream extends FSOutputSummer
             if(DFSClient.LOG.isDebugEnabled()) {
             if(DFSClient.LOG.isDebugEnabled()) {
               DFSClient.LOG.debug("Allocating new block");
               DFSClient.LOG.debug("Allocating new block");
             }
             }
-            nodes = nextBlockOutputStream();
+            setPipeline(nextBlockOutputStream());
             initDataStreaming();
             initDataStreaming();
           } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
           } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
             if(DFSClient.LOG.isDebugEnabled()) {
             if(DFSClient.LOG.isDebugEnabled()) {
@@ -917,9 +926,10 @@ public class DFSOutputStream extends FSOutputSummer
       //get a new datanode
       //get a new datanode
       final DatanodeInfo[] original = nodes;
       final DatanodeInfo[] original = nodes;
       final LocatedBlock lb = dfsClient.namenode.getAdditionalDatanode(
       final LocatedBlock lb = dfsClient.namenode.getAdditionalDatanode(
-          src, block, nodes, failed.toArray(new DatanodeInfo[failed.size()]),
+          src, block, nodes, storageIDs,
+          failed.toArray(new DatanodeInfo[failed.size()]),
           1, dfsClient.clientName);
           1, dfsClient.clientName);
-      nodes = lb.getLocations();
+      setPipeline(lb);
 
 
       //find the new datanode
       //find the new datanode
       final int d = findNewDatanode(original);
       final int d = findNewDatanode(original);
@@ -1019,7 +1029,14 @@ public class DFSOutputStream extends FSOutputSummer
           System.arraycopy(nodes, 0, newnodes, 0, errorIndex);
           System.arraycopy(nodes, 0, newnodes, 0, errorIndex);
           System.arraycopy(nodes, errorIndex+1, newnodes, errorIndex,
           System.arraycopy(nodes, errorIndex+1, newnodes, errorIndex,
               newnodes.length-errorIndex);
               newnodes.length-errorIndex);
-          nodes = newnodes;
+
+          final String[] newStorageIDs = new String[newnodes.length];
+          System.arraycopy(storageIDs, 0, newStorageIDs, 0, errorIndex);
+          System.arraycopy(storageIDs, errorIndex+1, newStorageIDs, errorIndex,
+              newStorageIDs.length-errorIndex);
+          
+          setPipeline(newnodes, newStorageIDs);
+
           hasError = false;
           hasError = false;
           lastException.set(null);
           lastException.set(null);
           errorIndex = -1;
           errorIndex = -1;
@@ -1055,7 +1072,8 @@ public class DFSOutputStream extends FSOutputSummer
         // update pipeline at the namenode
         // update pipeline at the namenode
         ExtendedBlock newBlock = new ExtendedBlock(
         ExtendedBlock newBlock = new ExtendedBlock(
             block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS);
             block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS);
-        dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock, nodes);
+        dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
+            nodes, storageIDs);
         // update client side generation stamp
         // update client side generation stamp
         block = newBlock;
         block = newBlock;
       }
       }
@@ -1068,7 +1086,7 @@ public class DFSOutputStream extends FSOutputSummer
      * Must get block ID and the IDs of the destinations from the namenode.
      * Must get block ID and the IDs of the destinations from the namenode.
      * Returns the list of target datanodes.
      * Returns the list of target datanodes.
      */
      */
-    private DatanodeInfo[] nextBlockOutputStream() throws IOException {
+    private LocatedBlock nextBlockOutputStream() throws IOException {
       LocatedBlock lb = null;
       LocatedBlock lb = null;
       DatanodeInfo[] nodes = null;
       DatanodeInfo[] nodes = null;
       int count = dfsClient.getConf().nBlockWriteRetry;
       int count = dfsClient.getConf().nBlockWriteRetry;
@@ -1110,7 +1128,7 @@ public class DFSOutputStream extends FSOutputSummer
       if (!success) {
       if (!success) {
         throw new IOException("Unable to create new block.");
         throw new IOException("Unable to create new block.");
       }
       }
-      return nodes;
+      return lb;
     }
     }
 
 
     // connects to the first datanode in the pipeline
     // connects to the first datanode in the pipeline

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -145,6 +145,23 @@ public class DFSUtil {
     return SECURE_RANDOM.get();
     return SECURE_RANDOM.get();
   }
   }
 
 
+  /** Shuffle the elements in the given array. */
+  public static <T> T[] shuffle(final T[] array) {
+    if (array != null && array.length > 0) {
+      final Random random = getRandom();
+      for (int n = array.length; n > 1; ) {
+        final int randomIndex = random.nextInt(n);
+        n--;
+        if (n != randomIndex) {
+          final T tmp = array[randomIndex];
+          array[randomIndex] = array[n];
+          array[n] = tmp;
+        }
+      }
+    }
+    return array;
+  }
+
   /**
   /**
    * Compartor for sorting DataNodeInfo[] based on decommissioned states.
    * Compartor for sorting DataNodeInfo[] based on decommissioned states.
    * Decommissioned nodes are moved to the end of the array on sorting with
    * Decommissioned nodes are moved to the end of the array on sorting with

+ 35 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StorageType.java

@@ -0,0 +1,35 @@
+/**
+ * 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 org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Defines the types of supported storage media. The default storage
+ * medium is assumed to be DISK.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public enum StorageType {
+  DISK,
+  SSD;
+
+  public static StorageType DEFAULT = DISK;
+}

+ 19 - 22
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java

@@ -19,7 +19,9 @@ package org.apache.hadoop.hdfs.protocol;
 
 
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
+import java.util.Random;
 
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
@@ -250,33 +252,28 @@ public class BlockListAsLongs implements Iterable<Block> {
   }
   }
 
 
   /**
   /**
-   * The block-id of the indexTh block
-   * @param index - the block whose block-id is desired
-   * @return the block-id
+   * Corrupt the generation stamp of the block with the given index.
+   * Not meant to be used outside of tests.
    */
    */
-  @Deprecated
-  public long getBlockId(final int index)  {
-    return blockId(index);
-  }
-  
-  /**
-   * The block-len of the indexTh block
-   * @param index - the block whose block-len is desired
-   * @return - the block-len
-   */
-  @Deprecated
-  public long getBlockLen(final int index)  {
-    return blockLength(index);
+  @VisibleForTesting
+  public long corruptBlockGSForTesting(final int blockIndex, Random rand) {
+    long oldGS = blockList[index2BlockId(blockIndex) + 2];
+    while (blockList[index2BlockId(blockIndex) + 2] == oldGS) {
+      blockList[index2BlockId(blockIndex) + 2] = rand.nextInt();
+    }
+    return oldGS;
   }
   }
 
 
   /**
   /**
-   * The generation stamp of the indexTh block
-   * @param index - the block whose block-len is desired
-   * @return - the generation stamp
+   * Corrupt the length of the block with the given index by truncation.
+   * Not meant to be used outside of tests.
    */
    */
-  @Deprecated
-  public long getBlockGenStamp(final int index)  {
-    return blockGenerationStamp(index);
+  @VisibleForTesting
+  public long corruptBlockLengthForTesting(final int blockIndex, Random rand) {
+    long oldLength = blockList[index2BlockId(blockIndex) + 1];
+    blockList[index2BlockId(blockIndex) + 1] =
+        rand.nextInt((int) oldLength - 1);
+    return oldLength;
   }
   }
   
   
   /**
   /**

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -357,7 +357,8 @@ public interface ClientProtocol {
    */
    */
   @Idempotent
   @Idempotent
   public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk,
   public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk,
-      final DatanodeInfo[] existings, final DatanodeInfo[] excludes,
+      final DatanodeInfo[] existings, final String[] existingStorageIDs,
+      final DatanodeInfo[] excludes,
       final int numAdditionalNodes, final String clientName
       final int numAdditionalNodes, final String clientName
       ) throws AccessControlException, FileNotFoundException,
       ) throws AccessControlException, FileNotFoundException,
           SafeModeException, UnresolvedLinkException, IOException;
           SafeModeException, UnresolvedLinkException, IOException;
@@ -986,7 +987,7 @@ public interface ClientProtocol {
    */
    */
   @AtMostOnce
   @AtMostOnce
   public void updatePipeline(String clientName, ExtendedBlock oldBlock, 
   public void updatePipeline(String clientName, ExtendedBlock oldBlock, 
-      ExtendedBlock newBlock, DatanodeID[] newNodes)
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
       throws IOException;
       throws IOException;
 
 
   /**
   /**

+ 37 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java

@@ -21,6 +21,8 @@ package org.apache.hadoop.hdfs.protocol;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
 /**
  * This class represents the primary identifier for a Datanode.
  * This class represents the primary identifier for a Datanode.
  * Datanodes are identified by how they can be contacted (hostname
  * Datanodes are identified by how they can be contacted (hostname
@@ -40,37 +42,46 @@ public class DatanodeID implements Comparable<DatanodeID> {
   private String ipAddr;     // IP address
   private String ipAddr;     // IP address
   private String hostName;   // hostname claimed by datanode
   private String hostName;   // hostname claimed by datanode
   private String peerHostName; // hostname from the actual connection
   private String peerHostName; // hostname from the actual connection
-  private String storageID;  // unique per cluster storageID
   private int xferPort;      // data streaming port
   private int xferPort;      // data streaming port
   private int infoPort;      // info server port
   private int infoPort;      // info server port
   private int infoSecurePort; // info server port
   private int infoSecurePort; // info server port
   private int ipcPort;       // IPC server port
   private int ipcPort;       // IPC server port
 
 
+  /**
+   * UUID identifying a given datanode. For upgraded Datanodes this is the
+   * same as the StorageID that was previously used by this Datanode. 
+   * For newly formatted Datanodes it is a UUID.
+   */
+  private String datanodeUuid = null;
+
   public DatanodeID(DatanodeID from) {
   public DatanodeID(DatanodeID from) {
     this(from.getIpAddr(),
     this(from.getIpAddr(),
         from.getHostName(),
         from.getHostName(),
-        from.getStorageID(),
+        from.getDatanodeUuid(),
         from.getXferPort(),
         from.getXferPort(),
         from.getInfoPort(),
         from.getInfoPort(),
         from.getInfoSecurePort(),
         from.getInfoSecurePort(),
         from.getIpcPort());
         from.getIpcPort());
     this.peerHostName = from.getPeerHostName();
     this.peerHostName = from.getPeerHostName();
   }
   }
-  
+
   /**
   /**
    * Create a DatanodeID
    * Create a DatanodeID
    * @param ipAddr IP
    * @param ipAddr IP
    * @param hostName hostname
    * @param hostName hostname
-   * @param storageID data storage ID
+   * @param datanodeUuid data node ID, UUID for new Datanodes, may be the
+   *                     storage ID for pre-UUID datanodes. NULL if unknown
+   *                     e.g. if this is a new datanode. A new UUID will
+   *                     be assigned by the namenode.
    * @param xferPort data transfer port
    * @param xferPort data transfer port
    * @param infoPort info server port 
    * @param infoPort info server port 
    * @param ipcPort ipc server port
    * @param ipcPort ipc server port
    */
    */
-  public DatanodeID(String ipAddr, String hostName, String storageID,
+  public DatanodeID(String ipAddr, String hostName, String datanodeUuid,
       int xferPort, int infoPort, int infoSecurePort, int ipcPort) {
       int xferPort, int infoPort, int infoSecurePort, int ipcPort) {
     this.ipAddr = ipAddr;
     this.ipAddr = ipAddr;
     this.hostName = hostName;
     this.hostName = hostName;
-    this.storageID = storageID;
+    this.datanodeUuid = checkDatanodeUuid(datanodeUuid);
     this.xferPort = xferPort;
     this.xferPort = xferPort;
     this.infoPort = infoPort;
     this.infoPort = infoPort;
     this.infoSecurePort = infoSecurePort;
     this.infoSecurePort = infoSecurePort;
@@ -85,8 +96,24 @@ public class DatanodeID implements Comparable<DatanodeID> {
     this.peerHostName = peerHostName;
     this.peerHostName = peerHostName;
   }
   }
   
   
-  public void setStorageID(String storageID) {
-    this.storageID = storageID;
+  /**
+   * @return data node ID.
+   */
+  public String getDatanodeUuid() {
+    return datanodeUuid;
+  }
+
+  @VisibleForTesting
+  public void setDatanodeUuidForTesting(String datanodeUuid) {
+    this.datanodeUuid = datanodeUuid;
+  }
+
+  private String checkDatanodeUuid(String uuid) {
+    if (uuid == null || uuid.isEmpty()) {
+      return null;
+    } else {
+      return uuid;
+    }
   }
   }
 
 
   /**
   /**
@@ -168,13 +195,6 @@ public class DatanodeID implements Comparable<DatanodeID> {
     return useHostname ? getIpcAddrWithHostname() : getIpcAddr();
     return useHostname ? getIpcAddrWithHostname() : getIpcAddr();
   }
   }
 
 
-  /**
-   * @return data storage ID.
-   */
-  public String getStorageID() {
-    return storageID;
-  }
-
   /**
   /**
    * @return xferPort (the port for data streaming)
    * @return xferPort (the port for data streaming)
    */
    */
@@ -212,12 +232,12 @@ public class DatanodeID implements Comparable<DatanodeID> {
       return false;
       return false;
     }
     }
     return (getXferAddr().equals(((DatanodeID)to).getXferAddr()) &&
     return (getXferAddr().equals(((DatanodeID)to).getXferAddr()) &&
-            storageID.equals(((DatanodeID)to).getStorageID()));
+        datanodeUuid.equals(((DatanodeID)to).getDatanodeUuid()));
   }
   }
   
   
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
-    return getXferAddr().hashCode()^ storageID.hashCode();
+    return getXferAddr().hashCode()^ datanodeUuid.hashCode();
   }
   }
   
   
   @Override
   @Override

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -115,7 +115,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
       final long lastUpdate, final int xceiverCount,
       final long lastUpdate, final int xceiverCount,
       final AdminStates adminState) {
       final AdminStates adminState) {
-    this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getStorageID(),
+    this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getDatanodeUuid(),
         nodeID.getXferPort(), nodeID.getInfoPort(), nodeID.getInfoSecurePort(),
         nodeID.getXferPort(), nodeID.getInfoPort(), nodeID.getInfoSecurePort(),
         nodeID.getIpcPort(), capacity, dfsUsed, remaining, blockPoolUsed,
         nodeID.getIpcPort(), capacity, dfsUsed, remaining, blockPoolUsed,
         cacheCapacity, cacheUsed, lastUpdate, xceiverCount, location,
         cacheCapacity, cacheUsed, lastUpdate, xceiverCount, location,
@@ -124,13 +124,13 @@ public class DatanodeInfo extends DatanodeID implements Node {
 
 
   /** Constructor */
   /** Constructor */
   public DatanodeInfo(final String ipAddr, final String hostName,
   public DatanodeInfo(final String ipAddr, final String hostName,
-      final String storageID, final int xferPort, final int infoPort,
+      final String datanodeUuid, final int xferPort, final int infoPort,
       final int infoSecurePort, final int ipcPort,
       final int infoSecurePort, final int ipcPort,
       final long capacity, final long dfsUsed, final long remaining,
       final long capacity, final long dfsUsed, final long remaining,
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
       final long lastUpdate, final int xceiverCount,
       final long lastUpdate, final int xceiverCount,
       final String networkLocation, final AdminStates adminState) {
       final String networkLocation, final AdminStates adminState) {
-    super(ipAddr, hostName, storageID, xferPort, infoPort,
+    super(ipAddr, hostName, datanodeUuid, xferPort, infoPort,
             infoSecurePort, ipcPort);
             infoSecurePort, ipcPort);
     this.capacity = capacity;
     this.capacity = capacity;
     this.dfsUsed = dfsUsed;
     this.dfsUsed = dfsUsed;

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java

@@ -107,7 +107,10 @@ public class LayoutVersion {
         "block IDs in the edits log and image files"),
         "block IDs in the edits log and image files"),
     EDITLOG_SUPPORT_RETRYCACHE(-47, "Record ClientId and CallId in editlog to " 
     EDITLOG_SUPPORT_RETRYCACHE(-47, "Record ClientId and CallId in editlog to " 
         + "enable rebuilding retry cache in case of HA failover"),
         + "enable rebuilding retry cache in case of HA failover"),
-    CACHING(-48, "Support for cache pools and path-based caching");
+    CACHING(-48, "Support for cache pools and path-based caching"),
+    ADD_DATANODE_AND_STORAGE_UUIDS(-49, "Replace StorageID with DatanodeUuid."
+        + " Use distinct StorageUuid per storage directory.");
+
     
     
     final int lv;
     final int lv;
     final int ancestorLV;
     final int ancestorLV;
@@ -248,3 +251,4 @@ public class LayoutVersion {
     throw new AssertionError("All layout versions are reserved.");
     throw new AssertionError("All layout versions are reserved.");
   }
   }
 }
 }
+

+ 39 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java

@@ -21,7 +21,9 @@ import java.util.List;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
@@ -40,6 +42,10 @@ public class LocatedBlock {
   private ExtendedBlock b;
   private ExtendedBlock b;
   private long offset;  // offset of the first byte of the block in the file
   private long offset;  // offset of the first byte of the block in the file
   private DatanodeInfo[] locs;
   private DatanodeInfo[] locs;
+  /** Storage ID for each replica */
+  private String[] storageIDs;
+  // Storage type for each replica, if reported.
+  private StorageType[] storageTypes;
   // corrupt flag is true if all of the replicas of a block are corrupt.
   // corrupt flag is true if all of the replicas of a block are corrupt.
   // else false. If block has few corrupt replicas, they are filtered and 
   // else false. If block has few corrupt replicas, they are filtered and 
   // their locations are not part of this object
   // their locations are not part of this object
@@ -54,20 +60,34 @@ public class LocatedBlock {
   private static final DatanodeInfo[] EMPTY_LOCS = new DatanodeInfo[0];
   private static final DatanodeInfo[] EMPTY_LOCS = new DatanodeInfo[0];
 
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs) {
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs) {
-    this(b, locs, -1); // startOffset is unknown
-  }
-
-  public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset) {
-    this(b, locs, startOffset, false);
+    this(b, locs, -1, false); // startOffset is unknown
   }
   }
 
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset, 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset, 
                       boolean corrupt) {
                       boolean corrupt) {
-    this(b, locs, startOffset, corrupt, EMPTY_LOCS);
+    this(b, locs, null, null, startOffset, corrupt, EMPTY_LOCS);
+  }
+
+  public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages) {
+    this(b, storages, -1, false); // startOffset is unknown
   }
   }
 
 
-  public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset,
-      boolean corrupt, DatanodeInfo[] cachedLocs) {
+  public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs,
+                      String[] storageIDs, StorageType[] storageTypes) {
+    this(b, locs, storageIDs, storageTypes, -1, false, EMPTY_LOCS);
+  }
+
+  public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages,
+      long startOffset, boolean corrupt) {
+    this(b, DatanodeStorageInfo.toDatanodeInfos(storages),
+        DatanodeStorageInfo.toStorageIDs(storages),
+        DatanodeStorageInfo.toStorageTypes(storages),
+        startOffset, corrupt, EMPTY_LOCS); // startOffset is unknown
+  }
+
+  public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs,
+                      StorageType[] storageTypes, long startOffset,
+                      boolean corrupt, DatanodeInfo[] cachedLocs) {
     this.b = b;
     this.b = b;
     this.offset = startOffset;
     this.offset = startOffset;
     this.corrupt = corrupt;
     this.corrupt = corrupt;
@@ -76,6 +96,8 @@ public class LocatedBlock {
     } else {
     } else {
       this.locs = locs;
       this.locs = locs;
     }
     }
+    this.storageIDs = storageIDs;
+    this.storageTypes = storageTypes;
     Preconditions.checkArgument(cachedLocs != null,
     Preconditions.checkArgument(cachedLocs != null,
         "cachedLocs should not be null, use a different constructor");
         "cachedLocs should not be null, use a different constructor");
     if (cachedLocs.length == 0) {
     if (cachedLocs.length == 0) {
@@ -100,7 +122,15 @@ public class LocatedBlock {
   public DatanodeInfo[] getLocations() {
   public DatanodeInfo[] getLocations() {
     return locs;
     return locs;
   }
   }
+
+  public StorageType[] getStorageTypes() {
+    return storageTypes;
+  }
   
   
+  public String[] getStorageIDs() {
+    return storageIDs;
+  }
+
   public long getStartOffset() {
   public long getStartOffset() {
     return offset;
     return offset;
   }
   }
@@ -161,3 +191,4 @@ public class LocatedBlock {
         + "}";
         + "}";
   }
   }
 }
 }
+

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

@@ -51,7 +51,7 @@ public class UnregisteredNodeException extends IOException {
    */
    */
   public UnregisteredNodeException(DatanodeID nodeID, DatanodeInfo storedNode) {
   public UnregisteredNodeException(DatanodeID nodeID, DatanodeInfo storedNode) {
     super("Data node " + nodeID + " is attempting to report storage ID " 
     super("Data node " + nodeID + " is attempting to report storage ID " 
-          + nodeID.getStorageID() + ". Node " 
+          + nodeID.getDatanodeUuid() + ". Node "
           + storedNode + " is expected to serve this storage.");
           + storedNode + " is expected to serve this storage.");
   }
   }
 }
 }

+ 12 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -455,14 +455,17 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throws ServiceException {
       throws ServiceException {
     try {
     try {
       List<DatanodeInfoProto> existingList = req.getExistingsList();
       List<DatanodeInfoProto> existingList = req.getExistingsList();
+      List<String> existingStorageIDsList = req.getExistingStorageUuidsList();
       List<DatanodeInfoProto> excludesList = req.getExcludesList();
       List<DatanodeInfoProto> excludesList = req.getExcludesList();
-      LocatedBlock result = server.getAdditionalDatanode(
-          req.getSrc(), PBHelper.convert(req.getBlk()),
+      LocatedBlock result = server.getAdditionalDatanode(req.getSrc(),
+          PBHelper.convert(req.getBlk()),
           PBHelper.convert(existingList.toArray(
           PBHelper.convert(existingList.toArray(
               new DatanodeInfoProto[existingList.size()])),
               new DatanodeInfoProto[existingList.size()])),
+          existingStorageIDsList.toArray(
+              new String[existingStorageIDsList.size()]),
           PBHelper.convert(excludesList.toArray(
           PBHelper.convert(excludesList.toArray(
               new DatanodeInfoProto[excludesList.size()])), 
               new DatanodeInfoProto[excludesList.size()])), 
-              req.getNumAdditionalNodes(), req.getClientName());
+          req.getNumAdditionalNodes(), req.getClientName());
       return GetAdditionalDatanodeResponseProto.newBuilder().setBlock(
       return GetAdditionalDatanodeResponseProto.newBuilder().setBlock(
           PBHelper.convert(result))
           PBHelper.convert(result))
           .build();
           .build();
@@ -863,10 +866,12 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       UpdatePipelineRequestProto req) throws ServiceException {
       UpdatePipelineRequestProto req) throws ServiceException {
     try {
     try {
       List<DatanodeIDProto> newNodes = req.getNewNodesList();
       List<DatanodeIDProto> newNodes = req.getNewNodesList();
-      server
-          .updatePipeline(req.getClientName(), PBHelper.convert(req
-              .getOldBlock()), PBHelper.convert(req.getNewBlock()), PBHelper
-              .convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()])));
+      List<String> newStorageIDs = req.getStorageIDsList();
+      server.updatePipeline(req.getClientName(),
+          PBHelper.convert(req.getOldBlock()),
+          PBHelper.convert(req.getNewBlock()),
+          PBHelper.convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()])),
+          newStorageIDs.toArray(new String[newStorageIDs.size()]));
       return VOID_UPDATEPIPELINE_RESPONSE;
       return VOID_UPDATEPIPELINE_RESPONSE;
     } catch (IOException e) {
     } catch (IOException e) {
       throw new ServiceException(e);
       throw new ServiceException(e);

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -360,7 +360,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public LocatedBlock getAdditionalDatanode(String src, ExtendedBlock blk,
   public LocatedBlock getAdditionalDatanode(String src, ExtendedBlock blk,
-      DatanodeInfo[] existings, DatanodeInfo[] excludes,
+      DatanodeInfo[] existings, String[] existingStorageIDs,
+      DatanodeInfo[] excludes,
       int numAdditionalNodes, String clientName) throws AccessControlException,
       int numAdditionalNodes, String clientName) throws AccessControlException,
       FileNotFoundException, SafeModeException, UnresolvedLinkException,
       FileNotFoundException, SafeModeException, UnresolvedLinkException,
       IOException {
       IOException {
@@ -369,6 +370,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setSrc(src)
         .setSrc(src)
         .setBlk(PBHelper.convert(blk))
         .setBlk(PBHelper.convert(blk))
         .addAllExistings(PBHelper.convert(existings))
         .addAllExistings(PBHelper.convert(existings))
+        .addAllExistingStorageUuids(Arrays.asList(existingStorageIDs))
         .addAllExcludes(PBHelper.convert(excludes))
         .addAllExcludes(PBHelper.convert(excludes))
         .setNumAdditionalNodes(numAdditionalNodes)
         .setNumAdditionalNodes(numAdditionalNodes)
         .setClientName(clientName)
         .setClientName(clientName)
@@ -805,12 +807,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
-      ExtendedBlock newBlock, DatanodeID[] newNodes) throws IOException {
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException {
     UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
     UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
         .setClientName(clientName)
         .setClientName(clientName)
         .setOldBlock(PBHelper.convert(oldBlock))
         .setOldBlock(PBHelper.convert(oldBlock))
         .setNewBlock(PBHelper.convert(newBlock))
         .setNewBlock(PBHelper.convert(newBlock))
         .addAllNewNodes(Arrays.asList(PBHelper.convert(newNodes)))
         .addAllNewNodes(Arrays.asList(PBHelper.convert(newNodes)))
+        .addAllStorageIDs(storageIDs == null ? null : Arrays.asList(storageIDs))
         .build();
         .build();
     try {
     try {
       rpcProxy.updatePipeline(null, req);
       rpcProxy.updatePipeline(null, req);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java

@@ -245,7 +245,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
     for (StorageReceivedDeletedBlocks storageBlock : receivedAndDeletedBlocks) {
     for (StorageReceivedDeletedBlocks storageBlock : receivedAndDeletedBlocks) {
       StorageReceivedDeletedBlocksProto.Builder repBuilder = 
       StorageReceivedDeletedBlocksProto.Builder repBuilder = 
           StorageReceivedDeletedBlocksProto.newBuilder();
           StorageReceivedDeletedBlocksProto.newBuilder();
-      repBuilder.setStorageID(storageBlock.getStorageID());
+      repBuilder.setStorageUuid(storageBlock.getStorageID());
       for (ReceivedDeletedBlockInfo rdBlock : storageBlock.getBlocks()) {
       for (ReceivedDeletedBlockInfo rdBlock : storageBlock.getBlocks()) {
         repBuilder.addBlocks(PBHelper.convert(rdBlock));
         repBuilder.addBlocks(PBHelper.convert(rdBlock));
       }
       }

+ 3 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java

@@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlo
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlockReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlockReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
@@ -102,14 +101,8 @@ public class DatanodeProtocolServerSideTranslatorPB implements
       HeartbeatRequestProto request) throws ServiceException {
       HeartbeatRequestProto request) throws ServiceException {
     HeartbeatResponse response;
     HeartbeatResponse response;
     try {
     try {
-      List<StorageReportProto> list = request.getReportsList();
-      StorageReport[] report = new StorageReport[list.size()];
-      int i = 0;
-      for (StorageReportProto p : list) {
-        report[i++] = new StorageReport(p.getStorageID(), p.getFailed(),
-            p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
-            p.getBlockPoolUsed());
-      }
+      final StorageReport[] report = PBHelper.convertStorageReports(
+          request.getReportsList());
       response = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()),
       response = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()),
           report, request.getCacheCapacity(), request.getCacheUsed(),
           report, request.getCacheCapacity(), request.getCacheUsed(),
           request.getXmitsInProgress(),
           request.getXmitsInProgress(),
@@ -198,7 +191,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
       for (int j = 0; j < list.size(); j++) {
       for (int j = 0; j < list.size(); j++) {
         rdBlocks[j] = PBHelper.convert(list.get(j));
         rdBlocks[j] = PBHelper.convert(list.get(j));
       }
       }
-      info[i] = new StorageReceivedDeletedBlocks(sBlock.getStorageID(), rdBlocks);
+      info[i] = new StorageReceivedDeletedBlocks(sBlock.getStorageUuid(), rdBlocks);
     }
     }
     try {
     try {
       impl.blockReceivedAndDeleted(PBHelper.convert(request.getRegistration()),
       impl.blockReceivedAndDeleted(PBHelper.convert(request.getRegistration()),

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java

@@ -82,6 +82,6 @@ public class InterDatanodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
       throw new ServiceException(e);
     }
     }
     return UpdateReplicaUnderRecoveryResponseProto.newBuilder()
     return UpdateReplicaUnderRecoveryResponseProto.newBuilder()
-        .setStorageID(storageID).build();
+        .setStorageUuid(storageID).build();
   }
   }
 }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java

@@ -109,7 +109,7 @@ public class InterDatanodeProtocolTranslatorPB implements
         .setNewLength(newLength).setRecoveryId(recoveryId).build();
         .setNewLength(newLength).setRecoveryId(recoveryId).build();
     try {
     try {
       return rpcProxy.updateReplicaUnderRecovery(NULL_CONTROLLER, req
       return rpcProxy.updateReplicaUnderRecovery(NULL_CONTROLLER, req
-          ).getStorageID();
+          ).getStorageUuid();
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }

+ 128 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
 import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
@@ -57,12 +58,12 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclStatusProto;
@@ -133,6 +134,8 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -264,17 +267,20 @@ public class PBHelper {
 
 
   // DatanodeId
   // DatanodeId
   public static DatanodeID convert(DatanodeIDProto dn) {
   public static DatanodeID convert(DatanodeIDProto dn) {
-    return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getStorageID(),
+    return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(),
         dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
         dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
         .getInfoSecurePort() : 0, dn.getIpcPort());
         .getInfoSecurePort() : 0, dn.getIpcPort());
   }
   }
 
 
   public static DatanodeIDProto convert(DatanodeID dn) {
   public static DatanodeIDProto convert(DatanodeID dn) {
+    // For wire compatibility with older versions we transmit the StorageID
+    // which is the same as the DatanodeUuid. Since StorageID is a required
+    // field we pass the empty string if the DatanodeUuid is not yet known.
     return DatanodeIDProto.newBuilder()
     return DatanodeIDProto.newBuilder()
         .setIpAddr(dn.getIpAddr())
         .setIpAddr(dn.getIpAddr())
         .setHostName(dn.getHostName())
         .setHostName(dn.getHostName())
-        .setStorageID(dn.getStorageID())
         .setXferPort(dn.getXferPort())
         .setXferPort(dn.getXferPort())
+        .setDatanodeUuid(dn.getDatanodeUuid() != null ? dn.getDatanodeUuid() : "")
         .setInfoPort(dn.getInfoPort())
         .setInfoPort(dn.getInfoPort())
         .setInfoSecurePort(dn.getInfoSecurePort())
         .setInfoSecurePort(dn.getInfoSecurePort())
         .setIpcPort(dn.getIpcPort()).build();
         .setIpcPort(dn.getIpcPort()).build();
@@ -316,12 +322,16 @@ public class PBHelper {
   public static BlockWithLocationsProto convert(BlockWithLocations blk) {
   public static BlockWithLocationsProto convert(BlockWithLocations blk) {
     return BlockWithLocationsProto.newBuilder()
     return BlockWithLocationsProto.newBuilder()
         .setBlock(convert(blk.getBlock()))
         .setBlock(convert(blk.getBlock()))
-        .addAllStorageIDs(Arrays.asList(blk.getStorageIDs())).build();
+        .addAllDatanodeUuids(Arrays.asList(blk.getDatanodeUuids()))
+        .addAllStorageUuids(Arrays.asList(blk.getStorageIDs())).build();
   }
   }
 
 
   public static BlockWithLocations convert(BlockWithLocationsProto b) {
   public static BlockWithLocations convert(BlockWithLocationsProto b) {
-    return new BlockWithLocations(convert(b.getBlock()), b.getStorageIDsList()
-        .toArray(new String[0]));
+    final List<String> datanodeUuids = b.getDatanodeUuidsList();
+    final List<String> storageUuids = b.getStorageUuidsList();
+    return new BlockWithLocations(convert(b.getBlock()),
+        datanodeUuids.toArray(new String[datanodeUuids.size()]),
+        storageUuids.toArray(new String[storageUuids.size()]));
   }
   }
 
 
   public static BlocksWithLocationsProto convert(BlocksWithLocations blks) {
   public static BlocksWithLocationsProto convert(BlocksWithLocations blks) {
@@ -623,6 +633,17 @@ public class PBHelper {
         "Found additional cached replica locations that are not in the set of"
         "Found additional cached replica locations that are not in the set of"
         + " storage-backed locations!");
         + " storage-backed locations!");
 
 
+    StorageType[] storageTypes = b.getStorageTypes();
+    if (storageTypes != null) {
+      for (int i = 0; i < storageTypes.length; ++i) {
+        builder.addStorageTypes(PBHelper.convertStorageType(storageTypes[i]));
+      }
+    }
+    final String[] storageIDs = b.getStorageIDs();
+    if (storageIDs != null) {
+      builder.addAllStorageIDs(Arrays.asList(storageIDs));
+    }
+
     return builder.setB(PBHelper.convert(b.getBlock()))
     return builder.setB(PBHelper.convert(b.getBlock()))
         .setBlockToken(PBHelper.convert(b.getBlockToken()))
         .setBlockToken(PBHelper.convert(b.getBlockToken()))
         .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
         .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
@@ -635,6 +656,25 @@ public class PBHelper {
     for (int i = 0; i < locs.size(); i++) {
     for (int i = 0; i < locs.size(); i++) {
       targets[i] = PBHelper.convert(locs.get(i));
       targets[i] = PBHelper.convert(locs.get(i));
     }
     }
+
+    final int storageTypesCount = proto.getStorageTypesCount();
+    final StorageType[] storageTypes;
+    if (storageTypesCount == 0) {
+      storageTypes = null;
+    } else {
+      Preconditions.checkState(storageTypesCount == locs.size());
+      storageTypes = convertStorageTypeProtos(proto.getStorageTypesList());
+    }
+
+    final int storageIDsCount = proto.getStorageIDsCount();
+    final String[] storageIDs;
+    if (storageIDsCount == 0) {
+      storageIDs = null;
+    } else {
+      Preconditions.checkState(storageIDsCount == locs.size());
+      storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
+    }
+
     // Set values from the isCached list, re-using references from loc
     // Set values from the isCached list, re-using references from loc
     List<DatanodeInfo> cachedLocs = new ArrayList<DatanodeInfo>(locs.size());
     List<DatanodeInfo> cachedLocs = new ArrayList<DatanodeInfo>(locs.size());
     List<Boolean> isCachedList = proto.getIsCachedList();
     List<Boolean> isCachedList = proto.getIsCachedList();
@@ -645,7 +685,7 @@ public class PBHelper {
     }
     }
 
 
     LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets,
     LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets,
-        proto.getOffset(), proto.getCorrupt(),
+        storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
         cachedLocs.toArray(new DatanodeInfo[0]));
         cachedLocs.toArray(new DatanodeInfo[0]));
     lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
     lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
 
 
@@ -789,7 +829,8 @@ public class PBHelper {
     for (int i = 0; i < blocks.length; i++) {
     for (int i = 0; i < blocks.length; i++) {
       builder.addBlocks(PBHelper.convert(blocks[i]));
       builder.addBlocks(PBHelper.convert(blocks[i]));
     }
     }
-    builder.addAllTargets(PBHelper.convert(cmd.getTargets()));
+    builder.addAllTargets(convert(cmd.getTargets()))
+           .addAllTargetStorageUuids(convert(cmd.getTargetStorageIDs()));
     return builder.build();
     return builder.build();
   }
   }
   
   
@@ -822,6 +863,15 @@ public class PBHelper {
     return Arrays.asList(ret);
     return Arrays.asList(ret);
   }
   }
 
 
+  private static List<StorageUuidsProto> convert(String[][] targetStorageUuids) {
+    StorageUuidsProto[] ret = new StorageUuidsProto[targetStorageUuids.length];
+    for (int i = 0; i < targetStorageUuids.length; i++) {
+      ret[i] = StorageUuidsProto.newBuilder()
+          .addAllStorageUuids(Arrays.asList(targetStorageUuids[i])).build();
+    }
+    return Arrays.asList(ret);
+  }
+
   public static DatanodeCommandProto convert(DatanodeCommand datanodeCommand) {
   public static DatanodeCommandProto convert(DatanodeCommand datanodeCommand) {
     DatanodeCommandProto.Builder builder = DatanodeCommandProto.newBuilder();
     DatanodeCommandProto.Builder builder = DatanodeCommandProto.newBuilder();
     if (datanodeCommand == null) {
     if (datanodeCommand == null) {
@@ -901,6 +951,14 @@ public class PBHelper {
     for (int i = 0; i < targetList.size(); i++) {
     for (int i = 0; i < targetList.size(); i++) {
       targets[i] = PBHelper.convert(targetList.get(i));
       targets[i] = PBHelper.convert(targetList.get(i));
     }
     }
+
+    List<StorageUuidsProto> targetStorageUuidsList = blkCmd.getTargetStorageUuidsList();
+    String[][] targetStorageIDs = new String[targetStorageUuidsList.size()][];
+    for(int i = 0; i < targetStorageIDs.length; i++) {
+      List<String> storageIDs = targetStorageUuidsList.get(i).getStorageUuidsList();
+      targetStorageIDs[i] = storageIDs.toArray(new String[storageIDs.size()]);
+    }
+
     int action = DatanodeProtocol.DNA_UNKNOWN;
     int action = DatanodeProtocol.DNA_UNKNOWN;
     switch (blkCmd.getAction()) {
     switch (blkCmd.getAction()) {
     case TRANSFER:
     case TRANSFER:
@@ -915,7 +973,8 @@ public class PBHelper {
     default:
     default:
       throw new AssertionError("Unknown action type: " + blkCmd.getAction());
       throw new AssertionError("Unknown action type: " + blkCmd.getAction());
     }
     }
-    return new BlockCommand(action, blkCmd.getBlockPoolId(), blocks, targets);
+    return new BlockCommand(action, blkCmd.getBlockPoolId(), blocks, targets,
+        targetStorageIDs);
   }
   }
 
 
   public static BlockIdCommand convert(BlockIdCommandProto blkIdCmd) {
   public static BlockIdCommand convert(BlockIdCommandProto blkIdCmd) {
@@ -1445,11 +1504,12 @@ public class PBHelper {
 
 
   public static DatanodeStorageProto convert(DatanodeStorage s) {
   public static DatanodeStorageProto convert(DatanodeStorage s) {
     return DatanodeStorageProto.newBuilder()
     return DatanodeStorageProto.newBuilder()
-        .setState(PBHelper.convert(s.getState()))
-        .setStorageID(s.getStorageID()).build();
+        .setState(PBHelper.convertState(s.getState()))
+        .setStorageType(PBHelper.convertStorageType(s.getStorageType()))
+        .setStorageUuid(s.getStorageID()).build();
   }
   }
 
 
-  private static StorageState convert(State state) {
+  private static StorageState convertState(State state) {
     switch(state) {
     switch(state) {
     case READ_ONLY:
     case READ_ONLY:
       return StorageState.READ_ONLY;
       return StorageState.READ_ONLY;
@@ -1459,11 +1519,26 @@ public class PBHelper {
     }
     }
   }
   }
 
 
+  private static StorageTypeProto convertStorageType(
+      StorageType type) {
+    switch(type) {
+    case DISK:
+      return StorageTypeProto.DISK;
+    case SSD:
+      return StorageTypeProto.SSD;
+    default:
+      throw new IllegalStateException(
+          "BUG: StorageType not found, type=" + type);
+    }
+  }
+
   public static DatanodeStorage convert(DatanodeStorageProto s) {
   public static DatanodeStorage convert(DatanodeStorageProto s) {
-    return new DatanodeStorage(s.getStorageID(), PBHelper.convert(s.getState()));
+    return new DatanodeStorage(s.getStorageUuid(),
+                               PBHelper.convertState(s.getState()),
+                               PBHelper.convertType(s.getStorageType()));
   }
   }
 
 
-  private static State convert(StorageState state) {
+  private static State convertState(StorageState state) {
     switch(state) {
     switch(state) {
     case READ_ONLY:
     case READ_ONLY:
       return DatanodeStorage.State.READ_ONLY;
       return DatanodeStorage.State.READ_ONLY;
@@ -1473,14 +1548,50 @@ public class PBHelper {
     }
     }
   }
   }
 
 
+  private static StorageType convertType(StorageTypeProto type) {
+    switch(type) {
+      case DISK:
+        return StorageType.DISK;
+      case SSD:
+        return StorageType.SSD;
+      default:
+        throw new IllegalStateException(
+            "BUG: StorageTypeProto not found, type=" + type);
+    }
+  }
+
+  private static StorageType[] convertStorageTypeProtos(
+      List<StorageTypeProto> storageTypesList) {
+    final StorageType[] storageTypes = new StorageType[storageTypesList.size()];
+    for (int i = 0; i < storageTypes.length; ++i) {
+      storageTypes[i] = PBHelper.convertType(storageTypesList.get(i));
+    }
+    return storageTypes;
+  }
+
   public static StorageReportProto convert(StorageReport r) {
   public static StorageReportProto convert(StorageReport r) {
     StorageReportProto.Builder builder = StorageReportProto.newBuilder()
     StorageReportProto.Builder builder = StorageReportProto.newBuilder()
         .setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
         .setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
         .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
         .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
-        .setStorageID(r.getStorageID());
+        .setStorageUuid(r.getStorageID());
     return builder.build();
     return builder.build();
   }
   }
 
 
+  public static StorageReport convert(StorageReportProto p) {
+    return new StorageReport(p.getStorageUuid(), p.getFailed(),
+        p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
+        p.getBlockPoolUsed());
+  }
+
+  public static StorageReport[] convertStorageReports(
+      List<StorageReportProto> list) {
+    final StorageReport[] report = new StorageReport[list.size()];
+    for (int i = 0; i < report.length; i++) {
+      report[i] = convert(list.get(i));
+    }
+    return report;
+  }
+
   public static JournalInfo convert(JournalInfoProto info) {
   public static JournalInfo convert(JournalInfoProto info) {
     int lv = info.hasLayoutVersion() ? info.getLayoutVersion() : 0;
     int lv = info.hasLayoutVersion() ? info.getLayoutVersion() : 0;
     int nsID = info.hasNamespaceID() ? info.getNamespaceID() : 0;
     int nsID = info.hasNamespaceID() ? info.getNamespaceID() : 0;
@@ -1838,3 +1949,4 @@ public class PBHelper {
     return GetAclStatusResponseProto.newBuilder().setResult(r).build();
     return GetAclStatusResponseProto.newBuilder().setResult(r).build();
   }
   }
 }
 }
+

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java

@@ -109,7 +109,7 @@ interface AsyncLogger {
    * Fetch the list of edit logs available on the remote node.
    * Fetch the list of edit logs available on the remote node.
    */
    */
   public ListenableFuture<RemoteEditLogManifest> getEditLogManifest(
   public ListenableFuture<RemoteEditLogManifest> getEditLogManifest(
-      long fromTxnId, boolean forReading, boolean inProgressOk);
+      long fromTxnId, boolean inProgressOk);
 
 
   /**
   /**
    * Prepare recovery. See the HDFS-3077 design document for details.
    * Prepare recovery. See the HDFS-3077 design document for details.

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java

@@ -261,13 +261,13 @@ class AsyncLoggerSet {
   }
   }
 
 
   public QuorumCall<AsyncLogger, RemoteEditLogManifest> getEditLogManifest(
   public QuorumCall<AsyncLogger, RemoteEditLogManifest> getEditLogManifest(
-      long fromTxnId, boolean forReading, boolean inProgressOk) {
+      long fromTxnId, boolean inProgressOk) {
     Map<AsyncLogger,
     Map<AsyncLogger,
         ListenableFuture<RemoteEditLogManifest>> calls
         ListenableFuture<RemoteEditLogManifest>> calls
         = Maps.newHashMap();
         = Maps.newHashMap();
     for (AsyncLogger logger : loggers) {
     for (AsyncLogger logger : loggers) {
       ListenableFuture<RemoteEditLogManifest> future =
       ListenableFuture<RemoteEditLogManifest> future =
-          logger.getEditLogManifest(fromTxnId, forReading, inProgressOk);
+          logger.getEditLogManifest(fromTxnId, inProgressOk);
       calls.put(logger, future);
       calls.put(logger, future);
     }
     }
     return QuorumCall.create(calls);
     return QuorumCall.create(calls);

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java

@@ -181,6 +181,7 @@ public class IPCLoggerChannel implements AsyncLogger {
   
   
   @Override
   @Override
   public void close() {
   public void close() {
+    QuorumJournalManager.LOG.info("Closing", new Exception());
     // No more tasks may be submitted after this point.
     // No more tasks may be submitted after this point.
     executor.shutdown();
     executor.shutdown();
     if (proxy != null) {
     if (proxy != null) {
@@ -520,13 +521,12 @@ public class IPCLoggerChannel implements AsyncLogger {
 
 
   @Override
   @Override
   public ListenableFuture<RemoteEditLogManifest> getEditLogManifest(
   public ListenableFuture<RemoteEditLogManifest> getEditLogManifest(
-      final long fromTxnId, final boolean forReading, 
-      final boolean inProgressOk) {
+      final long fromTxnId, final boolean inProgressOk) {
     return executor.submit(new Callable<RemoteEditLogManifest>() {
     return executor.submit(new Callable<RemoteEditLogManifest>() {
       @Override
       @Override
       public RemoteEditLogManifest call() throws IOException {
       public RemoteEditLogManifest call() throws IOException {
         GetEditLogManifestResponseProto ret = getProxy().getEditLogManifest(
         GetEditLogManifestResponseProto ret = getProxy().getEditLogManifest(
-            journalId, fromTxnId, forReading, inProgressOk);
+            journalId, fromTxnId, inProgressOk);
         // Update the http port, since we need this to build URLs to any of the
         // Update the http port, since we need this to build URLs to any of the
         // returned logs.
         // returned logs.
         constructHttpServerURI(ret);
         constructHttpServerURI(ret);

+ 2 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java

@@ -449,18 +449,13 @@ public class QuorumJournalManager implements JournalManager {
   public void close() throws IOException {
   public void close() throws IOException {
     loggers.close();
     loggers.close();
   }
   }
-  
-  public void selectInputStreams(Collection<EditLogInputStream> streams,
-      long fromTxnId, boolean inProgressOk) throws IOException {
-    selectInputStreams(streams, fromTxnId, inProgressOk, true);
-  }
 
 
   @Override
   @Override
   public void selectInputStreams(Collection<EditLogInputStream> streams,
   public void selectInputStreams(Collection<EditLogInputStream> streams,
-      long fromTxnId, boolean inProgressOk, boolean forReading) throws IOException {
+      long fromTxnId, boolean inProgressOk) throws IOException {
 
 
     QuorumCall<AsyncLogger, RemoteEditLogManifest> q =
     QuorumCall<AsyncLogger, RemoteEditLogManifest> q =
-        loggers.getEditLogManifest(fromTxnId, forReading, inProgressOk);
+        loggers.getEditLogManifest(fromTxnId, inProgressOk);
     Map<AsyncLogger, RemoteEditLogManifest> resps =
     Map<AsyncLogger, RemoteEditLogManifest> resps =
         loggers.waitForWriteQuorum(q, selectInputStreamsTimeoutMs,
         loggers.waitForWriteQuorum(q, selectInputStreamsTimeoutMs,
             "selectInputStreams");
             "selectInputStreams");

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java

@@ -123,14 +123,12 @@ public interface QJournalProtocol {
   /**
   /**
    * @param jid the journal from which to enumerate edits
    * @param jid the journal from which to enumerate edits
    * @param sinceTxId the first transaction which the client cares about
    * @param sinceTxId the first transaction which the client cares about
-   * @param forReading whether or not the caller intends to read from the edit
-   *        logs
    * @param inProgressOk whether or not to check the in-progress edit log 
    * @param inProgressOk whether or not to check the in-progress edit log 
    *        segment       
    *        segment       
    * @return a list of edit log segments since the given transaction ID.
    * @return a list of edit log segments since the given transaction ID.
    */
    */
   public GetEditLogManifestResponseProto getEditLogManifest(String jid,
   public GetEditLogManifestResponseProto getEditLogManifest(String jid,
-      long sinceTxId, boolean forReading, boolean inProgressOk)
+      long sinceTxId, boolean inProgressOk)
       throws IOException;
       throws IOException;
   
   
   /**
   /**

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java

@@ -203,7 +203,6 @@ public class QJournalProtocolServerSideTranslatorPB implements QJournalProtocolP
       return impl.getEditLogManifest(
       return impl.getEditLogManifest(
           request.getJid().getIdentifier(),
           request.getJid().getIdentifier(),
           request.getSinceTxId(),
           request.getSinceTxId(),
-          request.getForReading(),
           request.getInProgressOk());
           request.getInProgressOk());
     } catch (IOException e) {
     } catch (IOException e) {
       throw new ServiceException(e);
       throw new ServiceException(e);

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java

@@ -228,14 +228,13 @@ public class QJournalProtocolTranslatorPB implements ProtocolMetaInterface,
 
 
   @Override
   @Override
   public GetEditLogManifestResponseProto getEditLogManifest(String jid,
   public GetEditLogManifestResponseProto getEditLogManifest(String jid,
-      long sinceTxId, boolean forReading, boolean inProgressOk)
+      long sinceTxId, boolean inProgressOk)
       throws IOException {
       throws IOException {
     try {
     try {
       return rpcProxy.getEditLogManifest(NULL_CONTROLLER,
       return rpcProxy.getEditLogManifest(NULL_CONTROLLER,
           GetEditLogManifestRequestProto.newBuilder()
           GetEditLogManifestRequestProto.newBuilder()
             .setJid(convertJournalId(jid))
             .setJid(convertJournalId(jid))
             .setSinceTxId(sinceTxId)
             .setSinceTxId(sinceTxId)
-            .setForReading(forReading)
             .setInProgressOk(inProgressOk)
             .setInProgressOk(inProgressOk)
             .build());
             .build());
     } catch (ServiceException e) {
     } catch (ServiceException e) {

+ 2 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java

@@ -630,15 +630,12 @@ class Journal implements Closeable {
    * @see QJournalProtocol#getEditLogManifest(String, long)
    * @see QJournalProtocol#getEditLogManifest(String, long)
    */
    */
   public RemoteEditLogManifest getEditLogManifest(long sinceTxId,
   public RemoteEditLogManifest getEditLogManifest(long sinceTxId,
-      boolean forReading, boolean inProgressOk) throws IOException {
+      boolean inProgressOk) throws IOException {
     // No need to checkRequest() here - anyone may ask for the list
     // No need to checkRequest() here - anyone may ask for the list
     // of segments.
     // of segments.
     checkFormatted();
     checkFormatted();
     
     
-    // if this is for reading, ignore the in-progress editlog segment
-    inProgressOk = forReading ? false : inProgressOk;
-    List<RemoteEditLog> logs = fjm.getRemoteEditLogs(sinceTxId, forReading,
-        inProgressOk);
+    List<RemoteEditLog> logs = fjm.getRemoteEditLogs(sinceTxId, inProgressOk);
     
     
     if (inProgressOk) {
     if (inProgressOk) {
       RemoteEditLog log = null;
       RemoteEditLog log = null;

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java

@@ -178,11 +178,11 @@ class JournalNodeRpcServer implements QJournalProtocol {
   @SuppressWarnings("deprecation")
   @SuppressWarnings("deprecation")
   @Override
   @Override
   public GetEditLogManifestResponseProto getEditLogManifest(String jid,
   public GetEditLogManifestResponseProto getEditLogManifest(String jid,
-      long sinceTxId, boolean forReading, boolean inProgressOk)
+      long sinceTxId, boolean inProgressOk)
       throws IOException {
       throws IOException {
     
     
     RemoteEditLogManifest manifest = jn.getOrCreateJournal(jid)
     RemoteEditLogManifest manifest = jn.getOrCreateJournal(jid)
-        .getEditLogManifest(sinceTxId, forReading, inProgressOk);
+        .getEditLogManifest(sinceTxId, inProgressOk);
     
     
     return GetEditLogManifestResponseProto.newBuilder()
     return GetEditLogManifestResponseProto.newBuilder()
         .setManifest(PBHelper.convert(manifest))
         .setManifest(PBHelper.convert(manifest))

+ 41 - 63
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.balancer;
 package org.apache.hadoop.hdfs.server.balancer;
 
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkArgument;
-
 import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
 import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
 
 
 import java.io.BufferedInputStream;
 import java.io.BufferedInputStream;
@@ -221,9 +220,9 @@ public class Balancer {
   private Map<Block, BalancerBlock> globalBlockList
   private Map<Block, BalancerBlock> globalBlockList
                  = new HashMap<Block, BalancerBlock>();
                  = new HashMap<Block, BalancerBlock>();
   private MovedBlocks movedBlocks = new MovedBlocks();
   private MovedBlocks movedBlocks = new MovedBlocks();
-  // Map storage IDs to BalancerDatanodes
-  private Map<String, BalancerDatanode> datanodes
-                 = new HashMap<String, BalancerDatanode>();
+  /** Map (datanodeUuid -> BalancerDatanodes) */
+  private final Map<String, BalancerDatanode> datanodeMap
+      = new HashMap<String, BalancerDatanode>();
   
   
   private NetworkTopology cluster;
   private NetworkTopology cluster;
 
 
@@ -241,6 +240,14 @@ public class Balancer {
     private PendingBlockMove() {
     private PendingBlockMove() {
     }
     }
     
     
+    @Override
+    public String toString() {
+      final Block b = block.getBlock();
+      return b + " with size=" + b.getNumBytes() + " from "
+          + source.getDisplayName() + " to " + target.getDisplayName()
+          + " through " + proxySource.getDisplayName();
+    }
+
     /* choose a block & a proxy source for this pendingMove 
     /* choose a block & a proxy source for this pendingMove 
      * whose source & target have already been chosen.
      * whose source & target have already been chosen.
      * 
      * 
@@ -272,11 +279,7 @@ public class Balancer {
             if ( chooseProxySource() ) {
             if ( chooseProxySource() ) {
               movedBlocks.add(block);
               movedBlocks.add(block);
               if (LOG.isDebugEnabled()) {
               if (LOG.isDebugEnabled()) {
-                LOG.debug("Decided to move block "+ block.getBlockId()
-                    +" with a length of "+StringUtils.byteDesc(block.getNumBytes())
-                    + " bytes from " + source.getDisplayName()
-                    + " to " + target.getDisplayName()
-                    + " using proxy source " + proxySource.getDisplayName() );
+                LOG.debug("Decided to move " + this);
               }
               }
               return true;
               return true;
             }
             }
@@ -292,26 +295,27 @@ public class Balancer {
      */
      */
     private boolean chooseProxySource() {
     private boolean chooseProxySource() {
       final DatanodeInfo targetDN = target.getDatanode();
       final DatanodeInfo targetDN = target.getDatanode();
-      boolean find = false;
-      for (BalancerDatanode loc : block.getLocations()) {
-        // check if there is replica which is on the same rack with the target
-        if (cluster.isOnSameRack(loc.getDatanode(), targetDN) && addTo(loc)) {
-          find = true;
-          // if cluster is not nodegroup aware or the proxy is on the same 
-          // nodegroup with target, then we already find the nearest proxy
-          if (!cluster.isNodeGroupAware() 
-              || cluster.isOnSameNodeGroup(loc.getDatanode(), targetDN)) {
+      // if node group is supported, first try add nodes in the same node group
+      if (cluster.isNodeGroupAware()) {
+        for (BalancerDatanode loc : block.getLocations()) {
+          if (cluster.isOnSameNodeGroup(loc.getDatanode(), targetDN) && addTo(loc)) {
             return true;
             return true;
           }
           }
         }
         }
-        
-        if (!find) {
-          // find out a non-busy replica out of rack of target
-          find = addTo(loc);
+      }
+      // check if there is replica which is on the same rack with the target
+      for (BalancerDatanode loc : block.getLocations()) {
+        if (cluster.isOnSameRack(loc.getDatanode(), targetDN) && addTo(loc)) {
+          return true;
         }
         }
       }
       }
-      
-      return find;
+      // find out a non-busy replica
+      for (BalancerDatanode loc : block.getLocations()) {
+        if (addTo(loc)) {
+          return true;
+        }
+      }
+      return false;
     }
     }
     
     
     // add a BalancerDatanode as proxy source for specific block movement
     // add a BalancerDatanode as proxy source for specific block movement
@@ -352,17 +356,9 @@ public class Balancer {
         sendRequest(out);
         sendRequest(out);
         receiveResponse(in);
         receiveResponse(in);
         bytesMoved.inc(block.getNumBytes());
         bytesMoved.inc(block.getNumBytes());
-        LOG.info( "Moving block " + block.getBlock().getBlockId() +
-              " from "+ source.getDisplayName() + " to " +
-              target.getDisplayName() + " through " +
-              proxySource.getDisplayName() +
-              " is succeeded." );
+        LOG.info("Successfully moved " + this);
       } catch (IOException e) {
       } catch (IOException e) {
-        LOG.warn("Error moving block "+block.getBlockId()+
-            " from " + source.getDisplayName() + " to " +
-            target.getDisplayName() + " through " +
-            proxySource.getDisplayName() +
-            ": "+e.getMessage());
+        LOG.warn("Failed to move " + this + ": " + e.getMessage());
       } finally {
       } finally {
         IOUtils.closeStream(out);
         IOUtils.closeStream(out);
         IOUtils.closeStream(in);
         IOUtils.closeStream(in);
@@ -414,9 +410,7 @@ public class Balancer {
         @Override
         @Override
         public void run() {
         public void run() {
           if (LOG.isDebugEnabled()) {
           if (LOG.isDebugEnabled()) {
-            LOG.debug("Starting moving "+ block.getBlockId() +
-                " from " + proxySource.getDisplayName() + " to " +
-                target.getDisplayName());
+            LOG.debug("Start moving " + PendingBlockMove.this);
           }
           }
           dispatch();
           dispatch();
         }
         }
@@ -463,11 +457,6 @@ public class Balancer {
       return block;
       return block;
     }
     }
     
     
-    /* Return the block id */
-    private long getBlockId() {
-      return block.getBlockId();
-    }
-    
     /* Return the length of the block */
     /* Return the length of the block */
     private long getNumBytes() {
     private long getNumBytes() {
       return block.getNumBytes();
       return block.getNumBytes();
@@ -551,7 +540,7 @@ public class Balancer {
     
     
     /* Get the storage id of the datanode */
     /* Get the storage id of the datanode */
     protected String getStorageID() {
     protected String getStorageID() {
-      return datanode.getStorageID();
+      return datanode.getDatanodeUuid();
     }
     }
     
     
     /** Decide if still need to move more bytes */
     /** Decide if still need to move more bytes */
@@ -674,10 +663,10 @@ public class Balancer {
         
         
           synchronized (block) {
           synchronized (block) {
             // update locations
             // update locations
-            for ( String storageID : blk.getStorageIDs() ) {
-              BalancerDatanode datanode = datanodes.get(storageID);
+            for (String datanodeUuid : blk.getDatanodeUuids()) {
+              final BalancerDatanode d = datanodeMap.get(datanodeUuid);
               if (datanode != null) { // not an unknown datanode
               if (datanode != null) { // not an unknown datanode
-                block.addLocation(datanode);
+                block.addLocation(d);
               }
               }
             }
             }
           }
           }
@@ -851,16 +840,6 @@ public class Balancer {
                         DFSConfigKeys.DFS_BALANCER_DISPATCHERTHREADS_DEFAULT));
                         DFSConfigKeys.DFS_BALANCER_DISPATCHERTHREADS_DEFAULT));
   }
   }
   
   
-  /* Shuffle datanode array */
-  static private void shuffleArray(DatanodeInfo[] datanodes) {
-    for (int i=datanodes.length; i>1; i--) {
-      int randomIndex = DFSUtil.getRandom().nextInt(i);
-      DatanodeInfo tmp = datanodes[randomIndex];
-      datanodes[randomIndex] = datanodes[i-1];
-      datanodes[i-1] = tmp;
-    }
-  }
-  
   /* Given a data node set, build a network topology and decide
   /* Given a data node set, build a network topology and decide
    * over-utilized datanodes, above average utilized datanodes, 
    * over-utilized datanodes, above average utilized datanodes, 
    * below average utilized datanodes, and underutilized datanodes. 
    * below average utilized datanodes, and underutilized datanodes. 
@@ -890,8 +869,7 @@ public class Balancer {
      * an increasing order or a decreasing order.
      * an increasing order or a decreasing order.
      */  
      */  
     long overLoadedBytes = 0L, underLoadedBytes = 0L;
     long overLoadedBytes = 0L, underLoadedBytes = 0L;
-    shuffleArray(datanodes);
-    for (DatanodeInfo datanode : datanodes) {
+    for (DatanodeInfo datanode : DFSUtil.shuffle(datanodes)) {
       if (datanode.isDecommissioned() || datanode.isDecommissionInProgress()) {
       if (datanode.isDecommissioned() || datanode.isDecommissionInProgress()) {
         continue; // ignore decommissioning or decommissioned nodes
         continue; // ignore decommissioning or decommissioned nodes
       }
       }
@@ -922,13 +900,13 @@ public class Balancer {
               datanodeS.utilization)*datanodeS.datanode.getCapacity()/100.0);
               datanodeS.utilization)*datanodeS.datanode.getCapacity()/100.0);
         }
         }
       }
       }
-      this.datanodes.put(datanode.getStorageID(), datanodeS);
+      datanodeMap.put(datanode.getDatanodeUuid(), datanodeS);
     }
     }
 
 
     //logging
     //logging
     logNodes();
     logNodes();
     
     
-    assert (this.datanodes.size() == 
+    assert (this.datanodeMap.size() == 
       overUtilizedDatanodes.size()+underUtilizedDatanodes.size()+
       overUtilizedDatanodes.size()+underUtilizedDatanodes.size()+
       aboveAvgUtilizedDatanodes.size()+belowAvgUtilizedDatanodes.size())
       aboveAvgUtilizedDatanodes.size()+belowAvgUtilizedDatanodes.size())
       : "Mismatched number of datanodes";
       : "Mismatched number of datanodes";
@@ -1000,9 +978,9 @@ public class Balancer {
     // At last, match all remaining nodes
     // At last, match all remaining nodes
     chooseNodes(ANY_OTHER);
     chooseNodes(ANY_OTHER);
     
     
-    assert (datanodes.size() >= sources.size()+targets.size())
+    assert (datanodeMap.size() >= sources.size()+targets.size())
       : "Mismatched number of datanodes (" +
       : "Mismatched number of datanodes (" +
-      datanodes.size() + " total, " +
+      datanodeMap.size() + " total, " +
       sources.size() + " sources, " +
       sources.size() + " sources, " +
       targets.size() + " targets)";
       targets.size() + " targets)";
 
 
@@ -1303,7 +1281,7 @@ public class Balancer {
     this.aboveAvgUtilizedDatanodes.clear();
     this.aboveAvgUtilizedDatanodes.clear();
     this.belowAvgUtilizedDatanodes.clear();
     this.belowAvgUtilizedDatanodes.clear();
     this.underUtilizedDatanodes.clear();
     this.underUtilizedDatanodes.clear();
-    this.datanodes.clear();
+    this.datanodeMap.clear();
     this.sources.clear();
     this.sources.clear();
     this.targets.clear();  
     this.targets.clear();  
     this.policy.reset();
     this.policy.reset();

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

@@ -75,7 +75,7 @@ public interface BlockCollection {
    * and set the locations.
    * and set the locations.
    */
    */
   public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
   public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
-      DatanodeDescriptor[] locations) throws IOException;
+      DatanodeStorageInfo[] targets) throws IOException;
 
 
   /**
   /**
    * @return whether the block collection is under construction.
    * @return whether the block collection is under construction.

+ 91 - 41
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -21,6 +21,7 @@ import java.util.LinkedList;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.util.LightWeightGSet;
 import org.apache.hadoop.util.LightWeightGSet;
 
 
@@ -39,11 +40,11 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   private LightWeightGSet.LinkedElement nextLinkedElement;
   private LightWeightGSet.LinkedElement nextLinkedElement;
 
 
   /**
   /**
-   * This array contains triplets of references. For each i-th datanode the
-   * block belongs to triplets[3*i] is the reference to the DatanodeDescriptor
-   * and triplets[3*i+1] and triplets[3*i+2] are references to the previous and
-   * the next blocks, respectively, in the list of blocks belonging to this
-   * data-node.
+   * This array contains triplets of references. For each i-th storage, the
+   * block belongs to triplets[3*i] is the reference to the
+   * {@link DatanodeStorageInfo} and triplets[3*i+1] and triplets[3*i+2] are
+   * references to the previous and the next blocks, respectively, in the list
+   * of blocks belonging to this storage.
    * 
    * 
    * Using previous and next in Object triplets is done instead of a
    * Using previous and next in Object triplets is done instead of a
    * {@link LinkedList} list to efficiently use memory. With LinkedList the cost
    * {@link LinkedList} list to efficiently use memory. With LinkedList the cost
@@ -86,9 +87,14 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   }
   }
 
 
   public DatanodeDescriptor getDatanode(int index) {
   public DatanodeDescriptor getDatanode(int index) {
+    DatanodeStorageInfo storage = getStorageInfo(index);
+    return storage == null ? null : storage.getDatanodeDescriptor();
+  }
+
+  DatanodeStorageInfo getStorageInfo(int index) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert this.triplets != null : "BlockInfo is not initialized";
     assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
     assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
-    return (DatanodeDescriptor)triplets[index*3];
+    return (DatanodeStorageInfo)triplets[index*3];
   }
   }
 
 
   private BlockInfo getPrevious(int index) {
   private BlockInfo getPrevious(int index) {
@@ -111,14 +117,10 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     return info;
     return info;
   }
   }
 
 
-  private void setDatanode(int index, DatanodeDescriptor node, BlockInfo previous,
-      BlockInfo next) {
+  private void setStorageInfo(int index, DatanodeStorageInfo storage) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert this.triplets != null : "BlockInfo is not initialized";
-    int i = index * 3;
-    assert index >= 0 && i+2 < triplets.length : "Index is out of bound";
-    triplets[i] = node;
-    triplets[i+1] = previous;
-    triplets[i+2] = next;
+    assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
+    triplets[index*3] = storage;
   }
   }
 
 
   /**
   /**
@@ -190,22 +192,34 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   }
   }
 
 
   /**
   /**
-   * Add data-node this block belongs to.
+   * Add a {@link DatanodeStorageInfo} location for a block
    */
    */
-  public boolean addNode(DatanodeDescriptor node) {
-    if(findDatanode(node) >= 0) // the node is already there
-      return false;
+  boolean addStorage(DatanodeStorageInfo storage) {
+    boolean added = true;
+    int idx = findDatanode(storage.getDatanodeDescriptor());
+    if(idx >= 0) {
+      if (getStorageInfo(idx) == storage) { // the storage is already there
+        return false;
+      } else {
+        // The block is on the DN but belongs to a different storage.
+        // Update our state.
+        removeStorage(storage);
+        added = false;      // Just updating storage. Return false.
+      }
+    }
     // find the last null node
     // find the last null node
     int lastNode = ensureCapacity(1);
     int lastNode = ensureCapacity(1);
-    setDatanode(lastNode, node, null, null);
-    return true;
+    setStorageInfo(lastNode, storage);
+    setNext(lastNode, null);
+    setPrevious(lastNode, null);
+    return added;
   }
   }
 
 
   /**
   /**
-   * Remove data-node from the block.
+   * Remove {@link DatanodeStorageInfo} location for a block
    */
    */
-  public boolean removeNode(DatanodeDescriptor node) {
-    int dnIndex = findDatanode(node);
+  boolean removeStorage(DatanodeStorageInfo storage) {
+    int dnIndex = findStorageInfo(storage);
     if(dnIndex < 0) // the node is not found
     if(dnIndex < 0) // the node is not found
       return false;
       return false;
     assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
     assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
@@ -213,10 +227,13 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     // find the last not null node
     // find the last not null node
     int lastNode = numNodes()-1; 
     int lastNode = numNodes()-1; 
     // replace current node triplet by the lastNode one 
     // replace current node triplet by the lastNode one 
-    setDatanode(dnIndex, getDatanode(lastNode), getPrevious(lastNode),
-        getNext(lastNode));
+    setStorageInfo(dnIndex, getStorageInfo(lastNode));
+    setNext(dnIndex, getNext(lastNode)); 
+    setPrevious(dnIndex, getPrevious(lastNode)); 
     // set the last triplet to null
     // set the last triplet to null
-    setDatanode(lastNode, null, null, null);
+    setStorageInfo(lastNode, null);
+    setNext(lastNode, null); 
+    setPrevious(lastNode, null); 
     return true;
     return true;
   }
   }
 
 
@@ -236,37 +253,70 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     }
     }
     return -1;
     return -1;
   }
   }
+  /**
+   * Find specified DatanodeStorageInfo.
+   * @param dn
+   * @return index or -1 if not found.
+   */
+  int findStorageInfo(DatanodeInfo dn) {
+    int len = getCapacity();
+    for(int idx = 0; idx < len; idx++) {
+      DatanodeStorageInfo cur = getStorageInfo(idx);
+      if(cur == null)
+        break;
+      if(cur.getDatanodeDescriptor() == dn)
+        return idx;
+    }
+    return -1;
+  }
+  
+  /**
+   * Find specified DatanodeStorageInfo.
+   * @param storageInfo
+   * @return index or -1 if not found.
+   */
+  int findStorageInfo(DatanodeStorageInfo storageInfo) {
+    int len = getCapacity();
+    for(int idx = 0; idx < len; idx++) {
+      DatanodeStorageInfo cur = getStorageInfo(idx);
+      if(cur == storageInfo)
+        return idx;
+      if(cur == null)
+        break;
+    }
+    return -1;
+  }
 
 
   /**
   /**
    * Insert this block into the head of the list of blocks 
    * Insert this block into the head of the list of blocks 
-   * related to the specified DatanodeDescriptor.
+   * related to the specified DatanodeStorageInfo.
    * If the head is null then form a new list.
    * If the head is null then form a new list.
    * @return current block as the new head of the list.
    * @return current block as the new head of the list.
    */
    */
-  public BlockInfo listInsert(BlockInfo head, DatanodeDescriptor dn) {
-    int dnIndex = this.findDatanode(dn);
+  BlockInfo listInsert(BlockInfo head, DatanodeStorageInfo storage) {
+    int dnIndex = this.findStorageInfo(storage);
     assert dnIndex >= 0 : "Data node is not found: current";
     assert dnIndex >= 0 : "Data node is not found: current";
     assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
     assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
             "Block is already in the list and cannot be inserted.";
             "Block is already in the list and cannot be inserted.";
     this.setPrevious(dnIndex, null);
     this.setPrevious(dnIndex, null);
     this.setNext(dnIndex, head);
     this.setNext(dnIndex, head);
     if(head != null)
     if(head != null)
-      head.setPrevious(head.findDatanode(dn), this);
+      head.setPrevious(head.findStorageInfo(storage), this);
     return this;
     return this;
   }
   }
 
 
   /**
   /**
    * Remove this block from the list of blocks 
    * Remove this block from the list of blocks 
-   * related to the specified DatanodeDescriptor.
+   * related to the specified DatanodeStorageInfo.
    * If this block is the head of the list then return the next block as 
    * If this block is the head of the list then return the next block as 
    * the new head.
    * the new head.
    * @return the new head of the list or null if the list becomes
    * @return the new head of the list or null if the list becomes
-   * empty after deletion.
+   * empy after deletion.
    */
    */
-  public BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
+  BlockInfo listRemove(BlockInfo head, DatanodeStorageInfo storage) {
     if(head == null)
     if(head == null)
       return null;
       return null;
-    int dnIndex = this.findDatanode(dn);
+    int dnIndex = this.findStorageInfo(storage);
     if(dnIndex < 0) // this block is not on the data-node list
     if(dnIndex < 0) // this block is not on the data-node list
       return head;
       return head;
 
 
@@ -275,9 +325,9 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     this.setNext(dnIndex, null);
     this.setNext(dnIndex, null);
     this.setPrevious(dnIndex, null);
     this.setPrevious(dnIndex, null);
     if(prev != null)
     if(prev != null)
-      prev.setNext(prev.findDatanode(dn), next);
+      prev.setNext(prev.findStorageInfo(storage), next);
     if(next != null)
     if(next != null)
-      next.setPrevious(next.findDatanode(dn), prev);
+      next.setPrevious(next.findStorageInfo(storage), prev);
     if(this == head)  // removing the head
     if(this == head)  // removing the head
       head = next;
       head = next;
     return head;
     return head;
@@ -289,7 +339,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    *
    *
    * @return the new head of the list.
    * @return the new head of the list.
    */
    */
-  public BlockInfo moveBlockToHead(BlockInfo head, DatanodeDescriptor dn,
+  public BlockInfo moveBlockToHead(BlockInfo head, DatanodeStorageInfo storage,
       int curIndex, int headIndex) {
       int curIndex, int headIndex) {
     if (head == this) {
     if (head == this) {
       return this;
       return this;
@@ -298,9 +348,9 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     BlockInfo prev = this.setPrevious(curIndex, null);
     BlockInfo prev = this.setPrevious(curIndex, null);
 
 
     head.setPrevious(headIndex, this);
     head.setPrevious(headIndex, this);
-    prev.setNext(prev.findDatanode(dn), next);
+    prev.setNext(prev.findStorageInfo(storage), next);
     if (next != null)
     if (next != null)
-      next.setPrevious(next.findDatanode(dn), prev);
+      next.setPrevious(next.findStorageInfo(storage), prev);
     return this;
     return this;
   }
   }
 
 
@@ -328,10 +378,10 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * @return BlockInfoUnderConstruction -  an under construction block.
    * @return BlockInfoUnderConstruction -  an under construction block.
    */
    */
   public BlockInfoUnderConstruction convertToBlockUnderConstruction(
   public BlockInfoUnderConstruction convertToBlockUnderConstruction(
-      BlockUCState s, DatanodeDescriptor[] targets) {
+      BlockUCState s, DatanodeStorageInfo[] targets) {
     if(isComplete()) {
     if(isComplete()) {
-      return new BlockInfoUnderConstruction(
-          this, getBlockCollection().getBlockReplication(), s, targets);
+      return new BlockInfoUnderConstruction(this,
+          getBlockCollection().getBlockReplication(), s, targets);
     }
     }
     // the block is already under construction
     // the block is already under construction
     BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)this;
     BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)this;

+ 33 - 20
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java

@@ -63,12 +63,12 @@ public class BlockInfoUnderConstruction extends BlockInfo {
    * corresponding replicas.
    * corresponding replicas.
    */
    */
   static class ReplicaUnderConstruction extends Block {
   static class ReplicaUnderConstruction extends Block {
-    private DatanodeDescriptor expectedLocation;
+    private final DatanodeStorageInfo expectedLocation;
     private ReplicaState state;
     private ReplicaState state;
     private boolean chosenAsPrimary;
     private boolean chosenAsPrimary;
 
 
     ReplicaUnderConstruction(Block block,
     ReplicaUnderConstruction(Block block,
-                             DatanodeDescriptor target,
+                             DatanodeStorageInfo target,
                              ReplicaState state) {
                              ReplicaState state) {
       super(block);
       super(block);
       this.expectedLocation = target;
       this.expectedLocation = target;
@@ -82,7 +82,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
      * It is not guaranteed, but expected, that the data-node actually has
      * It is not guaranteed, but expected, that the data-node actually has
      * the replica.
      * the replica.
      */
      */
-    DatanodeDescriptor getExpectedLocation() {
+    private DatanodeStorageInfo getExpectedStorageLocation() {
       return expectedLocation;
       return expectedLocation;
     }
     }
 
 
@@ -118,7 +118,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
      * Is data-node the replica belongs to alive.
      * Is data-node the replica belongs to alive.
      */
      */
     boolean isAlive() {
     boolean isAlive() {
-      return expectedLocation.isAlive;
+      return expectedLocation.getDatanodeDescriptor().isAlive;
     }
     }
 
 
     @Override // Block
     @Override // Block
@@ -162,7 +162,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
    */
    */
   public BlockInfoUnderConstruction(Block blk, int replication,
   public BlockInfoUnderConstruction(Block blk, int replication,
                              BlockUCState state,
                              BlockUCState state,
-                             DatanodeDescriptor[] targets) {
+                             DatanodeStorageInfo[] targets) {
     super(blk, replication);
     super(blk, replication);
     assert getBlockUCState() != BlockUCState.COMPLETE :
     assert getBlockUCState() != BlockUCState.COMPLETE :
       "BlockInfoUnderConstruction cannot be in COMPLETE state";
       "BlockInfoUnderConstruction cannot be in COMPLETE state";
@@ -186,7 +186,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
   }
   }
 
 
   /** Set expected locations */
   /** Set expected locations */
-  public void setExpectedLocations(DatanodeDescriptor[] targets) {
+  public void setExpectedLocations(DatanodeStorageInfo[] targets) {
     int numLocations = targets == null ? 0 : targets.length;
     int numLocations = targets == null ? 0 : targets.length;
     this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
     this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
     for(int i = 0; i < numLocations; i++)
     for(int i = 0; i < numLocations; i++)
@@ -198,12 +198,12 @@ public class BlockInfoUnderConstruction extends BlockInfo {
    * Create array of expected replica locations
    * Create array of expected replica locations
    * (as has been assigned by chooseTargets()).
    * (as has been assigned by chooseTargets()).
    */
    */
-  public DatanodeDescriptor[] getExpectedLocations() {
+  public DatanodeStorageInfo[] getExpectedStorageLocations() {
     int numLocations = replicas == null ? 0 : replicas.size();
     int numLocations = replicas == null ? 0 : replicas.size();
-    DatanodeDescriptor[] locations = new DatanodeDescriptor[numLocations];
+    DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
     for(int i = 0; i < numLocations; i++)
     for(int i = 0; i < numLocations; i++)
-      locations[i] = replicas.get(i).getExpectedLocation();
-    return locations;
+      storages[i] = replicas.get(i).getExpectedStorageLocation();
+    return storages;
   }
   }
 
 
   /** Get the number of expected locations */
   /** Get the number of expected locations */
@@ -244,9 +244,9 @@ public class BlockInfoUnderConstruction extends BlockInfo {
     // The replica list is unchanged.
     // The replica list is unchanged.
     for (ReplicaUnderConstruction r : replicas) {
     for (ReplicaUnderConstruction r : replicas) {
       if (genStamp != r.getGenerationStamp()) {
       if (genStamp != r.getGenerationStamp()) {
-        r.getExpectedLocation().removeBlock(this);
+        r.getExpectedStorageLocation().removeBlock(this);
         NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
         NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
-            + "from location: " + r.getExpectedLocation());
+            + "from location: " + r.getExpectedStorageLocation());
       }
       }
     }
     }
   }
   }
@@ -302,31 +302,44 @@ public class BlockInfoUnderConstruction extends BlockInfo {
       if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) {
       if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) {
         continue;
         continue;
       }
       }
-      if (replicas.get(i).getExpectedLocation().getLastUpdate() > mostRecentLastUpdate) {
-        primary = replicas.get(i);
+      final ReplicaUnderConstruction ruc = replicas.get(i);
+      final long lastUpdate = ruc.getExpectedStorageLocation().getDatanodeDescriptor().getLastUpdate(); 
+      if (lastUpdate > mostRecentLastUpdate) {
         primaryNodeIndex = i;
         primaryNodeIndex = i;
-        mostRecentLastUpdate = primary.getExpectedLocation().getLastUpdate();
+        primary = ruc;
+        mostRecentLastUpdate = lastUpdate;
       }
       }
     }
     }
     if (primary != null) {
     if (primary != null) {
-      primary.getExpectedLocation().addBlockToBeRecovered(this);
+      primary.getExpectedStorageLocation().getDatanodeDescriptor().addBlockToBeRecovered(this);
       primary.setChosenAsPrimary(true);
       primary.setChosenAsPrimary(true);
       NameNode.blockStateChangeLog.info("BLOCK* " + this
       NameNode.blockStateChangeLog.info("BLOCK* " + this
         + " recovery started, primary=" + primary);
         + " recovery started, primary=" + primary);
     }
     }
   }
   }
 
 
-  void addReplicaIfNotPresent(DatanodeDescriptor dn,
+  void addReplicaIfNotPresent(DatanodeStorageInfo storage,
                      Block block,
                      Block block,
                      ReplicaState rState) {
                      ReplicaState rState) {
-    for (ReplicaUnderConstruction r : replicas) {
-      if (r.getExpectedLocation() == dn) {
+    Iterator<ReplicaUnderConstruction> it = replicas.iterator();
+    while (it.hasNext()) {
+      ReplicaUnderConstruction r = it.next();
+      if(r.getExpectedStorageLocation() == storage) {
         // Record the gen stamp from the report
         // Record the gen stamp from the report
         r.setGenerationStamp(block.getGenerationStamp());
         r.setGenerationStamp(block.getGenerationStamp());
         return;
         return;
+      } else if (r.getExpectedStorageLocation().getDatanodeDescriptor() ==
+          storage.getDatanodeDescriptor()) {
+
+        // The Datanode reported that the block is on a different storage
+        // than the one chosen by BlockPlacementPolicy. This can occur as
+        // we allow Datanodes to choose the target storage. Update our
+        // state by removing the stale entry and adding a new one.
+        it.remove();
+        break;
       }
       }
     }
     }
-    replicas.add(new ReplicaUnderConstruction(block, dn, rState));
+    replicas.add(new ReplicaUnderConstruction(block, storage, rState));
   }
   }
 
 
   @Override // BlockInfo
   @Override // BlockInfo

+ 185 - 165
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -34,6 +34,7 @@ import java.util.Map;
 import java.util.Queue;
 import java.util.Queue;
 import java.util.Set;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeMap;
+import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicLong;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -44,6 +45,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
@@ -70,8 +72,10 @@ import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -470,8 +474,8 @@ public class BlockManager {
   private void dumpBlockMeta(Block block, PrintWriter out) {
   private void dumpBlockMeta(Block block, PrintWriter out) {
     List<DatanodeDescriptor> containingNodes =
     List<DatanodeDescriptor> containingNodes =
                                       new ArrayList<DatanodeDescriptor>();
                                       new ArrayList<DatanodeDescriptor>();
-    List<DatanodeDescriptor> containingLiveReplicasNodes =
-      new ArrayList<DatanodeDescriptor>();
+    List<DatanodeStorageInfo> containingLiveReplicasNodes =
+      new ArrayList<DatanodeStorageInfo>();
     
     
     NumberReplicas numReplicas = new NumberReplicas();
     NumberReplicas numReplicas = new NumberReplicas();
     // source node returned is not used
     // source node returned is not used
@@ -498,9 +502,8 @@ public class BlockManager {
     Collection<DatanodeDescriptor> corruptNodes = 
     Collection<DatanodeDescriptor> corruptNodes = 
                                   corruptReplicas.getNodes(block);
                                   corruptReplicas.getNodes(block);
     
     
-    for (Iterator<DatanodeDescriptor> jt = blocksMap.nodeIterator(block);
-         jt.hasNext();) {
-      DatanodeDescriptor node = jt.next();
+    for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
+      final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       String state = "";
       String state = "";
       if (corruptNodes != null && corruptNodes.contains(node)) {
       if (corruptNodes != null && corruptNodes.contains(node)) {
         state = "(corrupt)";
         state = "(corrupt)";
@@ -509,7 +512,7 @@ public class BlockManager {
         state = "(decommissioned)";
         state = "(decommissioned)";
       }
       }
       
       
-      if (node.areBlockContentsStale()) {
+      if (storage.areBlockContentsStale()) {
         state += " (block deletions maybe out of date)";
         state += " (block deletions maybe out of date)";
       }
       }
       out.print(" " + node + state + " : ");
       out.print(" " + node + state + " : ");
@@ -660,10 +663,9 @@ public class BlockManager {
     assert oldBlock == getStoredBlock(oldBlock) :
     assert oldBlock == getStoredBlock(oldBlock) :
       "last block of the file is not in blocksMap";
       "last block of the file is not in blocksMap";
 
 
-    DatanodeDescriptor[] targets = getNodes(oldBlock);
+    DatanodeStorageInfo[] targets = getStorages(oldBlock);
 
 
-    BlockInfoUnderConstruction ucBlock =
-      bc.setLastBlock(oldBlock, targets);
+    BlockInfoUnderConstruction ucBlock = bc.setLastBlock(oldBlock, targets);
     blocksMap.replaceBlock(ucBlock);
     blocksMap.replaceBlock(ucBlock);
 
 
     // Remove block from replication queue.
     // Remove block from replication queue.
@@ -673,9 +675,8 @@ public class BlockManager {
     pendingReplications.remove(ucBlock);
     pendingReplications.remove(ucBlock);
 
 
     // remove this block from the list of pending blocks to be deleted. 
     // remove this block from the list of pending blocks to be deleted. 
-    for (DatanodeDescriptor dd : targets) {
-      String datanodeId = dd.getStorageID();
-      invalidateBlocks.remove(datanodeId, oldBlock);
+    for (DatanodeStorageInfo storage : targets) {
+      invalidateBlocks.remove(storage.getStorageID(), oldBlock);
     }
     }
     
     
     // Adjust safe-mode totals, since under-construction blocks don't
     // Adjust safe-mode totals, since under-construction blocks don't
@@ -694,18 +695,17 @@ public class BlockManager {
   /**
   /**
    * Get all valid locations of the block
    * Get all valid locations of the block
    */
    */
-  private List<String> getValidLocations(Block block) {
-    ArrayList<String> machineSet =
-      new ArrayList<String>(blocksMap.numNodes(block));
-    for(Iterator<DatanodeDescriptor> it =
-      blocksMap.nodeIterator(block); it.hasNext();) {
-      String storageID = it.next().getStorageID();
+  private List<DatanodeStorageInfo> getValidLocations(Block block) {
+    final List<DatanodeStorageInfo> locations
+        = new ArrayList<DatanodeStorageInfo>(blocksMap.numNodes(block));
+    for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
+      final String storageID = storage.getStorageID();
       // filter invalidate replicas
       // filter invalidate replicas
       if(!invalidateBlocks.contains(storageID, block)) {
       if(!invalidateBlocks.contains(storageID, block)) {
-        machineSet.add(storageID);
+        locations.add(storage);
       }
       }
     }
     }
-    return machineSet;
+    return locations;
   }
   }
   
   
   private List<LocatedBlock> createLocatedBlockList(final BlockInfo[] blocks,
   private List<LocatedBlock> createLocatedBlockList(final BlockInfo[] blocks,
@@ -773,9 +773,9 @@ public class BlockManager {
             + ", blk=" + blk);
             + ", blk=" + blk);
       }
       }
       final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)blk;
       final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)blk;
-      final DatanodeDescriptor[] locations = uc.getExpectedLocations();
+      final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
       final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
       final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
-      return new LocatedBlock(eb, locations, pos, false);
+      return new LocatedBlock(eb, storages, pos, false);
     }
     }
 
 
     // get block locations
     // get block locations
@@ -790,15 +790,14 @@ public class BlockManager {
     final int numNodes = blocksMap.numNodes(blk);
     final int numNodes = blocksMap.numNodes(blk);
     final boolean isCorrupt = numCorruptNodes == numNodes;
     final boolean isCorrupt = numCorruptNodes == numNodes;
     final int numMachines = isCorrupt ? numNodes: numNodes - numCorruptNodes;
     final int numMachines = isCorrupt ? numNodes: numNodes - numCorruptNodes;
-    final DatanodeDescriptor[] machines = new DatanodeDescriptor[numMachines];
+    final DatanodeStorageInfo[] machines = new DatanodeStorageInfo[numMachines];
     int j = 0;
     int j = 0;
     if (numMachines > 0) {
     if (numMachines > 0) {
-      for(Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(blk);
-          it.hasNext();) {
-        final DatanodeDescriptor d = it.next();
+      for(DatanodeStorageInfo storage : blocksMap.getStorages(blk)) {
+        final DatanodeDescriptor d = storage.getDatanodeDescriptor();
         final boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(blk, d);
         final boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(blk, d);
         if (isCorrupt || (!isCorrupt && !replicaCorrupt))
         if (isCorrupt || (!isCorrupt && !replicaCorrupt))
-          machines[j++] = d;
+          machines[j++] = storage;
       }
       }
     }
     }
     assert j == machines.length :
     assert j == machines.length :
@@ -990,13 +989,20 @@ public class BlockManager {
     }
     }
 
 
     node.resetBlocks();
     node.resetBlocks();
-    invalidateBlocks.remove(node.getStorageID());
+    invalidateBlocks.remove(node.getDatanodeUuid());
     
     
     // If the DN hasn't block-reported since the most recent
     // If the DN hasn't block-reported since the most recent
     // failover, then we may have been holding up on processing
     // failover, then we may have been holding up on processing
     // over-replicated blocks because of it. But we can now
     // over-replicated blocks because of it. But we can now
     // process those blocks.
     // process those blocks.
-    if (node.areBlockContentsStale()) {
+    boolean stale = false;
+    for(DatanodeStorageInfo storage : node.getStorageInfos()) {
+      if (storage.areBlockContentsStale()) {
+        stale = true;
+        break;
+      }
+    }
+    if (stale) {
       rescanPostponedMisreplicatedBlocks();
       rescanPostponedMisreplicatedBlocks();
     }
     }
   }
   }
@@ -1015,9 +1021,8 @@ public class BlockManager {
    */
    */
   private void addToInvalidates(Block b) {
   private void addToInvalidates(Block b) {
     StringBuilder datanodes = new StringBuilder();
     StringBuilder datanodes = new StringBuilder();
-    for (Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(b); it
-        .hasNext();) {
-      DatanodeDescriptor node = it.next();
+    for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
+      final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       invalidateBlocks.add(b, node, false);
       invalidateBlocks.add(b, node, false);
       datanodes.append(node).append(" ");
       datanodes.append(node).append(" ");
     }
     }
@@ -1035,7 +1040,7 @@ public class BlockManager {
    * for logging purposes
    * for logging purposes
    */
    */
   public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
   public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
-      final DatanodeInfo dn, String reason) throws IOException {
+      final DatanodeInfo dn, String storageID, String reason) throws IOException {
     assert namesystem.hasWriteLock();
     assert namesystem.hasWriteLock();
     final BlockInfo storedBlock = getStoredBlock(blk.getLocalBlock());
     final BlockInfo storedBlock = getStoredBlock(blk.getLocalBlock());
     if (storedBlock == null) {
     if (storedBlock == null) {
@@ -1048,11 +1053,11 @@ public class BlockManager {
       return;
       return;
     }
     }
     markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason,
     markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason,
-        Reason.CORRUPTION_REPORTED), dn);
+        Reason.CORRUPTION_REPORTED), dn, storageID);
   }
   }
 
 
   private void markBlockAsCorrupt(BlockToMarkCorrupt b,
   private void markBlockAsCorrupt(BlockToMarkCorrupt b,
-                                  DatanodeInfo dn) throws IOException {
+      DatanodeInfo dn, String storageID) throws IOException {
     DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
     DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
     if (node == null) {
     if (node == null) {
       throw new IOException("Cannot mark " + b
       throw new IOException("Cannot mark " + b
@@ -1068,7 +1073,7 @@ public class BlockManager {
     } 
     } 
 
 
     // Add replica to the data-node if it is not already there
     // Add replica to the data-node if it is not already there
-    node.addBlock(b.stored);
+    node.addBlock(storageID, b.stored);
 
 
     // Add this replica to corruptReplicas Map
     // Add this replica to corruptReplicas Map
     corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason,
     corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason,
@@ -1193,7 +1198,7 @@ public class BlockManager {
   @VisibleForTesting
   @VisibleForTesting
   int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
   int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
     int requiredReplication, numEffectiveReplicas;
     int requiredReplication, numEffectiveReplicas;
-    List<DatanodeDescriptor> containingNodes, liveReplicaNodes;
+    List<DatanodeDescriptor> containingNodes;
     DatanodeDescriptor srcNode;
     DatanodeDescriptor srcNode;
     BlockCollection bc = null;
     BlockCollection bc = null;
     int additionalReplRequired;
     int additionalReplRequired;
@@ -1218,7 +1223,7 @@ public class BlockManager {
 
 
             // get a source data-node
             // get a source data-node
             containingNodes = new ArrayList<DatanodeDescriptor>();
             containingNodes = new ArrayList<DatanodeDescriptor>();
-            liveReplicaNodes = new ArrayList<DatanodeDescriptor>();
+            List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<DatanodeStorageInfo>();
             NumberReplicas numReplicas = new NumberReplicas();
             NumberReplicas numReplicas = new NumberReplicas();
             srcNode = chooseSourceDatanode(
             srcNode = chooseSourceDatanode(
                 block, containingNodes, liveReplicaNodes, numReplicas,
                 block, containingNodes, liveReplicaNodes, numReplicas,
@@ -1277,7 +1282,7 @@ public class BlockManager {
     namesystem.writeLock();
     namesystem.writeLock();
     try {
     try {
       for(ReplicationWork rw : work){
       for(ReplicationWork rw : work){
-        DatanodeDescriptor[] targets = rw.targets;
+        final DatanodeStorageInfo[] targets = rw.targets;
         if(targets == null || targets.length == 0){
         if(targets == null || targets.length == 0){
           rw.targets = null;
           rw.targets = null;
           continue;
           continue;
@@ -1315,7 +1320,8 @@ public class BlockManager {
 
 
           if ( (numReplicas.liveReplicas() >= requiredReplication) &&
           if ( (numReplicas.liveReplicas() >= requiredReplication) &&
                (!blockHasEnoughRacks(block)) ) {
                (!blockHasEnoughRacks(block)) ) {
-            if (rw.srcNode.getNetworkLocation().equals(targets[0].getNetworkLocation())) {
+            if (rw.srcNode.getNetworkLocation().equals(
+                targets[0].getDatanodeDescriptor().getNetworkLocation())) {
               //No use continuing, unless a new rack in this case
               //No use continuing, unless a new rack in this case
               continue;
               continue;
             }
             }
@@ -1324,15 +1330,13 @@ public class BlockManager {
           // Add block to the to be replicated list
           // Add block to the to be replicated list
           rw.srcNode.addBlockToBeReplicated(block, targets);
           rw.srcNode.addBlockToBeReplicated(block, targets);
           scheduledWork++;
           scheduledWork++;
-
-          for (DatanodeDescriptor dn : targets) {
-            dn.incBlocksScheduled();
-          }
+          DatanodeStorageInfo.incrementBlocksScheduled(targets);
 
 
           // Move the block-replication into a "pending" state.
           // Move the block-replication into a "pending" state.
           // The reason we use 'pending' is so we can retry
           // The reason we use 'pending' is so we can retry
           // replications that fail after an appropriate amount of time.
           // replications that fail after an appropriate amount of time.
-          pendingReplications.increment(block, targets);
+          pendingReplications.increment(block,
+              DatanodeStorageInfo.toDatanodeDescriptors(targets));
           if(blockLog.isDebugEnabled()) {
           if(blockLog.isDebugEnabled()) {
             blockLog.debug(
             blockLog.debug(
                 "BLOCK* block " + block
                 "BLOCK* block " + block
@@ -1352,12 +1356,12 @@ public class BlockManager {
     if (blockLog.isInfoEnabled()) {
     if (blockLog.isInfoEnabled()) {
       // log which blocks have been scheduled for replication
       // log which blocks have been scheduled for replication
       for(ReplicationWork rw : work){
       for(ReplicationWork rw : work){
-        DatanodeDescriptor[] targets = rw.targets;
+        DatanodeStorageInfo[] targets = rw.targets;
         if (targets != null && targets.length != 0) {
         if (targets != null && targets.length != 0) {
           StringBuilder targetList = new StringBuilder("datanode(s)");
           StringBuilder targetList = new StringBuilder("datanode(s)");
           for (int k = 0; k < targets.length; k++) {
           for (int k = 0; k < targets.length; k++) {
             targetList.append(' ');
             targetList.append(' ');
-            targetList.append(targets[k]);
+            targetList.append(targets[k].getDatanodeDescriptor());
           }
           }
           blockLog.info("BLOCK* ask " + rw.srcNode
           blockLog.info("BLOCK* ask " + rw.srcNode
               + " to replicate " + rw.block + " to " + targetList);
               + " to replicate " + rw.block + " to " + targetList);
@@ -1381,15 +1385,16 @@ public class BlockManager {
    * @see BlockPlacementPolicy#chooseTarget(String, int, Node,
    * @see BlockPlacementPolicy#chooseTarget(String, int, Node,
    *      List, boolean, Set, long)
    *      List, boolean, Set, long)
    */
    */
-  public DatanodeDescriptor[] chooseTarget(final String src,
+  public DatanodeStorageInfo[] chooseTarget(final String src,
       final int numOfReplicas, final DatanodeDescriptor client,
       final int numOfReplicas, final DatanodeDescriptor client,
       final Set<Node> excludedNodes,
       final Set<Node> excludedNodes,
       final long blocksize, List<String> favoredNodes) throws IOException {
       final long blocksize, List<String> favoredNodes) throws IOException {
     List<DatanodeDescriptor> favoredDatanodeDescriptors = 
     List<DatanodeDescriptor> favoredDatanodeDescriptors = 
         getDatanodeDescriptors(favoredNodes);
         getDatanodeDescriptors(favoredNodes);
-    final DatanodeDescriptor targets[] = blockplacement.chooseTarget(src,
+    final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src,
         numOfReplicas, client, excludedNodes, blocksize, 
         numOfReplicas, client, excludedNodes, blocksize, 
-        favoredDatanodeDescriptors);
+        // TODO: get storage type from file
+        favoredDatanodeDescriptors, StorageType.DEFAULT);
     if (targets.length < minReplication) {
     if (targets.length < minReplication) {
       throw new IOException("File " + src + " could only be replicated to "
       throw new IOException("File " + src + " could only be replicated to "
           + targets.length + " nodes instead of minReplication (="
           + targets.length + " nodes instead of minReplication (="
@@ -1450,12 +1455,11 @@ public class BlockManager {
    *         the given block
    *         the given block
    */
    */
    @VisibleForTesting
    @VisibleForTesting
-   DatanodeDescriptor chooseSourceDatanode(
-                                    Block block,
-                                    List<DatanodeDescriptor> containingNodes,
-                                    List<DatanodeDescriptor> nodesContainingLiveReplicas,
-                                    NumberReplicas numReplicas,
-                                    int priority) {
+   DatanodeDescriptor chooseSourceDatanode(Block block,
+       List<DatanodeDescriptor> containingNodes,
+       List<DatanodeStorageInfo>  nodesContainingLiveReplicas,
+       NumberReplicas numReplicas,
+       int priority) {
     containingNodes.clear();
     containingNodes.clear();
     nodesContainingLiveReplicas.clear();
     nodesContainingLiveReplicas.clear();
     DatanodeDescriptor srcNode = null;
     DatanodeDescriptor srcNode = null;
@@ -1463,12 +1467,12 @@ public class BlockManager {
     int decommissioned = 0;
     int decommissioned = 0;
     int corrupt = 0;
     int corrupt = 0;
     int excess = 0;
     int excess = 0;
-    Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
+    
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
-    while(it.hasNext()) {
-      DatanodeDescriptor node = it.next();
+    for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
+      final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       LightWeightLinkedSet<Block> excessBlocks =
       LightWeightLinkedSet<Block> excessBlocks =
-        excessReplicateMap.get(node.getStorageID());
+        excessReplicateMap.get(node.getDatanodeUuid());
       if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
       if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
         corrupt++;
         corrupt++;
       else if (node.isDecommissionInProgress() || node.isDecommissioned())
       else if (node.isDecommissionInProgress() || node.isDecommissioned())
@@ -1476,7 +1480,7 @@ public class BlockManager {
       else if (excessBlocks != null && excessBlocks.contains(block)) {
       else if (excessBlocks != null && excessBlocks.contains(block)) {
         excess++;
         excess++;
       } else {
       } else {
-        nodesContainingLiveReplicas.add(node);
+        nodesContainingLiveReplicas.add(storage);
         live++;
         live++;
       }
       }
       containingNodes.add(node);
       containingNodes.add(node);
@@ -1608,10 +1612,11 @@ public class BlockManager {
   }
   }
 
 
   /**
   /**
-   * The given datanode is reporting all its blocks.
-   * Update the (machine-->blocklist) and (block-->machinelist) maps.
+   * The given storage is reporting all its blocks.
+   * Update the (storage-->block list) and (block-->storage list) maps.
    */
    */
-  public void processReport(final DatanodeID nodeID, final String poolId,
+  public void processReport(final DatanodeID nodeID,
+      final DatanodeStorage storage, final String poolId,
       final BlockListAsLongs newReport) throws IOException {
       final BlockListAsLongs newReport) throws IOException {
     namesystem.writeLock();
     namesystem.writeLock();
     final long startTime = Time.now(); //after acquiring write lock
     final long startTime = Time.now(); //after acquiring write lock
@@ -1625,26 +1630,28 @@ public class BlockManager {
 
 
       // To minimize startup time, we discard any second (or later) block reports
       // To minimize startup time, we discard any second (or later) block reports
       // that we receive while still in startup phase.
       // that we receive while still in startup phase.
-      if (namesystem.isInStartupSafeMode() && !node.isFirstBlockReport()) {
+      final DatanodeStorageInfo storageInfo = node.updateStorage(storage);
+      if (namesystem.isInStartupSafeMode()
+          && storageInfo.getBlockReportCount() > 0) {
         blockLog.info("BLOCK* processReport: "
         blockLog.info("BLOCK* processReport: "
             + "discarded non-initial block report from " + nodeID
             + "discarded non-initial block report from " + nodeID
             + " because namenode still in startup phase");
             + " because namenode still in startup phase");
         return;
         return;
       }
       }
 
 
-      if (node.numBlocks() == 0) {
+      if (storageInfo.numBlocks() == 0) {
         // The first block report can be processed a lot more efficiently than
         // The first block report can be processed a lot more efficiently than
         // ordinary block reports.  This shortens restart times.
         // ordinary block reports.  This shortens restart times.
-        processFirstBlockReport(node, newReport);
+        processFirstBlockReport(node, storage.getStorageID(), newReport);
       } else {
       } else {
-        processReport(node, newReport);
+        processReport(node, storage, newReport);
       }
       }
       
       
       // Now that we have an up-to-date block report, we know that any
       // Now that we have an up-to-date block report, we know that any
       // deletions from a previous NN iteration have been accounted for.
       // deletions from a previous NN iteration have been accounted for.
-      boolean staleBefore = node.areBlockContentsStale();
-      node.receivedBlockReport();
-      if (staleBefore && !node.areBlockContentsStale()) {
+      boolean staleBefore = storageInfo.areBlockContentsStale();
+      storageInfo.receivedBlockReport();
+      if (staleBefore && !storageInfo.areBlockContentsStale()) {
         LOG.info("BLOCK* processReport: Received first block report from "
         LOG.info("BLOCK* processReport: Received first block report from "
             + node + " after starting up or becoming active. Its block "
             + node + " after starting up or becoming active. Its block "
             + "contents are no longer considered stale");
             + "contents are no longer considered stale");
@@ -1698,28 +1705,30 @@ public class BlockManager {
   }
   }
   
   
   private void processReport(final DatanodeDescriptor node,
   private void processReport(final DatanodeDescriptor node,
+      final DatanodeStorage storage,
       final BlockListAsLongs report) throws IOException {
       final BlockListAsLongs report) throws IOException {
     // Normal case:
     // Normal case:
     // Modify the (block-->datanode) map, according to the difference
     // Modify the (block-->datanode) map, according to the difference
     // between the old and new block report.
     // between the old and new block report.
     //
     //
     Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
     Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
-    Collection<Block> toRemove = new LinkedList<Block>();
+    Collection<Block> toRemove = new TreeSet<Block>();
     Collection<Block> toInvalidate = new LinkedList<Block>();
     Collection<Block> toInvalidate = new LinkedList<Block>();
     Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
     Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
     Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
     Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
-    reportDiff(node, report, toAdd, toRemove, toInvalidate, toCorrupt, toUC);
+    reportDiff(node, storage, report,
+        toAdd, toRemove, toInvalidate, toCorrupt, toUC);
 
 
     // Process the blocks on each queue
     // Process the blocks on each queue
     for (StatefulBlockInfo b : toUC) { 
     for (StatefulBlockInfo b : toUC) { 
-      addStoredBlockUnderConstruction(b, node);
+      addStoredBlockUnderConstruction(b, node, storage.getStorageID());
     }
     }
     for (Block b : toRemove) {
     for (Block b : toRemove) {
       removeStoredBlock(b, node);
       removeStoredBlock(b, node);
     }
     }
     int numBlocksLogged = 0;
     int numBlocksLogged = 0;
     for (BlockInfo b : toAdd) {
     for (BlockInfo b : toAdd) {
-      addStoredBlock(b, node, null, numBlocksLogged < maxNumBlocksToLog);
+      addStoredBlock(b, node, storage.getStorageID(), null, numBlocksLogged < maxNumBlocksToLog);
       numBlocksLogged++;
       numBlocksLogged++;
     }
     }
     if (numBlocksLogged > maxNumBlocksToLog) {
     if (numBlocksLogged > maxNumBlocksToLog) {
@@ -1733,7 +1742,7 @@ public class BlockManager {
       addToInvalidates(b, node);
       addToInvalidates(b, node);
     }
     }
     for (BlockToMarkCorrupt b : toCorrupt) {
     for (BlockToMarkCorrupt b : toCorrupt) {
-      markBlockAsCorrupt(b, node);
+      markBlockAsCorrupt(b, node, storage.getStorageID());
     }
     }
   }
   }
 
 
@@ -1749,10 +1758,11 @@ public class BlockManager {
    * @throws IOException 
    * @throws IOException 
    */
    */
   private void processFirstBlockReport(final DatanodeDescriptor node,
   private void processFirstBlockReport(final DatanodeDescriptor node,
+      final String storageID,
       final BlockListAsLongs report) throws IOException {
       final BlockListAsLongs report) throws IOException {
     if (report == null) return;
     if (report == null) return;
     assert (namesystem.hasWriteLock());
     assert (namesystem.hasWriteLock());
-    assert (node.numBlocks() == 0);
+    assert (node.getStorageInfo(storageID).numBlocks() == 0);
     BlockReportIterator itBR = report.getBlockReportIterator();
     BlockReportIterator itBR = report.getBlockReportIterator();
 
 
     while(itBR.hasNext()) {
     while(itBR.hasNext()) {
@@ -1761,7 +1771,7 @@ public class BlockManager {
       
       
       if (shouldPostponeBlocksFromFuture &&
       if (shouldPostponeBlocksFromFuture &&
           namesystem.isGenStampInFuture(iblk)) {
           namesystem.isGenStampInFuture(iblk)) {
-        queueReportedBlock(node, iblk, reportedState,
+        queueReportedBlock(node, storageID, iblk, reportedState,
             QUEUE_REASON_FUTURE_GENSTAMP);
             QUEUE_REASON_FUTURE_GENSTAMP);
         continue;
         continue;
       }
       }
@@ -1778,10 +1788,10 @@ public class BlockManager {
         if (shouldPostponeBlocksFromFuture) {
         if (shouldPostponeBlocksFromFuture) {
           // In the Standby, we may receive a block report for a file that we
           // In the Standby, we may receive a block report for a file that we
           // just have an out-of-date gen-stamp or state for, for example.
           // just have an out-of-date gen-stamp or state for, for example.
-          queueReportedBlock(node, iblk, reportedState,
+          queueReportedBlock(node, storageID, iblk, reportedState,
               QUEUE_REASON_CORRUPT_STATE);
               QUEUE_REASON_CORRUPT_STATE);
         } else {
         } else {
-          markBlockAsCorrupt(c, node);
+          markBlockAsCorrupt(c, node, storageID);
         }
         }
         continue;
         continue;
       }
       }
@@ -1789,7 +1799,7 @@ public class BlockManager {
       // If block is under construction, add this replica to its list
       // If block is under construction, add this replica to its list
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
         ((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent(
         ((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent(
-            node, iblk, reportedState);
+            node.getStorageInfo(storageID), iblk, reportedState);
         // OpenFileBlocks only inside snapshots also will be added to safemode
         // OpenFileBlocks only inside snapshots also will be added to safemode
         // threshold. So we need to update such blocks to safemode
         // threshold. So we need to update such blocks to safemode
         // refer HDFS-5283
         // refer HDFS-5283
@@ -1802,22 +1812,25 @@ public class BlockManager {
       }      
       }      
       //add replica if appropriate
       //add replica if appropriate
       if (reportedState == ReplicaState.FINALIZED) {
       if (reportedState == ReplicaState.FINALIZED) {
-        addStoredBlockImmediate(storedBlock, node);
+        addStoredBlockImmediate(storedBlock, node, storageID);
       }
       }
     }
     }
   }
   }
 
 
-  private void reportDiff(DatanodeDescriptor dn, 
+  private void reportDiff(DatanodeDescriptor dn, DatanodeStorage storage, 
       BlockListAsLongs newReport, 
       BlockListAsLongs newReport, 
       Collection<BlockInfo> toAdd,              // add to DatanodeDescriptor
       Collection<BlockInfo> toAdd,              // add to DatanodeDescriptor
       Collection<Block> toRemove,           // remove from DatanodeDescriptor
       Collection<Block> toRemove,           // remove from DatanodeDescriptor
       Collection<Block> toInvalidate,       // should be removed from DN
       Collection<Block> toInvalidate,       // should be removed from DN
       Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
       Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
       Collection<StatefulBlockInfo> toUC) { // add to under-construction list
       Collection<StatefulBlockInfo> toUC) { // add to under-construction list
+
+    final DatanodeStorageInfo storageInfo = dn.updateStorage(storage);
+
     // place a delimiter in the list which separates blocks 
     // place a delimiter in the list which separates blocks 
     // that have been reported from those that have not
     // that have been reported from those that have not
     BlockInfo delimiter = new BlockInfo(new Block(), 1);
     BlockInfo delimiter = new BlockInfo(new Block(), 1);
-    boolean added = dn.addBlock(delimiter);
+    boolean added = storageInfo.addBlock(delimiter);
     assert added : "Delimiting block cannot be present in the node";
     assert added : "Delimiting block cannot be present in the node";
     int headIndex = 0; //currently the delimiter is in the head of the list
     int headIndex = 0; //currently the delimiter is in the head of the list
     int curIndex;
     int curIndex;
@@ -1829,20 +1842,21 @@ public class BlockManager {
     while(itBR.hasNext()) {
     while(itBR.hasNext()) {
       Block iblk = itBR.next();
       Block iblk = itBR.next();
       ReplicaState iState = itBR.getCurrentReplicaState();
       ReplicaState iState = itBR.getCurrentReplicaState();
-      BlockInfo storedBlock = processReportedBlock(dn, iblk, iState,
-                                  toAdd, toInvalidate, toCorrupt, toUC);
+      BlockInfo storedBlock = processReportedBlock(dn, storage.getStorageID(),
+          iblk, iState, toAdd, toInvalidate, toCorrupt, toUC);
+
       // move block to the head of the list
       // move block to the head of the list
       if (storedBlock != null && (curIndex = storedBlock.findDatanode(dn)) >= 0) {
       if (storedBlock != null && (curIndex = storedBlock.findDatanode(dn)) >= 0) {
-        headIndex = dn.moveBlockToHead(storedBlock, curIndex, headIndex);
+        headIndex = storageInfo.moveBlockToHead(storedBlock, curIndex, headIndex);
       }
       }
     }
     }
+
     // collect blocks that have not been reported
     // collect blocks that have not been reported
     // all of them are next to the delimiter
     // all of them are next to the delimiter
-    Iterator<? extends Block> it = new DatanodeDescriptor.BlockIterator(
-        delimiter.getNext(0), dn);
+    Iterator<BlockInfo> it = storageInfo.new BlockIterator(delimiter.getNext(0));
     while(it.hasNext())
     while(it.hasNext())
       toRemove.add(it.next());
       toRemove.add(it.next());
-    dn.removeBlock(delimiter);
+    storageInfo.removeBlock(delimiter);
   }
   }
 
 
   /**
   /**
@@ -1876,7 +1890,8 @@ public class BlockManager {
    * @return the up-to-date stored block, if it should be kept.
    * @return the up-to-date stored block, if it should be kept.
    *         Otherwise, null.
    *         Otherwise, null.
    */
    */
-  private BlockInfo processReportedBlock(final DatanodeDescriptor dn, 
+  private BlockInfo processReportedBlock(final DatanodeDescriptor dn,
+      final String storageID,
       final Block block, final ReplicaState reportedState, 
       final Block block, final ReplicaState reportedState, 
       final Collection<BlockInfo> toAdd, 
       final Collection<BlockInfo> toAdd, 
       final Collection<Block> toInvalidate, 
       final Collection<Block> toInvalidate, 
@@ -1891,7 +1906,7 @@ public class BlockManager {
   
   
     if (shouldPostponeBlocksFromFuture &&
     if (shouldPostponeBlocksFromFuture &&
         namesystem.isGenStampInFuture(block)) {
         namesystem.isGenStampInFuture(block)) {
-      queueReportedBlock(dn, block, reportedState,
+      queueReportedBlock(dn, storageID, block, reportedState,
           QUEUE_REASON_FUTURE_GENSTAMP);
           QUEUE_REASON_FUTURE_GENSTAMP);
       return null;
       return null;
     }
     }
@@ -1912,7 +1927,7 @@ public class BlockManager {
     }
     }
 
 
     // Ignore replicas already scheduled to be removed from the DN
     // Ignore replicas already scheduled to be removed from the DN
-    if(invalidateBlocks.contains(dn.getStorageID(), block)) {
+    if(invalidateBlocks.contains(dn.getDatanodeUuid(), block)) {
 /*  TODO: following assertion is incorrect, see HDFS-2668
 /*  TODO: following assertion is incorrect, see HDFS-2668
 assert storedBlock.findDatanode(dn) < 0 : "Block " + block
 assert storedBlock.findDatanode(dn) < 0 : "Block " + block
         + " in recentInvalidatesSet should not appear in DN " + dn; */
         + " in recentInvalidatesSet should not appear in DN " + dn; */
@@ -1926,7 +1941,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
         // If the block is an out-of-date generation stamp or state,
         // If the block is an out-of-date generation stamp or state,
         // but we're the standby, we shouldn't treat it as corrupt,
         // but we're the standby, we shouldn't treat it as corrupt,
         // but instead just queue it for later processing.
         // but instead just queue it for later processing.
-        queueReportedBlock(dn, storedBlock, reportedState,
+        queueReportedBlock(dn, storageID, storedBlock, reportedState,
             QUEUE_REASON_CORRUPT_STATE);
             QUEUE_REASON_CORRUPT_STATE);
       } else {
       } else {
         toCorrupt.add(c);
         toCorrupt.add(c);
@@ -1955,7 +1970,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
    * standby node. @see PendingDataNodeMessages.
    * standby node. @see PendingDataNodeMessages.
    * @param reason a textual reason to report in the debug logs
    * @param reason a textual reason to report in the debug logs
    */
    */
-  private void queueReportedBlock(DatanodeDescriptor dn, Block block,
+  private void queueReportedBlock(DatanodeDescriptor dn, String storageID, Block block,
       ReplicaState reportedState, String reason) {
       ReplicaState reportedState, String reason) {
     assert shouldPostponeBlocksFromFuture;
     assert shouldPostponeBlocksFromFuture;
     
     
@@ -1965,7 +1980,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
           " from datanode " + dn + " for later processing " +
           " from datanode " + dn + " for later processing " +
           "because " + reason + ".");
           "because " + reason + ".");
     }
     }
-    pendingDNMessages.enqueueReportedBlock(dn, block, reportedState);
+    pendingDNMessages.enqueueReportedBlock(dn, storageID, block, reportedState);
   }
   }
 
 
   /**
   /**
@@ -1988,8 +2003,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Processing previouly queued message " + rbi);
         LOG.debug("Processing previouly queued message " + rbi);
       }
       }
-      processAndHandleReportedBlock(
-          rbi.getNode(), rbi.getBlock(), rbi.getReportedState(), null);
+      processAndHandleReportedBlock(rbi.getNode(), rbi.getStorageID(), 
+          rbi.getBlock(), rbi.getReportedState(), null);
     }
     }
   }
   }
   
   
@@ -2106,19 +2121,21 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       return false;
       return false;
     }
     }
   }
   }
-  
+
   void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
   void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
-      DatanodeDescriptor node) throws IOException {
+      DatanodeDescriptor node, String storageID) throws IOException {
     BlockInfoUnderConstruction block = ucBlock.storedBlock;
     BlockInfoUnderConstruction block = ucBlock.storedBlock;
-    block.addReplicaIfNotPresent(node, ucBlock.reportedBlock, ucBlock.reportedState);
+    block.addReplicaIfNotPresent(node.getStorageInfo(storageID),
+        ucBlock.reportedBlock, ucBlock.reportedState);
+
     if (ucBlock.reportedState == ReplicaState.FINALIZED && block.findDatanode(node) < 0) {
     if (ucBlock.reportedState == ReplicaState.FINALIZED && block.findDatanode(node) < 0) {
-      addStoredBlock(block, node, null, true);
+      addStoredBlock(block, node, storageID, null, true);
     }
     }
-  }
-  
+  } 
+
   /**
   /**
    * Faster version of
    * Faster version of
-   * {@link #addStoredBlock(BlockInfo, DatanodeDescriptor, DatanodeDescriptor, boolean)}
+   * {@link #addStoredBlock(BlockInfo, DatanodeDescriptor, String, DatanodeDescriptor, boolean)}
    * , intended for use with initial block report at startup. If not in startup
    * , intended for use with initial block report at startup. If not in startup
    * safe mode, will call standard addStoredBlock(). Assumes this method is
    * safe mode, will call standard addStoredBlock(). Assumes this method is
    * called "immediately" so there is no need to refresh the storedBlock from
    * called "immediately" so there is no need to refresh the storedBlock from
@@ -2129,17 +2146,17 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
    * @throws IOException
    * @throws IOException
    */
    */
   private void addStoredBlockImmediate(BlockInfo storedBlock,
   private void addStoredBlockImmediate(BlockInfo storedBlock,
-                               DatanodeDescriptor node)
+      DatanodeDescriptor node, String storageID)
   throws IOException {
   throws IOException {
     assert (storedBlock != null && namesystem.hasWriteLock());
     assert (storedBlock != null && namesystem.hasWriteLock());
     if (!namesystem.isInStartupSafeMode() 
     if (!namesystem.isInStartupSafeMode() 
         || namesystem.isPopulatingReplQueues()) {
         || namesystem.isPopulatingReplQueues()) {
-      addStoredBlock(storedBlock, node, null, false);
+      addStoredBlock(storedBlock, node, storageID, null, false);
       return;
       return;
     }
     }
 
 
     // just add it
     // just add it
-    node.addBlock(storedBlock);
+    node.addBlock(storageID, storedBlock);
 
 
     // Now check for completion of blocks and safe block count
     // Now check for completion of blocks and safe block count
     int numCurrentReplica = countLiveNodes(storedBlock);
     int numCurrentReplica = countLiveNodes(storedBlock);
@@ -2162,6 +2179,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
    */
    */
   private Block addStoredBlock(final BlockInfo block,
   private Block addStoredBlock(final BlockInfo block,
                                DatanodeDescriptor node,
                                DatanodeDescriptor node,
+                               String storageID,
                                DatanodeDescriptor delNodeHint,
                                DatanodeDescriptor delNodeHint,
                                boolean logEveryBlock)
                                boolean logEveryBlock)
   throws IOException {
   throws IOException {
@@ -2187,7 +2205,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     assert bc != null : "Block must belong to a file";
     assert bc != null : "Block must belong to a file";
 
 
     // add block to the datanode
     // add block to the datanode
-    boolean added = node.addBlock(storedBlock);
+    boolean added = node.addBlock(storageID, storedBlock);
 
 
     int curReplicaDelta;
     int curReplicaDelta;
     if (added) {
     if (added) {
@@ -2447,19 +2465,19 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     Collection<DatanodeDescriptor> nonExcess = new ArrayList<DatanodeDescriptor>();
     Collection<DatanodeDescriptor> nonExcess = new ArrayList<DatanodeDescriptor>();
     Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
     Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
         .getNodes(block);
         .getNodes(block);
-    for (Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
-         it.hasNext();) {
-      DatanodeDescriptor cur = it.next();
-      if (cur.areBlockContentsStale()) {
+    for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
+      final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
+      if (storage.areBlockContentsStale()) {
         LOG.info("BLOCK* processOverReplicatedBlock: " +
         LOG.info("BLOCK* processOverReplicatedBlock: " +
             "Postponing processing of over-replicated " +
             "Postponing processing of over-replicated " +
-            block + " since datanode " + cur + " does not yet have up-to-date " +
+            block + " since storage + " + storage
+            + "datanode " + cur + " does not yet have up-to-date " +
             "block information.");
             "block information.");
         postponeBlock(block);
         postponeBlock(block);
         return;
         return;
       }
       }
       LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(cur
       LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(cur
-          .getStorageID());
+          .getDatanodeUuid());
       if (excessBlocks == null || !excessBlocks.contains(block)) {
       if (excessBlocks == null || !excessBlocks.contains(block)) {
         if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
         if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
           // exclude corrupt replicas
           // exclude corrupt replicas
@@ -2548,10 +2566,10 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
 
 
   private void addToExcessReplicate(DatanodeInfo dn, Block block) {
   private void addToExcessReplicate(DatanodeInfo dn, Block block) {
     assert namesystem.hasWriteLock();
     assert namesystem.hasWriteLock();
-    LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(dn.getStorageID());
+    LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(dn.getDatanodeUuid());
     if (excessBlocks == null) {
     if (excessBlocks == null) {
       excessBlocks = new LightWeightLinkedSet<Block>();
       excessBlocks = new LightWeightLinkedSet<Block>();
-      excessReplicateMap.put(dn.getStorageID(), excessBlocks);
+      excessReplicateMap.put(dn.getDatanodeUuid(), excessBlocks);
     }
     }
     if (excessBlocks.add(block)) {
     if (excessBlocks.add(block)) {
       excessBlocksCount.incrementAndGet();
       excessBlocksCount.incrementAndGet();
@@ -2599,7 +2617,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       // in "excess" there.
       // in "excess" there.
       //
       //
       LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(node
       LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(node
-          .getStorageID());
+          .getDatanodeUuid());
       if (excessBlocks != null) {
       if (excessBlocks != null) {
         if (excessBlocks.remove(block)) {
         if (excessBlocks.remove(block)) {
           excessBlocksCount.decrementAndGet();
           excessBlocksCount.decrementAndGet();
@@ -2608,7 +2626,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
                 + block + " is removed from excessBlocks");
                 + block + " is removed from excessBlocks");
           }
           }
           if (excessBlocks.size() == 0) {
           if (excessBlocks.size() == 0) {
-            excessReplicateMap.remove(node.getStorageID());
+            excessReplicateMap.remove(node.getDatanodeUuid());
           }
           }
         }
         }
       }
       }
@@ -2623,12 +2641,18 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
    * return the length of the added block; 0 if the block is not added
    * return the length of the added block; 0 if the block is not added
    */
    */
   private long addBlock(Block block, List<BlockWithLocations> results) {
   private long addBlock(Block block, List<BlockWithLocations> results) {
-    final List<String> machineSet = getValidLocations(block);
-    if(machineSet.size() == 0) {
+    final List<DatanodeStorageInfo> locations = getValidLocations(block);
+    if(locations.size() == 0) {
       return 0;
       return 0;
     } else {
     } else {
-      results.add(new BlockWithLocations(block, 
-          machineSet.toArray(new String[machineSet.size()])));
+      final String[] datanodeUuids = new String[locations.size()];
+      final String[] storageIDs = new String[datanodeUuids.length];
+      for(int i = 0; i < locations.size(); i++) {
+        final DatanodeStorageInfo s = locations.get(i);
+        datanodeUuids[i] = s.getDatanodeDescriptor().getDatanodeUuid();
+        storageIDs[i] = s.getStorageID();
+      }
+      results.add(new BlockWithLocations(block, datanodeUuids, storageIDs));
       return block.getNumBytes();
       return block.getNumBytes();
     }
     }
   }
   }
@@ -2637,12 +2661,12 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
    * The given node is reporting that it received a certain block.
    * The given node is reporting that it received a certain block.
    */
    */
   @VisibleForTesting
   @VisibleForTesting
-  void addBlock(DatanodeDescriptor node, Block block, String delHint)
+  void addBlock(DatanodeDescriptor node, String storageID, Block block, String delHint)
       throws IOException {
       throws IOException {
-    // decrement number of blocks scheduled to this datanode.
+    // Decrement number of blocks scheduled to this datanode.
     // for a retry request (of DatanodeProtocol#blockReceivedAndDeleted with 
     // for a retry request (of DatanodeProtocol#blockReceivedAndDeleted with 
     // RECEIVED_BLOCK), we currently also decrease the approximate number. 
     // RECEIVED_BLOCK), we currently also decrease the approximate number. 
-    node.decBlocksScheduled();
+    node.decrementBlocksScheduled();
 
 
     // get the deletion hint node
     // get the deletion hint node
     DatanodeDescriptor delHintNode = null;
     DatanodeDescriptor delHintNode = null;
@@ -2658,11 +2682,12 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     // Modify the blocks->datanode map and node's map.
     // Modify the blocks->datanode map and node's map.
     //
     //
     pendingReplications.decrement(block, node);
     pendingReplications.decrement(block, node);
-    processAndHandleReportedBlock(node, block, ReplicaState.FINALIZED,
+    processAndHandleReportedBlock(node, storageID, block, ReplicaState.FINALIZED,
         delHintNode);
         delHintNode);
   }
   }
   
   
-  private void processAndHandleReportedBlock(DatanodeDescriptor node, Block block,
+  private void processAndHandleReportedBlock(DatanodeDescriptor node,
+      String storageID, Block block,
       ReplicaState reportedState, DatanodeDescriptor delHintNode)
       ReplicaState reportedState, DatanodeDescriptor delHintNode)
       throws IOException {
       throws IOException {
     // blockReceived reports a finalized block
     // blockReceived reports a finalized block
@@ -2670,7 +2695,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     Collection<Block> toInvalidate = new LinkedList<Block>();
     Collection<Block> toInvalidate = new LinkedList<Block>();
     Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
     Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
     Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
     Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
-    processReportedBlock(node, block, reportedState,
+    processReportedBlock(node, storageID, block, reportedState,
                               toAdd, toInvalidate, toCorrupt, toUC);
                               toAdd, toInvalidate, toCorrupt, toUC);
     // the block is only in one of the to-do lists
     // the block is only in one of the to-do lists
     // if it is in none then data-node already has it
     // if it is in none then data-node already has it
@@ -2678,11 +2703,11 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       : "The block should be only in one of the lists.";
       : "The block should be only in one of the lists.";
 
 
     for (StatefulBlockInfo b : toUC) { 
     for (StatefulBlockInfo b : toUC) { 
-      addStoredBlockUnderConstruction(b, node);
+      addStoredBlockUnderConstruction(b, node, storageID);
     }
     }
     long numBlocksLogged = 0;
     long numBlocksLogged = 0;
     for (BlockInfo b : toAdd) {
     for (BlockInfo b : toAdd) {
-      addStoredBlock(b, node, delHintNode, numBlocksLogged < maxNumBlocksToLog);
+      addStoredBlock(b, node, storageID, delHintNode, numBlocksLogged < maxNumBlocksToLog);
       numBlocksLogged++;
       numBlocksLogged++;
     }
     }
     if (numBlocksLogged > maxNumBlocksToLog) {
     if (numBlocksLogged > maxNumBlocksToLog) {
@@ -2696,7 +2721,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       addToInvalidates(b, node);
       addToInvalidates(b, node);
     }
     }
     for (BlockToMarkCorrupt b : toCorrupt) {
     for (BlockToMarkCorrupt b : toCorrupt) {
-      markBlockAsCorrupt(b, node);
+      markBlockAsCorrupt(b, node, storageID);
     }
     }
   }
   }
 
 
@@ -2708,7 +2733,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
    * This method must be called with FSNamesystem lock held.
    * This method must be called with FSNamesystem lock held.
    */
    */
   public void processIncrementalBlockReport(final DatanodeID nodeID,
   public void processIncrementalBlockReport(final DatanodeID nodeID,
-      final String poolId, final ReceivedDeletedBlockInfo blockInfos[])
+      final String poolId, final StorageReceivedDeletedBlocks srdb)
       throws IOException {
       throws IOException {
     assert namesystem.hasWriteLock();
     assert namesystem.hasWriteLock();
     int received = 0;
     int received = 0;
@@ -2724,19 +2749,19 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
           "Got incremental block report from unregistered or dead node");
           "Got incremental block report from unregistered or dead node");
     }
     }
 
 
-    for (ReceivedDeletedBlockInfo rdbi : blockInfos) {
+    for (ReceivedDeletedBlockInfo rdbi : srdb.getBlocks()) {
       switch (rdbi.getStatus()) {
       switch (rdbi.getStatus()) {
       case DELETED_BLOCK:
       case DELETED_BLOCK:
         removeStoredBlock(rdbi.getBlock(), node);
         removeStoredBlock(rdbi.getBlock(), node);
         deleted++;
         deleted++;
         break;
         break;
       case RECEIVED_BLOCK:
       case RECEIVED_BLOCK:
-        addBlock(node, rdbi.getBlock(), rdbi.getDelHints());
+        addBlock(node, srdb.getStorageID(), rdbi.getBlock(), rdbi.getDelHints());
         received++;
         received++;
         break;
         break;
       case RECEIVING_BLOCK:
       case RECEIVING_BLOCK:
         receiving++;
         receiving++;
-        processAndHandleReportedBlock(node, rdbi.getBlock(),
+        processAndHandleReportedBlock(node, srdb.getStorageID(), rdbi.getBlock(),
             ReplicaState.RBW, null);
             ReplicaState.RBW, null);
         break;
         break;
       default:
       default:
@@ -2768,24 +2793,23 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     int corrupt = 0;
     int corrupt = 0;
     int excess = 0;
     int excess = 0;
     int stale = 0;
     int stale = 0;
-    Iterator<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(b);
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
-    while (nodeIter.hasNext()) {
-      DatanodeDescriptor node = nodeIter.next();
+    for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
+      final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) {
       if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) {
         corrupt++;
         corrupt++;
       } else if (node.isDecommissionInProgress() || node.isDecommissioned()) {
       } else if (node.isDecommissionInProgress() || node.isDecommissioned()) {
         decommissioned++;
         decommissioned++;
       } else {
       } else {
         LightWeightLinkedSet<Block> blocksExcess = excessReplicateMap.get(node
         LightWeightLinkedSet<Block> blocksExcess = excessReplicateMap.get(node
-            .getStorageID());
+            .getDatanodeUuid());
         if (blocksExcess != null && blocksExcess.contains(b)) {
         if (blocksExcess != null && blocksExcess.contains(b)) {
           excess++;
           excess++;
         } else {
         } else {
           live++;
           live++;
         }
         }
       }
       }
-      if (node.areBlockContentsStale()) {
+      if (storage.areBlockContentsStale()) {
         stale++;
         stale++;
       }
       }
     }
     }
@@ -2808,10 +2832,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     }
     }
     // else proceed with fast case
     // else proceed with fast case
     int live = 0;
     int live = 0;
-    Iterator<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(b);
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
-    while (nodeIter.hasNext()) {
-      DatanodeDescriptor node = nodeIter.next();
+    for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
+      final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       if ((nodesCorrupt == null) || (!nodesCorrupt.contains(node)))
       if ((nodesCorrupt == null) || (!nodesCorrupt.contains(node)))
         live++;
         live++;
     }
     }
@@ -2823,10 +2846,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     int curReplicas = num.liveReplicas();
     int curReplicas = num.liveReplicas();
     int curExpectedReplicas = getReplication(block);
     int curExpectedReplicas = getReplication(block);
     BlockCollection bc = blocksMap.getBlockCollection(block);
     BlockCollection bc = blocksMap.getBlockCollection(block);
-    Iterator<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(block);
     StringBuilder nodeList = new StringBuilder();
     StringBuilder nodeList = new StringBuilder();
-    while (nodeIter.hasNext()) {
-      DatanodeDescriptor node = nodeIter.next();
+    for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
+      final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       nodeList.append(node);
       nodeList.append(node);
       nodeList.append(" ");
       nodeList.append(" ");
     }
     }
@@ -2923,14 +2945,13 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     return blocksMap.size();
     return blocksMap.size();
   }
   }
 
 
-  public DatanodeDescriptor[] getNodes(BlockInfo block) {
-    DatanodeDescriptor[] nodes =
-      new DatanodeDescriptor[block.numNodes()];
-    Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
-    for (int i = 0; it != null && it.hasNext(); i++) {
-      nodes[i] = it.next();
+  public DatanodeStorageInfo[] getStorages(BlockInfo block) {
+    final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[block.numNodes()];
+    int i = 0;
+    for(DatanodeStorageInfo s : blocksMap.getStorages(block)) {
+      storages[i++] = s;
     }
     }
-    return nodes;
+    return storages;
   }
   }
 
 
   public int getTotalBlocks() {
   public int getTotalBlocks() {
@@ -3059,9 +3080,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
                                   corruptReplicas.getNodes(b);
                                   corruptReplicas.getNodes(b);
     int numExpectedReplicas = getReplication(b);
     int numExpectedReplicas = getReplication(b);
     String rackName = null;
     String rackName = null;
-    for (Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(b); 
-         it.hasNext();) {
-      DatanodeDescriptor cur = it.next();
+    for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
+      final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
       if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
       if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
         if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
         if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
           if (numExpectedReplicas == 1 ||
           if (numExpectedReplicas == 1 ||
@@ -3105,8 +3125,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
   }
   }
 
 
   /** @return an iterator of the datanodes. */
   /** @return an iterator of the datanodes. */
-  public Iterator<DatanodeDescriptor> datanodeIterator(final Block block) {
-    return blocksMap.nodeIterator(block);
+  public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
+    return blocksMap.getStorages(block);
   }
   }
 
 
   public int numCorruptReplicas(Block block) {
   public int numCorruptReplicas(Block block) {
@@ -3257,24 +3277,24 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
 
 
     private DatanodeDescriptor srcNode;
     private DatanodeDescriptor srcNode;
     private List<DatanodeDescriptor> containingNodes;
     private List<DatanodeDescriptor> containingNodes;
-    private List<DatanodeDescriptor> liveReplicaNodes;
+    private List<DatanodeStorageInfo> liveReplicaStorages;
     private int additionalReplRequired;
     private int additionalReplRequired;
 
 
-    private DatanodeDescriptor targets[];
+    private DatanodeStorageInfo targets[];
     private int priority;
     private int priority;
 
 
     public ReplicationWork(Block block,
     public ReplicationWork(Block block,
         BlockCollection bc,
         BlockCollection bc,
         DatanodeDescriptor srcNode,
         DatanodeDescriptor srcNode,
         List<DatanodeDescriptor> containingNodes,
         List<DatanodeDescriptor> containingNodes,
-        List<DatanodeDescriptor> liveReplicaNodes,
+        List<DatanodeStorageInfo> liveReplicaStorages,
         int additionalReplRequired,
         int additionalReplRequired,
         int priority) {
         int priority) {
       this.block = block;
       this.block = block;
       this.bc = bc;
       this.bc = bc;
       this.srcNode = srcNode;
       this.srcNode = srcNode;
       this.containingNodes = containingNodes;
       this.containingNodes = containingNodes;
-      this.liveReplicaNodes = liveReplicaNodes;
+      this.liveReplicaStorages = liveReplicaStorages;
       this.additionalReplRequired = additionalReplRequired;
       this.additionalReplRequired = additionalReplRequired;
       this.priority = priority;
       this.priority = priority;
       this.targets = null;
       this.targets = null;
@@ -3283,8 +3303,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     private void chooseTargets(BlockPlacementPolicy blockplacement,
     private void chooseTargets(BlockPlacementPolicy blockplacement,
         Set<Node> excludedNodes) {
         Set<Node> excludedNodes) {
       targets = blockplacement.chooseTarget(bc.getName(),
       targets = blockplacement.chooseTarget(bc.getName(),
-          additionalReplRequired, srcNode, liveReplicaNodes, false,
-          excludedNodes, block.getNumBytes());
+          additionalReplRequired, srcNode, liveReplicaStorages, false,
+          excludedNodes, block.getNumBytes(), StorageType.DEFAULT);
     }
     }
   }
   }
 
 

+ 12 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java

@@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -67,13 +68,14 @@ public abstract class BlockPlacementPolicy {
    * @return array of DatanodeDescriptor instances chosen as target
    * @return array of DatanodeDescriptor instances chosen as target
    * and sorted as a pipeline.
    * and sorted as a pipeline.
    */
    */
-  public abstract DatanodeDescriptor[] chooseTarget(String srcPath,
+  public abstract DatanodeStorageInfo[] chooseTarget(String srcPath,
                                              int numOfReplicas,
                                              int numOfReplicas,
                                              Node writer,
                                              Node writer,
-                                             List<DatanodeDescriptor> chosenNodes,
+                                             List<DatanodeStorageInfo> chosen,
                                              boolean returnChosenNodes,
                                              boolean returnChosenNodes,
                                              Set<Node> excludedNodes,
                                              Set<Node> excludedNodes,
-                                             long blocksize);
+                                             long blocksize,
+                                             StorageType storageType);
   
   
   /**
   /**
    * Same as {@link #chooseTarget(String, int, Node, List, boolean, 
    * Same as {@link #chooseTarget(String, int, Node, List, boolean, 
@@ -82,16 +84,19 @@ public abstract class BlockPlacementPolicy {
    *          is only a hint and due to cluster state, namenode may not be 
    *          is only a hint and due to cluster state, namenode may not be 
    *          able to place the blocks on these datanodes.
    *          able to place the blocks on these datanodes.
    */
    */
-  DatanodeDescriptor[] chooseTarget(String src,
+  DatanodeStorageInfo[] chooseTarget(String src,
       int numOfReplicas, Node writer,
       int numOfReplicas, Node writer,
       Set<Node> excludedNodes,
       Set<Node> excludedNodes,
-      long blocksize, List<DatanodeDescriptor> favoredNodes) {
+      long blocksize,
+      List<DatanodeDescriptor> favoredNodes,
+      StorageType storageType) {
     // This class does not provide the functionality of placing
     // This class does not provide the functionality of placing
     // a block in favored datanodes. The implementations of this class
     // a block in favored datanodes. The implementations of this class
     // are expected to provide this functionality
     // are expected to provide this functionality
+
     return chooseTarget(src, numOfReplicas, writer, 
     return chooseTarget(src, numOfReplicas, writer, 
-        new ArrayList<DatanodeDescriptor>(numOfReplicas), false, excludedNodes, 
-        blocksize);
+        new ArrayList<DatanodeStorageInfo>(numOfReplicas), false,
+        excludedNodes, blocksize, storageType);
   }
   }
 
 
   /**
   /**

+ 186 - 139
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -29,11 +29,14 @@ import java.util.TreeSet;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
 import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.NodeBase;
@@ -103,99 +106,101 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
   }
   }
 
 
   @Override
   @Override
-  public DatanodeDescriptor[] chooseTarget(String srcPath,
+  public DatanodeStorageInfo[] chooseTarget(String srcPath,
                                     int numOfReplicas,
                                     int numOfReplicas,
                                     Node writer,
                                     Node writer,
-                                    List<DatanodeDescriptor> chosenNodes,
+                                    List<DatanodeStorageInfo> chosenNodes,
                                     boolean returnChosenNodes,
                                     boolean returnChosenNodes,
                                     Set<Node> excludedNodes,
                                     Set<Node> excludedNodes,
-                                    long blocksize) {
+                                    long blocksize,
+                                    StorageType storageType) {
     return chooseTarget(numOfReplicas, writer, chosenNodes, returnChosenNodes,
     return chooseTarget(numOfReplicas, writer, chosenNodes, returnChosenNodes,
-        excludedNodes, blocksize);
+        excludedNodes, blocksize, storageType);
   }
   }
 
 
   @Override
   @Override
-  DatanodeDescriptor[] chooseTarget(String src,
+  DatanodeStorageInfo[] chooseTarget(String src,
       int numOfReplicas,
       int numOfReplicas,
       Node writer,
       Node writer,
       Set<Node> excludedNodes,
       Set<Node> excludedNodes,
       long blocksize,
       long blocksize,
-      List<DatanodeDescriptor> favoredNodes) {
+      List<DatanodeDescriptor> favoredNodes,
+      StorageType storageType) {
     try {
     try {
       if (favoredNodes == null || favoredNodes.size() == 0) {
       if (favoredNodes == null || favoredNodes.size() == 0) {
         // Favored nodes not specified, fall back to regular block placement.
         // Favored nodes not specified, fall back to regular block placement.
         return chooseTarget(src, numOfReplicas, writer,
         return chooseTarget(src, numOfReplicas, writer,
-            new ArrayList<DatanodeDescriptor>(numOfReplicas), false, 
-            excludedNodes, blocksize);
+            new ArrayList<DatanodeStorageInfo>(numOfReplicas), false, 
+            excludedNodes, blocksize, storageType);
       }
       }
 
 
       Set<Node> favoriteAndExcludedNodes = excludedNodes == null ?
       Set<Node> favoriteAndExcludedNodes = excludedNodes == null ?
           new HashSet<Node>() : new HashSet<Node>(excludedNodes);
           new HashSet<Node>() : new HashSet<Node>(excludedNodes);
 
 
       // Choose favored nodes
       // Choose favored nodes
-      List<DatanodeDescriptor> results = new ArrayList<DatanodeDescriptor>();
+      List<DatanodeStorageInfo> results = new ArrayList<DatanodeStorageInfo>();
       boolean avoidStaleNodes = stats != null
       boolean avoidStaleNodes = stats != null
           && stats.isAvoidingStaleDataNodesForWrite();
           && stats.isAvoidingStaleDataNodesForWrite();
       for (int i = 0; i < Math.min(favoredNodes.size(), numOfReplicas); i++) {
       for (int i = 0; i < Math.min(favoredNodes.size(), numOfReplicas); i++) {
         DatanodeDescriptor favoredNode = favoredNodes.get(i);
         DatanodeDescriptor favoredNode = favoredNodes.get(i);
         // Choose a single node which is local to favoredNode.
         // Choose a single node which is local to favoredNode.
         // 'results' is updated within chooseLocalNode
         // 'results' is updated within chooseLocalNode
-        DatanodeDescriptor target = chooseLocalNode(favoredNode,
+        final DatanodeStorageInfo target = chooseLocalStorage(favoredNode,
             favoriteAndExcludedNodes, blocksize, 
             favoriteAndExcludedNodes, blocksize, 
-            getMaxNodesPerRack(results, 
-                numOfReplicas)[1], results, avoidStaleNodes);
+            getMaxNodesPerRack(results.size(), numOfReplicas)[1],
+            results, avoidStaleNodes, storageType);
         if (target == null) {
         if (target == null) {
           LOG.warn("Could not find a target for file " + src
           LOG.warn("Could not find a target for file " + src
               + " with favored node " + favoredNode); 
               + " with favored node " + favoredNode); 
           continue;
           continue;
         }
         }
-        favoriteAndExcludedNodes.add(target);
+        favoriteAndExcludedNodes.add(target.getDatanodeDescriptor());
       }
       }
 
 
       if (results.size() < numOfReplicas) {
       if (results.size() < numOfReplicas) {
         // Not enough favored nodes, choose other nodes.
         // Not enough favored nodes, choose other nodes.
         numOfReplicas -= results.size();
         numOfReplicas -= results.size();
-        DatanodeDescriptor[] remainingTargets = 
+        DatanodeStorageInfo[] remainingTargets = 
             chooseTarget(src, numOfReplicas, writer, results,
             chooseTarget(src, numOfReplicas, writer, results,
-                false, favoriteAndExcludedNodes, blocksize);
+                false, favoriteAndExcludedNodes, blocksize, storageType);
         for (int i = 0; i < remainingTargets.length; i++) {
         for (int i = 0; i < remainingTargets.length; i++) {
           results.add(remainingTargets[i]);
           results.add(remainingTargets[i]);
         }
         }
       }
       }
       return getPipeline(writer,
       return getPipeline(writer,
-          results.toArray(new DatanodeDescriptor[results.size()]));
+          results.toArray(new DatanodeStorageInfo[results.size()]));
     } catch (NotEnoughReplicasException nr) {
     } catch (NotEnoughReplicasException nr) {
       // Fall back to regular block placement disregarding favored nodes hint
       // Fall back to regular block placement disregarding favored nodes hint
       return chooseTarget(src, numOfReplicas, writer, 
       return chooseTarget(src, numOfReplicas, writer, 
-          new ArrayList<DatanodeDescriptor>(numOfReplicas), false, 
-          excludedNodes, blocksize);
+          new ArrayList<DatanodeStorageInfo>(numOfReplicas), false, 
+          excludedNodes, blocksize, storageType);
     }
     }
   }
   }
 
 
   /** This is the implementation. */
   /** This is the implementation. */
-  private DatanodeDescriptor[] chooseTarget(int numOfReplicas,
+  private DatanodeStorageInfo[] chooseTarget(int numOfReplicas,
                                     Node writer,
                                     Node writer,
-                                    List<DatanodeDescriptor> chosenNodes,
+                                    List<DatanodeStorageInfo> chosenStorage,
                                     boolean returnChosenNodes,
                                     boolean returnChosenNodes,
                                     Set<Node> excludedNodes,
                                     Set<Node> excludedNodes,
-                                    long blocksize) {
+                                    long blocksize,
+                                    StorageType storageType) {
     if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
     if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
-      return DatanodeDescriptor.EMPTY_ARRAY;
+      return DatanodeStorageInfo.EMPTY_ARRAY;
     }
     }
       
       
     if (excludedNodes == null) {
     if (excludedNodes == null) {
       excludedNodes = new HashSet<Node>();
       excludedNodes = new HashSet<Node>();
     }
     }
      
      
-    int[] result = getMaxNodesPerRack(chosenNodes, numOfReplicas);
+    int[] result = getMaxNodesPerRack(chosenStorage.size(), numOfReplicas);
     numOfReplicas = result[0];
     numOfReplicas = result[0];
     int maxNodesPerRack = result[1];
     int maxNodesPerRack = result[1];
       
       
-    List<DatanodeDescriptor> results = 
-      new ArrayList<DatanodeDescriptor>(chosenNodes);
-    for (DatanodeDescriptor node:chosenNodes) {
+    final List<DatanodeStorageInfo> results = new ArrayList<DatanodeStorageInfo>(chosenStorage);
+    for (DatanodeStorageInfo storage : chosenStorage) {
       // add localMachine and related nodes to excludedNodes
       // add localMachine and related nodes to excludedNodes
-      addToExcludedNodes(node, excludedNodes);
+      addToExcludedNodes(storage.getDatanodeDescriptor(), excludedNodes);
     }
     }
       
       
     if (!clusterMap.contains(writer)) {
     if (!clusterMap.contains(writer)) {
@@ -205,20 +210,19 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     boolean avoidStaleNodes = (stats != null
     boolean avoidStaleNodes = (stats != null
         && stats.isAvoidingStaleDataNodesForWrite());
         && stats.isAvoidingStaleDataNodesForWrite());
     Node localNode = chooseTarget(numOfReplicas, writer,
     Node localNode = chooseTarget(numOfReplicas, writer,
-        excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes);
+        excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
     if (!returnChosenNodes) {  
     if (!returnChosenNodes) {  
-      results.removeAll(chosenNodes);
+      results.removeAll(chosenStorage);
     }
     }
       
       
     // sorting nodes to form a pipeline
     // sorting nodes to form a pipeline
     return getPipeline((writer==null)?localNode:writer,
     return getPipeline((writer==null)?localNode:writer,
-                       results.toArray(new DatanodeDescriptor[results.size()]));
+                       results.toArray(new DatanodeStorageInfo[results.size()]));
   }
   }
 
 
-  private int[] getMaxNodesPerRack(List<DatanodeDescriptor> chosenNodes,
-      int numOfReplicas) {
+  private int[] getMaxNodesPerRack(int numOfChosen, int numOfReplicas) {
     int clusterSize = clusterMap.getNumOfLeaves();
     int clusterSize = clusterMap.getNumOfLeaves();
-    int totalNumOfReplicas = chosenNodes.size()+numOfReplicas;
+    int totalNumOfReplicas = numOfChosen + numOfReplicas;
     if (totalNumOfReplicas > clusterSize) {
     if (totalNumOfReplicas > clusterSize) {
       numOfReplicas -= (totalNumOfReplicas-clusterSize);
       numOfReplicas -= (totalNumOfReplicas-clusterSize);
       totalNumOfReplicas = clusterSize;
       totalNumOfReplicas = clusterSize;
@@ -243,8 +247,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                                           Set<Node> excludedNodes,
                                           Set<Node> excludedNodes,
                                           long blocksize,
                                           long blocksize,
                                           int maxNodesPerRack,
                                           int maxNodesPerRack,
-                                          List<DatanodeDescriptor> results,
-                                          final boolean avoidStaleNodes) {
+                                          List<DatanodeStorageInfo> results,
+                                          final boolean avoidStaleNodes,
+                                          StorageType storageType) {
     if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
     if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
       return writer;
       return writer;
     }
     }
@@ -253,7 +258,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     int numOfResults = results.size();
     int numOfResults = results.size();
     boolean newBlock = (numOfResults==0);
     boolean newBlock = (numOfResults==0);
     if ((writer == null || !(writer instanceof DatanodeDescriptor)) && !newBlock) {
     if ((writer == null || !(writer instanceof DatanodeDescriptor)) && !newBlock) {
-      writer = results.get(0);
+      writer = results.get(0).getDatanodeDescriptor();
     }
     }
 
 
     // Keep a copy of original excludedNodes
     // Keep a copy of original excludedNodes
@@ -261,42 +266,49 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         new HashSet<Node>(excludedNodes) : null;
         new HashSet<Node>(excludedNodes) : null;
     try {
     try {
       if (numOfResults == 0) {
       if (numOfResults == 0) {
-        writer = chooseLocalNode(writer, excludedNodes, blocksize,
-            maxNodesPerRack, results, avoidStaleNodes);
+        writer = chooseLocalStorage(writer, excludedNodes, blocksize,
+            maxNodesPerRack, results, avoidStaleNodes, storageType)
+                .getDatanodeDescriptor();
         if (--numOfReplicas == 0) {
         if (--numOfReplicas == 0) {
           return writer;
           return writer;
         }
         }
       }
       }
+      final DatanodeDescriptor dn0 = results.get(0).getDatanodeDescriptor();
       if (numOfResults <= 1) {
       if (numOfResults <= 1) {
-        chooseRemoteRack(1, results.get(0), excludedNodes, blocksize,
-            maxNodesPerRack, results, avoidStaleNodes);
+        chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack,
+            results, avoidStaleNodes, storageType);
         if (--numOfReplicas == 0) {
         if (--numOfReplicas == 0) {
           return writer;
           return writer;
         }
         }
       }
       }
       if (numOfResults <= 2) {
       if (numOfResults <= 2) {
-        if (clusterMap.isOnSameRack(results.get(0), results.get(1))) {
-          chooseRemoteRack(1, results.get(0), excludedNodes,
-                           blocksize, maxNodesPerRack, 
-                           results, avoidStaleNodes);
+        final DatanodeDescriptor dn1 = results.get(1).getDatanodeDescriptor();
+        if (clusterMap.isOnSameRack(dn0, dn1)) {
+          chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack,
+              results, avoidStaleNodes, storageType);
         } else if (newBlock){
         } else if (newBlock){
-          chooseLocalRack(results.get(1), excludedNodes, blocksize, 
-                          maxNodesPerRack, results, avoidStaleNodes);
+          chooseLocalRack(dn1, excludedNodes, blocksize, maxNodesPerRack,
+              results, avoidStaleNodes, storageType);
         } else {
         } else {
           chooseLocalRack(writer, excludedNodes, blocksize, maxNodesPerRack,
           chooseLocalRack(writer, excludedNodes, blocksize, maxNodesPerRack,
-              results, avoidStaleNodes);
+              results, avoidStaleNodes, storageType);
         }
         }
         if (--numOfReplicas == 0) {
         if (--numOfReplicas == 0) {
           return writer;
           return writer;
         }
         }
       }
       }
       chooseRandom(numOfReplicas, NodeBase.ROOT, excludedNodes, blocksize,
       chooseRandom(numOfReplicas, NodeBase.ROOT, excludedNodes, blocksize,
-          maxNodesPerRack, results, avoidStaleNodes);
+          maxNodesPerRack, results, avoidStaleNodes, storageType);
     } catch (NotEnoughReplicasException e) {
     } catch (NotEnoughReplicasException e) {
-      LOG.warn("Not able to place enough replicas, still in need of "
-               + (totalReplicasExpected - results.size()) + " to reach "
-               + totalReplicasExpected + "\n"
-               + e.getMessage());
+      final String message = "Failed to place enough replicas, still in need of "
+          + (totalReplicasExpected - results.size()) + " to reach "
+          + totalReplicasExpected + ".";
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(message, e);
+      } else {
+        LOG.warn(message + " " + e.getMessage());
+      }
+
       if (avoidStaleNodes) {
       if (avoidStaleNodes) {
         // Retry chooseTarget again, this time not avoiding stale nodes.
         // Retry chooseTarget again, this time not avoiding stale nodes.
 
 
@@ -304,14 +316,14 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         // not chosen because they were stale, decommissioned, etc.
         // not chosen because they were stale, decommissioned, etc.
         // We need to additionally exclude the nodes that were added to the 
         // We need to additionally exclude the nodes that were added to the 
         // result list in the successful calls to choose*() above.
         // result list in the successful calls to choose*() above.
-        for (Node node : results) {
-          oldExcludedNodes.add(node);
+        for (DatanodeStorageInfo resultStorage : results) {
+          oldExcludedNodes.add(resultStorage.getDatanodeDescriptor());
         }
         }
         // Set numOfReplicas, since it can get out of sync with the result list
         // Set numOfReplicas, since it can get out of sync with the result list
         // if the NotEnoughReplicasException was thrown in chooseRandom().
         // if the NotEnoughReplicasException was thrown in chooseRandom().
         numOfReplicas = totalReplicasExpected - results.size();
         numOfReplicas = totalReplicasExpected - results.size();
         return chooseTarget(numOfReplicas, writer, oldExcludedNodes, blocksize,
         return chooseTarget(numOfReplicas, writer, oldExcludedNodes, blocksize,
-            maxNodesPerRack, results, false);
+            maxNodesPerRack, results, false, storageType);
       }
       }
     }
     }
     return writer;
     return writer;
@@ -321,32 +333,36 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    * Choose <i>localMachine</i> as the target.
    * Choose <i>localMachine</i> as the target.
    * if <i>localMachine</i> is not available, 
    * if <i>localMachine</i> is not available, 
    * choose a node on the same rack
    * choose a node on the same rack
-   * @return the chosen node
+   * @return the chosen storage
    */
    */
-  protected DatanodeDescriptor chooseLocalNode(Node localMachine,
+  protected DatanodeStorageInfo chooseLocalStorage(Node localMachine,
                                              Set<Node> excludedNodes,
                                              Set<Node> excludedNodes,
                                              long blocksize,
                                              long blocksize,
                                              int maxNodesPerRack,
                                              int maxNodesPerRack,
-                                             List<DatanodeDescriptor> results,
-                                             boolean avoidStaleNodes)
+                                             List<DatanodeStorageInfo> results,
+                                             boolean avoidStaleNodes,
+                                             StorageType storageType)
       throws NotEnoughReplicasException {
       throws NotEnoughReplicasException {
     // if no local machine, randomly choose one node
     // if no local machine, randomly choose one node
     if (localMachine == null)
     if (localMachine == null)
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-          maxNodesPerRack, results, avoidStaleNodes);
+          maxNodesPerRack, results, avoidStaleNodes, storageType);
     if (preferLocalNode && localMachine instanceof DatanodeDescriptor) {
     if (preferLocalNode && localMachine instanceof DatanodeDescriptor) {
       DatanodeDescriptor localDatanode = (DatanodeDescriptor) localMachine;
       DatanodeDescriptor localDatanode = (DatanodeDescriptor) localMachine;
       // otherwise try local machine first
       // otherwise try local machine first
       if (excludedNodes.add(localMachine)) { // was not in the excluded list
       if (excludedNodes.add(localMachine)) { // was not in the excluded list
-        if (addIfIsGoodTarget(localDatanode, excludedNodes, blocksize,
-            maxNodesPerRack, false, results, avoidStaleNodes) >= 0) {
-          return localDatanode;
+        for(DatanodeStorageInfo localStorage : DFSUtil.shuffle(
+            localDatanode.getStorageInfos())) {
+          if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
+              maxNodesPerRack, false, results, avoidStaleNodes, storageType) >= 0) {
+            return localStorage;
+          }
         }
         }
       } 
       } 
     }      
     }      
     // try a node on local rack
     // try a node on local rack
     return chooseLocalRack(localMachine, excludedNodes, blocksize,
     return chooseLocalRack(localMachine, excludedNodes, blocksize,
-        maxNodesPerRack, results, avoidStaleNodes);
+        maxNodesPerRack, results, avoidStaleNodes, storageType);
   }
   }
   
   
   /**
   /**
@@ -368,27 +384,29 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    * in the cluster.
    * in the cluster.
    * @return the chosen node
    * @return the chosen node
    */
    */
-  protected DatanodeDescriptor chooseLocalRack(Node localMachine,
+  protected DatanodeStorageInfo chooseLocalRack(Node localMachine,
                                              Set<Node> excludedNodes,
                                              Set<Node> excludedNodes,
                                              long blocksize,
                                              long blocksize,
                                              int maxNodesPerRack,
                                              int maxNodesPerRack,
-                                             List<DatanodeDescriptor> results,
-                                             boolean avoidStaleNodes)
+                                             List<DatanodeStorageInfo> results,
+                                             boolean avoidStaleNodes,
+                                             StorageType storageType)
       throws NotEnoughReplicasException {
       throws NotEnoughReplicasException {
     // no local machine, so choose a random machine
     // no local machine, so choose a random machine
     if (localMachine == null) {
     if (localMachine == null) {
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-          maxNodesPerRack, results, avoidStaleNodes);
+          maxNodesPerRack, results, avoidStaleNodes, storageType);
     }
     }
       
       
     // choose one from the local rack
     // choose one from the local rack
     try {
     try {
       return chooseRandom(localMachine.getNetworkLocation(), excludedNodes,
       return chooseRandom(localMachine.getNetworkLocation(), excludedNodes,
-          blocksize, maxNodesPerRack, results, avoidStaleNodes);
+          blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
     } catch (NotEnoughReplicasException e1) {
     } catch (NotEnoughReplicasException e1) {
       // find the second replica
       // find the second replica
       DatanodeDescriptor newLocal=null;
       DatanodeDescriptor newLocal=null;
-      for(DatanodeDescriptor nextNode : results) {
+      for(DatanodeStorageInfo resultStorage : results) {
+        DatanodeDescriptor nextNode = resultStorage.getDatanodeDescriptor();
         if (nextNode != localMachine) {
         if (nextNode != localMachine) {
           newLocal = nextNode;
           newLocal = nextNode;
           break;
           break;
@@ -397,16 +415,16 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       if (newLocal != null) {
       if (newLocal != null) {
         try {
         try {
           return chooseRandom(newLocal.getNetworkLocation(), excludedNodes,
           return chooseRandom(newLocal.getNetworkLocation(), excludedNodes,
-              blocksize, maxNodesPerRack, results, avoidStaleNodes);
+              blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
         } catch(NotEnoughReplicasException e2) {
         } catch(NotEnoughReplicasException e2) {
           //otherwise randomly choose one from the network
           //otherwise randomly choose one from the network
           return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
           return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-              maxNodesPerRack, results, avoidStaleNodes);
+              maxNodesPerRack, results, avoidStaleNodes, storageType);
         }
         }
       } else {
       } else {
         //otherwise randomly choose one from the network
         //otherwise randomly choose one from the network
         return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
         return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-            maxNodesPerRack, results, avoidStaleNodes);
+            maxNodesPerRack, results, avoidStaleNodes, storageType);
       }
       }
     }
     }
   }
   }
@@ -423,48 +441,51 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                                 Set<Node> excludedNodes,
                                 Set<Node> excludedNodes,
                                 long blocksize,
                                 long blocksize,
                                 int maxReplicasPerRack,
                                 int maxReplicasPerRack,
-                                List<DatanodeDescriptor> results,
-                                boolean avoidStaleNodes)
+                                List<DatanodeStorageInfo> results,
+                                boolean avoidStaleNodes,
+                                StorageType storageType)
                                     throws NotEnoughReplicasException {
                                     throws NotEnoughReplicasException {
     int oldNumOfReplicas = results.size();
     int oldNumOfReplicas = results.size();
     // randomly choose one node from remote racks
     // randomly choose one node from remote racks
     try {
     try {
       chooseRandom(numOfReplicas, "~" + localMachine.getNetworkLocation(),
       chooseRandom(numOfReplicas, "~" + localMachine.getNetworkLocation(),
           excludedNodes, blocksize, maxReplicasPerRack, results,
           excludedNodes, blocksize, maxReplicasPerRack, results,
-          avoidStaleNodes);
+          avoidStaleNodes, storageType);
     } catch (NotEnoughReplicasException e) {
     } catch (NotEnoughReplicasException e) {
       chooseRandom(numOfReplicas-(results.size()-oldNumOfReplicas),
       chooseRandom(numOfReplicas-(results.size()-oldNumOfReplicas),
                    localMachine.getNetworkLocation(), excludedNodes, blocksize, 
                    localMachine.getNetworkLocation(), excludedNodes, blocksize, 
-                   maxReplicasPerRack, results, avoidStaleNodes);
+                   maxReplicasPerRack, results, avoidStaleNodes, storageType);
     }
     }
   }
   }
 
 
   /**
   /**
    * Randomly choose one target from the given <i>scope</i>.
    * Randomly choose one target from the given <i>scope</i>.
-   * @return the chosen node, if there is any.
+   * @return the chosen storage, if there is any.
    */
    */
-  protected DatanodeDescriptor chooseRandom(String scope,
+  protected DatanodeStorageInfo chooseRandom(String scope,
       Set<Node> excludedNodes,
       Set<Node> excludedNodes,
       long blocksize,
       long blocksize,
       int maxNodesPerRack,
       int maxNodesPerRack,
-      List<DatanodeDescriptor> results,
-      boolean avoidStaleNodes)
+      List<DatanodeStorageInfo> results,
+      boolean avoidStaleNodes,
+      StorageType storageType)
           throws NotEnoughReplicasException {
           throws NotEnoughReplicasException {
     return chooseRandom(1, scope, excludedNodes, blocksize, maxNodesPerRack,
     return chooseRandom(1, scope, excludedNodes, blocksize, maxNodesPerRack,
-        results, avoidStaleNodes);
+        results, avoidStaleNodes, storageType);
   }
   }
 
 
   /**
   /**
    * Randomly choose <i>numOfReplicas</i> targets from the given <i>scope</i>.
    * Randomly choose <i>numOfReplicas</i> targets from the given <i>scope</i>.
    * @return the first chosen node, if there is any.
    * @return the first chosen node, if there is any.
    */
    */
-  protected DatanodeDescriptor chooseRandom(int numOfReplicas,
+  protected DatanodeStorageInfo chooseRandom(int numOfReplicas,
                             String scope,
                             String scope,
                             Set<Node> excludedNodes,
                             Set<Node> excludedNodes,
                             long blocksize,
                             long blocksize,
                             int maxNodesPerRack,
                             int maxNodesPerRack,
-                            List<DatanodeDescriptor> results,
-                            boolean avoidStaleNodes)
+                            List<DatanodeStorageInfo> results,
+                            boolean avoidStaleNodes,
+                            StorageType storageType)
                                 throws NotEnoughReplicasException {
                                 throws NotEnoughReplicasException {
       
       
     int numOfAvailableNodes = clusterMap.countNumOfAvailableNodes(
     int numOfAvailableNodes = clusterMap.countNumOfAvailableNodes(
@@ -476,24 +497,32 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       builder.append("[");
       builder.append("[");
     }
     }
     boolean badTarget = false;
     boolean badTarget = false;
-    DatanodeDescriptor firstChosen = null;
+    DatanodeStorageInfo firstChosen = null;
     while(numOfReplicas > 0 && numOfAvailableNodes > 0) {
     while(numOfReplicas > 0 && numOfAvailableNodes > 0) {
       DatanodeDescriptor chosenNode = 
       DatanodeDescriptor chosenNode = 
           (DatanodeDescriptor)clusterMap.chooseRandom(scope);
           (DatanodeDescriptor)clusterMap.chooseRandom(scope);
       if (excludedNodes.add(chosenNode)) { //was not in the excluded list
       if (excludedNodes.add(chosenNode)) { //was not in the excluded list
         numOfAvailableNodes--;
         numOfAvailableNodes--;
 
 
-        int newExcludedNodes = addIfIsGoodTarget(chosenNode, excludedNodes,
-            blocksize, maxNodesPerRack, considerLoad, results, avoidStaleNodes);
-        if (newExcludedNodes >= 0) {
-          numOfReplicas--;
-          if (firstChosen == null) {
-            firstChosen = chosenNode;
+        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];
+            }
+            numOfAvailableNodes -= newExcludedNodes;
+            break;
           }
           }
-          numOfAvailableNodes -= newExcludedNodes;
-        } else {
-          badTarget = true;
         }
         }
+
+        // If no candidate storage was found on this DN then set badTarget.
+        badTarget = (i == storages.length);
       }
       }
     }
     }
       
       
@@ -512,43 +541,46 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
   }
   }
 
 
   /**
   /**
-   * If the given node is a good target, add it to the result list and
+   * If the given storage is a good target, add it to the result list and
    * update the set of excluded nodes.
    * update the set of excluded nodes.
    * @return -1 if the given is not a good target;
    * @return -1 if the given is not a good target;
    *         otherwise, return the number of nodes added to excludedNodes set.
    *         otherwise, return the number of nodes added to excludedNodes set.
    */
    */
-  int addIfIsGoodTarget(DatanodeDescriptor node,
+  int addIfIsGoodTarget(DatanodeStorageInfo storage,
       Set<Node> excludedNodes,
       Set<Node> excludedNodes,
       long blockSize,
       long blockSize,
       int maxNodesPerRack,
       int maxNodesPerRack,
       boolean considerLoad,
       boolean considerLoad,
-      List<DatanodeDescriptor> results,
-      boolean avoidStaleNodes) {
-    if (isGoodTarget(node, blockSize, maxNodesPerRack, considerLoad,
-        results, avoidStaleNodes)) {
-      results.add(node);
+      List<DatanodeStorageInfo> results,                           
+      boolean avoidStaleNodes,
+      StorageType storageType) {
+    if (isGoodTarget(storage, blockSize, maxNodesPerRack, considerLoad,
+        results, avoidStaleNodes, storageType)) {
+      results.add(storage);
       // add node and related nodes to excludedNode
       // add node and related nodes to excludedNode
-      return addToExcludedNodes(node, excludedNodes);
+      return addToExcludedNodes(storage.getDatanodeDescriptor(), excludedNodes);
     } else { 
     } else { 
       return -1;
       return -1;
     }
     }
   }
   }
 
 
-  private static void logNodeIsNotChosen(DatanodeDescriptor node, String reason) {
+  private static void logNodeIsNotChosen(DatanodeStorageInfo storage, String reason) {
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
+      final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       // build the error message for later use.
       // build the error message for later use.
       debugLoggingBuilder.get()
       debugLoggingBuilder.get()
           .append(node).append(": ")
           .append(node).append(": ")
-          .append("Node ").append(NodeBase.getPath(node))
+          .append("Storage ").append(storage)
+          .append("at node ").append(NodeBase.getPath(node))
           .append(" is not chosen because ")
           .append(" is not chosen because ")
           .append(reason);
           .append(reason);
     }
     }
   }
   }
 
 
   /**
   /**
-   * Determine if a node is a good target. 
+   * Determine if a storage is a good target. 
    * 
    * 
-   * @param node The target node
+   * @param storage The target storage
    * @param blockSize Size of block
    * @param blockSize Size of block
    * @param maxTargetPerRack Maximum number of targets per rack. The value of 
    * @param maxTargetPerRack Maximum number of targets per rack. The value of 
    *                       this parameter depends on the number of racks in 
    *                       this parameter depends on the number of racks in 
@@ -561,32 +593,43 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    *         does not have too much load, 
    *         does not have too much load, 
    *         and the rack does not have too many nodes.
    *         and the rack does not have too many nodes.
    */
    */
-  private boolean isGoodTarget(DatanodeDescriptor node,
+  private boolean isGoodTarget(DatanodeStorageInfo storage,
                                long blockSize, int maxTargetPerRack,
                                long blockSize, int maxTargetPerRack,
                                boolean considerLoad,
                                boolean considerLoad,
-                               List<DatanodeDescriptor> results,                           
-                               boolean avoidStaleNodes) {
-    // check if the node is (being) decommissed
+                               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);
+      return false;
+    }
+    if (storage.getState() == State.READ_ONLY) {
+      logNodeIsNotChosen(storage, "storage is read-only");
+      return false;
+    }
+    DatanodeDescriptor node = storage.getDatanodeDescriptor();
+    // check if the node is (being) decommissioned
     if (node.isDecommissionInProgress() || node.isDecommissioned()) {
     if (node.isDecommissionInProgress() || node.isDecommissioned()) {
-      logNodeIsNotChosen(node, "the node is (being) decommissioned ");
+      logNodeIsNotChosen(storage, "the node is (being) decommissioned ");
       return false;
       return false;
     }
     }
 
 
     if (avoidStaleNodes) {
     if (avoidStaleNodes) {
       if (node.isStale(this.staleInterval)) {
       if (node.isStale(this.staleInterval)) {
-        logNodeIsNotChosen(node, "the node is stale ");
+        logNodeIsNotChosen(storage, "the node is stale ");
         return false;
         return false;
       }
       }
     }
     }
     
     
-    long remaining = node.getRemaining() - 
-                     (node.getBlocksScheduled() * blockSize); 
-    // check the remaining capacity of the target machine
-    if (blockSize* HdfsConstants.MIN_BLOCKS_FOR_WRITE>remaining) {
-      logNodeIsNotChosen(node, "the node does not have enough space ");
+    final long requiredSize = blockSize * HdfsConstants.MIN_BLOCKS_FOR_WRITE;
+    final long scheduledSize = blockSize * node.getBlocksScheduled();
+    if (requiredSize > node.getRemaining() - scheduledSize) {
+      logNodeIsNotChosen(storage, "the node does not have enough space ");
       return false;
       return false;
     }
     }
-      
+
     // check the communication traffic of the target machine
     // check the communication traffic of the target machine
     if (considerLoad) {
     if (considerLoad) {
       double avgLoad = 0;
       double avgLoad = 0;
@@ -595,7 +638,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         avgLoad = (double)stats.getTotalLoad()/size;
         avgLoad = (double)stats.getTotalLoad()/size;
       }
       }
       if (node.getXceiverCount() > (2.0 * avgLoad)) {
       if (node.getXceiverCount() > (2.0 * avgLoad)) {
-        logNodeIsNotChosen(node, "the node is too busy ");
+        logNodeIsNotChosen(storage, "the node is too busy ");
         return false;
         return false;
       }
       }
     }
     }
@@ -603,13 +646,14 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     // check if the target rack has chosen too many nodes
     // check if the target rack has chosen too many nodes
     String rackname = node.getNetworkLocation();
     String rackname = node.getNetworkLocation();
     int counter=1;
     int counter=1;
-    for(Node result : results) {
-      if (rackname.equals(result.getNetworkLocation())) {
+    for(DatanodeStorageInfo resultStorage : results) {
+      if (rackname.equals(
+          resultStorage.getDatanodeDescriptor().getNetworkLocation())) {
         counter++;
         counter++;
       }
       }
     }
     }
     if (counter>maxTargetPerRack) {
     if (counter>maxTargetPerRack) {
-      logNodeIsNotChosen(node, "the rack has too many chosen nodes ");
+      logNodeIsNotChosen(storage, "the rack has too many chosen nodes ");
       return false;
       return false;
     }
     }
     return true;
     return true;
@@ -621,37 +665,40 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    * starts from the writer and traverses all <i>nodes</i>
    * starts from the writer and traverses all <i>nodes</i>
    * This is basically a traveling salesman problem.
    * This is basically a traveling salesman problem.
    */
    */
-  private DatanodeDescriptor[] getPipeline(Node writer,
-                                           DatanodeDescriptor[] nodes) {
-    if (nodes.length==0) return nodes;
-      
+  private DatanodeStorageInfo[] getPipeline(Node writer,
+      DatanodeStorageInfo[] storages) {
+    if (storages.length == 0) {
+      return storages;
+    }
+
     synchronized(clusterMap) {
     synchronized(clusterMap) {
       int index=0;
       int index=0;
       if (writer == null || !clusterMap.contains(writer)) {
       if (writer == null || !clusterMap.contains(writer)) {
-        writer = nodes[0];
+        writer = storages[0].getDatanodeDescriptor();
       }
       }
-      for(;index<nodes.length; index++) {
-        DatanodeDescriptor shortestNode = nodes[index];
-        int shortestDistance = clusterMap.getDistance(writer, shortestNode);
+      for(; index < storages.length; index++) {
+        DatanodeStorageInfo shortestStorage = storages[index];
+        int shortestDistance = clusterMap.getDistance(writer,
+            shortestStorage.getDatanodeDescriptor());
         int shortestIndex = index;
         int shortestIndex = index;
-        for(int i=index+1; i<nodes.length; i++) {
-          DatanodeDescriptor currentNode = nodes[i];
-          int currentDistance = clusterMap.getDistance(writer, currentNode);
+        for(int i = index + 1; i < storages.length; i++) {
+          int currentDistance = clusterMap.getDistance(writer,
+              storages[i].getDatanodeDescriptor());
           if (shortestDistance>currentDistance) {
           if (shortestDistance>currentDistance) {
             shortestDistance = currentDistance;
             shortestDistance = currentDistance;
-            shortestNode = currentNode;
+            shortestStorage = storages[i];
             shortestIndex = i;
             shortestIndex = i;
           }
           }
         }
         }
         //switch position index & shortestIndex
         //switch position index & shortestIndex
         if (index != shortestIndex) {
         if (index != shortestIndex) {
-          nodes[shortestIndex] = nodes[index];
-          nodes[index] = shortestNode;
+          storages[shortestIndex] = storages[index];
+          storages[index] = shortestStorage;
         }
         }
-        writer = shortestNode;
+        writer = shortestStorage.getDatanodeDescriptor();
       }
       }
     }
     }
-    return nodes;
+    return storages;
   }
   }
 
 
   @Override
   @Override

+ 59 - 56
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java

@@ -25,6 +25,8 @@ import java.util.Map;
 import java.util.Set;
 import java.util.Set;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
 import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
@@ -64,81 +66,87 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
    * @return the chosen node
    * @return the chosen node
    */
    */
   @Override
   @Override
-  protected DatanodeDescriptor chooseLocalNode(Node localMachine,
+  protected DatanodeStorageInfo chooseLocalStorage(Node localMachine,
       Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
       Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
-      List<DatanodeDescriptor> results, boolean avoidStaleNodes)
-        throws NotEnoughReplicasException {
+      List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
+      StorageType storageType) throws NotEnoughReplicasException {
     // if no local machine, randomly choose one node
     // if no local machine, randomly choose one node
     if (localMachine == null)
     if (localMachine == null)
       return chooseRandom(NodeBase.ROOT, excludedNodes, 
       return chooseRandom(NodeBase.ROOT, excludedNodes, 
-          blocksize, maxNodesPerRack, results, avoidStaleNodes);
+          blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
 
 
+    // otherwise try local machine first
     if (localMachine instanceof DatanodeDescriptor) {
     if (localMachine instanceof DatanodeDescriptor) {
       DatanodeDescriptor localDataNode = (DatanodeDescriptor)localMachine;
       DatanodeDescriptor localDataNode = (DatanodeDescriptor)localMachine;
-      // otherwise try local machine first
       if (excludedNodes.add(localMachine)) { // was not in the excluded list
       if (excludedNodes.add(localMachine)) { // was not in the excluded list
-        if (addIfIsGoodTarget(localDataNode, excludedNodes, blocksize,
-            maxNodesPerRack, false, results, avoidStaleNodes) >= 0) {
-          return localDataNode;
+        for(DatanodeStorageInfo localStorage : DFSUtil.shuffle(
+            localDataNode.getStorageInfos())) {
+          if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
+              maxNodesPerRack, false, results, avoidStaleNodes, storageType) >= 0) {
+            return localStorage;
+          }
         }
         }
       }
       }
     }
     }
 
 
     // try a node on local node group
     // try a node on local node group
-    DatanodeDescriptor chosenNode = chooseLocalNodeGroup(
+    DatanodeStorageInfo chosenStorage = chooseLocalNodeGroup(
         (NetworkTopologyWithNodeGroup)clusterMap, localMachine, excludedNodes, 
         (NetworkTopologyWithNodeGroup)clusterMap, localMachine, excludedNodes, 
-        blocksize, maxNodesPerRack, results, avoidStaleNodes);
-    if (chosenNode != null) {
-      return chosenNode;
+        blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
+    if (chosenStorage != null) {
+      return chosenStorage;
     }
     }
     // try a node on local rack
     // try a node on local rack
     return chooseLocalRack(localMachine, excludedNodes, 
     return chooseLocalRack(localMachine, excludedNodes, 
-        blocksize, maxNodesPerRack, results, avoidStaleNodes);
+        blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
+  }
+
+  /** @return the node of the second replica */
+  private static DatanodeDescriptor secondNode(Node localMachine,
+      List<DatanodeStorageInfo> results) {
+    // find the second replica
+    for(DatanodeStorageInfo nextStorage : results) {
+      DatanodeDescriptor nextNode = nextStorage.getDatanodeDescriptor();
+      if (nextNode != localMachine) {
+        return nextNode;
+      }
+    }
+    return null;
   }
   }
 
 
-  
   @Override
   @Override
-  protected DatanodeDescriptor chooseLocalRack(Node localMachine,
+  protected DatanodeStorageInfo chooseLocalRack(Node localMachine,
       Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
       Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
-      List<DatanodeDescriptor> results, boolean avoidStaleNodes)
-      throws NotEnoughReplicasException {
+      List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
+      StorageType storageType) throws NotEnoughReplicasException {
     // no local machine, so choose a random machine
     // no local machine, so choose a random machine
     if (localMachine == null) {
     if (localMachine == null) {
-      return chooseRandom(NodeBase.ROOT, excludedNodes, 
-                          blocksize, maxNodesPerRack, results, 
-                          avoidStaleNodes);
+      return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
+          maxNodesPerRack, results, avoidStaleNodes, storageType);
     }
     }
 
 
     // choose one from the local rack, but off-nodegroup
     // choose one from the local rack, but off-nodegroup
     try {
     try {
-      return chooseRandom(NetworkTopology.getFirstHalf(
-                              localMachine.getNetworkLocation()),
-                          excludedNodes, blocksize, 
-                          maxNodesPerRack, results, 
-                          avoidStaleNodes);
+      final String scope = NetworkTopology.getFirstHalf(localMachine.getNetworkLocation());
+      return chooseRandom(scope, excludedNodes, blocksize, maxNodesPerRack,
+          results, avoidStaleNodes, storageType);
     } catch (NotEnoughReplicasException e1) {
     } catch (NotEnoughReplicasException e1) {
       // find the second replica
       // find the second replica
-      DatanodeDescriptor newLocal=null;
-      for(DatanodeDescriptor nextNode : results) {
-        if (nextNode != localMachine) {
-          newLocal = nextNode;
-          break;
-        }
-      }
+      final DatanodeDescriptor newLocal = secondNode(localMachine, results);
       if (newLocal != null) {
       if (newLocal != null) {
         try {
         try {
           return chooseRandom(
           return chooseRandom(
               clusterMap.getRack(newLocal.getNetworkLocation()), excludedNodes,
               clusterMap.getRack(newLocal.getNetworkLocation()), excludedNodes,
-              blocksize, maxNodesPerRack, results, avoidStaleNodes);
+              blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
         } catch(NotEnoughReplicasException e2) {
         } catch(NotEnoughReplicasException e2) {
           //otherwise randomly choose one from the network
           //otherwise randomly choose one from the network
           return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
           return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-              maxNodesPerRack, results, avoidStaleNodes);
+              maxNodesPerRack, results, avoidStaleNodes, storageType);
         }
         }
       } else {
       } else {
         //otherwise randomly choose one from the network
         //otherwise randomly choose one from the network
         return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
         return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-            maxNodesPerRack, results, avoidStaleNodes);
+            maxNodesPerRack, results, avoidStaleNodes, storageType);
       }
       }
     }
     }
   }
   }
@@ -146,8 +154,9 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
   @Override
   @Override
   protected void chooseRemoteRack(int numOfReplicas,
   protected void chooseRemoteRack(int numOfReplicas,
       DatanodeDescriptor localMachine, Set<Node> excludedNodes,
       DatanodeDescriptor localMachine, Set<Node> excludedNodes,
-      long blocksize, int maxReplicasPerRack, List<DatanodeDescriptor> results,
-      boolean avoidStaleNodes) throws NotEnoughReplicasException {
+      long blocksize, int maxReplicasPerRack, List<DatanodeStorageInfo> results,
+      boolean avoidStaleNodes, StorageType storageType)
+          throws NotEnoughReplicasException {
     int oldNumOfReplicas = results.size();
     int oldNumOfReplicas = results.size();
 
 
     final String rackLocation = NetworkTopology.getFirstHalf(
     final String rackLocation = NetworkTopology.getFirstHalf(
@@ -155,12 +164,12 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
     try {
     try {
       // randomly choose from remote racks
       // randomly choose from remote racks
       chooseRandom(numOfReplicas, "~" + rackLocation, excludedNodes, blocksize,
       chooseRandom(numOfReplicas, "~" + rackLocation, excludedNodes, blocksize,
-          maxReplicasPerRack, results, avoidStaleNodes);
+          maxReplicasPerRack, results, avoidStaleNodes, storageType);
     } catch (NotEnoughReplicasException e) {
     } catch (NotEnoughReplicasException e) {
       // fall back to the local rack
       // fall back to the local rack
       chooseRandom(numOfReplicas - (results.size() - oldNumOfReplicas),
       chooseRandom(numOfReplicas - (results.size() - oldNumOfReplicas),
           rackLocation, excludedNodes, blocksize,
           rackLocation, excludedNodes, blocksize,
-          maxReplicasPerRack, results, avoidStaleNodes);
+          maxReplicasPerRack, results, avoidStaleNodes, storageType);
     }
     }
   }
   }
 
 
@@ -170,46 +179,40 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
    * if still no such node is available, choose a random node in the cluster.
    * if still no such node is available, choose a random node in the cluster.
    * @return the chosen node
    * @return the chosen node
    */
    */
-  private DatanodeDescriptor chooseLocalNodeGroup(
+  private DatanodeStorageInfo chooseLocalNodeGroup(
       NetworkTopologyWithNodeGroup clusterMap, Node localMachine,
       NetworkTopologyWithNodeGroup clusterMap, Node localMachine,
       Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
       Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
-      List<DatanodeDescriptor> results, boolean avoidStaleNodes)
-      throws NotEnoughReplicasException {
+      List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
+      StorageType storageType) throws NotEnoughReplicasException {
     // no local machine, so choose a random machine
     // no local machine, so choose a random machine
     if (localMachine == null) {
     if (localMachine == null) {
-      return chooseRandom(NodeBase.ROOT, excludedNodes, 
-      blocksize, maxNodesPerRack, results, avoidStaleNodes);
+      return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
+          maxNodesPerRack, results, avoidStaleNodes, storageType);
     }
     }
 
 
     // choose one from the local node group
     // choose one from the local node group
     try {
     try {
       return chooseRandom(
       return chooseRandom(
           clusterMap.getNodeGroup(localMachine.getNetworkLocation()),
           clusterMap.getNodeGroup(localMachine.getNetworkLocation()),
-          excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes);
+          excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes,
+          storageType);
     } catch (NotEnoughReplicasException e1) {
     } catch (NotEnoughReplicasException e1) {
-      // find the second replica
-      DatanodeDescriptor newLocal=null;
-      for(DatanodeDescriptor nextNode : results) {
-        if (nextNode != localMachine) {
-          newLocal = nextNode;
-          break;
-        }
-      }
+      final DatanodeDescriptor newLocal = secondNode(localMachine, results);
       if (newLocal != null) {
       if (newLocal != null) {
         try {
         try {
           return chooseRandom(
           return chooseRandom(
               clusterMap.getNodeGroup(newLocal.getNetworkLocation()),
               clusterMap.getNodeGroup(newLocal.getNetworkLocation()),
               excludedNodes, blocksize, maxNodesPerRack, results,
               excludedNodes, blocksize, maxNodesPerRack, results,
-              avoidStaleNodes);
+              avoidStaleNodes, storageType);
         } catch(NotEnoughReplicasException e2) {
         } catch(NotEnoughReplicasException e2) {
           //otherwise randomly choose one from the network
           //otherwise randomly choose one from the network
           return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
           return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-              maxNodesPerRack, results, avoidStaleNodes);
+              maxNodesPerRack, results, avoidStaleNodes, storageType);
         }
         }
       } else {
       } else {
         //otherwise randomly choose one from the network
         //otherwise randomly choose one from the network
         return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
         return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
-            maxNodesPerRack, results, avoidStaleNodes);
+            maxNodesPerRack, results, avoidStaleNodes, storageType);
       }
       }
     }
     }
   }
   }

+ 15 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java

@@ -29,11 +29,11 @@ import org.apache.hadoop.util.LightWeightGSet;
  * the datanodes that store the block.
  * the datanodes that store the block.
  */
  */
 class BlocksMap {
 class BlocksMap {
-  private static class NodeIterator implements Iterator<DatanodeDescriptor> {
+  private static class StorageIterator implements Iterator<DatanodeStorageInfo> {
     private BlockInfo blockInfo;
     private BlockInfo blockInfo;
     private int nextIdx = 0;
     private int nextIdx = 0;
       
       
-    NodeIterator(BlockInfo blkInfo) {
+    StorageIterator(BlockInfo blkInfo) {
       this.blockInfo = blkInfo;
       this.blockInfo = blkInfo;
     }
     }
 
 
@@ -44,8 +44,8 @@ class BlocksMap {
     }
     }
 
 
     @Override
     @Override
-    public DatanodeDescriptor next() {
-      return blockInfo.getDatanode(nextIdx++);
+    public DatanodeStorageInfo next() {
+      return blockInfo.getStorageInfo(nextIdx++);
     }
     }
 
 
     @Override
     @Override
@@ -115,18 +115,23 @@ class BlocksMap {
 
 
   /**
   /**
    * Searches for the block in the BlocksMap and 
    * Searches for the block in the BlocksMap and 
-   * returns Iterator that iterates through the nodes the block belongs to.
+   * returns {@link Iterable} of the storages the block belongs to.
    */
    */
-  Iterator<DatanodeDescriptor> nodeIterator(Block b) {
-    return nodeIterator(blocks.get(b));
+  Iterable<DatanodeStorageInfo> getStorages(Block b) {
+    return getStorages(blocks.get(b));
   }
   }
 
 
   /**
   /**
    * For a block that has already been retrieved from the BlocksMap
    * For a block that has already been retrieved from the BlocksMap
-   * returns Iterator that iterates through the nodes the block belongs to.
+   * returns {@link Iterable} of the storages the block belongs to.
    */
    */
-  Iterator<DatanodeDescriptor> nodeIterator(BlockInfo storedBlock) {
-    return new NodeIterator(storedBlock);
+  Iterable<DatanodeStorageInfo> getStorages(final BlockInfo storedBlock) {
+    return new Iterable<DatanodeStorageInfo>() {
+      @Override
+      public Iterator<DatanodeStorageInfo> iterator() {
+        return new StorageIterator(storedBlock);
+      }
+    };
   }
   }
 
 
   /** counts number of containing nodes. Better than using iterator. */
   /** counts number of containing nodes. Better than using iterator. */

+ 147 - 185
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -18,23 +18,29 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
+import java.util.Map;
 import java.util.Queue;
 import java.util.Queue;
 
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 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.LightWeightHashSet;
 import org.apache.hadoop.hdfs.util.LightWeightHashSet;
 import org.apache.hadoop.util.IntrusiveCollection;
 import org.apache.hadoop.util.IntrusiveCollection;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
 /**
  * This class extends the DatanodeInfo class with ephemeral information (eg
  * This class extends the DatanodeInfo class with ephemeral information (eg
  * health, capacity, what blocks are associated with the Datanode) that is
  * health, capacity, what blocks are associated with the Datanode) that is
@@ -43,6 +49,7 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public class DatanodeDescriptor extends DatanodeInfo {
 public class DatanodeDescriptor extends DatanodeInfo {
+  public static final Log LOG = LogFactory.getLog(DatanodeDescriptor.class);
   public static final DatanodeDescriptor[] EMPTY_ARRAY = {};
   public static final DatanodeDescriptor[] EMPTY_ARRAY = {};
 
 
   // Stores status of decommissioning.
   // Stores status of decommissioning.
@@ -54,9 +61,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
   @InterfaceStability.Evolving
   @InterfaceStability.Evolving
   public static class BlockTargetPair {
   public static class BlockTargetPair {
     public final Block block;
     public final Block block;
-    public final DatanodeDescriptor[] targets;    
+    public final DatanodeStorageInfo[] targets;    
 
 
-    BlockTargetPair(Block block, DatanodeDescriptor[] targets) {
+    BlockTargetPair(Block block, DatanodeStorageInfo[] targets) {
       this.block = block;
       this.block = block;
       this.targets = targets;
       this.targets = targets;
     }
     }
@@ -99,6 +106,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
     }
   }
   }
 
 
+  private final Map<String, DatanodeStorageInfo> storageMap = 
+      new HashMap<String, DatanodeStorageInfo>();
+
   /**
   /**
    * A list of CachedBlock objects on this datanode.
    * A list of CachedBlock objects on this datanode.
    */
    */
@@ -164,37 +174,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
    */
    */
   private long lastCachingDirectiveSentTimeMs;
   private long lastCachingDirectiveSentTimeMs;
 
 
-  /**
-   * Head of the list of blocks on the datanode
-   */
-  private volatile BlockInfo blockList = null;
-  /**
-   * Number of blocks on the datanode
-   */
-  private int numBlocks = 0;
-
   // isAlive == heartbeats.contains(this)
   // isAlive == heartbeats.contains(this)
   // This is an optimization, because contains takes O(n) time on Arraylist
   // This is an optimization, because contains takes O(n) time on Arraylist
   public boolean isAlive = false;
   public boolean isAlive = false;
   public boolean needKeyUpdate = false;
   public boolean needKeyUpdate = false;
 
 
-  /**
-   * Set to false on any NN failover, and reset to true
-   * whenever a block report is received.
-   */
-  private boolean heartbeatedSinceFailover = false;
-  
-  /**
-   * At startup or at any failover, the DNs in the cluster may
-   * have pending block deletions from a previous incarnation
-   * of the NameNode. Thus, we consider their block contents
-   * stale until we have received a block report. When a DN
-   * is considered stale, any replicas on it are transitively
-   * considered stale. If any block has at least one stale replica,
-   * then no invalidations will be processed for this block.
-   * See HDFS-1972.
-   */
-  private boolean blockContentsStale = true;
   
   
   // A system administrator can tune the balancer bandwidth parameter
   // A system administrator can tune the balancer bandwidth parameter
   // (dfs.balance.bandwidthPerSec) dynamically by calling
   // (dfs.balance.bandwidthPerSec) dynamically by calling
@@ -213,7 +197,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
   private LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
 
 
   /* Variables for maintaining number of blocks scheduled to be written to
   /* Variables for maintaining number of blocks scheduled to be written to
-   * this datanode. This count is approximate and might be slightly bigger
+   * this storage. This count is approximate and might be slightly bigger
    * in case of errors (e.g. datanode does not report if an error occurs
    * in case of errors (e.g. datanode does not report if an error occurs
    * while writing the block).
    * while writing the block).
    */
    */
@@ -223,9 +207,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min
   private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min
   private int volumeFailures = 0;
   private int volumeFailures = 0;
   
   
-  /** Set to false after processing first block report */
-  private boolean firstBlockReport = true;
-  
   /** 
   /** 
    * When set to true, the node is not in include list and is not allowed
    * When set to true, the node is not in include list and is not allowed
    * to communicate with the namenode
    * to communicate with the namenode
@@ -237,7 +218,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * @param nodeID id of the data node
    * @param nodeID id of the data node
    */
    */
   public DatanodeDescriptor(DatanodeID nodeID) {
   public DatanodeDescriptor(DatanodeID nodeID) {
-    this(nodeID, 0L, 0L, 0L, 0L, 0L, 0L, 0, 0);
+    super(nodeID);
+    updateHeartbeat(StorageReport.EMPTY_ARRAY, 0L, 0L, 0, 0);
   }
   }
 
 
   /**
   /**
@@ -247,104 +229,60 @@ public class DatanodeDescriptor extends DatanodeInfo {
    */
    */
   public DatanodeDescriptor(DatanodeID nodeID, 
   public DatanodeDescriptor(DatanodeID nodeID, 
                             String networkLocation) {
                             String networkLocation) {
-    this(nodeID, networkLocation, 0L, 0L, 0L, 0L, 0L, 0L, 0, 0);
-  }
-  
-  /**
-   * DatanodeDescriptor constructor
-   * @param nodeID id of the data node
-   * @param capacity capacity of the data node
-   * @param dfsUsed space used by the data node
-   * @param remaining remaining capacity of the data node
-   * @param bpused space used by the block pool corresponding to this namenode
-   * @param cacheCapacity cache capacity of the data node
-   * @param cacheUsed cache used on the data node
-   * @param xceiverCount # of data transfers at the data node
-   */
-  public DatanodeDescriptor(DatanodeID nodeID, 
-                            long capacity,
-                            long dfsUsed,
-                            long remaining,
-                            long bpused,
-                            long cacheCapacity,
-                            long cacheUsed,
-                            int xceiverCount,
-                            int failedVolumes) {
-    super(nodeID);
-    updateHeartbeat(capacity, dfsUsed, remaining, bpused, cacheCapacity,
-        cacheUsed, xceiverCount, failedVolumes);
+    super(nodeID, networkLocation);
+    updateHeartbeat(StorageReport.EMPTY_ARRAY, 0L, 0L, 0, 0);
   }
   }
 
 
   /**
   /**
-   * DatanodeDescriptor constructor
-   * @param nodeID id of the data node
-   * @param networkLocation location of the data node in network
-   * @param capacity capacity of the data node, including space used by non-dfs
-   * @param dfsUsed the used space by dfs datanode
-   * @param remaining remaining capacity of the data node
-   * @param bpused space used by the block pool corresponding to this namenode
-   * @param cacheCapacity cache capacity of the data node
-   * @param cacheUsed cache used on the data node
-   * @param xceiverCount # of data transfers at the data node
+   * Add data-node to the block. Add block to the head of the list of blocks
+   * belonging to the data-node.
    */
    */
-  public DatanodeDescriptor(DatanodeID nodeID,
-                            String networkLocation,
-                            long capacity,
-                            long dfsUsed,
-                            long remaining,
-                            long bpused,
-                            long cacheCapacity,
-                            long cacheUsed,
-                            int xceiverCount,
-                            int failedVolumes) {
-    super(nodeID, networkLocation);
-    updateHeartbeat(capacity, dfsUsed, remaining, bpused, cacheCapacity,
-        cacheUsed, xceiverCount, failedVolumes);
+  public boolean addBlock(String storageID, BlockInfo b) {
+    DatanodeStorageInfo s = getStorageInfo(storageID);
+    if (s != null) {
+      return s.addBlock(b);
+    }
+    return false;
   }
   }
 
 
-  /**
-   * Add datanode to the block.
-   * Add block to the head of the list of blocks belonging to the data-node.
-   */
-  public boolean addBlock(BlockInfo b) {
-    if(!b.addNode(this))
-      return false;
-    // add to the head of the data-node list
-    blockList = b.listInsert(blockList, this);
-    numBlocks++;
-    return true;
+  DatanodeStorageInfo getStorageInfo(String storageID) {
+    synchronized (storageMap) {
+      return storageMap.get(storageID);
+    }
   }
   }
-  
-  /**
-   * Remove block from the list of blocks belonging to the data-node.
-   * Remove datanode from the block.
-   */
-  public boolean removeBlock(BlockInfo b) {
-    blockList = b.listRemove(blockList, this);
-    if ( b.removeNode(this) ) {
-      numBlocks--;
-      return true;
-    } else {
-      return false;
+  DatanodeStorageInfo[] getStorageInfos() {
+    synchronized (storageMap) {
+      final Collection<DatanodeStorageInfo> storages = storageMap.values();
+      return storages.toArray(new DatanodeStorageInfo[storages.size()]);
     }
     }
   }
   }
 
 
   /**
   /**
-   * Move block to the head of the list of blocks belonging to the data-node.
-   * @return the index of the head of the blockList
+   * Remove block from the list of blocks belonging to the data-node. Remove
+   * data-node from the block.
    */
    */
-  int moveBlockToHead(BlockInfo b, int curIndex, int headIndex) {
-    blockList = b.moveBlockToHead(blockList, this, curIndex, headIndex);
-    return curIndex;
+  boolean removeBlock(BlockInfo b) {
+    int index = b.findStorageInfo(this);
+    // if block exists on this datanode
+    if (index >= 0) {
+      DatanodeStorageInfo s = b.getStorageInfo(index);
+      if (s != null) {
+        return s.removeBlock(b);
+      }
+    }
+    return false;
   }
   }
-
+  
   /**
   /**
-   * Used for testing only
-   * @return the head of the blockList
+   * Remove block from the list of blocks belonging to the data-node. Remove
+   * data-node from the block.
    */
    */
-  @VisibleForTesting
-  protected BlockInfo getHead(){
-    return blockList;
+  boolean removeBlock(String storageID, BlockInfo b) {
+    DatanodeStorageInfo s = getStorageInfo(storageID);
+    if (s != null) {
+      return s.removeBlock(b);
+    }
+    return false;
   }
   }
 
 
   /**
   /**
@@ -355,9 +293,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * @return the new block
    * @return the new block
    */
    */
   public BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) {
   public BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) {
-    boolean done = removeBlock(oldBlock);
+    int index = oldBlock.findStorageInfo(this);
+    DatanodeStorageInfo s = oldBlock.getStorageInfo(index);
+    boolean done = s.removeBlock(oldBlock);
     assert done : "Old block should belong to the data-node when replacing";
     assert done : "Old block should belong to the data-node when replacing";
-    done = addBlock(newBlock);
+
+    done = s.addBlock(newBlock);
     assert done : "New block should not belong to the data-node when replacing";
     assert done : "New block should not belong to the data-node when replacing";
     return newBlock;
     return newBlock;
   }
   }
@@ -368,7 +309,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     setBlockPoolUsed(0);
     setBlockPoolUsed(0);
     setDfsUsed(0);
     setDfsUsed(0);
     setXceiverCount(0);
     setXceiverCount(0);
-    this.blockList = null;
     this.invalidateBlocks.clear();
     this.invalidateBlocks.clear();
     this.volumeFailures = 0;
     this.volumeFailures = 0;
     // pendingCached, cached, and pendingUncached are protected by the
     // pendingCached, cached, and pendingUncached are protected by the
@@ -392,66 +332,97 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
   }
 
 
   public int numBlocks() {
   public int numBlocks() {
-    return numBlocks;
+    int blocks = 0;
+    for (DatanodeStorageInfo entry : getStorageInfos()) {
+      blocks += entry.numBlocks();
+    }
+    return blocks;
   }
   }
 
 
   /**
   /**
    * Updates stats from datanode heartbeat.
    * Updates stats from datanode heartbeat.
    */
    */
-  public void updateHeartbeat(long capacity, long dfsUsed, long remaining,
-      long blockPoolUsed, long cacheCapacity, long cacheUsed, int xceiverCount,
-      int volFailures) {
-    setCapacity(capacity);
-    setRemaining(remaining);
-    setBlockPoolUsed(blockPoolUsed);
-    setDfsUsed(dfsUsed);
+  public void updateHeartbeat(StorageReport[] reports, long cacheCapacity,
+      long cacheUsed, int xceiverCount, int volFailures) {
+    long totalCapacity = 0;
+    long totalRemaining = 0;
+    long totalBlockPoolUsed = 0;
+    long totalDfsUsed = 0;
+
     setCacheCapacity(cacheCapacity);
     setCacheCapacity(cacheCapacity);
     setCacheUsed(cacheUsed);
     setCacheUsed(cacheUsed);
     setXceiverCount(xceiverCount);
     setXceiverCount(xceiverCount);
     setLastUpdate(Time.now());    
     setLastUpdate(Time.now());    
     this.volumeFailures = volFailures;
     this.volumeFailures = volFailures;
-    this.heartbeatedSinceFailover = true;
+    for (StorageReport report : reports) {
+      DatanodeStorageInfo storage = storageMap.get(report.getStorageID());
+      if (storage == null) {
+        // This is seen during cluster initialization when the heartbeat
+        // is received before the initial block reports from each storage.
+        storage = updateStorage(new DatanodeStorage(report.getStorageID()));
+      }
+      storage.receivedHeartbeat(report);
+      totalCapacity += report.getCapacity();
+      totalRemaining += report.getRemaining();
+      totalBlockPoolUsed += report.getBlockPoolUsed();
+      totalDfsUsed += report.getDfsUsed();
+    }
     rollBlocksScheduled(getLastUpdate());
     rollBlocksScheduled(getLastUpdate());
+
+    // Update total metrics for the node.
+    setCapacity(totalCapacity);
+    setRemaining(totalRemaining);
+    setBlockPoolUsed(totalBlockPoolUsed);
+    setDfsUsed(totalDfsUsed);
   }
   }
 
 
-  /**
-   * Iterates over the list of blocks belonging to the datanode.
-   */
-  public static class BlockIterator implements Iterator<BlockInfo> {
-    private BlockInfo current;
-    private DatanodeDescriptor node;
-      
-    BlockIterator(BlockInfo head, DatanodeDescriptor dn) {
-      this.current = head;
-      this.node = dn;
+  private static class BlockIterator implements Iterator<BlockInfo> {
+    private int index = 0;
+    private final List<Iterator<BlockInfo>> iterators;
+    
+    private BlockIterator(final DatanodeStorageInfo... storages) {
+      List<Iterator<BlockInfo>> iterators = new ArrayList<Iterator<BlockInfo>>();
+      for (DatanodeStorageInfo e : storages) {
+        iterators.add(e.getBlockIterator());
+      }
+      this.iterators = Collections.unmodifiableList(iterators);
     }
     }
 
 
     @Override
     @Override
     public boolean hasNext() {
     public boolean hasNext() {
-      return current != null;
+      update();
+      return !iterators.isEmpty() && iterators.get(index).hasNext();
     }
     }
 
 
     @Override
     @Override
     public BlockInfo next() {
     public BlockInfo next() {
-      BlockInfo res = current;
-      current = current.getNext(current.findDatanode(node));
-      return res;
+      update();
+      return iterators.get(index).next();
     }
     }
-
+    
     @Override
     @Override
-    public void remove()  {
-      throw new UnsupportedOperationException("Sorry. can't remove.");
+    public void remove() {
+      throw new UnsupportedOperationException("Remove unsupported.");
+    }
+    
+    private void update() {
+      while(index < iterators.size() - 1 && !iterators.get(index).hasNext()) {
+        index++;
+      }
     }
     }
   }
   }
 
 
-  public Iterator<BlockInfo> getBlockIterator() {
-    return new BlockIterator(this.blockList, this);
+  Iterator<BlockInfo> getBlockIterator() {
+    return new BlockIterator(getStorageInfos());
+  }
+  Iterator<BlockInfo> getBlockIterator(final String storageID) {
+    return new BlockIterator(getStorageInfo(storageID));
   }
   }
 
 
   /**
   /**
    * Store block replication work.
    * Store block replication work.
    */
    */
-  void addBlockToBeReplicated(Block block, DatanodeDescriptor[] targets) {
+  void addBlockToBeReplicated(Block block, DatanodeStorageInfo[] targets) {
     assert(block != null && targets != null && targets.length > 0);
     assert(block != null && targets != null && targets.length > 0);
     replicateBlocks.offer(new BlockTargetPair(block, targets));
     replicateBlocks.offer(new BlockTargetPair(block, targets));
   }
   }
@@ -526,18 +497,14 @@ public class DatanodeDescriptor extends DatanodeInfo {
   public int getBlocksScheduled() {
   public int getBlocksScheduled() {
     return currApproxBlocksScheduled + prevApproxBlocksScheduled;
     return currApproxBlocksScheduled + prevApproxBlocksScheduled;
   }
   }
-  
-  /**
-   * Increments counter for number of blocks scheduled. 
-   */
-  public void incBlocksScheduled() {
+
+  /** Increment the number of blocks scheduled. */
+  void incrementBlocksScheduled() {
     currApproxBlocksScheduled++;
     currApproxBlocksScheduled++;
   }
   }
   
   
-  /**
-   * Decrements counter for number of blocks scheduled.
-   */
-  void decBlocksScheduled() {
+  /** Decrement the number of blocks scheduled. */
+  void decrementBlocksScheduled() {
     if (prevApproxBlocksScheduled > 0) {
     if (prevApproxBlocksScheduled > 0) {
       prevApproxBlocksScheduled--;
       prevApproxBlocksScheduled--;
     } else if (currApproxBlocksScheduled > 0) {
     } else if (currApproxBlocksScheduled > 0) {
@@ -546,12 +513,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
     // its ok if both counters are zero.
     // its ok if both counters are zero.
   }
   }
   
   
-  /**
-   * Adjusts curr and prev number of blocks scheduled every few minutes.
-   */
+  /** Adjusts curr and prev number of blocks scheduled every few minutes. */
   private void rollBlocksScheduled(long now) {
   private void rollBlocksScheduled(long now) {
-    if ((now - lastBlocksScheduledRollTime) > 
-        BLOCKS_SCHEDULED_ROLL_INTERVAL) {
+    if (now - lastBlocksScheduledRollTime > BLOCKS_SCHEDULED_ROLL_INTERVAL) {
       prevApproxBlocksScheduled = currApproxBlocksScheduled;
       prevApproxBlocksScheduled = currApproxBlocksScheduled;
       currApproxBlocksScheduled = 0;
       currApproxBlocksScheduled = 0;
       lastBlocksScheduledRollTime = now;
       lastBlocksScheduledRollTime = now;
@@ -647,7 +611,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
   @Override
   @Override
   public void updateRegInfo(DatanodeID nodeReg) {
   public void updateRegInfo(DatanodeID nodeReg) {
     super.updateRegInfo(nodeReg);
     super.updateRegInfo(nodeReg);
-    firstBlockReport = true; // must re-process IBR after re-registration
+    
+    // must re-process IBR after re-registration
+    for(DatanodeStorageInfo storage : getStorageInfos()) {
+      storage.setBlockReportCount(0);
+    }
   }
   }
 
 
   /**
   /**
@@ -664,26 +632,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     this.bandwidth = bandwidth;
     this.bandwidth = bandwidth;
   }
   }
 
 
-  public boolean areBlockContentsStale() {
-    return blockContentsStale;
-  }
-
-  public void markStaleAfterFailover() {
-    heartbeatedSinceFailover = false;
-    blockContentsStale = true;
-  }
-
-  public void receivedBlockReport() {
-    if (heartbeatedSinceFailover) {
-      blockContentsStale = false;
-    }
-    firstBlockReport = false;
-  }
-  
-  boolean isFirstBlockReport() {
-    return firstBlockReport;
-  }
-
   @Override
   @Override
   public String dumpDatanode() {
   public String dumpDatanode() {
     StringBuilder sb = new StringBuilder(super.dumpDatanode());
     StringBuilder sb = new StringBuilder(super.dumpDatanode());
@@ -702,6 +650,19 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return sb.toString();
     return sb.toString();
   }
   }
 
 
+  DatanodeStorageInfo updateStorage(DatanodeStorage s) {
+    synchronized (storageMap) {
+      DatanodeStorageInfo storage = storageMap.get(s.getStorageID());
+      if (storage == null) {
+        LOG.info("Adding new storage ID " + s.getStorageID() +
+                 " for DN " + getXferAddr());
+        storage = new DatanodeStorageInfo(this, s);
+        storageMap.put(s.getStorageID(), storage);
+      }
+      return storage;
+    }
+  }
+
   /**
   /**
    * @return   The time at which we last sent caching directives to this 
    * @return   The time at which we last sent caching directives to this 
    *           DataNode, in monotonic milliseconds.
    *           DataNode, in monotonic milliseconds.
@@ -718,3 +679,4 @@ public class DatanodeDescriptor extends DatanodeInfo {
     this.lastCachingDirectiveSentTimeMs = time;
     this.lastCachingDirectiveSentTimeMs = time;
   }
   }
 }
 }
+

+ 53 - 60
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -424,9 +424,13 @@ public class DatanodeManager {
   }
   }
 
 
 
 
-  /** Get a datanode descriptor given corresponding storageID */
-  DatanodeDescriptor getDatanode(final String storageID) {
-    return datanodeMap.get(storageID);
+  /** Get a datanode descriptor given corresponding DatanodeUUID */
+  DatanodeDescriptor getDatanode(final String datanodeUuid) {
+    if (datanodeUuid == null) {
+      return null;
+    }
+
+    return datanodeMap.get(datanodeUuid);
   }
   }
 
 
   /**
   /**
@@ -438,7 +442,7 @@ public class DatanodeManager {
    */
    */
   public DatanodeDescriptor getDatanode(DatanodeID nodeID
   public DatanodeDescriptor getDatanode(DatanodeID nodeID
       ) throws UnregisteredNodeException {
       ) throws UnregisteredNodeException {
-    final DatanodeDescriptor node = getDatanode(nodeID.getStorageID());
+    final DatanodeDescriptor node = getDatanode(nodeID.getDatanodeUuid());
     if (node == null) 
     if (node == null) 
       return null;
       return null;
     if (!node.getXferAddr().equals(nodeID.getXferAddr())) {
     if (!node.getXferAddr().equals(nodeID.getXferAddr())) {
@@ -451,6 +455,20 @@ public class DatanodeManager {
     return node;
     return node;
   }
   }
 
 
+  public DatanodeStorageInfo[] getDatanodeStorageInfos(
+      DatanodeID[] datanodeID, String[] storageIDs)
+          throws UnregisteredNodeException {
+    if (datanodeID.length == 0) {
+      return null;
+    }
+    final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[datanodeID.length];
+    for(int i = 0; i < datanodeID.length; i++) {
+      final DatanodeDescriptor dd = getDatanode(datanodeID[i]);
+      storages[i] = dd.getStorageInfo(storageIDs[i]);
+    }
+    return storages; 
+  }
+
   /** Prints information about all datanodes. */
   /** Prints information about all datanodes. */
   void datanodeDump(final PrintWriter out) {
   void datanodeDump(final PrintWriter out) {
     synchronized (datanodeMap) {
     synchronized (datanodeMap) {
@@ -528,7 +546,7 @@ public class DatanodeManager {
     // remove  from host2DatanodeMap the datanodeDescriptor removed
     // remove  from host2DatanodeMap the datanodeDescriptor removed
     // from datanodeMap before adding node to host2DatanodeMap.
     // from datanodeMap before adding node to host2DatanodeMap.
     synchronized(datanodeMap) {
     synchronized(datanodeMap) {
-      host2DatanodeMap.remove(datanodeMap.put(node.getStorageID(), node));
+      host2DatanodeMap.remove(datanodeMap.put(node.getDatanodeUuid(), node));
     }
     }
 
 
     networktopology.add(node); // may throw InvalidTopologyException
     networktopology.add(node); // may throw InvalidTopologyException
@@ -543,7 +561,7 @@ public class DatanodeManager {
 
 
   /** Physically remove node from datanodeMap. */
   /** Physically remove node from datanodeMap. */
   private void wipeDatanode(final DatanodeID node) {
   private void wipeDatanode(final DatanodeID node) {
-    final String key = node.getStorageID();
+    final String key = node.getDatanodeUuid();
     synchronized (datanodeMap) {
     synchronized (datanodeMap) {
       host2DatanodeMap.remove(datanodeMap.remove(key));
       host2DatanodeMap.remove(datanodeMap.remove(key));
     }
     }
@@ -705,8 +723,10 @@ public class DatanodeManager {
   /** Start decommissioning the specified datanode. */
   /** Start decommissioning the specified datanode. */
   private void startDecommission(DatanodeDescriptor node) {
   private void startDecommission(DatanodeDescriptor node) {
     if (!node.isDecommissionInProgress() && !node.isDecommissioned()) {
     if (!node.isDecommissionInProgress() && !node.isDecommissioned()) {
-      LOG.info("Start Decommissioning " + node + " with " + 
-          node.numBlocks() +  " blocks");
+      for (DatanodeStorageInfo storage : node.getStorageInfos()) {
+        LOG.info("Start Decommissioning " + node + " " + storage
+            + " with " + storage.numBlocks() + " blocks");
+      }
       heartbeatManager.startDecommission(node);
       heartbeatManager.startDecommission(node);
       node.decommissioningStatus.setStartTime(now());
       node.decommissioningStatus.setStartTime(now());
       
       
@@ -728,24 +748,6 @@ public class DatanodeManager {
     }
     }
   }
   }
 
 
-  /**
-   * Generate new storage ID.
-   * 
-   * @return unique storage ID
-   * 
-   * Note: that collisions are still possible if somebody will try 
-   * to bring in a data storage from a different cluster.
-   */
-  private String newStorageID() {
-    String newID = null;
-    while(newID == null) {
-      newID = "DS" + Integer.toString(DFSUtil.getRandom().nextInt());
-      if (datanodeMap.get(newID) != null)
-        newID = null;
-    }
-    return newID;
-  }
-
   /**
   /**
    * Register the given datanode with the namenode. NB: the given
    * Register the given datanode with the namenode. NB: the given
    * registration is mutated and given back to the datanode.
    * registration is mutated and given back to the datanode.
@@ -784,9 +786,9 @@ public class DatanodeManager {
       }
       }
         
         
       NameNode.stateChangeLog.info("BLOCK* registerDatanode: from "
       NameNode.stateChangeLog.info("BLOCK* registerDatanode: from "
-          + nodeReg + " storage " + nodeReg.getStorageID());
+          + nodeReg + " storage " + nodeReg.getDatanodeUuid());
   
   
-      DatanodeDescriptor nodeS = datanodeMap.get(nodeReg.getStorageID());
+      DatanodeDescriptor nodeS = getDatanode(nodeReg.getDatanodeUuid());
       DatanodeDescriptor nodeN = host2DatanodeMap.getDatanodeByXferAddr(
       DatanodeDescriptor nodeN = host2DatanodeMap.getDatanodeByXferAddr(
           nodeReg.getIpAddr(), nodeReg.getXferPort());
           nodeReg.getIpAddr(), nodeReg.getXferPort());
         
         
@@ -821,7 +823,7 @@ public class DatanodeManager {
          */        
          */        
           NameNode.stateChangeLog.info("BLOCK* registerDatanode: " + nodeS
           NameNode.stateChangeLog.info("BLOCK* registerDatanode: " + nodeS
               + " is replaced by " + nodeReg + " with the same storageID "
               + " is replaced by " + nodeReg + " with the same storageID "
-              + nodeReg.getStorageID());
+              + nodeReg.getDatanodeUuid());
         }
         }
         
         
         boolean success = false;
         boolean success = false;
@@ -853,20 +855,8 @@ public class DatanodeManager {
           }
           }
         }
         }
         return;
         return;
-      } 
-  
-      // this is a new datanode serving a new data storage
-      if ("".equals(nodeReg.getStorageID())) {
-        // this data storage has never been registered
-        // it is either empty or was created by pre-storageID version of DFS
-        nodeReg.setStorageID(newStorageID());
-        if (NameNode.stateChangeLog.isDebugEnabled()) {
-          NameNode.stateChangeLog.debug(
-              "BLOCK* NameSystem.registerDatanode: "
-              + "new storageID " + nodeReg.getStorageID() + " assigned.");
-        }
       }
       }
-      
+
       DatanodeDescriptor nodeDescr 
       DatanodeDescriptor nodeDescr 
         = new DatanodeDescriptor(nodeReg, NetworkTopology.DEFAULT_RACK);
         = new DatanodeDescriptor(nodeReg, NetworkTopology.DEFAULT_RACK);
       boolean success = false;
       boolean success = false;
@@ -1234,10 +1224,10 @@ public class DatanodeManager {
 
 
   /** Handle heartbeat from datanodes. */
   /** Handle heartbeat from datanodes. */
   public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
   public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
-      final String blockPoolId,
-      long capacity, long dfsUsed, long remaining, long blockPoolUsed,
-      long cacheCapacity, long cacheUsed, int xceiverCount, int maxTransfers,
-      int failedVolumes) throws IOException {
+      StorageReport[] reports, final String blockPoolId,
+      long cacheCapacity, long cacheUsed, int xceiverCount, 
+      int maxTransfers, int failedVolumes
+      ) throws IOException {
     synchronized (heartbeatManager) {
     synchronized (heartbeatManager) {
       synchronized (datanodeMap) {
       synchronized (datanodeMap) {
         DatanodeDescriptor nodeinfo = null;
         DatanodeDescriptor nodeinfo = null;
@@ -1257,9 +1247,9 @@ public class DatanodeManager {
           return new DatanodeCommand[]{RegisterCommand.REGISTER};
           return new DatanodeCommand[]{RegisterCommand.REGISTER};
         }
         }
 
 
-        heartbeatManager.updateHeartbeat(nodeinfo, capacity, dfsUsed,
-            remaining, blockPoolUsed, cacheCapacity, cacheUsed, xceiverCount,
-            failedVolumes);
+        heartbeatManager.updateHeartbeat(nodeinfo, reports,
+                                         cacheCapacity, cacheUsed,
+                                         xceiverCount, failedVolumes);
 
 
         // If we are in safemode, do not send back any recovery / replication
         // If we are in safemode, do not send back any recovery / replication
         // requests. Don't even drain the existing queue of work.
         // requests. Don't even drain the existing queue of work.
@@ -1274,32 +1264,32 @@ public class DatanodeManager {
           BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
           BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
               blocks.length);
               blocks.length);
           for (BlockInfoUnderConstruction b : blocks) {
           for (BlockInfoUnderConstruction b : blocks) {
-            DatanodeDescriptor[] expectedLocations = b.getExpectedLocations();
+            final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
             // Skip stale nodes during recovery - not heart beated for some time (30s by default).
             // Skip stale nodes during recovery - not heart beated for some time (30s by default).
-            List<DatanodeDescriptor> recoveryLocations =
-                new ArrayList<DatanodeDescriptor>(expectedLocations.length);
-            for (int i = 0; i < expectedLocations.length; i++) {
-              if (!expectedLocations[i].isStale(this.staleInterval)) {
-                recoveryLocations.add(expectedLocations[i]);
+            final List<DatanodeStorageInfo> recoveryLocations =
+                new ArrayList<DatanodeStorageInfo>(storages.length);
+            for (int i = 0; i < storages.length; i++) {
+              if (!storages[i].getDatanodeDescriptor().isStale(staleInterval)) {
+                recoveryLocations.add(storages[i]);
               }
               }
             }
             }
             // If we only get 1 replica after eliminating stale nodes, then choose all
             // If we only get 1 replica after eliminating stale nodes, then choose all
             // replicas for recovery and let the primary data node handle failures.
             // replicas for recovery and let the primary data node handle failures.
             if (recoveryLocations.size() > 1) {
             if (recoveryLocations.size() > 1) {
-              if (recoveryLocations.size() != expectedLocations.length) {
+              if (recoveryLocations.size() != storages.length) {
                 LOG.info("Skipped stale nodes for recovery : " +
                 LOG.info("Skipped stale nodes for recovery : " +
-                    (expectedLocations.length - recoveryLocations.size()));
+                    (storages.length - recoveryLocations.size()));
               }
               }
               brCommand.add(new RecoveringBlock(
               brCommand.add(new RecoveringBlock(
                   new ExtendedBlock(blockPoolId, b),
                   new ExtendedBlock(blockPoolId, b),
-                  recoveryLocations.toArray(new DatanodeDescriptor[recoveryLocations.size()]),
+                  DatanodeStorageInfo.toDatanodeInfos(recoveryLocations),
                   b.getBlockRecoveryId()));
                   b.getBlockRecoveryId()));
             } else {
             } else {
               // If too many replicas are stale, then choose all replicas to participate
               // If too many replicas are stale, then choose all replicas to participate
               // in block recovery.
               // in block recovery.
               brCommand.add(new RecoveringBlock(
               brCommand.add(new RecoveringBlock(
                   new ExtendedBlock(blockPoolId, b),
                   new ExtendedBlock(blockPoolId, b),
-                  expectedLocations,
+                  DatanodeStorageInfo.toDatanodeInfos(storages),
                   b.getBlockRecoveryId()));
                   b.getBlockRecoveryId()));
             }
             }
           }
           }
@@ -1416,7 +1406,9 @@ public class DatanodeManager {
     LOG.info("Marking all datandoes as stale");
     LOG.info("Marking all datandoes as stale");
     synchronized (datanodeMap) {
     synchronized (datanodeMap) {
       for (DatanodeDescriptor dn : datanodeMap.values()) {
       for (DatanodeDescriptor dn : datanodeMap.values()) {
-        dn.markStaleAfterFailover();
+        for(DatanodeStorageInfo storage : dn.getStorageInfos()) {
+          storage.markStaleAfterFailover();
+        }
       }
       }
     }
     }
   }
   }
@@ -1455,3 +1447,4 @@ public class DatanodeManager {
     this.shouldSendCachingCommands = shouldSendCachingCommands;
     this.shouldSendCachingCommands = shouldSendCachingCommands;
   }
   }
 }
 }
+

+ 288 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java

@@ -0,0 +1,288 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+
+/**
+ * A Datanode has one or more storages. A storage in the Datanode is represented
+ * by this class.
+ */
+public class DatanodeStorageInfo {
+  public static final DatanodeStorageInfo[] EMPTY_ARRAY = {};
+
+  public static DatanodeInfo[] toDatanodeInfos(DatanodeStorageInfo[] storages) {
+    return toDatanodeInfos(Arrays.asList(storages));
+  }
+  static DatanodeInfo[] toDatanodeInfos(List<DatanodeStorageInfo> storages) {
+    final DatanodeInfo[] datanodes = new DatanodeInfo[storages.size()];
+    for(int i = 0; i < storages.size(); i++) {
+      datanodes[i] = storages.get(i).getDatanodeDescriptor();
+    }
+    return datanodes;
+  }
+
+  static DatanodeDescriptor[] toDatanodeDescriptors(
+      DatanodeStorageInfo[] storages) {
+    DatanodeDescriptor[] datanodes = new DatanodeDescriptor[storages.length];
+    for (int i = 0; i < storages.length; ++i) {
+      datanodes[i] = storages[i].getDatanodeDescriptor();
+    }
+    return datanodes;
+  }
+
+  public static String[] toStorageIDs(DatanodeStorageInfo[] storages) {
+    String[] storageIDs = new String[storages.length];
+    for(int i = 0; i < storageIDs.length; i++) {
+      storageIDs[i] = storages[i].getStorageID();
+    }
+    return storageIDs;
+  }
+
+  public static StorageType[] toStorageTypes(DatanodeStorageInfo[] storages) {
+    StorageType[] storageTypes = new StorageType[storages.length];
+    for(int i = 0; i < storageTypes.length; i++) {
+      storageTypes[i] = storages[i].getStorageType();
+    }
+    return storageTypes;
+  }
+
+  /**
+   * Iterates over the list of blocks belonging to the data-node.
+   */
+  class BlockIterator implements Iterator<BlockInfo> {
+    private BlockInfo current;
+
+    BlockIterator(BlockInfo head) {
+      this.current = head;
+    }
+
+    public boolean hasNext() {
+      return current != null;
+    }
+
+    public BlockInfo next() {
+      BlockInfo res = current;
+      current = current.getNext(current.findStorageInfo(DatanodeStorageInfo.this));
+      return res;
+    }
+
+    public void remove() {
+      throw new UnsupportedOperationException("Sorry. can't remove.");
+    }
+  }
+
+  private final DatanodeDescriptor dn;
+  private final String storageID;
+  private final StorageType storageType;
+  private final State state;
+
+  private long capacity;
+  private long dfsUsed;
+  private long remaining;
+  private long blockPoolUsed;
+
+  private volatile BlockInfo blockList = null;
+  private int numBlocks = 0;
+
+  /** The number of block reports received */
+  private int blockReportCount = 0;
+
+  /**
+   * Set to false on any NN failover, and reset to true
+   * whenever a block report is received.
+   */
+  private boolean heartbeatedSinceFailover = false;
+
+  /**
+   * At startup or at failover, the storages in the cluster may have pending
+   * block deletions from a previous incarnation of the NameNode. The block
+   * contents are considered as stale until a block report is received. When a
+   * storage is considered as stale, the replicas on it are also considered as
+   * stale. If any block has at least one stale replica, then no invalidations
+   * will be processed for this block. See HDFS-1972.
+   */
+  private boolean blockContentsStale = true;
+
+  DatanodeStorageInfo(DatanodeDescriptor dn, DatanodeStorage s) {
+    this.dn = dn;
+    this.storageID = s.getStorageID();
+    this.storageType = s.getStorageType();
+    this.state = s.getState();
+  }
+
+  int getBlockReportCount() {
+    return blockReportCount;
+  }
+
+  void setBlockReportCount(int blockReportCount) {
+    this.blockReportCount = blockReportCount;
+  }
+
+  boolean areBlockContentsStale() {
+    return blockContentsStale;
+  }
+
+  void markStaleAfterFailover() {
+    heartbeatedSinceFailover = false;
+    blockContentsStale = true;
+  }
+
+  void receivedHeartbeat(StorageReport report) {
+    updateState(report);
+    heartbeatedSinceFailover = true;
+  }
+
+  void receivedBlockReport() {
+    if (heartbeatedSinceFailover) {
+      blockContentsStale = false;
+    }
+    blockReportCount++;
+  }
+
+  @VisibleForTesting
+  public void setUtilizationForTesting(long capacity, long dfsUsed,
+                      long remaining, long blockPoolUsed) {
+    this.capacity = capacity;
+    this.dfsUsed = dfsUsed;
+    this.remaining = remaining;
+    this.blockPoolUsed = blockPoolUsed;
+  }
+  
+  State getState() {
+    return this.state;
+  }
+  
+  String getStorageID() {
+    return storageID;
+  }
+
+  StorageType getStorageType() {
+    return storageType;
+  }
+
+  long getCapacity() {
+    return capacity;
+  }
+
+  long getDfsUsed() {
+    return dfsUsed;
+  }
+
+  long getRemaining() {
+    return remaining;
+  }
+
+  long getBlockPoolUsed() {
+    return blockPoolUsed;
+  }
+
+  boolean addBlock(BlockInfo b) {
+    if(!b.addStorage(this))
+      return false;
+    // add to the head of the data-node list
+    blockList = b.listInsert(blockList, this);
+    numBlocks++;
+    return true;
+  }
+
+  boolean removeBlock(BlockInfo b) {
+    blockList = b.listRemove(blockList, this);
+    if (b.removeStorage(this)) {
+      numBlocks--;
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  int numBlocks() {
+    return numBlocks;
+  }
+  
+  Iterator<BlockInfo> getBlockIterator() {
+    return new BlockIterator(blockList);
+
+  }
+
+  /**
+   * Move block to the head of the list of blocks belonging to the data-node.
+   * @return the index of the head of the blockList
+   */
+  int moveBlockToHead(BlockInfo b, int curIndex, int headIndex) {
+    blockList = b.moveBlockToHead(blockList, this, curIndex, headIndex);
+    return curIndex;
+  }
+
+  /**
+   * Used for testing only
+   * @return the head of the blockList
+   */
+  @VisibleForTesting
+  BlockInfo getBlockListHeadForTesting(){
+    return blockList;
+  }
+
+  void updateState(StorageReport r) {
+    capacity = r.getCapacity();
+    dfsUsed = r.getDfsUsed();
+    remaining = r.getRemaining();
+    blockPoolUsed = r.getBlockPoolUsed();
+  }
+
+  public DatanodeDescriptor getDatanodeDescriptor() {
+    return dn;
+  }
+
+  /** Increment the number of blocks scheduled for each given storage */ 
+  public static void incrementBlocksScheduled(DatanodeStorageInfo... storages) {
+    for (DatanodeStorageInfo s : storages) {
+      s.getDatanodeDescriptor().incrementBlocksScheduled();
+    }
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    } else if (obj == null || !(obj instanceof DatanodeStorageInfo)) {
+      return false;
+    }
+    final DatanodeStorageInfo that = (DatanodeStorageInfo)obj;
+    return this.storageID.equals(that.storageID);
+  }
+
+  @Override
+  public int hashCode() {
+    return storageID.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "[" + storageType + "]" + storageID + ":" + state;
+  }
+}

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

@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 
 
@@ -181,7 +182,7 @@ class HeartbeatManager implements DatanodeStatistics {
       addDatanode(d);
       addDatanode(d);
 
 
       //update its timestamp
       //update its timestamp
-      d.updateHeartbeat(0L, 0L, 0L, 0L, 0L, 0L, 0, 0);
+      d.updateHeartbeat(StorageReport.EMPTY_ARRAY, 0L, 0L, 0, 0);
     }
     }
   }
   }
 
 
@@ -203,11 +204,11 @@ class HeartbeatManager implements DatanodeStatistics {
   }
   }
 
 
   synchronized void updateHeartbeat(final DatanodeDescriptor node,
   synchronized void updateHeartbeat(final DatanodeDescriptor node,
-      long capacity, long dfsUsed, long remaining, long blockPoolUsed,
-      long cacheCapacity, long cacheUsed, int xceiverCount, int failedVolumes) {
+      StorageReport[] reports, long cacheCapacity, long cacheUsed,
+      int xceiverCount, int failedVolumes) {
     stats.subtract(node);
     stats.subtract(node);
-    node.updateHeartbeat(capacity, dfsUsed, remaining, blockPoolUsed,
-        cacheCapacity, cacheUsed, xceiverCount, failedVolumes);
+    node.updateHeartbeat(reports, cacheCapacity, cacheUsed,
+      xceiverCount, failedVolumes);
     stats.add(node);
     stats.add(node);
   }
   }
 
 
@@ -358,3 +359,4 @@ class HeartbeatManager implements DatanodeStatistics {
     }
     }
   }
   }
 }
 }
+

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

@@ -78,10 +78,10 @@ class InvalidateBlocks {
    */
    */
   synchronized void add(final Block block, final DatanodeInfo datanode,
   synchronized void add(final Block block, final DatanodeInfo datanode,
       final boolean log) {
       final boolean log) {
-    LightWeightHashSet<Block> set = node2blocks.get(datanode.getStorageID());
+    LightWeightHashSet<Block> set = node2blocks.get(datanode.getDatanodeUuid());
     if (set == null) {
     if (set == null) {
       set = new LightWeightHashSet<Block>();
       set = new LightWeightHashSet<Block>();
-      node2blocks.put(datanode.getStorageID(), set);
+      node2blocks.put(datanode.getDatanodeUuid(), set);
     }
     }
     if (set.add(block)) {
     if (set.add(block)) {
       numBlocks++;
       numBlocks++;

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

@@ -34,5 +34,5 @@ public interface MutableBlockCollection extends BlockCollection {
    * and set the locations.
    * and set the locations.
    */
    */
   public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
   public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
-      DatanodeDescriptor[] locations) throws IOException;
+      DatanodeStorageInfo[] storages) throws IOException;
 }
 }

+ 9 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java

@@ -42,11 +42,13 @@ class PendingDataNodeMessages {
   static class ReportedBlockInfo {
   static class ReportedBlockInfo {
     private final Block block;
     private final Block block;
     private final DatanodeDescriptor dn;
     private final DatanodeDescriptor dn;
+    private final String storageID;
     private final ReplicaState reportedState;
     private final ReplicaState reportedState;
 
 
-    ReportedBlockInfo(DatanodeDescriptor dn, Block block,
+    ReportedBlockInfo(DatanodeDescriptor dn, String storageID, Block block,
         ReplicaState reportedState) {
         ReplicaState reportedState) {
       this.dn = dn;
       this.dn = dn;
+      this.storageID = storageID;
       this.block = block;
       this.block = block;
       this.reportedState = reportedState;
       this.reportedState = reportedState;
     }
     }
@@ -58,6 +60,10 @@ class PendingDataNodeMessages {
     DatanodeDescriptor getNode() {
     DatanodeDescriptor getNode() {
       return dn;
       return dn;
     }
     }
+    
+    String getStorageID() {
+      return storageID;
+    }
 
 
     ReplicaState getReportedState() {
     ReplicaState getReportedState() {
       return reportedState;
       return reportedState;
@@ -70,11 +76,11 @@ class PendingDataNodeMessages {
     }
     }
   }
   }
   
   
-  void enqueueReportedBlock(DatanodeDescriptor dn, Block block,
+  void enqueueReportedBlock(DatanodeDescriptor dn, String storageID, Block block,
       ReplicaState reportedState) {
       ReplicaState reportedState) {
     block = new Block(block);
     block = new Block(block);
     getBlockQueue(block).add(
     getBlockQueue(block).add(
-        new ReportedBlockInfo(dn, block, reportedState));
+        new ReportedBlockInfo(dn, storageID, block, reportedState));
     count++;
     count++;
   }
   }
   
   

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -236,6 +236,8 @@ public abstract class Storage extends StorageInfo {
     final boolean useLock;        // flag to enable storage lock
     final boolean useLock;        // flag to enable storage lock
     final StorageDirType dirType; // storage dir type
     final StorageDirType dirType; // storage dir type
     FileLock lock;                // storage lock
     FileLock lock;                // storage lock
+
+    private String storageUuid = null;      // Storage directory identifier.
     
     
     public StorageDirectory(File dir) {
     public StorageDirectory(File dir) {
       // default dirType is null
       // default dirType is null
@@ -246,6 +248,14 @@ public abstract class Storage extends StorageInfo {
       this(dir, dirType, true);
       this(dir, dirType, true);
     }
     }
     
     
+    public void setStorageUuid(String storageUuid) {
+      this.storageUuid = storageUuid;
+    }
+
+    public String getStorageUuid() {
+      return storageUuid;
+    }
+
     /**
     /**
      * Constructor
      * Constructor
      * @param dir directory corresponding to the storage
      * @param dir directory corresponding to the storage

+ 34 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java

@@ -27,6 +27,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -160,31 +161,32 @@ class BPOfferService {
   synchronized NamespaceInfo getNamespaceInfo() {
   synchronized NamespaceInfo getNamespaceInfo() {
     return bpNSInfo;
     return bpNSInfo;
   }
   }
-  
+
   @Override
   @Override
   public String toString() {
   public String toString() {
     if (bpNSInfo == null) {
     if (bpNSInfo == null) {
       // If we haven't yet connected to our NN, we don't yet know our
       // If we haven't yet connected to our NN, we don't yet know our
       // own block pool ID.
       // own block pool ID.
       // If _none_ of the block pools have connected yet, we don't even
       // If _none_ of the block pools have connected yet, we don't even
-      // know the storage ID of this DN.
-      String storageId = dn.getStorageId();
-      if (storageId == null || "".equals(storageId)) {
-        storageId = "unknown";
+      // know the DatanodeID ID of this DN.
+      String datanodeUuid = dn.getDatanodeUuid();
+
+      if (datanodeUuid == null || datanodeUuid.isEmpty()) {
+        datanodeUuid = "unassigned";
       }
       }
-      return "Block pool <registering> (storage id " + storageId +
-        ")";
+      return "Block pool <registering> (Datanode Uuid " + datanodeUuid + ")";
     } else {
     } else {
       return "Block pool " + getBlockPoolId() +
       return "Block pool " + getBlockPoolId() +
-        " (storage id " + dn.getStorageId() +
-        ")";
+          " (Datanode Uuid " + dn.getDatanodeUuid() +
+          ")";
     }
     }
   }
   }
   
   
-  void reportBadBlocks(ExtendedBlock block) {
+  void reportBadBlocks(ExtendedBlock block,
+                       String storageUuid, StorageType storageType) {
     checkBlock(block);
     checkBlock(block);
     for (BPServiceActor actor : bpServices) {
     for (BPServiceActor actor : bpServices) {
-      actor.reportBadBlocks(block);
+      actor.reportBadBlocks(block, storageUuid, storageType);
     }
     }
   }
   }
   
   
@@ -193,7 +195,8 @@ class BPOfferService {
    * till namenode is informed before responding with success to the
    * till namenode is informed before responding with success to the
    * client? For now we don't.
    * client? For now we don't.
    */
    */
-  void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) {
+  void notifyNamenodeReceivedBlock(
+      ExtendedBlock block, String delHint, String storageUuid) {
     checkBlock(block);
     checkBlock(block);
     checkDelHint(delHint);
     checkDelHint(delHint);
     ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
     ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
@@ -202,7 +205,7 @@ class BPOfferService {
         delHint);
         delHint);
 
 
     for (BPServiceActor actor : bpServices) {
     for (BPServiceActor actor : bpServices) {
-      actor.notifyNamenodeBlockImmediately(bInfo);
+      actor.notifyNamenodeBlockImmediately(bInfo, storageUuid);
     }
     }
   }
   }
 
 
@@ -219,23 +222,23 @@ class BPOfferService {
         "delHint is null");
         "delHint is null");
   }
   }
 
 
-  void notifyNamenodeDeletedBlock(ExtendedBlock block) {
+  void notifyNamenodeDeletedBlock(ExtendedBlock block, String storageUuid) {
     checkBlock(block);
     checkBlock(block);
     ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
     ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
        block.getLocalBlock(), BlockStatus.DELETED_BLOCK, null);
        block.getLocalBlock(), BlockStatus.DELETED_BLOCK, null);
     
     
     for (BPServiceActor actor : bpServices) {
     for (BPServiceActor actor : bpServices) {
-      actor.notifyNamenodeDeletedBlock(bInfo);
+      actor.notifyNamenodeDeletedBlock(bInfo, storageUuid);
     }
     }
   }
   }
   
   
-  void notifyNamenodeReceivingBlock(ExtendedBlock block) {
+  void notifyNamenodeReceivingBlock(ExtendedBlock block, String storageUuid) {
     checkBlock(block);
     checkBlock(block);
     ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
     ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
        block.getLocalBlock(), BlockStatus.RECEIVING_BLOCK, null);
        block.getLocalBlock(), BlockStatus.RECEIVING_BLOCK, null);
     
     
     for (BPServiceActor actor : bpServices) {
     for (BPServiceActor actor : bpServices) {
-      actor.notifyNamenodeBlockImmediately(bInfo);
+      actor.notifyNamenodeBlockImmediately(bInfo, storageUuid);
     }
     }
   }
   }
 
 
@@ -274,12 +277,22 @@ class BPOfferService {
   synchronized void verifyAndSetNamespaceInfo(NamespaceInfo nsInfo) throws IOException {
   synchronized void verifyAndSetNamespaceInfo(NamespaceInfo nsInfo) throws IOException {
     if (this.bpNSInfo == null) {
     if (this.bpNSInfo == null) {
       this.bpNSInfo = nsInfo;
       this.bpNSInfo = nsInfo;
-      
+      boolean success = false;
+
       // Now that we know the namespace ID, etc, we can pass this to the DN.
       // Now that we know the namespace ID, etc, we can pass this to the DN.
       // The DN can now initialize its local storage if we are the
       // The DN can now initialize its local storage if we are the
       // first BP to handshake, etc.
       // first BP to handshake, etc.
-      dn.initBlockPool(this);
-      return;
+      try {
+        dn.initBlockPool(this);
+        success = true;
+      } finally {
+        if (!success) {
+          // The datanode failed to initialize the BP. We need to reset
+          // the namespace info so that other BPService actors still have
+          // a chance to set it, and re-initialize the datanode.
+          this.bpNSInfo = null;
+        }
+      }
     } else {
     } else {
       checkNSEquality(bpNSInfo.getBlockPoolID(), nsInfo.getBlockPoolID(),
       checkNSEquality(bpNSInfo.getBlockPoolID(), nsInfo.getBlockPoolID(),
           "Blockpool ID");
           "Blockpool ID");
@@ -328,7 +341,7 @@ class BPOfferService {
     }
     }
   }
   }
 
 
-  synchronized DatanodeRegistration createRegistration() {
+  synchronized DatanodeRegistration createRegistration() throws IOException {
     Preconditions.checkState(bpNSInfo != null,
     Preconditions.checkState(bpNSInfo != null,
         "getRegistration() can only be called after initial handshake");
         "getRegistration() can only be called after initial handshake");
     return dn.createBPRegistration(bpNSInfo);
     return dn.createBPRegistration(bpNSInfo);

+ 204 - 74
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -22,7 +22,6 @@ import static org.apache.hadoop.util.Time.now;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.SocketTimeoutException;
 import java.net.SocketTimeoutException;
-import java.net.URI;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
@@ -31,6 +30,7 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionUtil;
 import org.apache.hadoop.util.VersionUtil;
@@ -100,9 +99,9 @@ class BPServiceActor implements Runnable {
    * keyed by block ID, contains the pending changes which have yet to be
    * keyed by block ID, contains the pending changes which have yet to be
    * reported to the NN. Access should be synchronized on this object.
    * reported to the NN. Access should be synchronized on this object.
    */
    */
-  private final Map<Long, ReceivedDeletedBlockInfo> pendingIncrementalBR 
-    = Maps.newHashMap();
-  
+  private final Map<String, PerStoragePendingIncrementalBR>
+      pendingIncrementalBRperStorage = Maps.newHashMap();
+
   private volatile int pendingReceivedRequests = 0;
   private volatile int pendingReceivedRequests = 0;
   private volatile boolean shouldServiceRun = true;
   private volatile boolean shouldServiceRun = true;
   private final DataNode dn;
   private final DataNode dn;
@@ -244,12 +243,15 @@ class BPServiceActor implements Runnable {
     resetBlockReportTime = true; // reset future BRs for randomness
     resetBlockReportTime = true; // reset future BRs for randomness
   }
   }
 
 
-  void reportBadBlocks(ExtendedBlock block) {
+  void reportBadBlocks(ExtendedBlock block,
+      String storageUuid, StorageType storageType) {
     if (bpRegistration == null) {
     if (bpRegistration == null) {
       return;
       return;
     }
     }
     DatanodeInfo[] dnArr = { new DatanodeInfo(bpRegistration) };
     DatanodeInfo[] dnArr = { new DatanodeInfo(bpRegistration) };
-    LocatedBlock[] blocks = { new LocatedBlock(block, dnArr) }; 
+    String[] uuids = { storageUuid };
+    StorageType[] types = { storageType };
+    LocatedBlock[] blocks = { new LocatedBlock(block, dnArr, uuids, types) };
     
     
     try {
     try {
       bpNamenode.reportBadBlocks(blocks);  
       bpNamenode.reportBadBlocks(blocks);  
@@ -263,69 +265,120 @@ class BPServiceActor implements Runnable {
   }
   }
   
   
   /**
   /**
-   * Report received blocks and delete hints to the Namenode
-   * 
+   * Report received blocks and delete hints to the Namenode for each
+   * storage.
+   *
    * @throws IOException
    * @throws IOException
    */
    */
   private void reportReceivedDeletedBlocks() throws IOException {
   private void reportReceivedDeletedBlocks() throws IOException {
 
 
-    // check if there are newly received blocks
-    ReceivedDeletedBlockInfo[] receivedAndDeletedBlockArray = null;
-    synchronized (pendingIncrementalBR) {
-      int numBlocks = pendingIncrementalBR.size();
-      if (numBlocks > 0) {
-        //
-        // Send newly-received and deleted blockids to namenode
-        //
-        receivedAndDeletedBlockArray = pendingIncrementalBR
-            .values().toArray(new ReceivedDeletedBlockInfo[numBlocks]);
+    // Generate a list of the pending reports for each storage under the lock
+    Map<String, ReceivedDeletedBlockInfo[]> blockArrays = Maps.newHashMap();
+    synchronized (pendingIncrementalBRperStorage) {
+      for (Map.Entry<String, PerStoragePendingIncrementalBR> entry :
+           pendingIncrementalBRperStorage.entrySet()) {
+        final String storageUuid = entry.getKey();
+        final PerStoragePendingIncrementalBR perStorageMap = entry.getValue();
+
+        if (perStorageMap.getBlockInfoCount() > 0) {
+          // Send newly-received and deleted blockids to namenode
+          ReceivedDeletedBlockInfo[] rdbi = perStorageMap.dequeueBlockInfos();
+          pendingReceivedRequests =
+              (pendingReceivedRequests > rdbi.length ?
+                  (pendingReceivedRequests - rdbi.length) : 0);
+          blockArrays.put(storageUuid, rdbi);
+        }
       }
       }
-      pendingIncrementalBR.clear();
     }
     }
-    if (receivedAndDeletedBlockArray != null) {
+
+    // Send incremental block reports to the Namenode outside the lock
+    for (Map.Entry<String, ReceivedDeletedBlockInfo[]> entry :
+         blockArrays.entrySet()) {
+      final String storageUuid = entry.getKey();
+      final ReceivedDeletedBlockInfo[] rdbi = entry.getValue();
+
       StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
       StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
-          bpRegistration.getStorageID(), receivedAndDeletedBlockArray) };
+          storageUuid, rdbi) };
       boolean success = false;
       boolean success = false;
       try {
       try {
-        bpNamenode.blockReceivedAndDeleted(bpRegistration, bpos.getBlockPoolId(),
-            report);
+        bpNamenode.blockReceivedAndDeleted(bpRegistration,
+            bpos.getBlockPoolId(), report);
         success = true;
         success = true;
       } finally {
       } finally {
-        synchronized (pendingIncrementalBR) {
-          if (!success) {
+        if (!success) {
+          synchronized (pendingIncrementalBRperStorage) {
             // If we didn't succeed in sending the report, put all of the
             // If we didn't succeed in sending the report, put all of the
-            // blocks back onto our queue, but only in the case where we didn't
-            // put something newer in the meantime.
-            for (ReceivedDeletedBlockInfo rdbi : receivedAndDeletedBlockArray) {
-              if (!pendingIncrementalBR.containsKey(rdbi.getBlock().getBlockId())) {
-                pendingIncrementalBR.put(rdbi.getBlock().getBlockId(), rdbi);
-              }
-            }
+            // blocks back onto our queue, but only in the case where we
+            // didn't put something newer in the meantime.
+            PerStoragePendingIncrementalBR perStorageMap =
+                pendingIncrementalBRperStorage.get(storageUuid);
+            pendingReceivedRequests += perStorageMap.putMissingBlockInfos(rdbi);
           }
           }
-          pendingReceivedRequests = pendingIncrementalBR.size();
         }
         }
       }
       }
     }
     }
   }
   }
 
 
+  /**
+   * Retrieve the incremental BR state for a given storage UUID
+   * @param storageUuid
+   * @return
+   */
+  private PerStoragePendingIncrementalBR getIncrementalBRMapForStorage(
+      String storageUuid) {
+    PerStoragePendingIncrementalBR mapForStorage =
+        pendingIncrementalBRperStorage.get(storageUuid);
+
+    if (mapForStorage == null) {
+      // This is the first time we are adding incremental BR state for
+      // this storage so create a new map. This is required once per
+      // storage, per service actor.
+      mapForStorage = new PerStoragePendingIncrementalBR();
+      pendingIncrementalBRperStorage.put(storageUuid, mapForStorage);
+    }
+
+    return mapForStorage;
+  }
+
+  /**
+   * Add a blockInfo for notification to NameNode. If another entry
+   * exists for the same block it is removed.
+   *
+   * Caller must synchronize access using pendingIncrementalBRperStorage.
+   * @param bInfo
+   * @param storageUuid
+   */
+  void addPendingReplicationBlockInfo(ReceivedDeletedBlockInfo bInfo,
+      String storageUuid) {
+    // Make sure another entry for the same block is first removed.
+    // There may only be one such entry.
+    for (Map.Entry<String, PerStoragePendingIncrementalBR> entry :
+          pendingIncrementalBRperStorage.entrySet()) {
+      if (entry.getValue().removeBlockInfo(bInfo)) {
+        break;
+      }
+    }
+    getIncrementalBRMapForStorage(storageUuid).putBlockInfo(bInfo);
+  }
+
   /*
   /*
    * Informing the name node could take a long long time! Should we wait
    * Informing the name node could take a long long time! Should we wait
    * till namenode is informed before responding with success to the
    * till namenode is informed before responding with success to the
    * client? For now we don't.
    * client? For now we don't.
    */
    */
-  void notifyNamenodeBlockImmediately(ReceivedDeletedBlockInfo bInfo) {
-    synchronized (pendingIncrementalBR) {
-      pendingIncrementalBR.put(
-          bInfo.getBlock().getBlockId(), bInfo);
+  void notifyNamenodeBlockImmediately(
+      ReceivedDeletedBlockInfo bInfo, String storageUuid) {
+    synchronized (pendingIncrementalBRperStorage) {
+      addPendingReplicationBlockInfo(bInfo, storageUuid);
       pendingReceivedRequests++;
       pendingReceivedRequests++;
-      pendingIncrementalBR.notifyAll();
+      pendingIncrementalBRperStorage.notifyAll();
     }
     }
   }
   }
 
 
-  void notifyNamenodeDeletedBlock(ReceivedDeletedBlockInfo bInfo) {
-    synchronized (pendingIncrementalBR) {
-      pendingIncrementalBR.put(
-          bInfo.getBlock().getBlockId(), bInfo);
+  void notifyNamenodeDeletedBlock(
+      ReceivedDeletedBlockInfo bInfo, String storageUuid) {
+    synchronized (pendingIncrementalBRperStorage) {
+      addPendingReplicationBlockInfo(bInfo, storageUuid);
     }
     }
   }
   }
 
 
@@ -334,13 +387,13 @@ class BPServiceActor implements Runnable {
    */
    */
   @VisibleForTesting
   @VisibleForTesting
   void triggerBlockReportForTests() {
   void triggerBlockReportForTests() {
-    synchronized (pendingIncrementalBR) {
+    synchronized (pendingIncrementalBRperStorage) {
       lastBlockReport = 0;
       lastBlockReport = 0;
       lastHeartbeat = 0;
       lastHeartbeat = 0;
-      pendingIncrementalBR.notifyAll();
+      pendingIncrementalBRperStorage.notifyAll();
       while (lastBlockReport == 0) {
       while (lastBlockReport == 0) {
         try {
         try {
-          pendingIncrementalBR.wait(100);
+          pendingIncrementalBRperStorage.wait(100);
         } catch (InterruptedException e) {
         } catch (InterruptedException e) {
           return;
           return;
         }
         }
@@ -350,12 +403,12 @@ class BPServiceActor implements Runnable {
   
   
   @VisibleForTesting
   @VisibleForTesting
   void triggerHeartbeatForTests() {
   void triggerHeartbeatForTests() {
-    synchronized (pendingIncrementalBR) {
+    synchronized (pendingIncrementalBRperStorage) {
       lastHeartbeat = 0;
       lastHeartbeat = 0;
-      pendingIncrementalBR.notifyAll();
+      pendingIncrementalBRperStorage.notifyAll();
       while (lastHeartbeat == 0) {
       while (lastHeartbeat == 0) {
         try {
         try {
-          pendingIncrementalBR.wait(100);
+          pendingIncrementalBRperStorage.wait(100);
         } catch (InterruptedException e) {
         } catch (InterruptedException e) {
           return;
           return;
         }
         }
@@ -365,13 +418,13 @@ class BPServiceActor implements Runnable {
 
 
   @VisibleForTesting
   @VisibleForTesting
   void triggerDeletionReportForTests() {
   void triggerDeletionReportForTests() {
-    synchronized (pendingIncrementalBR) {
+    synchronized (pendingIncrementalBRperStorage) {
       lastDeletedReport = 0;
       lastDeletedReport = 0;
-      pendingIncrementalBR.notifyAll();
+      pendingIncrementalBRperStorage.notifyAll();
 
 
       while (lastDeletedReport == 0) {
       while (lastDeletedReport == 0) {
         try {
         try {
-          pendingIncrementalBR.wait(100);
+          pendingIncrementalBRperStorage.wait(100);
         } catch (InterruptedException e) {
         } catch (InterruptedException e) {
           return;
           return;
         }
         }
@@ -395,23 +448,38 @@ class BPServiceActor implements Runnable {
       // a FINALIZED one.
       // a FINALIZED one.
       reportReceivedDeletedBlocks();
       reportReceivedDeletedBlocks();
 
 
+      // Send one block report per known storage.
+
       // Create block report
       // Create block report
       long brCreateStartTime = now();
       long brCreateStartTime = now();
-      BlockListAsLongs bReport = dn.getFSDataset().getBlockReport(
-          bpos.getBlockPoolId());
+      long totalBlockCount = 0;
+
+      Map<DatanodeStorage, BlockListAsLongs> perVolumeBlockLists =
+          dn.getFSDataset().getBlockReports(bpos.getBlockPoolId());
 
 
       // Send block report
       // Send block report
       long brSendStartTime = now();
       long brSendStartTime = now();
-      StorageBlockReport[] report = { new StorageBlockReport(
-          new DatanodeStorage(bpRegistration.getStorageID()),
-          bReport.getBlockListAsLongs()) };
-      cmd = bpNamenode.blockReport(bpRegistration, bpos.getBlockPoolId(), report);
+      StorageBlockReport[] reports =
+          new StorageBlockReport[perVolumeBlockLists.size()];
+
+      int i = 0;
+      for(Map.Entry<DatanodeStorage, BlockListAsLongs> kvPair : perVolumeBlockLists.entrySet()) {
+        DatanodeStorage dnStorage = kvPair.getKey();
+        BlockListAsLongs blockList = kvPair.getValue();
+        totalBlockCount += blockList.getNumberOfBlocks();
+
+        reports[i++] =
+            new StorageBlockReport(
+              dnStorage, blockList.getBlockListAsLongs());
+      }
+
+      cmd = bpNamenode.blockReport(bpRegistration, bpos.getBlockPoolId(), reports);
 
 
       // Log the block report processing stats from Datanode perspective
       // Log the block report processing stats from Datanode perspective
       long brSendCost = now() - brSendStartTime;
       long brSendCost = now() - brSendStartTime;
       long brCreateCost = brSendStartTime - brCreateStartTime;
       long brCreateCost = brSendStartTime - brCreateStartTime;
       dn.getMetrics().addBlockReport(brSendCost);
       dn.getMetrics().addBlockReport(brSendCost);
-      LOG.info("BlockReport of " + bReport.getNumberOfBlocks()
+      LOG.info("BlockReport of " + totalBlockCount
           + " blocks took " + brCreateCost + " msec to generate and "
           + " blocks took " + brCreateCost + " msec to generate and "
           + brSendCost + " msecs for RPC and NN processing");
           + brSendCost + " msecs for RPC and NN processing");
 
 
@@ -466,17 +534,15 @@ class BPServiceActor implements Runnable {
   }
   }
   
   
   HeartbeatResponse sendHeartBeat() throws IOException {
   HeartbeatResponse sendHeartBeat() throws IOException {
+    StorageReport[] reports =
+        dn.getFSDataset().getStorageReports(bpos.getBlockPoolId());
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Sending heartbeat from service actor: " + this);
-    }
-    // reports number of failed volumes
-    StorageReport[] report = { new StorageReport(bpRegistration.getStorageID(),
-        false,
-        dn.getFSDataset().getCapacity(),
-        dn.getFSDataset().getDfsUsed(),
-        dn.getFSDataset().getRemaining(),
-        dn.getFSDataset().getBlockPoolUsed(bpos.getBlockPoolId())) };
-    return bpNamenode.sendHeartbeat(bpRegistration, report,
+      LOG.debug("Sending heartbeat with " + reports.length +
+                " storage reports from service actor: " + this);
+    }
+
+    return bpNamenode.sendHeartbeat(bpRegistration,
+        reports,
         dn.getFSDataset().getCacheCapacity(),
         dn.getFSDataset().getCacheCapacity(),
         dn.getFSDataset().getCacheUsed(),
         dn.getFSDataset().getCacheUsed(),
         dn.getXmitsInProgress(),
         dn.getXmitsInProgress(),
@@ -496,9 +562,9 @@ class BPServiceActor implements Runnable {
   }
   }
   
   
   private String formatThreadName() {
   private String formatThreadName() {
-    Collection<URI> dataDirs = DataNode.getStorageDirs(dn.getConf());
-    return "DataNode: [" +
-      StringUtils.uriToString(dataDirs.toArray(new URI[0])) + "] " +
+    Collection<StorageLocation> dataDirs =
+        DataNode.getStorageLocations(dn.getConf());
+    return "DataNode: [" + dataDirs.toString() + "] " +
       " heartbeating to " + nnAddr;
       " heartbeating to " + nnAddr;
   }
   }
   
   
@@ -608,10 +674,10 @@ class BPServiceActor implements Runnable {
         //
         //
         long waitTime = dnConf.heartBeatInterval - 
         long waitTime = dnConf.heartBeatInterval - 
         (Time.now() - lastHeartbeat);
         (Time.now() - lastHeartbeat);
-        synchronized(pendingIncrementalBR) {
+        synchronized(pendingIncrementalBRperStorage) {
           if (waitTime > 0 && pendingReceivedRequests == 0) {
           if (waitTime > 0 && pendingReceivedRequests == 0) {
             try {
             try {
-              pendingIncrementalBR.wait(waitTime);
+              pendingIncrementalBRperStorage.wait(waitTime);
             } catch (InterruptedException ie) {
             } catch (InterruptedException ie) {
               LOG.warn("BPOfferService for " + this + " interrupted");
               LOG.warn("BPOfferService for " + this + " interrupted");
             }
             }
@@ -782,4 +848,68 @@ class BPServiceActor implements Runnable {
     }
     }
   }
   }
 
 
+  private static class PerStoragePendingIncrementalBR {
+    private Map<Long, ReceivedDeletedBlockInfo> pendingIncrementalBR =
+        Maps.newHashMap();
+
+    /**
+     * Return the number of blocks on this storage that have pending
+     * incremental block reports.
+     * @return
+     */
+    int getBlockInfoCount() {
+      return pendingIncrementalBR.size();
+    }
+
+    /**
+     * Dequeue and return all pending incremental block report state.
+     * @return
+     */
+    ReceivedDeletedBlockInfo[] dequeueBlockInfos() {
+      ReceivedDeletedBlockInfo[] blockInfos =
+          pendingIncrementalBR.values().toArray(
+              new ReceivedDeletedBlockInfo[getBlockInfoCount()]);
+
+      pendingIncrementalBR.clear();
+      return blockInfos;
+    }
+
+    /**
+     * Add blocks from blockArray to pendingIncrementalBR, unless the
+     * block already exists in pendingIncrementalBR.
+     * @param blockArray list of blocks to add.
+     * @return the number of missing blocks that we added.
+     */
+    int putMissingBlockInfos(ReceivedDeletedBlockInfo[] blockArray) {
+      int blocksPut = 0;
+      for (ReceivedDeletedBlockInfo rdbi : blockArray) {
+        if (!pendingIncrementalBR.containsKey(rdbi.getBlock().getBlockId())) {
+          pendingIncrementalBR.put(rdbi.getBlock().getBlockId(), rdbi);
+          ++blocksPut;
+        }
+      }
+      return blocksPut;
+    }
+
+    /**
+     * Add pending incremental block report for a single block.
+     * @param blockID
+     * @param blockInfo
+     */
+    void putBlockInfo(ReceivedDeletedBlockInfo blockInfo) {
+      pendingIncrementalBR.put(blockInfo.getBlock().getBlockId(), blockInfo);
+    }
+
+    /**
+     * Remove pending incremental block report for a single block if it
+     * exists.
+     *
+     * @param blockInfo
+     * @return true if a report was removed, false if no report existed for
+     *         the given block.
+     */
+    boolean removeBlockInfo(ReceivedDeletedBlockInfo blockInfo) {
+      return (pendingIncrementalBR.remove(blockInfo.getBlock().getBlockId()) != null);
+    }
+  }
 }
 }

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

@@ -187,7 +187,7 @@ class BlockPoolSliceScanner {
         + hours + " hours for block pool " + bpid);
         + hours + " hours for block pool " + bpid);
 
 
     // get the list of blocks and arrange them in random order
     // get the list of blocks and arrange them in random order
-    List<Block> arr = dataset.getFinalizedBlocks(blockPoolId);
+    List<FinalizedReplica> arr = dataset.getFinalizedBlocks(blockPoolId);
     Collections.shuffle(arr);
     Collections.shuffle(arr);
     
     
     long scanTime = -1;
     long scanTime = -1;

+ 13 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -162,7 +162,8 @@ class BlockReceiver implements Closeable {
         switch (stage) {
         switch (stage) {
         case PIPELINE_SETUP_CREATE:
         case PIPELINE_SETUP_CREATE:
           replicaInfo = datanode.data.createRbw(block);
           replicaInfo = datanode.data.createRbw(block);
-          datanode.notifyNamenodeReceivingBlock(block);
+          datanode.notifyNamenodeReceivingBlock(
+              block, replicaInfo.getStorageUuid());
           break;
           break;
         case PIPELINE_SETUP_STREAMING_RECOVERY:
         case PIPELINE_SETUP_STREAMING_RECOVERY:
           replicaInfo = datanode.data.recoverRbw(
           replicaInfo = datanode.data.recoverRbw(
@@ -176,7 +177,8 @@ class BlockReceiver implements Closeable {
                 block.getLocalBlock());
                 block.getLocalBlock());
           }
           }
           block.setGenerationStamp(newGs);
           block.setGenerationStamp(newGs);
-          datanode.notifyNamenodeReceivingBlock(block);
+          datanode.notifyNamenodeReceivingBlock(
+              block, replicaInfo.getStorageUuid());
           break;
           break;
         case PIPELINE_SETUP_APPEND_RECOVERY:
         case PIPELINE_SETUP_APPEND_RECOVERY:
           replicaInfo = datanode.data.recoverAppend(block, newGs, minBytesRcvd);
           replicaInfo = datanode.data.recoverAppend(block, newGs, minBytesRcvd);
@@ -185,7 +187,8 @@ class BlockReceiver implements Closeable {
                 block.getLocalBlock());
                 block.getLocalBlock());
           }
           }
           block.setGenerationStamp(newGs);
           block.setGenerationStamp(newGs);
-          datanode.notifyNamenodeReceivingBlock(block);
+          datanode.notifyNamenodeReceivingBlock(
+              block, replicaInfo.getStorageUuid());
           break;
           break;
         case TRANSFER_RBW:
         case TRANSFER_RBW:
         case TRANSFER_FINALIZED:
         case TRANSFER_FINALIZED:
@@ -252,6 +255,10 @@ class BlockReceiver implements Closeable {
   /** Return the datanode object. */
   /** Return the datanode object. */
   DataNode getDataNode() {return datanode;}
   DataNode getDataNode() {return datanode;}
 
 
+  String getStorageUuid() {
+    return replicaInfo.getStorageUuid();
+  }
+
   /**
   /**
    * close files.
    * close files.
    */
    */
@@ -1073,14 +1080,15 @@ class BlockReceiver implements Closeable {
           : 0;
           : 0;
       block.setNumBytes(replicaInfo.getNumBytes());
       block.setNumBytes(replicaInfo.getNumBytes());
       datanode.data.finalizeBlock(block);
       datanode.data.finalizeBlock(block);
-      datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT);
+      datanode.closeBlock(
+          block, DataNode.EMPTY_DEL_HINT, replicaInfo.getStorageUuid());
       if (ClientTraceLog.isInfoEnabled() && isClient) {
       if (ClientTraceLog.isInfoEnabled() && isClient) {
         long offset = 0;
         long offset = 0;
         DatanodeRegistration dnR = datanode.getDNRegistrationForBP(block
         DatanodeRegistration dnR = datanode.getDNRegistrationForBP(block
             .getBlockPoolId());
             .getBlockPoolId());
         ClientTraceLog.info(String.format(DN_CLIENTTRACE_FORMAT, inAddr,
         ClientTraceLog.info(String.format(DN_CLIENTTRACE_FORMAT, inAddr,
             myAddr, block.getNumBytes(), "HDFS_WRITE", clientname, offset,
             myAddr, block.getNumBytes(), "HDFS_WRITE", clientname, offset,
-            dnR.getStorageID(), block, endTime - startTime));
+            dnR.getDatanodeUuid(), block, endTime - startTime));
       } else {
       } else {
         LOG.info("Received " + block + " size " + block.getNumBytes()
         LOG.info("Received " + block + " size " + block.getNumBytes()
             + " from " + inAddr);
             + " from " + inAddr);

+ 171 - 123
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -17,10 +17,40 @@
  */
  */
 package org.apache.hadoop.hdfs.server.datanode;
 package org.apache.hadoop.hdfs.server.datanode;
 
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.protobuf.BlockingService;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+import static org.apache.hadoop.util.ExitUtil.terminate;
+
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
+import java.net.URI;
+import java.net.UnknownHostException;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.SocketChannel;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.management.ObjectName;
+
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -38,21 +68,42 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.net.DomainPeerServer;
 import org.apache.hadoop.hdfs.net.DomainPeerServer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
-import org.apache.hadoop.hdfs.protocol.*;
-import org.apache.hadoop.hdfs.protocol.datatransfer.*;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InterDatanodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InterDatanodeProtocolService;
-import org.apache.hadoop.hdfs.protocolPB.*;
-import org.apache.hadoop.hdfs.security.token.block.*;
+import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
+import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 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.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
-import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -61,7 +112,11 @@ import org.apache.hadoop.hdfs.server.datanode.web.resources.DatanodeWebHdfsMetho
 import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
 import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
 import org.apache.hadoop.hdfs.server.namenode.StreamFile;
 import org.apache.hadoop.hdfs.server.namenode.StreamFile;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
-import org.apache.hadoop.hdfs.server.protocol.*;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.resources.Param;
 import org.apache.hadoop.hdfs.web.resources.Param;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpConfig;
@@ -84,23 +139,21 @@ import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.util.*;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
+import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.util.JvmPauseMonitor;
+import org.apache.hadoop.util.ServicePlugin;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.VersionInfo;
 import org.mortbay.util.ajax.JSON;
 import org.mortbay.util.ajax.JSON;
 
 
-import java.io.*;
-import java.net.*;
-import java.nio.channels.ClosedByInterruptException;
-import java.nio.channels.SocketChannel;
-import java.security.PrivilegedExceptionAction;
-import java.util.*;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import javax.management.ObjectName;
-
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
-import static org.apache.hadoop.util.ExitUtil.terminate;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.BlockingService;
 
 
 /**********************************************************
 /**********************************************************
  * DataNode is a class (and program) that stores a set of
  * DataNode is a class (and program) that stores a set of
@@ -209,7 +262,7 @@ public class DataNode extends Configured
   private JvmPauseMonitor pauseMonitor;
   private JvmPauseMonitor pauseMonitor;
 
 
   private SecureResources secureResources = null;
   private SecureResources secureResources = null;
-  private AbstractList<File> dataDirs;
+  private List<StorageLocation> dataDirs;
   private Configuration conf;
   private Configuration conf;
   private final long maxNumberOfBlocksToLog;
   private final long maxNumberOfBlocksToLog;
 
 
@@ -219,21 +272,12 @@ public class DataNode extends Configured
   private final boolean getHdfsBlockLocationsEnabled;
   private final boolean getHdfsBlockLocationsEnabled;
   private ObjectName dataNodeInfoBeanName;
   private ObjectName dataNodeInfoBeanName;
 
 
-  /**
-   * Create the DataNode given a configuration and an array of dataDirs.
-   * 'dataDirs' is where the blocks are stored.
-   */
-  DataNode(final Configuration conf, 
-           final AbstractList<File> dataDirs) throws IOException {
-    this(conf, dataDirs, null);
-  }
-  
   /**
   /**
    * Create the DataNode given a configuration, an array of dataDirs,
    * Create the DataNode given a configuration, an array of dataDirs,
    * and a namenode proxy
    * and a namenode proxy
    */
    */
-  DataNode(final Configuration conf, 
-           final AbstractList<File> dataDirs,
+  DataNode(final Configuration conf,
+           final List<StorageLocation> dataDirs,
            final SecureResources resources) throws IOException {
            final SecureResources resources) throws IOException {
     super(conf);
     super(conf);
     this.maxNumberOfBlocksToLog = conf.getLong(DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
     this.maxNumberOfBlocksToLog = conf.getLong(DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
@@ -494,7 +538,7 @@ public class DataNode extends Configured
       directoryScanner.start();
       directoryScanner.start();
     } else {
     } else {
       LOG.info("Periodic Directory Tree Verification scan is disabled because " +
       LOG.info("Periodic Directory Tree Verification scan is disabled because " +
-               reason);
+                   reason);
     }
     }
   }
   }
   
   
@@ -566,10 +610,11 @@ public class DataNode extends Configured
   }
   }
   
   
   // calls specific to BP
   // calls specific to BP
-  protected void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) {
+  protected void notifyNamenodeReceivedBlock(
+      ExtendedBlock block, String delHint, String storageUuid) {
     BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
     BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
     if(bpos != null) {
     if(bpos != null) {
-      bpos.notifyNamenodeReceivedBlock(block, delHint); 
+      bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid);
     } else {
     } else {
       LOG.error("Cannot find BPOfferService for reporting block received for bpid="
       LOG.error("Cannot find BPOfferService for reporting block received for bpid="
           + block.getBlockPoolId());
           + block.getBlockPoolId());
@@ -577,10 +622,11 @@ public class DataNode extends Configured
   }
   }
   
   
   // calls specific to BP
   // calls specific to BP
-  protected void notifyNamenodeReceivingBlock(ExtendedBlock block) {
+  protected void notifyNamenodeReceivingBlock(
+      ExtendedBlock block, String storageUuid) {
     BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
     BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
     if(bpos != null) {
     if(bpos != null) {
-      bpos.notifyNamenodeReceivingBlock(block); 
+      bpos.notifyNamenodeReceivingBlock(block, storageUuid);
     } else {
     } else {
       LOG.error("Cannot find BPOfferService for reporting block receiving for bpid="
       LOG.error("Cannot find BPOfferService for reporting block receiving for bpid="
           + block.getBlockPoolId());
           + block.getBlockPoolId());
@@ -588,10 +634,10 @@ public class DataNode extends Configured
   }
   }
   
   
   /** Notify the corresponding namenode to delete the block. */
   /** Notify the corresponding namenode to delete the block. */
-  public void notifyNamenodeDeletedBlock(ExtendedBlock block) {
+  public void notifyNamenodeDeletedBlock(ExtendedBlock block, String storageUuid) {
     BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
     BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
     if (bpos != null) {
     if (bpos != null) {
-      bpos.notifyNamenodeDeletedBlock(block);
+      bpos.notifyNamenodeDeletedBlock(block, storageUuid);
     } else {
     } else {
       LOG.error("Cannot find BPOfferService for reporting block deleted for bpid="
       LOG.error("Cannot find BPOfferService for reporting block deleted for bpid="
           + block.getBlockPoolId());
           + block.getBlockPoolId());
@@ -603,7 +649,9 @@ public class DataNode extends Configured
    */
    */
   public void reportBadBlocks(ExtendedBlock block) throws IOException{
   public void reportBadBlocks(ExtendedBlock block) throws IOException{
     BPOfferService bpos = getBPOSForBlock(block);
     BPOfferService bpos = getBPOSForBlock(block);
-    bpos.reportBadBlocks(block);
+    FsVolumeSpi volume = getFSDataset().getVolume(block);
+    bpos.reportBadBlocks(
+        block, volume.getStorageID(), volume.getStorageType());
   }
   }
 
 
   /**
   /**
@@ -675,7 +723,7 @@ public class DataNode extends Configured
    * @throws IOException
    * @throws IOException
    */
    */
   void startDataNode(Configuration conf, 
   void startDataNode(Configuration conf, 
-                     AbstractList<File> dataDirs,
+                     List<StorageLocation> dataDirs,
                     // DatanodeProtocol namenode,
                     // DatanodeProtocol namenode,
                      SecureResources resources
                      SecureResources resources
                      ) throws IOException {
                      ) throws IOException {
@@ -736,19 +784,42 @@ public class DataNode extends Configured
     readaheadPool = ReadaheadPool.getInstance();
     readaheadPool = ReadaheadPool.getInstance();
   }
   }
   
   
+  public static String generateUuid() {
+    return UUID.randomUUID().toString();
+  }
+
+  /**
+   * Verify that the DatanodeUuid has been initialized. If this is a new
+   * datanode then we generate a new Datanode Uuid and persist it to disk.
+   *
+   * @throws IOException
+   */
+  private synchronized void checkDatanodeUuid() throws IOException {
+    if (storage.getDatanodeUuid() == null) {
+      storage.setDatanodeUuid(generateUuid());
+      storage.writeAll();
+      LOG.info("Generated and persisted new Datanode UUID " +
+               storage.getDatanodeUuid());
+    }
+  }
+
   /**
   /**
    * Create a DatanodeRegistration for a specific block pool.
    * Create a DatanodeRegistration for a specific block pool.
    * @param nsInfo the namespace info from the first part of the NN handshake
    * @param nsInfo the namespace info from the first part of the NN handshake
    */
    */
-  DatanodeRegistration createBPRegistration(NamespaceInfo nsInfo) {
+  DatanodeRegistration createBPRegistration(NamespaceInfo nsInfo)
+      throws IOException {
     StorageInfo storageInfo = storage.getBPStorage(nsInfo.getBlockPoolID());
     StorageInfo storageInfo = storage.getBPStorage(nsInfo.getBlockPoolID());
     if (storageInfo == null) {
     if (storageInfo == null) {
       // it's null in the case of SimulatedDataSet
       // it's null in the case of SimulatedDataSet
       storageInfo = new StorageInfo(nsInfo);
       storageInfo = new StorageInfo(nsInfo);
     }
     }
+
+    checkDatanodeUuid();
+
     DatanodeID dnId = new DatanodeID(
     DatanodeID dnId = new DatanodeID(
         streamingAddr.getAddress().getHostAddress(), hostName, 
         streamingAddr.getAddress().getHostAddress(), hostName, 
-        getStorageId(), getXferPort(), getInfoPort(),
+        storage.getDatanodeUuid(), getXferPort(), getInfoPort(),
             infoSecurePort, getIpcPort());
             infoSecurePort, getIpcPort());
     return new DatanodeRegistration(dnId, storageInfo, 
     return new DatanodeRegistration(dnId, storageInfo, 
         new ExportedBlockKeys(), VersionInfo.getVersion());
         new ExportedBlockKeys(), VersionInfo.getVersion());
@@ -767,16 +838,10 @@ public class DataNode extends Configured
       id = bpRegistration;
       id = bpRegistration;
     }
     }
 
 
-    if (storage.getStorageID().equals("")) {
-      // This is a fresh datanode, persist the NN-provided storage ID
-      storage.setStorageID(bpRegistration.getStorageID());
-      storage.writeAll();
-      LOG.info("New storage id " + bpRegistration.getStorageID()
-          + " is assigned to data-node " + bpRegistration);
-    } else if(!storage.getStorageID().equals(bpRegistration.getStorageID())) {
-      throw new IOException("Inconsistent storage IDs. Name-node returned "
-          + bpRegistration.getStorageID() 
-          + ". Expecting " + storage.getStorageID());
+    if(!storage.getDatanodeUuid().equals(bpRegistration.getDatanodeUuid())) {
+      throw new IOException("Inconsistent Datanode IDs. Name-node returned "
+          + bpRegistration.getDatanodeUuid()
+          + ". Expecting " + storage.getDatanodeUuid());
     }
     }
     
     
     registerBlockPoolWithSecretManager(bpRegistration, blockPoolId);
     registerBlockPoolWithSecretManager(bpRegistration, blockPoolId);
@@ -897,7 +962,7 @@ public class DataNode extends Configured
       final StorageInfo bpStorage = storage.getBPStorage(bpid);
       final StorageInfo bpStorage = storage.getBPStorage(bpid);
       LOG.info("Setting up storage: nsid=" + bpStorage.getNamespaceID()
       LOG.info("Setting up storage: nsid=" + bpStorage.getNamespaceID()
           + ";bpid=" + bpid + ";lv=" + storage.getLayoutVersion()
           + ";bpid=" + bpid + ";lv=" + storage.getLayoutVersion()
-          + ";nsInfo=" + nsInfo);
+          + ";nsInfo=" + nsInfo + ";dnuuid=" + storage.getDatanodeUuid());
     }
     }
 
 
     synchronized(this)  {
     synchronized(this)  {
@@ -924,10 +989,6 @@ public class DataNode extends Configured
     return streamingAddr.getPort();
     return streamingAddr.getPort();
   }
   }
   
   
-  String getStorageId() {
-    return storage.getStorageID();
-  }
-
   /**
   /**
    * @return name useful for logging
    * @return name useful for logging
    */
    */
@@ -1013,34 +1074,6 @@ public class DataNode extends Configured
     return metrics;
     return metrics;
   }
   }
   
   
-  public static void setNewStorageID(DatanodeID dnId) {
-    LOG.info("Datanode is " + dnId);
-    dnId.setStorageID(createNewStorageId(dnId.getXferPort()));
-  }
-  
-  /**
-   * @return a unique storage ID of form "DS-randInt-ipaddr-port-timestamp"
-   */
-  static String createNewStorageId(int port) {
-    // It is unlikely that we will create a non-unique storage ID
-    // for the following reasons:
-    // a) SecureRandom is a cryptographically strong random number generator
-    // b) IP addresses will likely differ on different hosts
-    // c) DataNode xfer ports will differ on the same host
-    // d) StorageIDs will likely be generated at different times (in ms)
-    // A conflict requires that all four conditions are violated.
-    // NB: The format of this string can be changed in the future without
-    // requiring that old SotrageIDs be updated.
-    String ip = "unknownIP";
-    try {
-      ip = DNS.getDefaultIP("default");
-    } catch (UnknownHostException ignored) {
-      LOG.warn("Could not find an IP address for the \"default\" inteface.");
-    }
-    int rand = DFSUtil.getSecureRandom().nextInt(Integer.MAX_VALUE);
-    return "DS-" + rand + "-" + ip + "-" + port + "-" + Time.now();
-  }
-  
   /** Ensure the authentication method is kerberos */
   /** Ensure the authentication method is kerberos */
   private void checkKerberosAuthMethod(String msg) throws IOException {
   private void checkKerberosAuthMethod(String msg) throws IOException {
     // User invoking the call must be same as the datanode user
     // User invoking the call must be same as the datanode user
@@ -1370,8 +1403,10 @@ public class DataNode extends Configured
     // Check if NN recorded length matches on-disk length 
     // Check if NN recorded length matches on-disk length 
     long onDiskLength = data.getLength(block);
     long onDiskLength = data.getLength(block);
     if (block.getNumBytes() > onDiskLength) {
     if (block.getNumBytes() > onDiskLength) {
+      FsVolumeSpi volume = getFSDataset().getVolume(block);
       // Shorter on-disk len indicates corruption so report NN the corrupt block
       // Shorter on-disk len indicates corruption so report NN the corrupt block
-      bpos.reportBadBlocks(block);
+      bpos.reportBadBlocks(
+          block, volume.getStorageID(), volume.getStorageType());
       LOG.warn("Can't replicate block " + block
       LOG.warn("Can't replicate block " + block
           + " because on-disk length " + onDiskLength 
           + " because on-disk length " + onDiskLength 
           + " is shorter than NameNode recorded length " + block.getNumBytes());
           + " is shorter than NameNode recorded length " + block.getNumBytes());
@@ -1635,11 +1670,11 @@ public class DataNode extends Configured
    * @param block
    * @param block
    * @param delHint
    * @param delHint
    */
    */
-  void closeBlock(ExtendedBlock block, String delHint) {
+  void closeBlock(ExtendedBlock block, String delHint, String storageUuid) {
     metrics.incrBlocksWritten();
     metrics.incrBlocksWritten();
     BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
     BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
     if(bpos != null) {
     if(bpos != null) {
-      bpos.notifyNamenodeReceivedBlock(block, delHint);
+      bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid);
     } else {
     } else {
       LOG.warn("Cannot find BPOfferService for reporting block received for bpid="
       LOG.warn("Cannot find BPOfferService for reporting block received for bpid="
           + block.getBlockPoolId());
           + block.getBlockPoolId());
@@ -1703,17 +1738,32 @@ public class DataNode extends Configured
       printUsage(System.err);
       printUsage(System.err);
       return null;
       return null;
     }
     }
-    Collection<URI> dataDirs = getStorageDirs(conf);
+    Collection<StorageLocation> dataLocations = getStorageLocations(conf);
     UserGroupInformation.setConfiguration(conf);
     UserGroupInformation.setConfiguration(conf);
     SecurityUtil.login(conf, DFS_DATANODE_KEYTAB_FILE_KEY,
     SecurityUtil.login(conf, DFS_DATANODE_KEYTAB_FILE_KEY,
         DFS_DATANODE_USER_NAME_KEY);
         DFS_DATANODE_USER_NAME_KEY);
-    return makeInstance(dataDirs, conf, resources);
+    return makeInstance(dataLocations, conf, resources);
   }
   }
 
 
-  static Collection<URI> getStorageDirs(Configuration conf) {
-    Collection<String> dirNames =
-      conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY);
-    return Util.stringCollectionAsURIs(dirNames);
+  public static List<StorageLocation> getStorageLocations(Configuration conf) {
+    Collection<String> rawLocations =
+        conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY);
+    List<StorageLocation> locations =
+        new ArrayList<StorageLocation>(rawLocations.size());
+
+    for(String locationString : rawLocations) {
+      final StorageLocation location;
+      try {
+        location = StorageLocation.parse(locationString);
+      } catch (IOException ioe) {
+        throw new IllegalArgumentException("Failed to parse conf property "
+            + DFS_DATANODE_DATA_DIR_KEY + ": " + locationString, ioe);
+      }
+
+      locations.add(location);
+    }
+
+    return locations;
   }
   }
 
 
   /** Instantiate & Start a single datanode daemon and wait for it to finish.
   /** Instantiate & Start a single datanode daemon and wait for it to finish.
@@ -1779,57 +1829,52 @@ public class DataNode extends Configured
    * no directory from this directory list can be created.
    * no directory from this directory list can be created.
    * @throws IOException
    * @throws IOException
    */
    */
-  static DataNode makeInstance(Collection<URI> dataDirs, Configuration conf,
-      SecureResources resources) throws IOException {
+  static DataNode makeInstance(Collection<StorageLocation> dataDirs,
+      Configuration conf, SecureResources resources) throws IOException {
     LocalFileSystem localFS = FileSystem.getLocal(conf);
     LocalFileSystem localFS = FileSystem.getLocal(conf);
     FsPermission permission = new FsPermission(
     FsPermission permission = new FsPermission(
         conf.get(DFS_DATANODE_DATA_DIR_PERMISSION_KEY,
         conf.get(DFS_DATANODE_DATA_DIR_PERMISSION_KEY,
                  DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT));
                  DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT));
     DataNodeDiskChecker dataNodeDiskChecker =
     DataNodeDiskChecker dataNodeDiskChecker =
         new DataNodeDiskChecker(permission);
         new DataNodeDiskChecker(permission);
-    ArrayList<File> dirs =
-        getDataDirsFromURIs(dataDirs, localFS, dataNodeDiskChecker);
+    List<StorageLocation> locations =
+        checkStorageLocations(dataDirs, localFS, dataNodeDiskChecker);
     DefaultMetricsSystem.initialize("DataNode");
     DefaultMetricsSystem.initialize("DataNode");
 
 
-    assert dirs.size() > 0 : "number of data directories should be > 0";
-    return new DataNode(conf, dirs, resources);
+    assert locations.size() > 0 : "number of data directories should be > 0";
+    return new DataNode(conf, locations, resources);
   }
   }
 
 
   // DataNode ctor expects AbstractList instead of List or Collection...
   // DataNode ctor expects AbstractList instead of List or Collection...
-  static ArrayList<File> getDataDirsFromURIs(Collection<URI> dataDirs,
+  static List<StorageLocation> checkStorageLocations(
+      Collection<StorageLocation> dataDirs,
       LocalFileSystem localFS, DataNodeDiskChecker dataNodeDiskChecker)
       LocalFileSystem localFS, DataNodeDiskChecker dataNodeDiskChecker)
           throws IOException {
           throws IOException {
-    ArrayList<File> dirs = new ArrayList<File>();
+    ArrayList<StorageLocation> locations = new ArrayList<StorageLocation>();
     StringBuilder invalidDirs = new StringBuilder();
     StringBuilder invalidDirs = new StringBuilder();
-    for (URI dirURI : dataDirs) {
-      if (!"file".equalsIgnoreCase(dirURI.getScheme())) {
-        LOG.warn("Unsupported URI schema in " + dirURI + ". Ignoring ...");
-        invalidDirs.append("\"").append(dirURI).append("\" ");
-        continue;
-      }
-      // drop any (illegal) authority in the URI for backwards compatibility
-      File dir = new File(dirURI.getPath());
+    for (StorageLocation location : dataDirs) {
+      final URI uri = location.getUri();
       try {
       try {
-        dataNodeDiskChecker.checkDir(localFS, new Path(dir.toURI()));
-        dirs.add(dir);
+        dataNodeDiskChecker.checkDir(localFS, new Path(uri));
+        locations.add(location);
       } catch (IOException ioe) {
       } catch (IOException ioe) {
         LOG.warn("Invalid " + DFS_DATANODE_DATA_DIR_KEY + " "
         LOG.warn("Invalid " + DFS_DATANODE_DATA_DIR_KEY + " "
-            + dir + " : ", ioe);
-        invalidDirs.append("\"").append(dirURI.getPath()).append("\" ");
+            + location.getFile() + " : ", ioe);
+        invalidDirs.append("\"").append(uri.getPath()).append("\" ");
       }
       }
     }
     }
-    if (dirs.size() == 0) {
+    if (locations.size() == 0) {
       throw new IOException("All directories in "
       throw new IOException("All directories in "
           + DFS_DATANODE_DATA_DIR_KEY + " are invalid: "
           + DFS_DATANODE_DATA_DIR_KEY + " are invalid: "
           + invalidDirs);
           + invalidDirs);
     }
     }
-    return dirs;
+    return locations;
   }
   }
 
 
   @Override
   @Override
   public String toString() {
   public String toString() {
     return "DataNode{data=" + data + ", localName='" + getDisplayName()
     return "DataNode{data=" + data + ", localName='" + getDisplayName()
-        + "', storageID='" + getStorageId() + "', xmitsInProgress="
+        + "', datanodeUuid='" + storage.getDatanodeUuid() + "', xmitsInProgress="
         + xmitsInProgress.get() + "}";
         + xmitsInProgress.get() + "}";
   }
   }
 
 
@@ -1883,7 +1928,6 @@ public class DataNode extends Configured
   }
   }
 
 
   /**
   /**
-   * This method is used for testing. 
    * Examples are adding and deleting blocks directly.
    * Examples are adding and deleting blocks directly.
    * The most common usage will be when the data node's storage is simulated.
    * The most common usage will be when the data node's storage is simulated.
    * 
    * 
@@ -1983,7 +2027,7 @@ public class DataNode extends Configured
     ExtendedBlock newBlock = new ExtendedBlock(oldBlock);
     ExtendedBlock newBlock = new ExtendedBlock(oldBlock);
     newBlock.setGenerationStamp(recoveryId);
     newBlock.setGenerationStamp(recoveryId);
     newBlock.setNumBytes(newLength);
     newBlock.setNumBytes(newLength);
-    notifyNamenodeReceivedBlock(newBlock, "");
+    notifyNamenodeReceivedBlock(newBlock, "", storageID);
     return storageID;
     return storageID;
   }
   }
 
 
@@ -2443,6 +2487,10 @@ public class DataNode extends Configured
     return dnConf;
     return dnConf;
   }
   }
 
 
+  public String getDatanodeUuid() {
+    return id == null ? null : id.getDatanodeUuid();
+  }
+
   boolean shouldRun() {
   boolean shouldRun() {
     return shouldRun;
     return shouldRun;
   }
   }

+ 69 - 42
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -24,13 +24,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.io.RandomAccessFile;
 import java.nio.channels.FileLock;
 import java.nio.channels.FileLock;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Properties;
+import java.util.*;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -50,6 +44,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Daemon;
@@ -71,8 +66,13 @@ public class DataStorage extends Storage {
   public final static String STORAGE_DIR_FINALIZED = "finalized";
   public final static String STORAGE_DIR_FINALIZED = "finalized";
   public final static String STORAGE_DIR_TMP = "tmp";
   public final static String STORAGE_DIR_TMP = "tmp";
 
 
-  /** Unique storage ID. {@see DataNode#createNewStorageId(int)} for details */
-  private String storageID;
+  /**
+   * Datanode UUID that this storage is currently attached to. This
+   *  is the same as the legacy StorageID for datanodes that were
+   *  upgraded from a pre-UUID version. For compatibility with prior
+   *  versions of Datanodes we cannot make this field a UUID.
+   */
+  private String datanodeUuid = null;
 
 
   // Flag to ensure we only initialize storage once
   // Flag to ensure we only initialize storage once
   private boolean initialized = false;
   private boolean initialized = false;
@@ -84,33 +84,29 @@ public class DataStorage extends Storage {
 
 
   DataStorage() {
   DataStorage() {
     super(NodeType.DATA_NODE);
     super(NodeType.DATA_NODE);
-    storageID = "";
   }
   }
   
   
   public StorageInfo getBPStorage(String bpid) {
   public StorageInfo getBPStorage(String bpid) {
     return bpStorageMap.get(bpid);
     return bpStorageMap.get(bpid);
   }
   }
   
   
-  public DataStorage(StorageInfo storageInfo, String strgID) {
+  public DataStorage(StorageInfo storageInfo) {
     super(NodeType.DATA_NODE, storageInfo);
     super(NodeType.DATA_NODE, storageInfo);
-    this.storageID = strgID;
   }
   }
 
 
-  /** @return storage ID. */
-  public synchronized String getStorageID() {
-    return storageID;
+  public synchronized String getDatanodeUuid() {
+    return datanodeUuid;
   }
   }
-  
-  synchronized void setStorageID(String newStorageID) {
-    this.storageID = newStorageID;
+
+  public synchronized void setDatanodeUuid(String newDatanodeUuid) {
+    this.datanodeUuid = newDatanodeUuid;
   }
   }
 
 
   /** Create an ID for this storage. */
   /** Create an ID for this storage. */
-  public synchronized void createStorageID(int datanodePort) {
-    if (storageID != null && !storageID.isEmpty()) {
-      return;
+  public synchronized void createStorageID(StorageDirectory sd) {
+    if (sd.getStorageUuid() == null) {
+      sd.setStorageUuid(DatanodeStorage.generateUuid());
     }
     }
-    storageID = DataNode.createNewStorageId(datanodePort);
   }
   }
   
   
   /**
   /**
@@ -128,7 +124,8 @@ public class DataStorage extends Storage {
    * @throws IOException
    * @throws IOException
    */
    */
   synchronized void recoverTransitionRead(DataNode datanode,
   synchronized void recoverTransitionRead(DataNode datanode,
-      NamespaceInfo nsInfo, Collection<File> dataDirs, StartupOption startOpt)
+      NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
+      StartupOption startOpt)
       throws IOException {
       throws IOException {
     if (initialized) {
     if (initialized) {
       // DN storage has been initialized, no need to do anything
       // DN storage has been initialized, no need to do anything
@@ -144,8 +141,8 @@ public class DataStorage extends Storage {
     // Format and recover.
     // Format and recover.
     this.storageDirs = new ArrayList<StorageDirectory>(dataDirs.size());
     this.storageDirs = new ArrayList<StorageDirectory>(dataDirs.size());
     ArrayList<StorageState> dataDirStates = new ArrayList<StorageState>(dataDirs.size());
     ArrayList<StorageState> dataDirStates = new ArrayList<StorageState>(dataDirs.size());
-    for(Iterator<File> it = dataDirs.iterator(); it.hasNext();) {
-      File dataDir = it.next();
+    for(Iterator<StorageLocation> it = dataDirs.iterator(); it.hasNext();) {
+      File dataDir = it.next().getFile();
       StorageDirectory sd = new StorageDirectory(dataDir);
       StorageDirectory sd = new StorageDirectory(dataDir);
       StorageState curState;
       StorageState curState;
       try {
       try {
@@ -162,7 +159,7 @@ public class DataStorage extends Storage {
         case NOT_FORMATTED: // format
         case NOT_FORMATTED: // format
           LOG.info("Storage directory " + dataDir + " is not formatted");
           LOG.info("Storage directory " + dataDir + " is not formatted");
           LOG.info("Formatting ...");
           LOG.info("Formatting ...");
-          format(sd, nsInfo);
+          format(sd, nsInfo, datanode.getDatanodeUuid());
           break;
           break;
         default:  // recovery part is common
         default:  // recovery part is common
           sd.doRecover(curState);
           sd.doRecover(curState);
@@ -191,11 +188,9 @@ public class DataStorage extends Storage {
       doTransition(datanode, getStorageDir(idx), nsInfo, startOpt);
       doTransition(datanode, getStorageDir(idx), nsInfo, startOpt);
       assert this.getLayoutVersion() == nsInfo.getLayoutVersion() :
       assert this.getLayoutVersion() == nsInfo.getLayoutVersion() :
         "Data-node and name-node layout versions must be the same.";
         "Data-node and name-node layout versions must be the same.";
+      createStorageID(getStorageDir(idx));
     }
     }
     
     
-    // make sure we have storage id set - if not - generate new one
-    createStorageID(datanode.getXferPort());
-    
     // 3. Update all storages. Some of them might have just been formatted.
     // 3. Update all storages. Some of them might have just been formatted.
     this.writeAll();
     this.writeAll();
     
     
@@ -214,14 +209,14 @@ public class DataStorage extends Storage {
    * @throws IOException on error
    * @throws IOException on error
    */
    */
   void recoverTransitionRead(DataNode datanode, String bpID, NamespaceInfo nsInfo,
   void recoverTransitionRead(DataNode datanode, String bpID, NamespaceInfo nsInfo,
-      Collection<File> dataDirs, StartupOption startOpt) throws IOException {
+      Collection<StorageLocation> dataDirs, StartupOption startOpt) throws IOException {
     // First ensure datanode level format/snapshot/rollback is completed
     // First ensure datanode level format/snapshot/rollback is completed
     recoverTransitionRead(datanode, nsInfo, dataDirs, startOpt);
     recoverTransitionRead(datanode, nsInfo, dataDirs, startOpt);
-    
+
     // Create list of storage directories for the block pool
     // Create list of storage directories for the block pool
     Collection<File> bpDataDirs = new ArrayList<File>();
     Collection<File> bpDataDirs = new ArrayList<File>();
-    for(Iterator<File> it = dataDirs.iterator(); it.hasNext();) {
-      File dnRoot = it.next();
+    for(StorageLocation dir : dataDirs) {
+      File dnRoot = dir.getFile();
       File bpRoot = BlockPoolSliceStorage.getBpRoot(bpID, new File(dnRoot,
       File bpRoot = BlockPoolSliceStorage.getBpRoot(bpID, new File(dnRoot,
           STORAGE_DIR_CURRENT));
           STORAGE_DIR_CURRENT));
       bpDataDirs.add(bpRoot);
       bpDataDirs.add(bpRoot);
@@ -263,19 +258,28 @@ public class DataStorage extends Storage {
     }
     }
   }
   }
 
 
-  void format(StorageDirectory sd, NamespaceInfo nsInfo) throws IOException {
+  void format(StorageDirectory sd, NamespaceInfo nsInfo,
+              String datanodeUuid) throws IOException {
     sd.clearDirectory(); // create directory
     sd.clearDirectory(); // create directory
     this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
     this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
     this.clusterID = nsInfo.getClusterID();
     this.clusterID = nsInfo.getClusterID();
     this.namespaceID = nsInfo.getNamespaceID();
     this.namespaceID = nsInfo.getNamespaceID();
     this.cTime = 0;
     this.cTime = 0;
-    // store storageID as it currently is
+    this.datanodeUuid = datanodeUuid;
+
+    if (sd.getStorageUuid() == null) {
+      // Assign a new Storage UUID.
+      sd.setStorageUuid(DatanodeStorage.generateUuid());
+    }
+
     writeProperties(sd);
     writeProperties(sd);
   }
   }
 
 
   /*
   /*
    * Set ClusterID, StorageID, StorageType, CTime into
    * Set ClusterID, StorageID, StorageType, CTime into
-   * DataStorage VERSION file
+   * DataStorage VERSION file.
+   * Always called just before writing the properties to
+   * the VERSION file.
   */
   */
   @Override
   @Override
   protected void setPropertiesFromFields(Properties props, 
   protected void setPropertiesFromFields(Properties props, 
@@ -285,7 +289,13 @@ public class DataStorage extends Storage {
     props.setProperty("clusterID", clusterID);
     props.setProperty("clusterID", clusterID);
     props.setProperty("cTime", String.valueOf(cTime));
     props.setProperty("cTime", String.valueOf(cTime));
     props.setProperty("layoutVersion", String.valueOf(layoutVersion));
     props.setProperty("layoutVersion", String.valueOf(layoutVersion));
-    props.setProperty("storageID", getStorageID());
+    props.setProperty("storageID", sd.getStorageUuid());
+
+    String datanodeUuid = getDatanodeUuid();
+    if (datanodeUuid != null) {
+      props.setProperty("datanodeUuid", datanodeUuid);
+    }
+
     // Set NamespaceID in version before federation
     // Set NamespaceID in version before federation
     if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
     if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
       props.setProperty("namespaceID", String.valueOf(namespaceID));
       props.setProperty("namespaceID", String.valueOf(namespaceID));
@@ -295,6 +305,7 @@ public class DataStorage extends Storage {
   /*
   /*
    * Read ClusterID, StorageID, StorageType, CTime from 
    * Read ClusterID, StorageID, StorageType, CTime from 
    * DataStorage VERSION file and verify them.
    * DataStorage VERSION file and verify them.
+   * Always called just after reading the properties from the VERSION file.
    */
    */
   @Override
   @Override
   protected void setFieldsFromProperties(Properties props, StorageDirectory sd)
   protected void setFieldsFromProperties(Properties props, StorageDirectory sd)
@@ -318,20 +329,36 @@ public class DataStorage extends Storage {
       setNamespaceID(props, sd);
       setNamespaceID(props, sd);
     }
     }
     
     
+
     // valid storage id, storage id may be empty
     // valid storage id, storage id may be empty
     String ssid = props.getProperty("storageID");
     String ssid = props.getProperty("storageID");
     if (ssid == null) {
     if (ssid == null) {
       throw new InconsistentFSStateException(sd.getRoot(), "file "
       throw new InconsistentFSStateException(sd.getRoot(), "file "
           + STORAGE_FILE_VERSION + " is invalid.");
           + STORAGE_FILE_VERSION + " is invalid.");
     }
     }
-    String sid = getStorageID();
-    if (!(sid.equals("") || ssid.equals("") || sid.equals(ssid))) {
+    String sid = sd.getStorageUuid();
+    if (!(sid == null || sid.equals("") ||
+          ssid.equals("") || sid.equals(ssid))) {
       throw new InconsistentFSStateException(sd.getRoot(),
       throw new InconsistentFSStateException(sd.getRoot(),
           "has incompatible storage Id.");
           "has incompatible storage Id.");
     }
     }
-    
-    if (sid.equals("")) { // update id only if it was empty
-      setStorageID(ssid);
+
+    if (sid == null) { // update id only if it was null
+      sd.setStorageUuid(ssid);
+    }
+
+    // Update the datanode UUID if present.
+    if (props.getProperty("datanodeUuid") != null) {
+      String dnUuid = props.getProperty("datanodeUuid");
+
+      if (getDatanodeUuid() == null) {
+        setDatanodeUuid(dnUuid);
+      } else if (getDatanodeUuid().compareTo(dnUuid) != 0) {
+        throw new InconsistentFSStateException(sd.getRoot(),
+            "Root " + sd.getRoot() + ": DatanodeUuid=" + dnUuid +
+            ", does not match " + getDatanodeUuid() + " from other" +
+            " StorageDirectory.");
+      }
     }
     }
   }
   }
 
 

+ 11 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -284,7 +284,7 @@ class DataXceiver extends Receiver implements Runnable {
         BlockSender.ClientTraceLog.info(String.format(
         BlockSender.ClientTraceLog.info(String.format(
             "src: 127.0.0.1, dest: 127.0.0.1, op: REQUEST_SHORT_CIRCUIT_FDS," +
             "src: 127.0.0.1, dest: 127.0.0.1, op: REQUEST_SHORT_CIRCUIT_FDS," +
             " blockid: %s, srvID: %s, success: %b",
             " blockid: %s, srvID: %s, success: %b",
-            blk.getBlockId(), dnR.getStorageID(), (fis != null)
+            blk.getBlockId(), dnR.getDatanodeUuid(), (fis != null)
           ));
           ));
       }
       }
       if (fis != null) {
       if (fis != null) {
@@ -317,7 +317,7 @@ class DataXceiver extends Receiver implements Runnable {
       clientName.length() > 0 && ClientTraceLog.isInfoEnabled()
       clientName.length() > 0 && ClientTraceLog.isInfoEnabled()
         ? String.format(DN_CLIENTTRACE_FORMAT, localAddress, remoteAddress,
         ? String.format(DN_CLIENTTRACE_FORMAT, localAddress, remoteAddress,
             "%d", "HDFS_READ", clientName, "%d",
             "%d", "HDFS_READ", clientName, "%d",
-            dnR.getStorageID(), block, "%d")
+            dnR.getDatanodeUuid(), block, "%d")
         : dnR + " Served block " + block + " to " +
         : dnR + " Served block " + block + " to " +
             remoteAddress;
             remoteAddress;
 
 
@@ -447,6 +447,7 @@ class DataXceiver extends Receiver implements Runnable {
     String mirrorNode = null;           // the name:port of next target
     String mirrorNode = null;           // the name:port of next target
     String firstBadLink = "";           // first datanode that failed in connection setup
     String firstBadLink = "";           // first datanode that failed in connection setup
     Status mirrorInStatus = SUCCESS;
     Status mirrorInStatus = SUCCESS;
+    final String storageUuid;
     try {
     try {
       if (isDatanode || 
       if (isDatanode || 
           stage != BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
           stage != BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
@@ -457,8 +458,10 @@ class DataXceiver extends Receiver implements Runnable {
             stage, latestGenerationStamp, minBytesRcvd, maxBytesRcvd,
             stage, latestGenerationStamp, minBytesRcvd, maxBytesRcvd,
             clientname, srcDataNode, datanode, requestedChecksum,
             clientname, srcDataNode, datanode, requestedChecksum,
             cachingStrategy);
             cachingStrategy);
+        storageUuid = blockReceiver.getStorageUuid();
       } else {
       } else {
-        datanode.data.recoverClose(block, latestGenerationStamp, minBytesRcvd);
+        storageUuid = datanode.data.recoverClose(
+            block, latestGenerationStamp, minBytesRcvd);
       }
       }
 
 
       //
       //
@@ -590,7 +593,7 @@ class DataXceiver extends Receiver implements Runnable {
       // the block is finalized in the PacketResponder.
       // the block is finalized in the PacketResponder.
       if (isDatanode ||
       if (isDatanode ||
           stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
           stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
-        datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT);
+        datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT, storageUuid);
         LOG.info("Received " + block + " src: " + remoteAddress + " dest: "
         LOG.info("Received " + block + " src: " + remoteAddress + " dest: "
             + localAddress + " of size " + block.getNumBytes());
             + localAddress + " of size " + block.getNumBytes());
       }
       }
@@ -859,9 +862,11 @@ class DataXceiver extends Receiver implements Runnable {
           dataXceiverServer.balanceThrottler, null);
           dataXceiverServer.balanceThrottler, null);
                     
                     
       // notify name node
       // notify name node
-      datanode.notifyNamenodeReceivedBlock(block, delHint);
+      datanode.notifyNamenodeReceivedBlock(
+          block, delHint, blockReceiver.getStorageUuid());
 
 
-      LOG.info("Moved " + block + " from " + peer.getRemoteAddressString());
+      LOG.info("Moved " + block + " from " + peer.getRemoteAddressString()
+          + ", delHint=" + delHint);
       
       
     } catch (IOException ioe) {
     } catch (IOException ioe) {
       opStatus = ERROR;
       opStatus = ERROR;

+ 10 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.LinkedList;
@@ -230,10 +229,6 @@ public class DirectoryScanner implements Runnable {
       throw new RuntimeException(prefix + " is not a prefix of " + fullPath);
       throw new RuntimeException(prefix + " is not a prefix of " + fullPath);
     }
     }
 
 
-    ScanInfo(long blockId) {
-      this(blockId, null, null, null);
-    }
-
     ScanInfo(long blockId, File blockFile, File metaFile, FsVolumeSpi vol) {
     ScanInfo(long blockId, File blockFile, File metaFile, FsVolumeSpi vol) {
       this.blockId = blockId;
       this.blockId = blockId;
       String condensedVolPath = vol == null ? null :
       String condensedVolPath = vol == null ? null :
@@ -439,8 +434,8 @@ public class DirectoryScanner implements Runnable {
         diffs.put(bpid, diffRecord);
         diffs.put(bpid, diffRecord);
         
         
         statsRecord.totalBlocks = blockpoolReport.length;
         statsRecord.totalBlocks = blockpoolReport.length;
-        List<Block> bl = dataset.getFinalizedBlocks(bpid);
-        Block[] memReport = bl.toArray(new Block[bl.size()]);
+        List<FinalizedReplica> bl = dataset.getFinalizedBlocks(bpid);
+        FinalizedReplica[] memReport = bl.toArray(new FinalizedReplica[bl.size()]);
         Arrays.sort(memReport); // Sort based on blockId
         Arrays.sort(memReport); // Sort based on blockId
   
   
         int d = 0; // index for blockpoolReport
         int d = 0; // index for blockpoolReport
@@ -458,7 +453,8 @@ public class DirectoryScanner implements Runnable {
           }
           }
           if (info.getBlockId() > memBlock.getBlockId()) {
           if (info.getBlockId() > memBlock.getBlockId()) {
             // Block is missing on the disk
             // Block is missing on the disk
-            addDifference(diffRecord, statsRecord, memBlock.getBlockId());
+            addDifference(diffRecord, statsRecord,
+                          memBlock.getBlockId(), info.getVolume());
             m++;
             m++;
             continue;
             continue;
           }
           }
@@ -478,7 +474,9 @@ public class DirectoryScanner implements Runnable {
           m++;
           m++;
         }
         }
         while (m < memReport.length) {
         while (m < memReport.length) {
-          addDifference(diffRecord, statsRecord, memReport[m++].getBlockId());
+          FinalizedReplica current = memReport[m++];
+          addDifference(diffRecord, statsRecord,
+                        current.getBlockId(), current.getVolume());
         }
         }
         while (d < blockpoolReport.length) {
         while (d < blockpoolReport.length) {
           statsRecord.missingMemoryBlocks++;
           statsRecord.missingMemoryBlocks++;
@@ -502,10 +500,11 @@ public class DirectoryScanner implements Runnable {
 
 
   /** Block is not found on the disk */
   /** Block is not found on the disk */
   private void addDifference(LinkedList<ScanInfo> diffRecord,
   private void addDifference(LinkedList<ScanInfo> diffRecord,
-                             Stats statsRecord, long blockId) {
+                             Stats statsRecord, long blockId,
+                             FsVolumeSpi vol) {
     statsRecord.missingBlockFile++;
     statsRecord.missingBlockFile++;
     statsRecord.missingMetaFile++;
     statsRecord.missingMetaFile++;
-    diffRecord.add(new ScanInfo(blockId));
+    diffRecord.add(new ScanInfo(blockId, null, null, vol));
   }
   }
 
 
   /** Is the given volume still valid in the dataset? */
   /** Is the given volume still valid in the dataset? */

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/Replica.java

@@ -54,4 +54,9 @@ public interface Replica {
    * @return the number of bytes that are visible to readers
    * @return the number of bytes that are visible to readers
    */
    */
   public long getVisibleLength();
   public long getVisibleLength();
+
+  /**
+   * Return the storageUuid of the volume that stores this replica.
+   */
+  public String getStorageUuid();
 }
 }

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java

@@ -137,6 +137,14 @@ abstract public class ReplicaInfo extends Block implements Replica {
   void setVolume(FsVolumeSpi vol) {
   void setVolume(FsVolumeSpi vol) {
     this.volume = vol;
     this.volume = vol;
   }
   }
+
+  /**
+   * Get the storageUuid of the volume that stores this replica.
+   */
+  @Override
+  public String getStorageUuid() {
+    return volume.getStorageID();
+  }
   
   
   /**
   /**
    * Return the parent directory path where this replica is located
    * Return the parent directory path where this replica is located

+ 101 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java

@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.util.regex.Pattern;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.regex.Matcher;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.server.common.Util;
+
+/**
+ * Encapsulates the URI and storage medium that together describe a
+ * storage directory.
+ * The default storage medium is assumed to be DISK, if none is specified.
+ *
+ */
+@InterfaceAudience.Private
+public class StorageLocation {
+  final StorageType storageType;
+  final File file;
+
+  /** Regular expression that describes a storage uri with a storage type.
+   *  e.g. [Disk]/storages/storage1/
+   */
+  private static final Pattern regex = Pattern.compile("^\\[(\\w*)\\](.+)$");
+
+  private StorageLocation(StorageType storageType, URI uri) {
+    this.storageType = storageType;
+
+    if (uri.getScheme() == null ||
+        "file".equalsIgnoreCase(uri.getScheme())) {
+      // drop any (illegal) authority in the URI for backwards compatibility
+      this.file = new File(uri.getPath());
+    } else {
+      throw new IllegalArgumentException("Unsupported URI schema in " + uri);
+    }
+  }
+
+  public StorageType getStorageType() {
+    return this.storageType;
+  }
+
+  URI getUri() {
+    return file.toURI();
+  }
+
+  public File getFile() {
+    return this.file;
+  }
+
+  /**
+   * Attempt to parse a storage uri with storage class and URI. The storage
+   * class component of the uri is case-insensitive.
+   *
+   * @param rawLocation Location string of the format [type]uri, where [type] is
+   *                    optional.
+   * @return A StorageLocation object if successfully parsed, null otherwise.
+   *         Does not throw any exceptions.
+   */
+  static StorageLocation parse(String rawLocation) throws IOException {
+    Matcher matcher = regex.matcher(rawLocation);
+    StorageType storageType = StorageType.DEFAULT;
+    String location = rawLocation;
+
+    if (matcher.matches()) {
+      String classString = matcher.group(1);
+      location = matcher.group(2);
+      if (!classString.isEmpty()) {
+        storageType = StorageType.valueOf(classString.toUpperCase());
+      }
+    }
+
+    return new StorageLocation(storageType, Util.stringAsURI(location));
+  }
+
+  @Override
+  public String toString() {
+    return "[" + storageType + "]" + file.toURI();
+  }
+}

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

@@ -34,12 +34,15 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.Replica;
 import org.apache.hadoop.hdfs.server.datanode.Replica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 
 
@@ -86,17 +89,18 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   /** @return a list of volumes. */
   /** @return a list of volumes. */
   public List<V> getVolumes();
   public List<V> getVolumes();
 
 
+  /** @return one or more storage reports for attached volumes. */
+  public StorageReport[] getStorageReports(String bpid)
+      throws IOException;
+
   /** @return the volume that contains a replica of the block. */
   /** @return the volume that contains a replica of the block. */
   public V getVolume(ExtendedBlock b);
   public V getVolume(ExtendedBlock b);
 
 
   /** @return a volume information map (name => info). */
   /** @return a volume information map (name => info). */
   public Map<String, Object> getVolumeInfoMap();
   public Map<String, Object> getVolumeInfoMap();
 
 
-  /** @return a list of block pools. */
-  public String[] getBlockPoolList();
-
   /** @return a list of finalized blocks for the given block pool. */
   /** @return a list of finalized blocks for the given block pool. */
-  public List<Block> getFinalizedBlocks(String bpid);
+  public List<FinalizedReplica> getFinalizedBlocks(String bpid);
 
 
   /**
   /**
    * Check whether the in-memory block record matches the block on the disk,
    * Check whether the in-memory block record matches the block on the disk,
@@ -239,9 +243,10 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * @param b block
    * @param b block
    * @param newGS the new generation stamp for the replica
    * @param newGS the new generation stamp for the replica
    * @param expectedBlockLen the number of bytes the replica is expected to have
    * @param expectedBlockLen the number of bytes the replica is expected to have
+   * @return the storage uuid of the replica.
    * @throws IOException
    * @throws IOException
    */
    */
-  public void recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
+  public String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
       ) throws IOException;
       ) throws IOException;
   
   
   /**
   /**
@@ -262,12 +267,11 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   public void unfinalizeBlock(ExtendedBlock b) throws IOException;
   public void unfinalizeBlock(ExtendedBlock b) throws IOException;
 
 
   /**
   /**
-   * Returns the block report - the full list of blocks stored under a 
-   * block pool
+   * Returns one block report per volume.
    * @param bpid Block Pool Id
    * @param bpid Block Pool Id
-   * @return - the block report - the full list of blocks stored
+   * @return - a map of DatanodeStorage to block report for the volume.
    */
    */
-  public BlockListAsLongs getBlockReport(String bpid);
+  public Map<DatanodeStorage, BlockListAsLongs> getBlockReports(String bpid);
 
 
   /**
   /**
    * Returns the cache report - the full list of cached block IDs of a
    * Returns the cache report - the full list of cached block IDs of a
@@ -408,3 +412,4 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)
   public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)
       throws IOException;
       throws IOException;
 }
 }
+

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

@@ -20,10 +20,15 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset;
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 
 
+import org.apache.hadoop.hdfs.StorageType;
+
 /**
 /**
  * This is an interface for the underlying volume.
  * This is an interface for the underlying volume.
  */
  */
 public interface FsVolumeSpi {
 public interface FsVolumeSpi {
+  /** @return the StorageUuid of the volume */
+  public String getStorageID();
+
   /** @return a list of block pools. */
   /** @return a list of block pools. */
   public String[] getBlockPoolList();
   public String[] getBlockPoolList();
 
 
@@ -38,4 +43,6 @@ public interface FsVolumeSpi {
 
 
   /** @return the directory for the finalized blocks in the block pool. */
   /** @return the directory for the finalized blocks in the block pool. */
   public File getFinalizedDir(String bpid) throws IOException;
   public File getFinalizedDir(String bpid) throws IOException;
+  
+  public StorageType getStorageType();
 }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java

@@ -195,7 +195,7 @@ class FsDatasetAsyncDiskService {
             + " at file " + blockFile + ". Ignored.");
             + " at file " + blockFile + ". Ignored.");
       } else {
       } else {
         if(block.getLocalBlock().getNumBytes() != BlockCommand.NO_ACK){
         if(block.getLocalBlock().getNumBytes() != BlockCommand.NO_ACK){
-          datanode.notifyNamenodeDeletedBlock(block);
+          datanode.notifyNamenodeDeletedBlock(block, volume.getStorageID());
         }
         }
         volume.decDfsUsed(block.getBlockPoolId(), dfsBytes);
         volume.decDfsUsed(block.getBlockPoolId(), dfsBytes);
         LOG.info("Deleted " + block.getBlockPoolId() + " "
         LOG.info("Deleted " + block.getBlockPoolId() + " "

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

@@ -38,13 +38,14 @@ import javax.management.NotCompliantMBeanException;
 import javax.management.ObjectName;
 import javax.management.ObjectName;
 import javax.management.StandardMBean;
 import javax.management.StandardMBean;
 
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
@@ -53,6 +54,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -66,6 +68,7 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
@@ -76,7 +79,9 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosing
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum;
@@ -108,6 +113,26 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     return volumes.volumes;
     return volumes.volumes;
   }
   }
 
 
+  @Override // FsDatasetSpi
+  public StorageReport[] getStorageReports(String bpid)
+      throws IOException {
+    StorageReport[] reports;
+    synchronized (statsLock) {
+      reports = new StorageReport[volumes.volumes.size()];
+      int i = 0;
+      for (FsVolumeImpl volume : volumes.volumes) {
+        reports[i++] = new StorageReport(volume.getStorageID(),
+                                         false,
+                                         volume.getCapacity(),
+                                         volume.getDfsUsed(),
+                                         volume.getAvailable(),
+                                         volume.getBlockPoolUsed(bpid));
+      }
+    }
+
+    return reports;
+  }
+
   @Override
   @Override
   public synchronized FsVolumeImpl getVolume(final ExtendedBlock b) {
   public synchronized FsVolumeImpl getVolume(final ExtendedBlock b) {
     final ReplicaInfo r =  volumeMap.get(b.getBlockPoolId(), b.getLocalBlock());
     final ReplicaInfo r =  volumeMap.get(b.getBlockPoolId(), b.getLocalBlock());
@@ -169,11 +194,12 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     
     
   final DataNode datanode;
   final DataNode datanode;
   final FsVolumeList volumes;
   final FsVolumeList volumes;
-  final ReplicaMap volumeMap;
   final FsDatasetAsyncDiskService asyncDiskService;
   final FsDatasetAsyncDiskService asyncDiskService;
   final FsDatasetCache cacheManager;
   final FsDatasetCache cacheManager;
   private final int validVolsRequired;
   private final int validVolsRequired;
 
 
+  final ReplicaMap volumeMap;
+
   // Used for synchronizing access to usage stats
   // Used for synchronizing access to usage stats
   private final Object statsLock = new Object();
   private final Object statsLock = new Object();
 
 
@@ -190,6 +216,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
                   DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
                   DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
 
 
     String[] dataDirs = conf.getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
     String[] dataDirs = conf.getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
+    Collection<StorageLocation> dataLocations = DataNode.getStorageLocations(conf);
 
 
     int volsConfigured = (dataDirs == null) ? 0 : dataDirs.length;
     int volsConfigured = (dataDirs == null) ? 0 : dataDirs.length;
     int volsFailed = volsConfigured - storage.getNumStorageDirs();
     int volsFailed = volsConfigured - storage.getNumStorageDirs();
@@ -210,9 +237,15 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     final List<FsVolumeImpl> volArray = new ArrayList<FsVolumeImpl>(
     final List<FsVolumeImpl> volArray = new ArrayList<FsVolumeImpl>(
         storage.getNumStorageDirs());
         storage.getNumStorageDirs());
     for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
     for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
-      final File dir = storage.getStorageDir(idx).getCurrentDir();
-      volArray.add(new FsVolumeImpl(this, storage.getStorageID(), dir, conf));
-      LOG.info("Added volume - " + dir);
+      // TODO: getStorageTypeFromLocations() is only a temporary workaround and 
+      // should be replaced with getting storage type from DataStorage (missing 
+      // storage type now) directly.
+      Storage.StorageDirectory sd = storage.getStorageDir(idx);
+      final File dir = sd.getCurrentDir();
+      final StorageType storageType = getStorageTypeFromLocations(dataLocations, dir);
+      volArray.add(new FsVolumeImpl(this, sd.getStorageUuid(), dir, conf,
+          storageType));
+      LOG.info("Added volume - " + dir + ", StorageType: " + storageType);
     }
     }
     volumeMap = new ReplicaMap(this);
     volumeMap = new ReplicaMap(this);
 
 
@@ -223,7 +256,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
             RoundRobinVolumeChoosingPolicy.class,
             RoundRobinVolumeChoosingPolicy.class,
             VolumeChoosingPolicy.class), conf);
             VolumeChoosingPolicy.class), conf);
     volumes = new FsVolumeList(volArray, volsFailed, blockChooserImpl);
     volumes = new FsVolumeList(volArray, volsFailed, blockChooserImpl);
-    volumes.getVolumeMap(volumeMap);
+    volumes.initializeReplicaMaps(volumeMap);
 
 
     File[] roots = new File[storage.getNumStorageDirs()];
     File[] roots = new File[storage.getNumStorageDirs()];
     for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
     for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
@@ -231,7 +264,17 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     }
     }
     asyncDiskService = new FsDatasetAsyncDiskService(datanode, roots);
     asyncDiskService = new FsDatasetAsyncDiskService(datanode, roots);
     cacheManager = new FsDatasetCache(this);
     cacheManager = new FsDatasetCache(this);
-    registerMBean(storage.getStorageID());
+    registerMBean(datanode.getDatanodeUuid());
+  }
+
+  private StorageType getStorageTypeFromLocations(
+      Collection<StorageLocation> dataLocations, File dir) {
+    for (StorageLocation dataLocation : dataLocations) {
+      if (dataLocation.getFile().equals(dir)) {
+        return dataLocation.getStorageType();
+      }
+    }
+    return StorageType.DEFAULT;
   }
   }
 
 
   /**
   /**
@@ -336,9 +379,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   File getBlockFile(String bpid, Block b) throws IOException {
   File getBlockFile(String bpid, Block b) throws IOException {
     File f = validateBlockFile(bpid, b);
     File f = validateBlockFile(bpid, b);
     if(f == null) {
     if(f == null) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("b=" + b + ", volumeMap=" + volumeMap);
-      }
       throw new IOException("Block " + b + " is not valid.");
       throw new IOException("Block " + b + " is not valid.");
     }
     }
     return f;
     return f;
@@ -684,7 +724,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   }
   }
 
 
   @Override // FsDatasetSpi
   @Override // FsDatasetSpi
-  public void recoverClose(ExtendedBlock b, long newGS,
+  public String recoverClose(ExtendedBlock b, long newGS,
       long expectedBlockLen) throws IOException {
       long expectedBlockLen) throws IOException {
     LOG.info("Recover failed close " + b);
     LOG.info("Recover failed close " + b);
     // check replica's state
     // check replica's state
@@ -695,6 +735,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     if (replicaInfo.getState() == ReplicaState.RBW) {
     if (replicaInfo.getState() == ReplicaState.RBW) {
       finalizeReplica(b.getBlockPoolId(), replicaInfo);
       finalizeReplica(b.getBlockPoolId(), replicaInfo);
     }
     }
+    return replicaInfo.getStorageUuid();
   }
   }
   
   
   /**
   /**
@@ -995,56 +1036,68 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     return true;
     return true;
   }
   }
 
 
-  /**
-   * Generates a block report from the in-memory block map.
-   */
   @Override // FsDatasetSpi
   @Override // FsDatasetSpi
-  public BlockListAsLongs getBlockReport(String bpid) {
-    int size =  volumeMap.size(bpid);
-    ArrayList<ReplicaInfo> finalized = new ArrayList<ReplicaInfo>(size);
-    ArrayList<ReplicaInfo> uc = new ArrayList<ReplicaInfo>();
-    if (size == 0) {
-      return new BlockListAsLongs(finalized, uc);
+  public List<Long> getCacheReport(String bpid) {
+    return cacheManager.getCachedBlocks(bpid);
+  }
+
+  @Override
+  public Map<DatanodeStorage, BlockListAsLongs> getBlockReports(String bpid) {
+    Map<DatanodeStorage, BlockListAsLongs> blockReportsMap =
+        new HashMap<DatanodeStorage, BlockListAsLongs>();
+
+    Map<String, ArrayList<ReplicaInfo>> finalized =
+        new HashMap<String, ArrayList<ReplicaInfo>>();
+    Map<String, ArrayList<ReplicaInfo>> uc =
+        new HashMap<String, ArrayList<ReplicaInfo>>();
+
+    for (FsVolumeSpi v : volumes.volumes) {
+      finalized.put(v.getStorageID(), new ArrayList<ReplicaInfo>());
+      uc.put(v.getStorageID(), new ArrayList<ReplicaInfo>());
     }
     }
-    
+
     synchronized(this) {
     synchronized(this) {
       for (ReplicaInfo b : volumeMap.replicas(bpid)) {
       for (ReplicaInfo b : volumeMap.replicas(bpid)) {
         switch(b.getState()) {
         switch(b.getState()) {
-        case FINALIZED:
-          finalized.add(b);
-          break;
-        case RBW:
-        case RWR:
-          uc.add(b);
-          break;
-        case RUR:
-          ReplicaUnderRecovery rur = (ReplicaUnderRecovery)b;
-          uc.add(rur.getOriginalReplica());
-          break;
-        case TEMPORARY:
-          break;
-        default:
-          assert false : "Illegal ReplicaInfo state.";
+          case FINALIZED:
+            finalized.get(b.getVolume().getStorageID()).add(b);
+            break;
+          case RBW:
+          case RWR:
+            uc.get(b.getVolume().getStorageID()).add(b);
+            break;
+          case RUR:
+            ReplicaUnderRecovery rur = (ReplicaUnderRecovery)b;
+            uc.get(rur.getVolume().getStorageID()).add(rur.getOriginalReplica());
+            break;
+          case TEMPORARY:
+            break;
+          default:
+            assert false : "Illegal ReplicaInfo state.";
         }
         }
       }
       }
-      return new BlockListAsLongs(finalized, uc);
     }
     }
-  }
 
 
-  @Override // FsDatasetSpi
-  public List<Long> getCacheReport(String bpid) {
-    return cacheManager.getCachedBlocks(bpid);
+    for (FsVolumeImpl v : volumes.volumes) {
+      ArrayList<ReplicaInfo> finalizedList = finalized.get(v.getStorageID());
+      ArrayList<ReplicaInfo> ucList = uc.get(v.getStorageID());
+      blockReportsMap.put(v.toDatanodeStorage(),
+                          new BlockListAsLongs(finalizedList, ucList));
+    }
+
+    return blockReportsMap;
   }
   }
 
 
   /**
   /**
    * Get the list of finalized blocks from in-memory blockmap for a block pool.
    * Get the list of finalized blocks from in-memory blockmap for a block pool.
    */
    */
   @Override
   @Override
-  public synchronized List<Block> getFinalizedBlocks(String bpid) {
-    ArrayList<Block> finalized = new ArrayList<Block>(volumeMap.size(bpid));
+  public synchronized List<FinalizedReplica> getFinalizedBlocks(String bpid) {
+    ArrayList<FinalizedReplica> finalized =
+        new ArrayList<FinalizedReplica>(volumeMap.size(bpid));
     for (ReplicaInfo b : volumeMap.replicas(bpid)) {
     for (ReplicaInfo b : volumeMap.replicas(bpid)) {
       if(b.getState() == ReplicaState.FINALIZED) {
       if(b.getState() == ReplicaState.FINALIZED) {
-        finalized.add(new Block(b));
+        finalized.add(new FinalizedReplica((FinalizedReplica)b));
       }
       }
     }
     }
     return finalized;
     return finalized;
@@ -1333,22 +1386,15 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   
   
   /**
   /**
    * Register the FSDataset MBean using the name
    * Register the FSDataset MBean using the name
-   *        "hadoop:service=DataNode,name=FSDatasetState-<storageid>"
+   *        "hadoop:service=DataNode,name=FSDatasetState-<datanodeUuid>"
    */
    */
-  void registerMBean(final String storageId) {
+  void registerMBean(final String datanodeUuid) {
     // We wrap to bypass standard mbean naming convetion.
     // We wrap to bypass standard mbean naming convetion.
     // This wraping can be removed in java 6 as it is more flexible in 
     // This wraping can be removed in java 6 as it is more flexible in 
     // package naming for mbeans and their impl.
     // package naming for mbeans and their impl.
-    StandardMBean bean;
-    String storageName;
-    if (storageId == null || storageId.equals("")) {// Temp fix for the uninitialized storage
-      storageName = "UndefinedStorageId" + DFSUtil.getRandom().nextInt();
-    } else {
-      storageName = storageId;
-    }
     try {
     try {
-      bean = new StandardMBean(this,FSDatasetMBean.class);
-      mbeanName = MBeans.register("DataNode", "FSDatasetState-" + storageName, bean);
+      StandardMBean bean = new StandardMBean(this,FSDatasetMBean.class);
+      mbeanName = MBeans.register("DataNode", "FSDatasetState-" + datanodeUuid, bean);
     } catch (NotCompliantMBeanException e) {
     } catch (NotCompliantMBeanException e) {
       LOG.warn("Error registering FSDatasetState MBean", e);
       LOG.warn("Error registering FSDatasetState MBean", e);
     }
     }
@@ -1724,7 +1770,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     LOG.info("Adding block pool " + bpid);
     LOG.info("Adding block pool " + bpid);
     volumes.addBlockPool(bpid, conf);
     volumes.addBlockPool(bpid, conf);
     volumeMap.initBlockPool(bpid);
     volumeMap.initBlockPool(bpid);
-    volumes.getVolumeMap(bpid, volumeMap);
+    volumes.getAllVolumesMap(bpid, volumeMap);
   }
   }
 
 
   @Override
   @Override
@@ -1734,11 +1780,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     volumes.removeBlockPool(bpid);
     volumes.removeBlockPool(bpid);
   }
   }
   
   
-  @Override
-  public String[] getBlockPoolList() {
-    return volumeMap.getBlockPoolList();
-  }
-  
   /**
   /**
    * Class for representing the Datanode volume information
    * Class for representing the Datanode volume information
    */
    */
@@ -1871,3 +1912,4 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     return new RollingLogsImpl(dir, prefix);
     return new RollingLogsImpl(dir, prefix);
   }
   }
 }
 }
+

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

@@ -34,9 +34,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -50,6 +52,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 class FsVolumeImpl implements FsVolumeSpi {
 class FsVolumeImpl implements FsVolumeSpi {
   private final FsDatasetImpl dataset;
   private final FsDatasetImpl dataset;
   private final String storageID;
   private final String storageID;
+  private final StorageType storageType;
   private final Map<String, BlockPoolSlice> bpSlices
   private final Map<String, BlockPoolSlice> bpSlices
       = new HashMap<String, BlockPoolSlice>();
       = new HashMap<String, BlockPoolSlice>();
   private final File currentDir;    // <StorageDirectory>/current
   private final File currentDir;    // <StorageDirectory>/current
@@ -64,7 +67,7 @@ class FsVolumeImpl implements FsVolumeSpi {
   private final ThreadPoolExecutor cacheExecutor;
   private final ThreadPoolExecutor cacheExecutor;
   
   
   FsVolumeImpl(FsDatasetImpl dataset, String storageID, File currentDir,
   FsVolumeImpl(FsDatasetImpl dataset, String storageID, File currentDir,
-      Configuration conf) throws IOException {
+      Configuration conf, StorageType storageType) throws IOException {
     this.dataset = dataset;
     this.dataset = dataset;
     this.storageID = storageID;
     this.storageID = storageID;
     this.reserved = conf.getLong(
     this.reserved = conf.getLong(
@@ -73,6 +76,7 @@ class FsVolumeImpl implements FsVolumeSpi {
     this.currentDir = currentDir; 
     this.currentDir = currentDir; 
     File parent = currentDir.getParentFile();
     File parent = currentDir.getParentFile();
     this.usage = new DF(parent, conf);
     this.usage = new DF(parent, conf);
+    this.storageType = storageType;
     final int maxNumThreads = dataset.datanode.getConf().getInt(
     final int maxNumThreads = dataset.datanode.getConf().getInt(
         DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_KEY,
         DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_KEY,
         DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_DEFAULT
         DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_DEFAULT
@@ -320,7 +324,19 @@ class FsVolumeImpl implements FsVolumeSpi {
     }
     }
   }
   }
 
 
-  String getStorageID() {
+  @Override
+  public String getStorageID() {
     return storageID;
     return storageID;
   }
   }
+  
+  @Override
+  public StorageType getStorageType() {
+    return storageType;
+  }
+  
+  DatanodeStorage toDatanodeStorage() {
+    return new DatanodeStorage(storageID, DatanodeStorage.State.NORMAL, storageType);
+  }
+
 }
 }
+

+ 17 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java

@@ -18,10 +18,7 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
+import java.util.*;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -56,6 +53,7 @@ class FsVolumeList {
    * @param blockSize free space needed on the volume
    * @param blockSize free space needed on the volume
    * @return next volume to store the block in.
    * @return next volume to store the block in.
    */
    */
+  // TODO should choose volume with storage type
   synchronized FsVolumeImpl getNextVolume(long blockSize) throws IOException {
   synchronized FsVolumeImpl getNextVolume(long blockSize) throws IOException {
     return blockChooser.chooseVolume(volumes, blockSize);
     return blockChooser.chooseVolume(volumes, blockSize);
   }
   }
@@ -92,27 +90,32 @@ class FsVolumeList {
     return remaining;
     return remaining;
   }
   }
     
     
-  void getVolumeMap(ReplicaMap volumeMap) throws IOException {
+  void initializeReplicaMaps(ReplicaMap globalReplicaMap) throws IOException {
     for (FsVolumeImpl v : volumes) {
     for (FsVolumeImpl v : volumes) {
-      v.getVolumeMap(volumeMap);
+      v.getVolumeMap(globalReplicaMap);
     }
     }
   }
   }
   
   
-  void getVolumeMap(String bpid, ReplicaMap volumeMap) throws IOException {
+  void getAllVolumesMap(String bpid, ReplicaMap volumeMap) throws IOException {
     long totalStartTime = System.currentTimeMillis();
     long totalStartTime = System.currentTimeMillis();
     for (FsVolumeImpl v : volumes) {
     for (FsVolumeImpl v : volumes) {
-      FsDatasetImpl.LOG.info("Adding replicas to map for block pool " + bpid +
-          " on volume " + v + "...");
-      long startTime = System.currentTimeMillis();
-      v.getVolumeMap(bpid, volumeMap);
-      long timeTaken = System.currentTimeMillis() - startTime;
-      FsDatasetImpl.LOG.info("Time to add replicas to map for block pool " + bpid +
-          " on volume " + v + ": " + timeTaken + "ms");
+      getVolumeMap(bpid, v, volumeMap);
     }
     }
     long totalTimeTaken = System.currentTimeMillis() - totalStartTime;
     long totalTimeTaken = System.currentTimeMillis() - totalStartTime;
     FsDatasetImpl.LOG.info("Total time to add all replicas to map: "
     FsDatasetImpl.LOG.info("Total time to add all replicas to map: "
         + totalTimeTaken + "ms");
         + totalTimeTaken + "ms");
   }
   }
+
+  void getVolumeMap(String bpid, FsVolumeImpl volume, ReplicaMap volumeMap)
+      throws IOException {
+    FsDatasetImpl.LOG.info("Adding replicas to map for block pool " + bpid +
+                               " on volume " + volume + "...");
+    long startTime = System.currentTimeMillis();
+    volume.getVolumeMap(bpid, volumeMap);
+    long timeTaken = System.currentTimeMillis() - startTime;
+    FsDatasetImpl.LOG.info("Time to add replicas to map for block pool " + bpid +
+                               " on volume " + volume + ": " + timeTaken + "ms");
+  }
     
     
   /**
   /**
    * Calls {@link FsVolumeImpl#checkDirs()} on each volume, removing any
    * Calls {@link FsVolumeImpl#checkDirs()} on each volume, removing any

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

@@ -117,6 +117,13 @@ class ReplicaMap {
       return  m.put(replicaInfo.getBlockId(), replicaInfo);
       return  m.put(replicaInfo.getBlockId(), replicaInfo);
     }
     }
   }
   }
+
+  /**
+   * Add all entries from the given replica map into the local replica map.
+   */
+  void addAll(ReplicaMap other) {
+    map.putAll(other.map);
+  }
   
   
   /**
   /**
    * Remove the replica's meta information from the map that matches
    * Remove the replica's meta information from the map that matches

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

@@ -77,7 +77,7 @@ class BackupJournalManager implements JournalManager {
 
 
   @Override
   @Override
   public void selectInputStreams(Collection<EditLogInputStream> streams,
   public void selectInputStreams(Collection<EditLogInputStream> streams,
-      long fromTxnId, boolean inProgressOk, boolean forReading) {
+      long fromTxnId, boolean inProgressOk) {
     // This JournalManager is never used for input. Therefore it cannot
     // This JournalManager is never used for input. Therefore it cannot
     // return any transactions
     // return any transactions
   }
   }

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

@@ -25,7 +25,7 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 /**
 /**
  * Quota feature for {@link INodeDirectory}. 
  * Quota feature for {@link INodeDirectory}. 
  */
  */
-public final class DirectoryWithQuotaFeature extends INodeDirectory.Feature {
+public final class DirectoryWithQuotaFeature implements INode.Feature {
   public static final long DEFAULT_NAMESPACE_QUOTA = Long.MAX_VALUE;
   public static final long DEFAULT_NAMESPACE_QUOTA = Long.MAX_VALUE;
   public static final long DEFAULT_DISKSPACE_QUOTA = HdfsConstants.QUOTA_RESET;
   public static final long DEFAULT_DISKSPACE_QUOTA = HdfsConstants.QUOTA_RESET;
 
 

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

@@ -62,6 +62,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
@@ -364,7 +365,7 @@ public class FSDirectory implements Closeable {
    * Add a block to the file. Returns a reference to the added block.
    * Add a block to the file. Returns a reference to the added block.
    */
    */
   BlockInfo addBlock(String path, INodesInPath inodesInPath, Block block,
   BlockInfo addBlock(String path, INodesInPath inodesInPath, Block block,
-      DatanodeDescriptor targets[]) throws IOException {
+      DatanodeStorageInfo[] targets) throws IOException {
     waitForReady();
     waitForReady();
 
 
     writeLock();
     writeLock();

+ 6 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -286,7 +286,7 @@ public class FSEditLog implements LogsPurgeable {
     // Safety check: we should never start a segment if there are
     // Safety check: we should never start a segment if there are
     // newer txids readable.
     // newer txids readable.
     List<EditLogInputStream> streams = new ArrayList<EditLogInputStream>();
     List<EditLogInputStream> streams = new ArrayList<EditLogInputStream>();
-    journalSet.selectInputStreams(streams, segmentTxId, true, true);
+    journalSet.selectInputStreams(streams, segmentTxId, true);
     if (!streams.isEmpty()) {
     if (!streams.isEmpty()) {
       String error = String.format("Cannot start writing at txid %s " +
       String error = String.format("Cannot start writing at txid %s " +
         "when there is a stream available for read: %s",
         "when there is a stream available for read: %s",
@@ -1037,7 +1037,7 @@ public class FSEditLog implements LogsPurgeable {
    */
    */
   public synchronized RemoteEditLogManifest getEditLogManifest(long fromTxId)
   public synchronized RemoteEditLogManifest getEditLogManifest(long fromTxId)
       throws IOException {
       throws IOException {
-    return journalSet.getEditLogManifest(fromTxId, true);
+    return journalSet.getEditLogManifest(fromTxId);
   }
   }
  
  
   /**
   /**
@@ -1332,22 +1332,14 @@ public class FSEditLog implements LogsPurgeable {
   
   
   @Override
   @Override
   public void selectInputStreams(Collection<EditLogInputStream> streams,
   public void selectInputStreams(Collection<EditLogInputStream> streams,
-      long fromTxId, boolean inProgressOk, boolean forReading) {
-    journalSet.selectInputStreams(streams, fromTxId, inProgressOk, forReading);
+      long fromTxId, boolean inProgressOk) throws IOException {
+    journalSet.selectInputStreams(streams, fromTxId, inProgressOk);
   }
   }
 
 
   public Collection<EditLogInputStream> selectInputStreams(
   public Collection<EditLogInputStream> selectInputStreams(
       long fromTxId, long toAtLeastTxId) throws IOException {
       long fromTxId, long toAtLeastTxId) throws IOException {
     return selectInputStreams(fromTxId, toAtLeastTxId, null, true);
     return selectInputStreams(fromTxId, toAtLeastTxId, null, true);
   }
   }
-
-  /** Select a list of input streams to load */
-  public Collection<EditLogInputStream> selectInputStreams(
-      long fromTxId, long toAtLeastTxId, MetaRecoveryContext recovery,
-      boolean inProgressOk) throws IOException {
-    return selectInputStreams(fromTxId, toAtLeastTxId, recovery, inProgressOk,
-        true);
-  }
   
   
   /**
   /**
    * Select a list of input streams.
    * Select a list of input streams.
@@ -1355,13 +1347,12 @@ public class FSEditLog implements LogsPurgeable {
    * @param fromTxId first transaction in the selected streams
    * @param fromTxId first transaction in the selected streams
    * @param toAtLeast the selected streams must contain this transaction
    * @param toAtLeast the selected streams must contain this transaction
    * @param inProgessOk set to true if in-progress streams are OK
    * @param inProgessOk set to true if in-progress streams are OK
-   * @param forReading whether or not to use the streams to load the edit log
    */
    */
   public synchronized Collection<EditLogInputStream> selectInputStreams(
   public synchronized Collection<EditLogInputStream> selectInputStreams(
       long fromTxId, long toAtLeastTxId, MetaRecoveryContext recovery,
       long fromTxId, long toAtLeastTxId, MetaRecoveryContext recovery,
-      boolean inProgressOk, boolean forReading) throws IOException {
+      boolean inProgressOk) throws IOException {
     List<EditLogInputStream> streams = new ArrayList<EditLogInputStream>();
     List<EditLogInputStream> streams = new ArrayList<EditLogInputStream>();
-    selectInputStreams(streams, fromTxId, inProgressOk, forReading);
+    selectInputStreams(streams, fromTxId, inProgressOk);
 
 
     try {
     try {
       checkForGaps(streams, fromTxId, toAtLeastTxId, inProgressOk);
       checkForGaps(streams, fromTxId, toAtLeastTxId, inProgressOk);

+ 61 - 68
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -149,6 +149,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
@@ -176,14 +177,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
-import org.apache.hadoop.hdfs.server.blockmanagement.OutOfV1GenerationStampsException;
+import org.apache.hadoop.hdfs.server.blockmanagement.*;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
@@ -220,7 +214,8 @@ import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -2584,7 +2579,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     }
 
 
     // choose targets for the new block to be allocated.
     // choose targets for the new block to be allocated.
-    final DatanodeDescriptor targets[] = getBlockManager().chooseTarget( 
+    final DatanodeStorageInfo targets[] = getBlockManager().chooseTarget( 
         src, replication, clientNode, excludedNodes, blockSize, favoredNodes);
         src, replication, clientNode, excludedNodes, blockSize, favoredNodes);
 
 
     // Part II.
     // Part II.
@@ -2711,7 +2706,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
             src + ". Returning previously allocated block " + lastBlockInFile);
             src + ". Returning previously allocated block " + lastBlockInFile);
         long offset = pendingFile.computeFileSize();
         long offset = pendingFile.computeFileSize();
         onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
         onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
-            ((BlockInfoUnderConstruction)lastBlockInFile).getExpectedLocations(),
+            ((BlockInfoUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
             offset);
             offset);
         return iip;
         return iip;
       } else {
       } else {
@@ -2729,11 +2724,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return iip;
     return iip;
   }
   }
 
 
-  LocatedBlock makeLocatedBlock(Block blk,
-                                        DatanodeInfo[] locs,
+  LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs,
                                         long offset) throws IOException {
                                         long offset) throws IOException {
     LocatedBlock lBlk = new LocatedBlock(
     LocatedBlock lBlk = new LocatedBlock(
-        getExtendedBlock(blk), locs, offset);
+        getExtendedBlock(blk), locs, offset, false);
     getBlockManager().setBlockToken(
     getBlockManager().setBlockToken(
         lBlk, BlockTokenSecretManager.AccessMode.WRITE);
         lBlk, BlockTokenSecretManager.AccessMode.WRITE);
     return lBlk;
     return lBlk;
@@ -2741,7 +2735,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
 
   /** @see NameNode#getAdditionalDatanode(String, ExtendedBlock, DatanodeInfo[], DatanodeInfo[], int, String) */
   /** @see NameNode#getAdditionalDatanode(String, ExtendedBlock, DatanodeInfo[], DatanodeInfo[], int, String) */
   LocatedBlock getAdditionalDatanode(String src, final ExtendedBlock blk,
   LocatedBlock getAdditionalDatanode(String src, final ExtendedBlock blk,
-      final DatanodeInfo[] existings,  final Set<Node> excludes,
+      final DatanodeInfo[] existings, final String[] storageIDs,
+      final Set<Node> excludes,
       final int numAdditionalNodes, final String clientName
       final int numAdditionalNodes, final String clientName
       ) throws IOException {
       ) throws IOException {
     //check if the feature is enabled
     //check if the feature is enabled
@@ -2749,7 +2744,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
 
     final DatanodeDescriptor clientnode;
     final DatanodeDescriptor clientnode;
     final long preferredblocksize;
     final long preferredblocksize;
-    final List<DatanodeDescriptor> chosen;
+    final List<DatanodeStorageInfo> chosen;
     checkOperation(OperationCategory.READ);
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     readLock();
     readLock();
@@ -2764,23 +2759,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       clientnode = file.getFileUnderConstructionFeature().getClientNode();
       clientnode = file.getFileUnderConstructionFeature().getClientNode();
       preferredblocksize = file.getPreferredBlockSize();
       preferredblocksize = file.getPreferredBlockSize();
 
 
-      //find datanode descriptors
-      chosen = new ArrayList<DatanodeDescriptor>();
-      for(DatanodeInfo d : existings) {
-        final DatanodeDescriptor descriptor = blockManager.getDatanodeManager(
-            ).getDatanode(d);
-        if (descriptor != null) {
-          chosen.add(descriptor);
-        }
-      }
+      //find datanode storages
+      final DatanodeManager dm = blockManager.getDatanodeManager();
+      chosen = Arrays.asList(dm.getDatanodeStorageInfos(existings, storageIDs));
     } finally {
     } finally {
       readUnlock();
       readUnlock();
     }
     }
 
 
     // choose new datanodes.
     // choose new datanodes.
-    final DatanodeInfo[] targets = blockManager.getBlockPlacementPolicy(
+    final DatanodeStorageInfo[] targets = blockManager.getBlockPlacementPolicy(
         ).chooseTarget(src, numAdditionalNodes, clientnode, chosen, true,
         ).chooseTarget(src, numAdditionalNodes, clientnode, chosen, true,
-        excludes, preferredblocksize);
+            // TODO: get storage type from the file
+        excludes, preferredblocksize, StorageType.DEFAULT);
     final LocatedBlock lb = new LocatedBlock(blk, targets);
     final LocatedBlock lb = new LocatedBlock(blk, targets);
     blockManager.setBlockToken(lb, AccessMode.COPY);
     blockManager.setBlockToken(lb, AccessMode.COPY);
     return lb;
     return lb;
@@ -2951,14 +2941,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @throws QuotaExceededException If addition of block exceeds space quota
    * @throws QuotaExceededException If addition of block exceeds space quota
    */
    */
   BlockInfo saveAllocatedBlock(String src, INodesInPath inodesInPath,
   BlockInfo saveAllocatedBlock(String src, INodesInPath inodesInPath,
-      Block newBlock, DatanodeDescriptor targets[]) throws IOException {
+      Block newBlock, DatanodeStorageInfo[] targets)
+          throws IOException {
     assert hasWriteLock();
     assert hasWriteLock();
     BlockInfo b = dir.addBlock(src, inodesInPath, newBlock, targets);
     BlockInfo b = dir.addBlock(src, inodesInPath, newBlock, targets);
     NameNode.stateChangeLog.info("BLOCK* allocateBlock: " + src + ". "
     NameNode.stateChangeLog.info("BLOCK* allocateBlock: " + src + ". "
         + getBlockPoolId() + " " + b);
         + getBlockPoolId() + " " + b);
-    for (DatanodeDescriptor dn : targets) {
-      dn.incBlocksScheduled();
-    }
+    DatanodeStorageInfo.incrementBlocksScheduled(targets);
     return b;
     return b;
   }
   }
 
 
@@ -3421,7 +3410,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   boolean isFileClosed(String src) 
   boolean isFileClosed(String src) 
       throws AccessControlException, UnresolvedLinkException,
       throws AccessControlException, UnresolvedLinkException,
       StandbyException, IOException {
       StandbyException, IOException {
-    FSPermissionChecker pc = getPermissionChecker();	
+    FSPermissionChecker pc = getPermissionChecker();  
     checkOperation(OperationCategory.READ);
     checkOperation(OperationCategory.READ);
     readLock();
     readLock();
     try {
     try {
@@ -3718,7 +3707,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)lastBlock;
       final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)lastBlock;
       // setup the last block locations from the blockManager if not known
       // setup the last block locations from the blockManager if not known
       if (uc.getNumExpectedLocations() == 0) {
       if (uc.getNumExpectedLocations() == 0) {
-        uc.setExpectedLocations(blockManager.getNodes(lastBlock));
+        uc.setExpectedLocations(blockManager.getStorages(lastBlock));
       }
       }
 
 
       if (uc.getNumExpectedLocations() == 0 && uc.getNumBytes() == 0) {
       if (uc.getNumExpectedLocations() == 0 && uc.getNumBytes() == 0) {
@@ -3917,32 +3906,39 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         // find the DatanodeDescriptor objects
         // find the DatanodeDescriptor objects
         // There should be no locations in the blockManager till now because the
         // There should be no locations in the blockManager till now because the
         // file is underConstruction
         // file is underConstruction
-        List<DatanodeDescriptor> targetList =
+        ArrayList<DatanodeDescriptor> trimmedTargets =
             new ArrayList<DatanodeDescriptor>(newtargets.length);
             new ArrayList<DatanodeDescriptor>(newtargets.length);
+        ArrayList<String> trimmedStorages =
+            new ArrayList<String>(newtargets.length);
         if (newtargets.length > 0) {
         if (newtargets.length > 0) {
-          for (DatanodeID newtarget : newtargets) {
+          for (int i = 0; i < newtargets.length; ++i) {
             // try to get targetNode
             // try to get targetNode
             DatanodeDescriptor targetNode =
             DatanodeDescriptor targetNode =
-                blockManager.getDatanodeManager().getDatanode(newtarget);
-            if (targetNode != null)
-              targetList.add(targetNode);
-            else if (LOG.isDebugEnabled()) {
-              LOG.debug("DatanodeDescriptor (=" + newtarget + ") not found");
+                blockManager.getDatanodeManager().getDatanode(newtargets[i]);
+            if (targetNode != null) {
+              trimmedTargets.add(targetNode);
+              trimmedStorages.add(newtargetstorages[i]);
+            } else if (LOG.isDebugEnabled()) {
+              LOG.debug("DatanodeDescriptor (=" + newtargets[i] + ") not found");
             }
             }
           }
           }
         }
         }
-        if ((closeFile) && !targetList.isEmpty()) {
+        if ((closeFile) && !trimmedTargets.isEmpty()) {
           // the file is getting closed. Insert block locations into blockManager.
           // the file is getting closed. Insert block locations into blockManager.
           // Otherwise fsck will report these blocks as MISSING, especially if the
           // Otherwise fsck will report these blocks as MISSING, especially if the
           // blocksReceived from Datanodes take a long time to arrive.
           // blocksReceived from Datanodes take a long time to arrive.
-          for (DatanodeDescriptor targetNode : targetList) {
-            targetNode.addBlock(storedBlock);
+          for (int i = 0; i < trimmedTargets.size(); i++) {
+            trimmedTargets.get(i).addBlock(
+              trimmedStorages.get(i), storedBlock);
           }
           }
         }
         }
+
         // add pipeline locations into the INodeUnderConstruction
         // add pipeline locations into the INodeUnderConstruction
-        DatanodeDescriptor[] targetArray =
-            new DatanodeDescriptor[targetList.size()];
-        iFile.setLastBlock(storedBlock, targetList.toArray(targetArray));
+        DatanodeStorageInfo[] trimmedStorageInfos =
+            blockManager.getDatanodeManager().getDatanodeStorageInfos(
+                trimmedTargets.toArray(new DatanodeID[trimmedTargets.size()]),
+                trimmedStorages.toArray(new String[trimmedStorages.size()]));
+        iFile.setLastBlock(storedBlock, trimmedStorageInfos);
       }
       }
 
 
       if (closeFile) {
       if (closeFile) {
@@ -4144,16 +4140,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @throws IOException
    * @throws IOException
    */
    */
   HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
   HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
-      long capacity, long dfsUsed, long remaining, long blockPoolUsed,
-      long cacheCapacity, long cacheUsed, int xceiverCount, int xmitsInProgress,
-      int failedVolumes) throws IOException {
+      StorageReport[] reports, long cacheCapacity, long cacheUsed,
+      int xceiverCount, int xmitsInProgress, int failedVolumes)
+        throws IOException {
     readLock();
     readLock();
     try {
     try {
       final int maxTransfer = blockManager.getMaxReplicationStreams()
       final int maxTransfer = blockManager.getMaxReplicationStreams()
           - xmitsInProgress;
           - xmitsInProgress;
       DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
       DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
-          nodeReg, blockPoolId, capacity, dfsUsed, remaining, blockPoolUsed,
-          cacheCapacity, cacheUsed, xceiverCount, maxTransfer, failedVolumes);
+          nodeReg, reports, blockPoolId, cacheCapacity, cacheUsed,
+          xceiverCount, maxTransfer, failedVolumes);
       return new HeartbeatResponse(cmds, createHaStatusHeartbeat());
       return new HeartbeatResponse(cmds, createHaStatusHeartbeat());
     } finally {
     } finally {
       readUnlock();
       readUnlock();
@@ -5314,11 +5310,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
   }
 
 
   public void processIncrementalBlockReport(final DatanodeID nodeID,
   public void processIncrementalBlockReport(final DatanodeID nodeID,
-      final String poolId, final ReceivedDeletedBlockInfo blockInfos[])
+      final String poolId, final StorageReceivedDeletedBlocks srdb)
       throws IOException {
       throws IOException {
     writeLock();
     writeLock();
     try {
     try {
-      blockManager.processIncrementalBlockReport(nodeID, poolId, blockInfos);
+      blockManager.processIncrementalBlockReport(nodeID, poolId, srdb);
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
     }
     }
@@ -5807,9 +5803,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       for (int i = 0; i < blocks.length; i++) {
       for (int i = 0; i < blocks.length; i++) {
         ExtendedBlock blk = blocks[i].getBlock();
         ExtendedBlock blk = blocks[i].getBlock();
         DatanodeInfo[] nodes = blocks[i].getLocations();
         DatanodeInfo[] nodes = blocks[i].getLocations();
+        String[] storageIDs = blocks[i].getStorageIDs();
         for (int j = 0; j < nodes.length; j++) {
         for (int j = 0; j < nodes.length; j++) {
-          DatanodeInfo dn = nodes[j];
-          blockManager.findAndMarkBlockAsCorrupt(blk, dn,
+          blockManager.findAndMarkBlockAsCorrupt(blk, nodes[j],
+              storageIDs == null ? null: storageIDs[j], 
               "client machine reported it");
               "client machine reported it");
         }
         }
       }
       }
@@ -5864,7 +5861,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @throws IOException if any error occurs
    * @throws IOException if any error occurs
    */
    */
   void updatePipeline(String clientName, ExtendedBlock oldBlock, 
   void updatePipeline(String clientName, ExtendedBlock oldBlock, 
-      ExtendedBlock newBlock, DatanodeID[] newNodes)
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
       throws IOException {
       throws IOException {
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
@@ -5885,7 +5882,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and "
       assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and "
         + oldBlock + " has different block identifier";
         + oldBlock + " has different block identifier";
       updatePipelineInternal(clientName, oldBlock, newBlock, newNodes,
       updatePipelineInternal(clientName, oldBlock, newBlock, newNodes,
-          cacheEntry != null);
+          newStorageIDs, cacheEntry != null);
       success = true;
       success = true;
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
@@ -5897,7 +5894,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
 
   /** @see #updatePipeline(String, ExtendedBlock, ExtendedBlock, DatanodeID[]) */
   /** @see #updatePipeline(String, ExtendedBlock, ExtendedBlock, DatanodeID[]) */
   private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock, 
   private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock, 
-      ExtendedBlock newBlock, DatanodeID[] newNodes, boolean logRetryCache)
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs,
+      boolean logRetryCache)
       throws IOException {
       throws IOException {
     assert hasWriteLock();
     assert hasWriteLock();
     // check the vadility of the block and lease holder name
     // check the vadility of the block and lease holder name
@@ -5920,15 +5918,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
     blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
 
 
     // find the DatanodeDescriptor objects
     // find the DatanodeDescriptor objects
-    final DatanodeManager dm = getBlockManager().getDatanodeManager();
-    DatanodeDescriptor[] descriptors = null;
-    if (newNodes.length > 0) {
-      descriptors = new DatanodeDescriptor[newNodes.length];
-      for(int i = 0; i < newNodes.length; i++) {
-        descriptors[i] = dm.getDatanode(newNodes[i]);
-      }
-    }
-    blockinfo.setExpectedLocations(descriptors);
+    final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager()
+        .getDatanodeStorageInfos(newNodes, newStorageIDs);
+    blockinfo.setExpectedLocations(storages);
 
 
     String src = leaseManager.findPath(pendingFile);
     String src = leaseManager.findPath(pendingFile);
     dir.persistBlocks(src, pendingFile, logRetryCache);
     dir.persistBlocks(src, pendingFile, logRetryCache);
@@ -6055,7 +6047,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @throws IOException
    * @throws IOException
    */
    */
   Collection<CorruptFileBlockInfo> listCorruptFileBlocks(String path,
   Collection<CorruptFileBlockInfo> listCorruptFileBlocks(String path,
-	String[] cookieTab) throws IOException {
+  String[] cookieTab) throws IOException {
     checkSuperuserPrivilege();
     checkSuperuserPrivilege();
     checkOperation(OperationCategory.READ);
     checkOperation(OperationCategory.READ);
     readLock();
     readLock();
@@ -7400,3 +7392,4 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     }
   }
   }
 }
 }
+

+ 5 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java

@@ -167,19 +167,13 @@ public class FileJournalManager implements JournalManager {
   /**
   /**
    * Find all editlog segments starting at or above the given txid.
    * Find all editlog segments starting at or above the given txid.
    * @param fromTxId the txnid which to start looking
    * @param fromTxId the txnid which to start looking
-   * @param forReading whether or not the caller intends to read from the edit
-   *        logs
    * @param inProgressOk whether or not to include the in-progress edit log 
    * @param inProgressOk whether or not to include the in-progress edit log 
    *        segment       
    *        segment       
    * @return a list of remote edit logs
    * @return a list of remote edit logs
    * @throws IOException if edit logs cannot be listed.
    * @throws IOException if edit logs cannot be listed.
    */
    */
   public List<RemoteEditLog> getRemoteEditLogs(long firstTxId,
   public List<RemoteEditLog> getRemoteEditLogs(long firstTxId,
-      boolean forReading, boolean inProgressOk) throws IOException {
-    // make sure not reading in-progress edit log, i.e., if forReading is true,
-    // we should ignore the in-progress edit log.
-    Preconditions.checkArgument(!(forReading && inProgressOk));
-    
+      boolean inProgressOk) throws IOException {
     File currentDir = sd.getCurrentDir();
     File currentDir = sd.getCurrentDir();
     List<EditLogFile> allLogFiles = matchEditLogs(currentDir);
     List<EditLogFile> allLogFiles = matchEditLogs(currentDir);
     List<RemoteEditLog> ret = Lists.newArrayListWithCapacity(
     List<RemoteEditLog> ret = Lists.newArrayListWithCapacity(
@@ -192,14 +186,9 @@ public class FileJournalManager implements JournalManager {
       if (elf.getFirstTxId() >= firstTxId) {
       if (elf.getFirstTxId() >= firstTxId) {
         ret.add(new RemoteEditLog(elf.firstTxId, elf.lastTxId));
         ret.add(new RemoteEditLog(elf.firstTxId, elf.lastTxId));
       } else if (elf.getFirstTxId() < firstTxId && firstTxId <= elf.getLastTxId()) {
       } else if (elf.getFirstTxId() < firstTxId && firstTxId <= elf.getLastTxId()) {
-        // If the firstTxId is in the middle of an edit log segment
-        if (forReading) {
-          // Note that this behavior is different from getLogFiles below.
-          throw new IllegalStateException("Asked for firstTxId " + firstTxId
-              + " which is in the middle of file " + elf.file);
-        } else {
-          ret.add(new RemoteEditLog(elf.firstTxId, elf.lastTxId));
-        }
+        // If the firstTxId is in the middle of an edit log segment. Return this
+        // anyway and let the caller figure out whether it wants to use it.
+        ret.add(new RemoteEditLog(elf.firstTxId, elf.lastTxId));
       }
       }
     }
     }
     
     
@@ -260,7 +249,7 @@ public class FileJournalManager implements JournalManager {
   @Override
   @Override
   synchronized public void selectInputStreams(
   synchronized public void selectInputStreams(
       Collection<EditLogInputStream> streams, long fromTxId,
       Collection<EditLogInputStream> streams, long fromTxId,
-      boolean inProgressOk, boolean forReading) throws IOException {
+      boolean inProgressOk) throws IOException {
     List<EditLogFile> elfs = matchEditLogs(sd.getCurrentDir());
     List<EditLogFile> elfs = matchEditLogs(sd.getCurrentDir());
     LOG.debug(this + ": selecting input streams starting at " + fromTxId + 
     LOG.debug(this + ": selecting input streams starting at " + fromTxId + 
         (inProgressOk ? " (inProgress ok) " : " (excluding inProgress) ") +
         (inProgressOk ? " (inProgress ok) " : " (excluding inProgress) ") +

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

@@ -29,7 +29,7 @@ import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
  * Feature for under-construction file.
  * Feature for under-construction file.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
-public class FileUnderConstructionFeature extends INodeFile.Feature {
+public class FileUnderConstructionFeature implements INode.Feature {
   private String clientName; // lease holder
   private String clientName; // lease holder
   private final String clientMachine;
   private final String clientMachine;
   // if client is a cluster node too.
   // if client is a cluster node too.

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

@@ -756,47 +756,6 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   /** INode feature such as {@link FileUnderConstructionFeature}
   /** INode feature such as {@link FileUnderConstructionFeature}
    *  and {@link DirectoryWithQuotaFeature}.
    *  and {@link DirectoryWithQuotaFeature}.
    */
    */
-  interface Feature<F extends Feature<F>> {
-    /** @return the next feature. */
-    public F getNextFeature();
-
-    /** Set the next feature. */
-    public void setNextFeature(F next);
-
-    /** Utility methods such as addFeature and removeFeature. */
-    static class Util {
-      /**
-       * Add a feature to the linked list.
-       * @return the new head.
-       */
-      static <F extends Feature<F>> F addFeature(F feature, F head) {
-        feature.setNextFeature(head);
-        return feature;
-      }
-
-      /**
-       * Remove a feature from the linked list.
-       * @return the new head.
-       */
-      static <F extends Feature<F>> F removeFeature(F feature, F head) {
-        if (feature == head) {
-          final F newHead = head.getNextFeature();
-          head.setNextFeature(null);
-          return newHead;
-        } else if (head != null) {
-          F prev = head;
-          F curr = head.getNextFeature();
-          for (; curr != null && curr != feature;
-              prev = curr, curr = curr.getNextFeature())
-            ;
-          if (curr != null) {
-            prev.setNextFeature(curr.getNextFeature());
-            curr.setNextFeature(null);
-            return head;
-          }
-        }
-        throw new IllegalStateException("Feature " + feature + " not found.");
-      }
-    }
+  public interface Feature {
   }
   }
 }
 }

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

@@ -45,20 +45,6 @@ import com.google.common.base.Preconditions;
  */
  */
 public class INodeDirectory extends INodeWithAdditionalFields
 public class INodeDirectory extends INodeWithAdditionalFields
     implements INodeDirectoryAttributes {
     implements INodeDirectoryAttributes {
-  /** Directory related features such as quota and snapshots. */
-  public static abstract class Feature implements INode.Feature<Feature> {
-    private Feature nextFeature;
-
-    @Override
-    public Feature getNextFeature() {
-      return nextFeature;
-    }
-
-    @Override
-    public void setNextFeature(Feature next) {
-      this.nextFeature = next;
-    }
-  }
 
 
   /** Cast INode to INodeDirectory. */
   /** Cast INode to INodeDirectory. */
   public static INodeDirectory valueOf(INode inode, Object path
   public static INodeDirectory valueOf(INode inode, Object path
@@ -78,9 +64,6 @@ public class INodeDirectory extends INodeWithAdditionalFields
 
 
   private List<INode> children = null;
   private List<INode> children = null;
   
   
-  /** A linked list of {@link Feature}s. */
-  private Feature headFeature = null;
-
   /** constructor */
   /** constructor */
   public INodeDirectory(long id, byte[] name, PermissionStatus permissions,
   public INodeDirectory(long id, byte[] name, PermissionStatus permissions,
       long mtime) {
       long mtime) {
@@ -102,7 +85,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
       }
       }
     }
     }
     if (copyFeatures) {
     if (copyFeatures) {
-      this.headFeature = other.headFeature;
+      this.features = other.features;
     }
     }
   }
   }
 
 
@@ -160,7 +143,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
    * otherwise, return null.
    * otherwise, return null.
    */
    */
   public final DirectoryWithQuotaFeature getDirectoryWithQuotaFeature() {
   public final DirectoryWithQuotaFeature getDirectoryWithQuotaFeature() {
-    for(Feature f = headFeature; f != null; f = f.nextFeature) {
+    for (Feature f : features) {
       if (f instanceof DirectoryWithQuotaFeature) {
       if (f instanceof DirectoryWithQuotaFeature) {
         return (DirectoryWithQuotaFeature)f;
         return (DirectoryWithQuotaFeature)f;
       }
       }
@@ -182,14 +165,6 @@ public class INodeDirectory extends INodeWithAdditionalFields
     return quota;
     return quota;
   }
   }
 
 
-  private void addFeature(Feature f) {
-    headFeature = INode.Feature.Util.addFeature(f, headFeature);
-  }
-
-  private void removeFeature(Feature f) {
-    headFeature = INode.Feature.Util.removeFeature(f, headFeature);
-  }
-
   private int searchChildren(byte[] name) {
   private int searchChildren(byte[] name) {
     return children == null? -1: Collections.binarySearch(children, name);
     return children == null? -1: Collections.binarySearch(children, name);
   }
   }

+ 5 - 35
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -27,10 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.*;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
@@ -44,23 +41,6 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class INodeFile extends INodeWithAdditionalFields
 public class INodeFile extends INodeWithAdditionalFields
     implements INodeFileAttributes, BlockCollection {
     implements INodeFileAttributes, BlockCollection {
-  /**
-   * A feature contains specific information for a type of INodeFile. E.g.,
-   * we can have separate features for Under-Construction and Snapshot.
-   */
-  public static abstract class Feature implements INode.Feature<Feature> {
-    private Feature nextFeature;
-
-    @Override
-    public Feature getNextFeature() {
-      return nextFeature;
-    }
-
-    @Override
-    public void setNextFeature(Feature next) {
-      this.nextFeature = next;
-    }
-  }
 
 
   /** The same as valueOf(inode, path, false). */
   /** The same as valueOf(inode, path, false). */
   public static INodeFile valueOf(INode inode, String path
   public static INodeFile valueOf(INode inode, String path
@@ -123,8 +103,6 @@ public class INodeFile extends INodeWithAdditionalFields
 
 
   private BlockInfo[] blocks;
   private BlockInfo[] blocks;
 
 
-  private Feature headFeature;
-
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
       long atime, BlockInfo[] blklist, short replication,
       long atime, BlockInfo[] blklist, short replication,
       long preferredBlockSize) {
       long preferredBlockSize) {
@@ -138,7 +116,7 @@ public class INodeFile extends INodeWithAdditionalFields
     super(that);
     super(that);
     this.header = that.header;
     this.header = that.header;
     this.blocks = that.blocks;
     this.blocks = that.blocks;
-    this.headFeature = that.headFeature;
+    this.features = that.features;
   }
   }
   
   
   public INodeFile(INodeFile that, FileDiffList diffs) {
   public INodeFile(INodeFile that, FileDiffList diffs) {
@@ -147,14 +125,6 @@ public class INodeFile extends INodeWithAdditionalFields
     this.addSnapshotFeature(diffs);
     this.addSnapshotFeature(diffs);
   }
   }
 
 
-  private void addFeature(Feature f) {
-    headFeature = INode.Feature.Util.addFeature(f, headFeature);
-  }
-
-  private void removeFeature(Feature f) {
-    headFeature = INode.Feature.Util.removeFeature(f, headFeature);
-  }
-
   /** @return true unconditionally. */
   /** @return true unconditionally. */
   @Override
   @Override
   public final boolean isFile() {
   public final boolean isFile() {
@@ -174,7 +144,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * otherwise, return null.
    * otherwise, return null.
    */
    */
   public final FileUnderConstructionFeature getFileUnderConstructionFeature() {
   public final FileUnderConstructionFeature getFileUnderConstructionFeature() {
-    for (Feature f = this.headFeature; f != null; f = f.nextFeature) {
+    for (Feature f : features) {
       if (f instanceof FileUnderConstructionFeature) {
       if (f instanceof FileUnderConstructionFeature) {
         return (FileUnderConstructionFeature) f;
         return (FileUnderConstructionFeature) f;
       }
       }
@@ -234,7 +204,7 @@ public class INodeFile extends INodeWithAdditionalFields
 
 
   @Override // BlockCollection, the file should be under construction
   @Override // BlockCollection, the file should be under construction
   public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
   public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
-      DatanodeDescriptor[] locations) throws IOException {
+      DatanodeStorageInfo[] locations) throws IOException {
     Preconditions.checkState(isUnderConstruction(),
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
         "file is no longer under construction");
 
 
@@ -286,7 +256,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * otherwise, return null.
    * otherwise, return null.
    */
    */
   public final FileWithSnapshotFeature getFileWithSnapshotFeature() {
   public final FileWithSnapshotFeature getFileWithSnapshotFeature() {
-    for (Feature f = headFeature; f != null; f = f.nextFeature) {
+    for (Feature f: features) {
       if (f instanceof FileWithSnapshotFeature) {
       if (f instanceof FileWithSnapshotFeature) {
         return (FileWithSnapshotFeature) f;
         return (FileWithSnapshotFeature) f;
       }
       }

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java

@@ -95,6 +95,9 @@ public abstract class INodeWithAdditionalFields extends INode
 
 
   /** For implementing {@link LinkedElement}. */
   /** For implementing {@link LinkedElement}. */
   private LinkedElement next = null;
   private LinkedElement next = null;
+  /** An array {@link Feature}s. */
+  private static final Feature[] EMPTY_FEATURE = new Feature[0];
+  protected Feature[] features = EMPTY_FEATURE;
 
 
   private INodeWithAdditionalFields(INode parent, long id, byte[] name,
   private INodeWithAdditionalFields(INode parent, long id, byte[] name,
       long permission, long modificationTime, long accessTime) {
       long permission, long modificationTime, long accessTime) {
@@ -262,4 +265,45 @@ public abstract class INodeWithAdditionalFields extends INode
   public final void setAccessTime(long accessTime) {
   public final void setAccessTime(long accessTime) {
     this.accessTime = accessTime;
     this.accessTime = accessTime;
   }
   }
+
+  protected void addFeature(Feature f) {
+    int size = features.length;
+    Feature[] arr = new Feature[size + 1];
+    if (size != 0) {
+      System.arraycopy(features, 0, arr, 0, size);
+    }
+    arr[size] = f;
+    features = arr;
+  }
+
+  protected void removeFeature(Feature f) {
+    int size = features.length;
+    Preconditions.checkState(size > 0, "Feature "
+        + f.getClass().getSimpleName() + " not found.");
+
+    if (size == 1) {
+      Preconditions.checkState(features[0] == f, "Feature "
+          + f.getClass().getSimpleName() + " not found.");
+      features = EMPTY_FEATURE;
+      return;
+    }
+
+    Feature[] arr = new Feature[size - 1];
+    int j = 0;
+    boolean overflow = false;
+    for (Feature f1 : features) {
+      if (f1 != f) {
+        if (j == size - 1) {
+          overflow = true;
+          break;
+        } else {
+          arr[j++] = f1;
+        }
+      }
+    }
+
+    Preconditions.checkState(!overflow && j == size - 1, "Feature "
+        + f.getClass().getSimpleName() + " not found.");
+    features = arr;
+  }
 }
 }

+ 6 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java

@@ -233,12 +233,10 @@ public class JournalSet implements JournalManager {
    *                         may not be sorted-- this is up to the caller.
    *                         may not be sorted-- this is up to the caller.
    * @param fromTxId         The transaction ID to start looking for streams at
    * @param fromTxId         The transaction ID to start looking for streams at
    * @param inProgressOk     Should we consider unfinalized streams?
    * @param inProgressOk     Should we consider unfinalized streams?
-   * @param forReading       Whether or not the caller intends to read from
-   *                         the returned streams.
    */
    */
   @Override
   @Override
   public void selectInputStreams(Collection<EditLogInputStream> streams,
   public void selectInputStreams(Collection<EditLogInputStream> streams,
-      long fromTxId, boolean inProgressOk, boolean forReading) {
+      long fromTxId, boolean inProgressOk) {
     final PriorityQueue<EditLogInputStream> allStreams = 
     final PriorityQueue<EditLogInputStream> allStreams = 
         new PriorityQueue<EditLogInputStream>(64,
         new PriorityQueue<EditLogInputStream>(64,
             EDIT_LOG_INPUT_STREAM_COMPARATOR);
             EDIT_LOG_INPUT_STREAM_COMPARATOR);
@@ -248,8 +246,7 @@ public class JournalSet implements JournalManager {
         continue;
         continue;
       }
       }
       try {
       try {
-        jas.getManager().selectInputStreams(allStreams, fromTxId, inProgressOk,
-            forReading);
+        jas.getManager().selectInputStreams(allStreams, fromTxId, inProgressOk);
       } catch (IOException ioe) {
       } catch (IOException ioe) {
         LOG.warn("Unable to determine input streams from " + jas.getManager() +
         LOG.warn("Unable to determine input streams from " + jas.getManager() +
             ". Skipping.", ioe);
             ". Skipping.", ioe);
@@ -582,20 +579,20 @@ public class JournalSet implements JournalManager {
   
   
   /**
   /**
    * Return a manifest of what finalized edit logs are available. All available
    * Return a manifest of what finalized edit logs are available. All available
-   * edit logs are returned starting from the transaction id passed.
+   * edit logs are returned starting from the transaction id passed. If
+   * 'fromTxId' falls in the middle of a log, that log is returned as well.
    * 
    * 
    * @param fromTxId Starting transaction id to read the logs.
    * @param fromTxId Starting transaction id to read the logs.
    * @return RemoteEditLogManifest object.
    * @return RemoteEditLogManifest object.
    */
    */
-  public synchronized RemoteEditLogManifest getEditLogManifest(long fromTxId,
-      boolean forReading) {
+  public synchronized RemoteEditLogManifest getEditLogManifest(long fromTxId) {
     // Collect RemoteEditLogs available from each FileJournalManager
     // Collect RemoteEditLogs available from each FileJournalManager
     List<RemoteEditLog> allLogs = Lists.newArrayList();
     List<RemoteEditLog> allLogs = Lists.newArrayList();
     for (JournalAndStream j : journals) {
     for (JournalAndStream j : journals) {
       if (j.getManager() instanceof FileJournalManager) {
       if (j.getManager() instanceof FileJournalManager) {
         FileJournalManager fjm = (FileJournalManager)j.getManager();
         FileJournalManager fjm = (FileJournalManager)j.getManager();
         try {
         try {
-          allLogs.addAll(fjm.getRemoteEditLogs(fromTxId, forReading, false));
+          allLogs.addAll(fjm.getRemoteEditLogs(fromTxId, false));
         } catch (Throwable t) {
         } catch (Throwable t) {
           LOG.warn("Cannot list edit logs in " + fjm, t);
           LOG.warn("Cannot list edit logs in " + fjm, t);
         }
         }

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

@@ -42,13 +42,11 @@ interface LogsPurgeable {
    * 
    * 
    * @param fromTxId the first transaction id we want to read
    * @param fromTxId the first transaction id we want to read
    * @param inProgressOk whether or not in-progress streams should be returned
    * @param inProgressOk whether or not in-progress streams should be returned
-   * @param forReading whether or not the caller intends to read from the edit logs
-   *
    * @return a list of streams
    * @return a list of streams
    * @throws IOException if the underlying storage has an error or is otherwise
    * @throws IOException if the underlying storage has an error or is otherwise
    * inaccessible
    * inaccessible
    */
    */
   void selectInputStreams(Collection<EditLogInputStream> streams,
   void selectInputStreams(Collection<EditLogInputStream> streams,
-      long fromTxId, boolean inProgressOk, boolean forReading) throws IOException;
+      long fromTxId, boolean inProgressOk) throws IOException;
   
   
 }
 }

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

@@ -108,7 +108,7 @@ public class NNStorageRetentionManager {
     long purgeLogsFrom = Math.max(0, minimumRequiredTxId - numExtraEditsToRetain);
     long purgeLogsFrom = Math.max(0, minimumRequiredTxId - numExtraEditsToRetain);
     
     
     ArrayList<EditLogInputStream> editLogs = new ArrayList<EditLogInputStream>();
     ArrayList<EditLogInputStream> editLogs = new ArrayList<EditLogInputStream>();
-    purgeableLogs.selectInputStreams(editLogs, purgeLogsFrom, false, false);
+    purgeableLogs.selectInputStreams(editLogs, purgeLogsFrom, false);
     Collections.sort(editLogs, new Comparator<EditLogInputStream>() {
     Collections.sort(editLogs, new Comparator<EditLogInputStream>() {
       @Override
       @Override
       public int compare(EditLogInputStream a, EditLogInputStream b) {
       public int compare(EditLogInputStream a, EditLogInputStream b) {

+ 26 - 22
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -43,8 +43,8 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -67,21 +67,21 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
@@ -97,9 +97,9 @@ import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
@@ -200,9 +200,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
 
     NamenodeProtocolServerSideTranslatorPB namenodeProtocolXlator = 
     NamenodeProtocolServerSideTranslatorPB namenodeProtocolXlator = 
         new NamenodeProtocolServerSideTranslatorPB(this);
         new NamenodeProtocolServerSideTranslatorPB(this);
-	  BlockingService NNPbService = NamenodeProtocolService
+    BlockingService NNPbService = NamenodeProtocolService
           .newReflectiveBlockingService(namenodeProtocolXlator);
           .newReflectiveBlockingService(namenodeProtocolXlator);
-	  
+    
     RefreshAuthorizationPolicyProtocolServerSideTranslatorPB refreshAuthPolicyXlator = 
     RefreshAuthorizationPolicyProtocolServerSideTranslatorPB refreshAuthPolicyXlator = 
         new RefreshAuthorizationPolicyProtocolServerSideTranslatorPB(this);
         new RefreshAuthorizationPolicyProtocolServerSideTranslatorPB(this);
     BlockingService refreshAuthService = RefreshAuthorizationPolicyProtocolService
     BlockingService refreshAuthService = RefreshAuthorizationPolicyProtocolService
@@ -222,7 +222,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
         new HAServiceProtocolServerSideTranslatorPB(this);
         new HAServiceProtocolServerSideTranslatorPB(this);
     BlockingService haPbService = HAServiceProtocolService
     BlockingService haPbService = HAServiceProtocolService
         .newReflectiveBlockingService(haServiceProtocolXlator);
         .newReflectiveBlockingService(haServiceProtocolXlator);
-	  
+    
     WritableRpcEngine.ensureInitialized();
     WritableRpcEngine.ensureInitialized();
 
 
     InetSocketAddress serviceRpcAddr = nn.getServiceRpcServerAddress(conf);
     InetSocketAddress serviceRpcAddr = nn.getServiceRpcServerAddress(conf);
@@ -573,7 +573,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
 
   @Override // ClientProtocol
   @Override // ClientProtocol
   public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk,
   public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk,
-      final DatanodeInfo[] existings, final DatanodeInfo[] excludes,
+      final DatanodeInfo[] existings, final String[] existingStorageIDs,
+      final DatanodeInfo[] excludes,
       final int numAdditionalNodes, final String clientName
       final int numAdditionalNodes, final String clientName
       ) throws IOException {
       ) throws IOException {
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
@@ -594,8 +595,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
         excludeSet.add(node);
         excludeSet.add(node);
       }
       }
     }
     }
-    return namesystem.getAdditionalDatanode(src, blk,
-        existings, excludeSet, numAdditionalNodes, clientName);
+    return namesystem.getAdditionalDatanode(src, blk, existings,
+        existingStorageIDs, excludeSet, numAdditionalNodes, clientName);
   }
   }
   /**
   /**
    * The client needs to give up on the block.
    * The client needs to give up on the block.
@@ -643,9 +644,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
 
   @Override // ClientProtocol
   @Override // ClientProtocol
   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
-      ExtendedBlock newBlock, DatanodeID[] newNodes)
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
       throws IOException {
       throws IOException {
-    namesystem.updatePipeline(clientName, oldBlock, newBlock, newNodes);
+    namesystem.updatePipeline(clientName, oldBlock, newBlock, newNodes, newStorageIDs);
   }
   }
   
   
   @Override // DatanodeProtocol
   @Override // DatanodeProtocol
@@ -967,24 +968,25 @@ class NameNodeRpcServer implements NamenodeProtocols {
       int xmitsInProgress, int xceiverCount,
       int xmitsInProgress, int xceiverCount,
       int failedVolumes) throws IOException {
       int failedVolumes) throws IOException {
     verifyRequest(nodeReg);
     verifyRequest(nodeReg);
-    return namesystem.handleHeartbeat(nodeReg, report[0].getCapacity(),
-        report[0].getDfsUsed(), report[0].getRemaining(),
-        report[0].getBlockPoolUsed(), dnCacheCapacity, dnCacheUsed,
-        xceiverCount, xmitsInProgress, failedVolumes);
+    return namesystem.handleHeartbeat(nodeReg, report,
+        dnCacheCapacity, dnCacheUsed, xceiverCount, xmitsInProgress,
+        failedVolumes);
   }
   }
 
 
   @Override // DatanodeProtocol
   @Override // DatanodeProtocol
   public DatanodeCommand blockReport(DatanodeRegistration nodeReg,
   public DatanodeCommand blockReport(DatanodeRegistration nodeReg,
       String poolId, StorageBlockReport[] reports) throws IOException {
       String poolId, StorageBlockReport[] reports) throws IOException {
     verifyRequest(nodeReg);
     verifyRequest(nodeReg);
-    BlockListAsLongs blist = new BlockListAsLongs(reports[0].getBlocks());
     if(blockStateChangeLog.isDebugEnabled()) {
     if(blockStateChangeLog.isDebugEnabled()) {
       blockStateChangeLog.debug("*BLOCK* NameNode.blockReport: "
       blockStateChangeLog.debug("*BLOCK* NameNode.blockReport: "
-           + "from " + nodeReg + " " + blist.getNumberOfBlocks()
-           + " blocks");
+           + "from " + nodeReg + ", reports.length=" + reports.length);
+    }
+    final BlockManager bm = namesystem.getBlockManager(); 
+    for(StorageBlockReport r : reports) {
+      final BlockListAsLongs blocks = new BlockListAsLongs(r.getBlocks());
+      bm.processReport(nodeReg, r.getStorage(), poolId, blocks);
     }
     }
 
 
-    namesystem.getBlockManager().processReport(nodeReg, poolId, blist);
     if (nn.getFSImage().isUpgradeFinalized() && !nn.isStandbyState())
     if (nn.getFSImage().isUpgradeFinalized() && !nn.isStandbyState())
       return new FinalizeCommand(poolId);
       return new FinalizeCommand(poolId);
     return null;
     return null;
@@ -1011,8 +1013,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
           +"from "+nodeReg+" "+receivedAndDeletedBlocks.length
           +"from "+nodeReg+" "+receivedAndDeletedBlocks.length
           +" blocks.");
           +" blocks.");
     }
     }
-    namesystem.processIncrementalBlockReport(
-        nodeReg, poolId, receivedAndDeletedBlocks[0].getBlocks());
+    for(StorageReceivedDeletedBlocks r : receivedAndDeletedBlocks) {
+      namesystem.processIncrementalBlockReport(nodeReg, poolId, r);
+    }
   }
   }
 
 
   @Override // DatanodeProtocol
   @Override // DatanodeProtocol
@@ -1314,3 +1317,4 @@ class NameNodeRpcServer implements NamenodeProtocols {
     return namesystem.getAclStatus(src);
     return namesystem.getAclStatus(src);
   }
   }
 }
 }
+

+ 6 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java

@@ -27,13 +27,7 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.net.URLEncoder;
 import java.net.URLEncoder;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 
 
 import javax.servlet.ServletContext;
 import javax.servlet.ServletContext;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequest;
@@ -50,6 +44,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -1113,13 +1108,12 @@ class NamenodeJspHelper {
         } 
         } 
 
 
         doc.startTag("replicas");
         doc.startTag("replicas");
-        for (final Iterator<DatanodeDescriptor> it = blockManager != null ?
-            blockManager.datanodeIterator(block) :
-            Collections.<DatanodeDescriptor>emptyList().iterator();
-            it.hasNext();) {
+        for(DatanodeStorageInfo storage : (blockManager != null ?
+                blockManager.getStorages(block) :
+                Collections.<DatanodeStorageInfo>emptyList())) {
           doc.startTag("replica");
           doc.startTag("replica");
 
 
-          DatanodeDescriptor dd = it.next();
+          DatanodeDescriptor dd = storage.getDatanodeDescriptor();
 
 
           doc.startTag("host_name");
           doc.startTag("host_name");
           doc.pcdata(dd.getHostName());
           doc.pcdata(dd.getHostName());

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

@@ -827,7 +827,7 @@ public class SecondaryNameNode implements Runnable {
 
 
       @Override
       @Override
       public void selectInputStreams(Collection<EditLogInputStream> streams,
       public void selectInputStreams(Collection<EditLogInputStream> streams,
-          long fromTxId, boolean inProgressOk, boolean forReading) {
+          long fromTxId, boolean inProgressOk) {
         Iterator<StorageDirectory> iter = storage.dirIterator();
         Iterator<StorageDirectory> iter = storage.dirIterator();
         while (iter.hasNext()) {
         while (iter.hasNext()) {
           StorageDirectory dir = iter.next();
           StorageDirectory dir = iter.next();

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

@@ -228,7 +228,7 @@ public class BootstrapStandby implements Tool, Configurable {
     try {
     try {
       Collection<EditLogInputStream> streams =
       Collection<EditLogInputStream> streams =
         image.getEditLog().selectInputStreams(
         image.getEditLog().selectInputStreams(
-          firstTxIdInLogs, curTxIdOnOtherNode, null, true, false);
+          firstTxIdInLogs, curTxIdOnOtherNode, null, true);
       for (EditLogInputStream stream : streams) {
       for (EditLogInputStream stream : streams) {
         IOUtils.closeStream(stream);
         IOUtils.closeStream(stream);
       }
       }

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

@@ -165,7 +165,7 @@ public class EditLogTailer {
   }
   }
   
   
   @VisibleForTesting
   @VisibleForTesting
-  void setEditLog(FSEditLog editLog) {
+  public void setEditLog(FSEditLog editLog) {
     this.editLog = editLog;
     this.editLog = editLog;
   }
   }
   
   

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

@@ -32,7 +32,7 @@ import org.apache.hadoop.hdfs.server.namenode.Quota;
  * Feature for file with snapshot-related information.
  * Feature for file with snapshot-related information.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
-public class FileWithSnapshotFeature extends INodeFile.Feature {
+public class FileWithSnapshotFeature implements INode.Feature {
   private final FileDiffList diffs;
   private final FileDiffList diffs;
   private boolean isCurrentFileDeleted = false;
   private boolean isCurrentFileDeleted = false;
   
   

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

@@ -53,6 +53,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -61,6 +62,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -160,7 +162,7 @@ public class NamenodeWebHdfsMethods {
 
 
   static DatanodeInfo chooseDatanode(final NameNode namenode,
   static DatanodeInfo chooseDatanode(final NameNode namenode,
       final String path, final HttpOpParam.Op op, final long openOffset,
       final String path, final HttpOpParam.Op op, final long openOffset,
-      final long blocksize, Configuration conf) throws IOException {
+      final long blocksize, final Configuration conf) throws IOException {
     final BlockManager bm = namenode.getNamesystem().getBlockManager();
     final BlockManager bm = namenode.getNamesystem().getBlockManager();
 
 
     if (op == PutOpParam.Op.CREATE) {
     if (op == PutOpParam.Op.CREATE) {
@@ -168,11 +170,13 @@ public class NamenodeWebHdfsMethods {
       final DatanodeDescriptor clientNode = bm.getDatanodeManager(
       final DatanodeDescriptor clientNode = bm.getDatanodeManager(
           ).getDatanodeByHost(getRemoteAddress());
           ).getDatanodeByHost(getRemoteAddress());
       if (clientNode != null) {
       if (clientNode != null) {
-        final DatanodeDescriptor[] datanodes = bm.getBlockPlacementPolicy()
+        final DatanodeStorageInfo[] storages = bm.getBlockPlacementPolicy()
             .chooseTarget(path, 1, clientNode,
             .chooseTarget(path, 1, clientNode,
-                new ArrayList<DatanodeDescriptor>(), false, null, blocksize);
-        if (datanodes.length > 0) {
-          return datanodes[0];
+                new ArrayList<DatanodeStorageInfo>(), false, null, blocksize,
+                // TODO: get storage type from the file
+                StorageType.DEFAULT);
+        if (storages.length > 0) {
+          return storages[0].getDatanodeDescriptor();
         }
         }
       }
       }
     } else if (op == GetOpParam.Op.OPEN
     } else if (op == GetOpParam.Op.OPEN

+ 19 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 
 
 /****************************************************
 /****************************************************
  * A BlockCommand is an instruction to a datanode 
  * A BlockCommand is an instruction to a datanode 
@@ -46,9 +47,10 @@ public class BlockCommand extends DatanodeCommand {
    */
    */
   public static final long NO_ACK = Long.MAX_VALUE;
   public static final long NO_ACK = Long.MAX_VALUE;
   
   
-  String poolId;
-  Block blocks[];
-  DatanodeInfo targets[][];
+  final String poolId;
+  final Block[] blocks;
+  final DatanodeInfo[][] targets;
+  final String[][] targetStorageIDs;
 
 
   /**
   /**
    * Create BlockCommand for transferring blocks to another datanode
    * Create BlockCommand for transferring blocks to another datanode
@@ -60,21 +62,26 @@ public class BlockCommand extends DatanodeCommand {
     this.poolId = poolId;
     this.poolId = poolId;
     blocks = new Block[blocktargetlist.size()]; 
     blocks = new Block[blocktargetlist.size()]; 
     targets = new DatanodeInfo[blocks.length][];
     targets = new DatanodeInfo[blocks.length][];
+    targetStorageIDs = new String[blocks.length][];
+
     for(int i = 0; i < blocks.length; i++) {
     for(int i = 0; i < blocks.length; i++) {
       BlockTargetPair p = blocktargetlist.get(i);
       BlockTargetPair p = blocktargetlist.get(i);
       blocks[i] = p.block;
       blocks[i] = p.block;
-      targets[i] = p.targets;
+      targets[i] = DatanodeStorageInfo.toDatanodeInfos(p.targets);
+      targetStorageIDs[i] = DatanodeStorageInfo.toStorageIDs(p.targets);
     }
     }
   }
   }
 
 
-  private static final DatanodeInfo[][] EMPTY_TARGET = {};
+  private static final DatanodeInfo[][] EMPTY_TARGET_DATANODES = {};
+  private static final String[][] EMPTY_TARGET_STORAGEIDS = {};
 
 
   /**
   /**
    * Create BlockCommand for the given action
    * Create BlockCommand for the given action
    * @param blocks blocks related to the action
    * @param blocks blocks related to the action
    */
    */
   public BlockCommand(int action, String poolId, Block blocks[]) {
   public BlockCommand(int action, String poolId, Block blocks[]) {
-    this(action, poolId, blocks, EMPTY_TARGET);
+    this(action, poolId, blocks, EMPTY_TARGET_DATANODES,
+        EMPTY_TARGET_STORAGEIDS);
   }
   }
 
 
   /**
   /**
@@ -82,11 +89,12 @@ public class BlockCommand extends DatanodeCommand {
    * @param blocks blocks related to the action
    * @param blocks blocks related to the action
    */
    */
   public BlockCommand(int action, String poolId, Block[] blocks,
   public BlockCommand(int action, String poolId, Block[] blocks,
-      DatanodeInfo[][] targets) {
+      DatanodeInfo[][] targets, String[][] targetStorageIDs) {
     super(action);
     super(action);
     this.poolId = poolId;
     this.poolId = poolId;
     this.blocks = blocks;
     this.blocks = blocks;
     this.targets = targets;
     this.targets = targets;
+    this.targetStorageIDs = targetStorageIDs;
   }
   }
   
   
   public String getBlockPoolId() {
   public String getBlockPoolId() {
@@ -100,4 +108,8 @@ public class BlockCommand extends DatanodeCommand {
   public DatanodeInfo[][] getTargets() {
   public DatanodeInfo[][] getTargets() {
     return targets;
     return targets;
   }
   }
+
+  public String[][] getTargetStorageIDs() {
+    return targetStorageIDs;
+  }
 }
 }

+ 28 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java

@@ -17,6 +17,8 @@
  */
  */
 package org.apache.hadoop.hdfs.server.protocol;
 package org.apache.hadoop.hdfs.server.protocol;
 
 
+import java.util.Arrays;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -34,12 +36,14 @@ public class BlocksWithLocations {
   @InterfaceAudience.Private
   @InterfaceAudience.Private
   @InterfaceStability.Evolving
   @InterfaceStability.Evolving
   public static class BlockWithLocations {
   public static class BlockWithLocations {
-    Block block;
-    String storageIDs[];
+    final Block block;
+    final String[] datanodeUuids;
+    final String[] storageIDs;
     
     
     /** constructor */
     /** constructor */
-    public BlockWithLocations(Block block, String[] storageIDs) {
+    public BlockWithLocations(Block block, String[] datanodeUuids, String[] storageIDs) {
       this.block = block;
       this.block = block;
+      this.datanodeUuids = datanodeUuids;
       this.storageIDs = storageIDs;
       this.storageIDs = storageIDs;
     }
     }
     
     
@@ -48,10 +52,30 @@ public class BlocksWithLocations {
       return block;
       return block;
     }
     }
     
     
-    /** get the block's locations */
+    /** get the block's datanode locations */
+    public String[] getDatanodeUuids() {
+      return datanodeUuids;
+    }
+
+    /** get the block's storage locations */
     public String[] getStorageIDs() {
     public String[] getStorageIDs() {
       return storageIDs;
       return storageIDs;
     }
     }
+    
+    @Override
+    public String toString() {
+      final StringBuilder b = new StringBuilder();
+      b.append(block);
+      if (datanodeUuids.length == 0) {
+        return b.append("[]").toString();
+      }
+      
+      b.append(storageIDs[0]).append('@').append(datanodeUuids[0]);
+      for(int i = 1; i < datanodeUuids.length; i++) {
+        b.append(", ").append(storageIDs[i]).append("@").append(datanodeUuids[i]);
+      }
+      return b.append("]").toString();
+    }
   }
   }
 
 
   private BlockWithLocations[] blocks;
   private BlockWithLocations[] blocks;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java

@@ -82,7 +82,7 @@ public class DatanodeRegistration extends DatanodeID
   public String toString() {
   public String toString() {
     return getClass().getSimpleName()
     return getClass().getSimpleName()
       + "(" + getIpAddr()
       + "(" + getIpAddr()
-      + ", storageID=" + getStorageID()
+      + ", datanodeUuid=" + getDatanodeUuid()
       + ", infoPort=" + getInfoPort()
       + ", infoPort=" + getInfoPort()
       + ", ipcPort=" + getIpcPort()
       + ", ipcPort=" + getIpcPort()
       + ", storageInfo=" + storageInfo
       + ", storageInfo=" + storageInfo

+ 45 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java

@@ -17,6 +17,10 @@
  */
  */
 package org.apache.hadoop.hdfs.server.protocol;
 package org.apache.hadoop.hdfs.server.protocol;
 
 
+import org.apache.hadoop.hdfs.StorageType;
+
+import java.util.UUID;
+
 /**
 /**
  * Class captures information of a storage in Datanode.
  * Class captures information of a storage in Datanode.
  */
  */
@@ -29,18 +33,21 @@ public class DatanodeStorage {
   
   
   private final String storageID;
   private final String storageID;
   private final State state;
   private final State state;
+  private final StorageType storageType;
 
 
   /**
   /**
-   * Create a storage with {@link State#NORMAL}.
+   * Create a storage with {@link State#NORMAL} and {@link StorageType#DEFAULT}.
+   *
    * @param storageID
    * @param storageID
    */
    */
   public DatanodeStorage(String storageID) {
   public DatanodeStorage(String storageID) {
-    this(storageID, State.NORMAL);
+    this(storageID, State.NORMAL, StorageType.DEFAULT);
   }
   }
 
 
-  public DatanodeStorage(String sid, State s) {
-    storageID = sid;
-    state = s;
+  public DatanodeStorage(String sid, State s, StorageType sm) {
+    this.storageID = sid;
+    this.state = s;
+    this.storageType = sm;
   }
   }
 
 
   public String getStorageID() {
   public String getStorageID() {
@@ -50,4 +57,37 @@ public class DatanodeStorage {
   public State getState() {
   public State getState() {
     return state;
     return state;
   }
   }
+
+  public StorageType getStorageType() {
+    return storageType;
+  }
+
+  /**
+   * Generate new storage ID. The format of this string can be changed
+   * in the future without requiring that old storage IDs be updated.
+   *
+   * @return unique storage ID
+   */
+  public static String generateUuid() {
+    return "DS-" + UUID.randomUUID();
+  }
+
+  @Override
+  public boolean equals(Object other){
+    if (other == this) {
+      return true;
+    }
+
+    if ((other == null) ||
+        !(other instanceof DatanodeStorage)) {
+      return false;
+    }
+    DatanodeStorage otherStorage = (DatanodeStorage) other;
+    return otherStorage.getStorageID().compareTo(getStorageID()) == 0;
+  }
+
+  @Override
+  public int hashCode() {
+    return getStorageID().hashCode();
+  }
 }
 }

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java

@@ -27,6 +27,8 @@ public class StorageReport {
   private final long dfsUsed;
   private final long dfsUsed;
   private final long remaining;
   private final long remaining;
   private final long blockPoolUsed;
   private final long blockPoolUsed;
+
+  public static final StorageReport[] EMPTY_ARRAY = {};
   
   
   public StorageReport(String sid, boolean failed, long capacity, long dfsUsed,
   public StorageReport(String sid, boolean failed, long capacity, long dfsUsed,
       long remaining, long bpUsed) {
       long remaining, long bpUsed) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -126,7 +126,7 @@ class ImageLoaderCurrent implements ImageLoader {
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
       -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
       -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
-      -40, -41, -42, -43, -44, -45, -46, -47, -48 };
+      -40, -41, -42, -43, -44, -45, -46, -47, -48, -49 };
   private int imageVersion = 0;
   private int imageVersion = 0;
   
   
   private final Map<Long, Boolean> subtreeMap = new HashMap<Long, Boolean>();
   private final Map<Long, Boolean> subtreeMap = new HashMap<Long, Boolean>();

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

@@ -276,10 +276,11 @@ public class JsonUtil {
       return null;
       return null;
     }
     }
 
 
+    // TODO: Fix storageID
     final Map<String, Object> m = new TreeMap<String, Object>();
     final Map<String, Object> m = new TreeMap<String, Object>();
     m.put("ipAddr", datanodeinfo.getIpAddr());
     m.put("ipAddr", datanodeinfo.getIpAddr());
     m.put("hostName", datanodeinfo.getHostName());
     m.put("hostName", datanodeinfo.getHostName());
-    m.put("storageID", datanodeinfo.getStorageID());
+    m.put("storageID", datanodeinfo.getDatanodeUuid());
     m.put("xferPort", datanodeinfo.getXferPort());
     m.put("xferPort", datanodeinfo.getXferPort());
     m.put("infoPort", datanodeinfo.getInfoPort());
     m.put("infoPort", datanodeinfo.getInfoPort());
     m.put("infoSecurePort", datanodeinfo.getInfoSecurePort());
     m.put("infoSecurePort", datanodeinfo.getInfoSecurePort());
@@ -329,6 +330,7 @@ public class JsonUtil {
       return null;
       return null;
     }
     }
 
 
+    // TODO: Fix storageID
     return new DatanodeInfo(
     return new DatanodeInfo(
         (String)m.get("ipAddr"),
         (String)m.get("ipAddr"),
         (String)m.get("hostName"),
         (String)m.get("hostName"),
@@ -412,7 +414,7 @@ public class JsonUtil {
         (Object[])m.get("cachedLocations"));
         (Object[])m.get("cachedLocations"));
 
 
     final LocatedBlock locatedblock = new LocatedBlock(b, locations,
     final LocatedBlock locatedblock = new LocatedBlock(b, locations,
-        startOffset, isCorrupt, cachedLocations);
+        null, null, startOffset, isCorrupt, cachedLocations);
     locatedblock.setBlockToken(toBlockToken((Map<?, ?>)m.get("blockToken")));
     locatedblock.setBlockToken(toBlockToken((Map<?, ?>)m.get("blockToken")));
     return locatedblock;
     return locatedblock;
   }
   }

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

@@ -142,6 +142,7 @@ message GetAdditionalDatanodeRequestProto {
   repeated DatanodeInfoProto excludes = 4;
   repeated DatanodeInfoProto excludes = 4;
   required uint32 numAdditionalNodes = 5;
   required uint32 numAdditionalNodes = 5;
   required string clientName = 6;
   required string clientName = 6;
+  repeated string existingStorageUuids = 7;
 }
 }
 
 
 message GetAdditionalDatanodeResponseProto {
 message GetAdditionalDatanodeResponseProto {
@@ -546,6 +547,7 @@ message UpdatePipelineRequestProto {
   required ExtendedBlockProto oldBlock = 2;
   required ExtendedBlockProto oldBlock = 2;
   required ExtendedBlockProto newBlock = 3;
   required ExtendedBlockProto newBlock = 3;
   repeated DatanodeIDProto newNodes = 4;
   repeated DatanodeIDProto newNodes = 4;
+  repeated string storageIDs = 5;
 }
 }
 
 
 message UpdatePipelineResponseProto { // void response
 message UpdatePipelineResponseProto { // void response

+ 6 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto

@@ -53,8 +53,9 @@ message DatanodeStorageProto {
     READ_ONLY = 1;
     READ_ONLY = 1;
   }
   }
 
 
-  required string storageID = 1;    // Unique identifier for the storage
+  required string storageUuid = 1;
   optional StorageState state = 2 [default = NORMAL];
   optional StorageState state = 2 [default = NORMAL];
+  optional StorageTypeProto storageType = 3 [default = DISK];
 }
 }
 
 
 /**
 /**
@@ -106,10 +107,12 @@ message BlockCommandProto {
     INVALIDATE = 2; // Invalidate blocks
     INVALIDATE = 2; // Invalidate blocks
     SHUTDOWN = 3;   // Shutdown the datanode
     SHUTDOWN = 3;   // Shutdown the datanode
   }
   }
+
   required Action action = 1;
   required Action action = 1;
   required string blockPoolId = 2;
   required string blockPoolId = 2;
   repeated BlockProto blocks = 3;
   repeated BlockProto blocks = 3;
   repeated DatanodeInfosProto targets = 4;
   repeated DatanodeInfosProto targets = 4;
+  repeated StorageUuidsProto targetStorageUuids = 5;
 }
 }
 
 
 /**
 /**
@@ -193,7 +196,7 @@ message HeartbeatRequestProto {
 }
 }
 
 
 message StorageReportProto {
 message StorageReportProto {
-  required string storageID = 1;
+  required string storageUuid = 1;
   optional bool failed = 2 [ default = false ];
   optional bool failed = 2 [ default = false ];
   optional uint64 capacity = 3 [ default = 0 ];
   optional uint64 capacity = 3 [ default = 0 ];
   optional uint64 dfsUsed = 4 [ default = 0 ];
   optional uint64 dfsUsed = 4 [ default = 0 ];
@@ -284,7 +287,7 @@ message ReceivedDeletedBlockInfoProto {
  * List of blocks received and deleted for a storage.
  * List of blocks received and deleted for a storage.
  */
  */
 message StorageReceivedDeletedBlocksProto {
 message StorageReceivedDeletedBlocksProto {
-  required string storageID = 1;
+  required string storageUuid = 1;
   repeated ReceivedDeletedBlockInfoProto blocks = 2;
   repeated ReceivedDeletedBlockInfoProto blocks = 2;
 }
 }
 
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto

@@ -65,7 +65,7 @@ message UpdateReplicaUnderRecoveryRequestProto {
  * Response returns updated block information
  * Response returns updated block information
  */
  */
 message UpdateReplicaUnderRecoveryResponseProto {
 message UpdateReplicaUnderRecoveryResponseProto {
-  required string storageID = 1; // ID of the storage that stores replica
+  optional string storageUuid = 1; // ID of the storage that stores replica
 }
 }
 
 
 /**
 /**

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