Browse Source

HDFS-14909. DFSNetworkTopology#chooseRandomWithStorageType() should not decrease storage count for excluded node which is already part of excluded scope. Contributed by Surendra Singh Lilhore.

Surendra Singh Lilhore 5 years ago
parent
commit
54dc6b7d72

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/DFSNetworkTopology.java

@@ -212,6 +212,10 @@ public class DFSNetworkTopology extends NetworkTopology {
     }
     if (excludedNodes != null) {
       for (Node excludedNode : excludedNodes) {
+        if (excludeRoot != null
+            && excludedNode.getNetworkLocation().startsWith(excludedScope)) {
+          continue;
+        }
         if (excludedNode instanceof DatanodeDescriptor) {
           availableCount -= ((DatanodeDescriptor) excludedNode)
               .hasStorageType(type) ? 1 : 0;

+ 30 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMissingBlocksAlert.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.AvailableSpaceBlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.junit.Assert;
 import org.junit.Test;
@@ -145,4 +146,33 @@ public class TestMissingBlocksAlert {
       }
     }
   }
+
+  @Test
+  public void testMissReplicatedBlockwithTwoRack() throws Exception {
+    Configuration conf = new Configuration();
+    //Start cluster with rack /default/rack1
+    String[] hosts = new String[] {"host0", "host1", "host2", "host3"};
+    String[] racks = new String[] {"/default/rack1", "/default/rack1",
+        "/default/rack1", "/default/rack1"};
+    conf.set(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY,
+        AvailableSpaceBlockPlacementPolicy.class.getName());
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4)
+        .hosts(hosts).racks(racks).build();
+    Path file = new Path("/file2");
+    try {
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      DFSTestUtil.createFile(dfs, file, 1024, (short) 2, 0);
+      dfs.getFileStatus(file);
+      //Add one more rack /default/rack2
+      cluster.startDataNodes(conf, 2, true, null,
+          new String[] {"/default/rack2", "/default/rack2"},
+          new String[] {"host4", "host5"}, null);
+      dfs.setReplication(file, (short) 3);
+      // wait for block replication
+      DFSTestUtil.waitForReplication(dfs, file, (short) 3, 60000);
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }