|
@@ -18,14 +18,10 @@
|
|
|
|
|
|
package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
|
|
|
|
-import static org.junit.Assert.assertEquals;
|
|
|
-import static org.junit.Assert.assertArrayEquals;
|
|
|
-import static org.junit.Assert.fail;
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
-
|
|
|
+import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.List;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
-
|
|
|
import org.apache.hadoop.hdfs.server.datanode.InternalDataNodeTestUtils;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
|
import org.slf4j.Logger;
|
|
@@ -48,6 +44,8 @@ import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.junit.Test;
|
|
|
import org.slf4j.event.Level;
|
|
|
|
|
|
+import static org.junit.Assert.*;
|
|
|
+
|
|
|
public class TestBlocksWithNotEnoughRacks {
|
|
|
public static final Logger LOG =
|
|
|
LoggerFactory.getLogger(TestBlocksWithNotEnoughRacks.class);
|
|
@@ -545,4 +543,105 @@ public class TestBlocksWithNotEnoughRacks {
|
|
|
hostsFileWriter.cleanup();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testMultipleReplicasScheduledForUpgradeDomain() throws Exception {
|
|
|
+ Configuration conf = getConf();
|
|
|
+ final short replicationFactor = 3;
|
|
|
+ final Path filePath = new Path("/testFile");
|
|
|
+
|
|
|
+ conf.set("dfs.block.replicator.classname",
|
|
|
+ "org.apache.hadoop.hdfs.server.blockmanagement." +
|
|
|
+ "BlockPlacementPolicyWithUpgradeDomain");
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .numDataNodes(6).build();
|
|
|
+ cluster.waitClusterUp();
|
|
|
+
|
|
|
+ List<DatanodeDescriptor> dnDescriptors = getDnDescriptors(cluster);
|
|
|
+
|
|
|
+ try {
|
|
|
+ // Create a file with one block with a replication factor of 3
|
|
|
+ // No upgrade domains are set.
|
|
|
+ final FileSystem fs = cluster.getFileSystem();
|
|
|
+ DFSTestUtil.createFile(fs, filePath, 1L, replicationFactor, 1L);
|
|
|
+ ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, filePath);
|
|
|
+
|
|
|
+ BlockManager bm = cluster.getNamesystem().getBlockManager();
|
|
|
+ BlockInfo storedBlock = bm.getStoredBlock(b.getLocalBlock());
|
|
|
+
|
|
|
+ // The block should be replicated OK - so Reconstruction Work will be null
|
|
|
+ BlockReconstructionWork work = bm.scheduleReconstruction(storedBlock, 2);
|
|
|
+ assertNull(work);
|
|
|
+ // Set the upgradeDomain to "3" for the 3 nodes hosting the block.
|
|
|
+ // Then alternately set the remaining 3 nodes to have an upgradeDomain
|
|
|
+ // of 0 or 1 giving a total of 3 upgradeDomains.
|
|
|
+ for (int i=0; i<storedBlock.getReplication(); i++) {
|
|
|
+ storedBlock.getDatanode(i).setUpgradeDomain("3");
|
|
|
+ }
|
|
|
+ int udInd = 0;
|
|
|
+ for (DatanodeDescriptor d : dnDescriptors) {
|
|
|
+ if (d.getUpgradeDomain() == null) {
|
|
|
+ d.setUpgradeDomain(Integer.toString(udInd % 2));
|
|
|
+ udInd++;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ // Now reconWork is non-null and 2 extra targets are needed
|
|
|
+ work = bm.scheduleReconstruction(storedBlock, 2);
|
|
|
+ assertEquals(2, work.getAdditionalReplRequired());
|
|
|
+
|
|
|
+ // Add the block to the replication queue and ensure it is replicated
|
|
|
+ // correctly.
|
|
|
+ bm.neededReconstruction.add(storedBlock, 3, 0, 0, replicationFactor);
|
|
|
+ DFSTestUtil.waitForReplication(cluster, b, 1, replicationFactor, 0, 3);
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testUnderReplicatedRespectsRacksAndUpgradeDomain()
|
|
|
+ throws Exception {
|
|
|
+ Configuration conf = getConf();
|
|
|
+ final short replicationFactor = 3;
|
|
|
+ final Path filePath = new Path("/testFile");
|
|
|
+
|
|
|
+ conf.set("dfs.block.replicator.classname",
|
|
|
+ "org.apache.hadoop.hdfs.server.blockmanagement." +
|
|
|
+ "BlockPlacementPolicyWithUpgradeDomain");
|
|
|
+
|
|
|
+ // All hosts are on two racks
|
|
|
+ String[] racks = {"/r1", "/r1", "/r1", "/r2", "/r2", "/r2"};
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .numDataNodes(6).racks(racks).build();
|
|
|
+ cluster.waitClusterUp();
|
|
|
+ List<DatanodeDescriptor> dnDescriptors = getDnDescriptors(cluster);
|
|
|
+ for (int i=0; i < dnDescriptors.size(); i++) {
|
|
|
+ dnDescriptors.get(i).setUpgradeDomain(Integer.toString(i%3));
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ final FileSystem fs = cluster.getFileSystem();
|
|
|
+ DFSTestUtil.createFile(fs, filePath, 1L, (short)1, 1L);
|
|
|
+ ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, filePath);
|
|
|
+ fs.setReplication(filePath, replicationFactor);
|
|
|
+ DFSTestUtil.waitForReplication(cluster, b, 2, replicationFactor, 0, 3);
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private List<DatanodeDescriptor> getDnDescriptors(MiniDFSCluster cluster)
|
|
|
+ throws IOException {
|
|
|
+ List<DatanodeDescriptor> dnDesc = new ArrayList<>();
|
|
|
+ DatanodeManager dnManager = cluster.getNamesystem().getBlockManager()
|
|
|
+ .getDatanodeManager();
|
|
|
+ for (DataNode dn : cluster.getDataNodes()) {
|
|
|
+ DatanodeDescriptor d = dnManager.getDatanode(dn.getDatanodeUuid());
|
|
|
+ if (d == null) {
|
|
|
+ throw new IOException("DatanodeDescriptor not found for DN "+
|
|
|
+ dn.getDatanodeUuid());
|
|
|
+ }
|
|
|
+ dnDesc.add(d);
|
|
|
+ }
|
|
|
+ return dnDesc;
|
|
|
+ }
|
|
|
}
|