Parcourir la source

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 il y a 10 ans
Parent
commit
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));