|
@@ -20,9 +20,14 @@ 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.util.ArrayList;
|
|
|
+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;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -143,6 +148,73 @@ public class TestBlocksWithNotEnoughRacks {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /*
|
|
|
+ * Initialize a cluster with datanodes on two different racks and shutdown
|
|
|
+ * all datanodes on one rack. Now create a file with a single block. Even
|
|
|
+ * though the block is sufficiently replicated, it violates the replica
|
|
|
+ * placement policy. Now restart the datanodes stopped earlier. Run the fsck
|
|
|
+ * command with -replicate option to schedule the replication of these
|
|
|
+ * mis-replicated blocks and verify if it indeed works as expected.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testMisReplicatedBlockUsesNewRack() throws Exception {
|
|
|
+ Configuration conf = getConf();
|
|
|
+ conf.setInt("dfs.namenode.heartbeat.recheck-interval", 500);
|
|
|
+
|
|
|
+ final short replicationFactor = 3;
|
|
|
+ final Path filePath = new Path("/testFile");
|
|
|
+ // All datanodes are on two different racks
|
|
|
+ String[] racks = new String[]{"/rack1", "/rack1", "/rack1", "/rack2"};
|
|
|
+
|
|
|
+ try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .numDataNodes(racks.length).racks(racks).build()) {
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
+ DatanodeRegistration reg = InternalDataNodeTestUtils.
|
|
|
+ getDNRegistrationForBP(cluster.getDataNodes().get(3), poolId);
|
|
|
+ // Shutdown datanode on rack2 and wait for it to be marked dead
|
|
|
+ cluster.stopDataNode(3);
|
|
|
+ DFSTestUtil.waitForDatanodeState(cluster, reg.getDatanodeUuid(),
|
|
|
+ false, 20000);
|
|
|
+
|
|
|
+ // Create a file with one block with a replication factor of 3
|
|
|
+ final FileSystem fs = cluster.getFileSystem();
|
|
|
+ DFSTestUtil.createFile(fs, filePath, 1L, replicationFactor, 1L);
|
|
|
+ ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, filePath);
|
|
|
+ DFSTestUtil.waitReplication(cluster.getFileSystem(), filePath,
|
|
|
+ replicationFactor);
|
|
|
+
|
|
|
+ // Add datanode on rack2 and wait for it be recognized as alive by NN
|
|
|
+ cluster.startDataNodes(conf, 1, true,
|
|
|
+ null, new String[]{"/rack2"});
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ try {
|
|
|
+ DFSTestUtil.waitForReplication(cluster, b, 2, replicationFactor, 0);
|
|
|
+ fail("NameNode should not have fixed the mis-replicated blocks" +
|
|
|
+ " automatically.");
|
|
|
+ } catch (TimeoutException e) {
|
|
|
+ //Expected.
|
|
|
+ }
|
|
|
+
|
|
|
+ String fsckOp = DFSTestUtil.runFsck(conf, 0, true, filePath.toString(),
|
|
|
+ "-replicate");
|
|
|
+ LOG.info("fsck response {}", fsckOp);
|
|
|
+ assertTrue(fsckOp.contains(
|
|
|
+ "/testFile: Replica placement policy is violated"));
|
|
|
+ assertTrue(fsckOp.contains(" Block should be additionally replicated" +
|
|
|
+ " on 1 more rack(s). Total number of racks in the cluster: 2"));
|
|
|
+
|
|
|
+ try {
|
|
|
+ DFSTestUtil.waitForReplication(cluster, b, 2, replicationFactor, 0);
|
|
|
+ } catch (TimeoutException e) {
|
|
|
+ fail("NameNode should have fixed the mis-replicated blocks as a" +
|
|
|
+ " result of fsck command.");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/*
|
|
|
* Creates a block with all datanodes on the same rack. Add additional
|
|
|
* datanodes on a different rack and increase the replication factor,
|