Преглед изворни кода

HDFS-7742. Favoring decommissioning node for replication can cause a block to stay
underreplicated for long periods. Contributed by Nathan Roberts.
(cherry picked from commit 04ee18ed48ceef34598f954ff40940abc9fde1d2)

(cherry picked from commit c4cedfc1d601127430c70ca8ca4d4e2ee2d1003d)
(cherry picked from commit c6b68a82adea8de488b255594d35db8e01f5fc8f)

Kihwal Lee пре 10 година
родитељ
комит
bc8728cd27

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

@@ -109,6 +109,9 @@ Release 2.6.1 - UNRELEASED
     HDFS-7960. The full block report should prune zombie storages even if
     they're not empty. (cmccabe and Eddy Xu via wang)
 
+    HDFS-7742. Favoring decommissioning node for replication can cause a block
+    to stay underreplicated for long periods (Nathan Roberts via kihwal)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

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

@@ -1652,7 +1652,8 @@ public class BlockManager {
       // If so, do not select the node as src node
       if ((nodesCorrupt != null) && nodesCorrupt.contains(node))
         continue;
-      if(priority != UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY
+      if(priority != UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY 
+          && !node.isDecommissionInProgress() 
           && node.getNumberOfBlocksToBeReplicated() >= maxReplicationStreams)
       {
         continue; // already reached replication limit
@@ -1667,13 +1668,12 @@ public class BlockManager {
       // never use already decommissioned nodes
       if(node.isDecommissioned())
         continue;
-      // we prefer nodes that are in DECOMMISSION_INPROGRESS state
-      if(node.isDecommissionInProgress() || srcNode == null) {
+
+      // We got this far, current node is a reasonable choice
+      if (srcNode == null) {
         srcNode = node;
         continue;
       }
-      if(srcNode.isDecommissionInProgress())
-        continue;
       // switch to a different node randomly
       // this to prevent from deterministically selecting the same node even
       // if the node failed to replicate the block on previous iterations

+ 42 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java

@@ -535,6 +535,48 @@ public class TestBlockManager {
             UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY));
   }
 
+  @Test
+  public void testFavorDecomUntilHardLimit() throws Exception {
+    bm.maxReplicationStreams = 0;
+    bm.replicationStreamsHardLimit = 1;
+
+    long blockId = 42;         // arbitrary
+    Block aBlock = new Block(blockId, 0, 0);
+    List<DatanodeDescriptor> origNodes = getNodes(0, 1);
+    // Add the block to the first node.
+    addBlockOnNodes(blockId,origNodes.subList(0,1));
+    origNodes.get(0).startDecommission();
+
+    List<DatanodeDescriptor> cntNodes = new LinkedList<DatanodeDescriptor>();
+    List<DatanodeStorageInfo> liveNodes = new LinkedList<DatanodeStorageInfo>();
+
+    assertNotNull("Chooses decommissioning source node for a normal replication"
+        + " if all available source nodes have reached their replication"
+        + " limits below the hard limit.",
+        bm.chooseSourceDatanode(
+            aBlock,
+            cntNodes,
+            liveNodes,
+            new NumberReplicas(),
+            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED));
+
+
+    // Increase the replication count to test replication count > hard limit
+    DatanodeStorageInfo targets[] = { origNodes.get(1).getStorageInfos()[0] };
+    origNodes.get(0).addBlockToBeReplicated(aBlock, targets);
+
+    assertNull("Does not choose a source decommissioning node for a normal"
+        + " replication when all available nodes exceed the hard limit.",
+        bm.chooseSourceDatanode(
+            aBlock,
+            cntNodes,
+            liveNodes,
+            new NumberReplicas(),
+            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED));
+  }
+
+
+
   @Test
   public void testSafeModeIBR() throws Exception {
     DatanodeDescriptor node = spy(nodes.get(0));