|
@@ -23,8 +23,9 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
-import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
|
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
@@ -39,9 +40,8 @@ import org.junit.Test;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
-import java.util.Arrays;
|
|
|
|
|
|
+import java.io.IOException;
|
|
import java.util.HashSet;
|
|
import java.util.HashSet;
|
|
-import java.util.List;
|
|
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
|
|
import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
|
|
@@ -55,14 +55,13 @@ public class TestReconstructStripedBlocksWithRackAwareness {
|
|
static {
|
|
static {
|
|
GenericTestUtils.setLogLevel(BlockPlacementPolicy.LOG, Level.ALL);
|
|
GenericTestUtils.setLogLevel(BlockPlacementPolicy.LOG, Level.ALL);
|
|
GenericTestUtils.setLogLevel(BlockManager.blockLog, Level.ALL);
|
|
GenericTestUtils.setLogLevel(BlockManager.blockLog, Level.ALL);
|
|
|
|
+ GenericTestUtils.setLogLevel(BlockManager.LOG, Level.ALL);
|
|
}
|
|
}
|
|
|
|
|
|
private static final String[] hosts = new String[]{"host1", "host2", "host3",
|
|
private static final String[] hosts = new String[]{"host1", "host2", "host3",
|
|
"host4", "host5", "host6", "host7", "host8", "host9", "host10"};
|
|
"host4", "host5", "host6", "host7", "host8", "host9", "host10"};
|
|
private static final String[] racks = new String[]{"/r1", "/r1", "/r2", "/r2",
|
|
private static final String[] racks = new String[]{"/r1", "/r1", "/r2", "/r2",
|
|
"/r3", "/r3", "/r4", "/r4", "/r5", "/r6"};
|
|
"/r3", "/r3", "/r4", "/r4", "/r5", "/r6"};
|
|
- private static final List<String> singleNodeRacks = Arrays.asList("host9", "host10");
|
|
|
|
- private static final short blockNum = (short) (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS);
|
|
|
|
|
|
|
|
private MiniDFSCluster cluster;
|
|
private MiniDFSCluster cluster;
|
|
private DistributedFileSystem fs;
|
|
private DistributedFileSystem fs;
|
|
@@ -94,6 +93,20 @@ public class TestReconstructStripedBlocksWithRackAwareness {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private MiniDFSCluster.DataNodeProperties stopDataNode(String hostname)
|
|
|
|
+ throws IOException {
|
|
|
|
+ MiniDFSCluster.DataNodeProperties dnProp = null;
|
|
|
|
+ for (int i = 0; i < cluster.getDataNodes().size(); i++) {
|
|
|
|
+ DataNode dn = cluster.getDataNodes().get(i);
|
|
|
|
+ if (dn.getDatanodeId().getHostName().equals(hostname)) {
|
|
|
|
+ dnProp = cluster.stopDataNode(i);
|
|
|
|
+ cluster.setDataNodeDead(dn.getDatanodeId());
|
|
|
|
+ LOG.info("stop datanode " + dn.getDatanodeId().getHostName());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return dnProp;
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* When there are all the internal blocks available but they are not placed on
|
|
* When there are all the internal blocks available but they are not placed on
|
|
* enough racks, NameNode should avoid normal decoding reconstruction but copy
|
|
* enough racks, NameNode should avoid normal decoding reconstruction but copy
|
|
@@ -102,24 +115,13 @@ public class TestReconstructStripedBlocksWithRackAwareness {
|
|
* In this test, we first need to create a scenario that a striped block has
|
|
* In this test, we first need to create a scenario that a striped block has
|
|
* all the internal blocks but distributed in <6 racks. Then we check if the
|
|
* all the internal blocks but distributed in <6 racks. Then we check if the
|
|
* replication monitor can correctly schedule the reconstruction work for it.
|
|
* replication monitor can correctly schedule the reconstruction work for it.
|
|
- *
|
|
|
|
- * For the 9 internal blocks + 5 racks setup, the test does the following:
|
|
|
|
- * 1. create a 6 rack cluster with 10 datanodes, where there are 2 racks only
|
|
|
|
- * containing 1 datanodes each
|
|
|
|
- * 2. for a striped block with 9 internal blocks, there must be one internal
|
|
|
|
- * block locating in a single-node rack. find this node and stop it
|
|
|
|
- * 3. namenode will trigger reconstruction for the block and since the cluster
|
|
|
|
- * has only 5 racks remaining, after the reconstruction we have 9 internal
|
|
|
|
- * blocks distributed in 5 racks.
|
|
|
|
- * 4. we bring the datanode back, now the cluster has 6 racks again
|
|
|
|
- * 5. let the datanode call reportBadBlock, this will make the namenode to
|
|
|
|
- * check if the striped block is placed in >= 6 racks, and the namenode will
|
|
|
|
- * put the block into the under-replicated queue
|
|
|
|
- * 6. now we can check if the replication monitor works as expected
|
|
|
|
*/
|
|
*/
|
|
@Test
|
|
@Test
|
|
public void testReconstructForNotEnoughRacks() throws Exception {
|
|
public void testReconstructForNotEnoughRacks() throws Exception {
|
|
|
|
+ MiniDFSCluster.DataNodeProperties host10 = stopDataNode("host10");
|
|
|
|
+
|
|
final Path file = new Path("/foo");
|
|
final Path file = new Path("/foo");
|
|
|
|
+ // the file's block is in 9 dn but 5 racks
|
|
DFSTestUtil.createFile(fs, file,
|
|
DFSTestUtil.createFile(fs, file,
|
|
BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS * 2, (short) 1, 0L);
|
|
BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS * 2, (short) 1, 0L);
|
|
Assert.assertEquals(0, bm.numOfUnderReplicatedBlocks());
|
|
Assert.assertEquals(0, bm.numOfUnderReplicatedBlocks());
|
|
@@ -128,39 +130,6 @@ public class TestReconstructStripedBlocksWithRackAwareness {
|
|
.getINode4Write(file.toString()).asFile();
|
|
.getINode4Write(file.toString()).asFile();
|
|
BlockInfoStriped blockInfo = (BlockInfoStriped) fileNode.getLastBlock();
|
|
BlockInfoStriped blockInfo = (BlockInfoStriped) fileNode.getLastBlock();
|
|
|
|
|
|
- // find the internal block located in the single node rack
|
|
|
|
- Block internalBlock = null;
|
|
|
|
- String hostToStop = null;
|
|
|
|
- for (DatanodeStorageInfo storage : blockInfo.storages) {
|
|
|
|
- if (singleNodeRacks.contains(storage.getDatanodeDescriptor().getHostName())) {
|
|
|
|
- hostToStop = storage.getDatanodeDescriptor().getHostName();
|
|
|
|
- internalBlock = blockInfo.getBlockOnStorage(storage);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- Assert.assertNotNull(internalBlock);
|
|
|
|
- Assert.assertNotNull(hostToStop);
|
|
|
|
-
|
|
|
|
- // delete the block on the chosen datanode
|
|
|
|
- cluster.corruptBlockOnDataNodesByDeletingBlockFile(
|
|
|
|
- new ExtendedBlock(bm.getBlockPoolId(), internalBlock));
|
|
|
|
-
|
|
|
|
- // stop the chosen datanode
|
|
|
|
- MiniDFSCluster.DataNodeProperties dnProp = null;
|
|
|
|
- for (int i = 0; i < cluster.getDataNodes().size(); i++) {
|
|
|
|
- DataNode dn = cluster.getDataNodes().get(i);
|
|
|
|
- if (dn.getDatanodeId().getHostName().equals(hostToStop)) {
|
|
|
|
- dnProp = cluster.stopDataNode(i);
|
|
|
|
- cluster.setDataNodeDead(dn.getDatanodeId());
|
|
|
|
- LOG.info("stop datanode " + dn.getDatanodeId().getHostName());
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- NetworkTopology topology = bm.getDatanodeManager().getNetworkTopology();
|
|
|
|
- Assert.assertEquals(5, topology.getNumOfRacks());
|
|
|
|
-
|
|
|
|
- // make sure the reconstruction work can finish
|
|
|
|
- // now we have 9 internal blocks in 5 racks
|
|
|
|
- DFSTestUtil.waitForReplication(fs, file, blockNum, 15 * 1000);
|
|
|
|
-
|
|
|
|
// we now should have 9 internal blocks distributed in 5 racks
|
|
// we now should have 9 internal blocks distributed in 5 racks
|
|
Set<String> rackSet = new HashSet<>();
|
|
Set<String> rackSet = new HashSet<>();
|
|
for (DatanodeStorageInfo storage : blockInfo.storages) {
|
|
for (DatanodeStorageInfo storage : blockInfo.storages) {
|
|
@@ -169,27 +138,25 @@ public class TestReconstructStripedBlocksWithRackAwareness {
|
|
Assert.assertEquals(5, rackSet.size());
|
|
Assert.assertEquals(5, rackSet.size());
|
|
|
|
|
|
// restart the stopped datanode
|
|
// restart the stopped datanode
|
|
- cluster.restartDataNode(dnProp);
|
|
|
|
|
|
+ cluster.restartDataNode(host10);
|
|
cluster.waitActive();
|
|
cluster.waitActive();
|
|
|
|
|
|
// make sure we have 6 racks again
|
|
// make sure we have 6 racks again
|
|
- topology = bm.getDatanodeManager().getNetworkTopology();
|
|
|
|
|
|
+ NetworkTopology topology = bm.getDatanodeManager().getNetworkTopology();
|
|
Assert.assertEquals(hosts.length, topology.getNumOfLeaves());
|
|
Assert.assertEquals(hosts.length, topology.getNumOfLeaves());
|
|
Assert.assertEquals(6, topology.getNumOfRacks());
|
|
Assert.assertEquals(6, topology.getNumOfRacks());
|
|
|
|
|
|
// pause all the heartbeats
|
|
// pause all the heartbeats
|
|
- DataNode badDn = null;
|
|
|
|
for (DataNode dn : cluster.getDataNodes()) {
|
|
for (DataNode dn : cluster.getDataNodes()) {
|
|
DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true);
|
|
DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true);
|
|
- if (dn.getDatanodeId().getHostName().equals(hostToStop)) {
|
|
|
|
- badDn = dn;
|
|
|
|
- }
|
|
|
|
}
|
|
}
|
|
- assert badDn != null;
|
|
|
|
- // let the DN report the bad block, so that the namenode will put the block
|
|
|
|
- // into under-replicated queue. note that the block still has 9 internal
|
|
|
|
- // blocks but in 5 racks
|
|
|
|
- badDn.reportBadBlocks(new ExtendedBlock(bm.getBlockPoolId(), internalBlock));
|
|
|
|
|
|
+
|
|
|
|
+ fsn.writeLock();
|
|
|
|
+ try {
|
|
|
|
+ bm.processMisReplicatedBlocks();
|
|
|
|
+ } finally {
|
|
|
|
+ fsn.writeUnlock();
|
|
|
|
+ }
|
|
|
|
|
|
// check if replication monitor correctly schedule the replication work
|
|
// check if replication monitor correctly schedule the replication work
|
|
boolean scheduled = false;
|
|
boolean scheduled = false;
|
|
@@ -210,4 +177,42 @@ public class TestReconstructStripedBlocksWithRackAwareness {
|
|
}
|
|
}
|
|
Assert.assertTrue(scheduled);
|
|
Assert.assertTrue(scheduled);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testChooseExcessReplicasToDelete() throws Exception {
|
|
|
|
+ MiniDFSCluster.DataNodeProperties host10 = stopDataNode("host10");
|
|
|
|
+
|
|
|
|
+ final Path file = new Path("/foo");
|
|
|
|
+ DFSTestUtil.createFile(fs, file,
|
|
|
|
+ BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS * 2, (short) 1, 0L);
|
|
|
|
+
|
|
|
|
+ // stop host1
|
|
|
|
+ MiniDFSCluster.DataNodeProperties host1 = stopDataNode("host1");
|
|
|
|
+ // bring host10 back
|
|
|
|
+ cluster.restartDataNode(host10);
|
|
|
|
+ cluster.waitActive();
|
|
|
|
+
|
|
|
|
+ // wait for reconstruction to finish
|
|
|
|
+ final short blockNum = (short) (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS);
|
|
|
|
+ DFSTestUtil.waitForReplication(fs, file, blockNum, 15 * 1000);
|
|
|
|
+
|
|
|
|
+ // restart host1
|
|
|
|
+ cluster.restartDataNode(host1);
|
|
|
|
+ cluster.waitActive();
|
|
|
|
+ for (DataNode dn : cluster.getDataNodes()) {
|
|
|
|
+ if (dn.getDatanodeId().getHostName().equals("host1")) {
|
|
|
|
+ DataNodeTestUtils.triggerBlockReport(dn);
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // make sure the excess replica is detected, and we delete host1's replica
|
|
|
|
+ // so that we have 6 racks
|
|
|
|
+ DFSTestUtil.waitForReplication(fs, file, blockNum, 15 * 1000);
|
|
|
|
+ LocatedBlocks blks = fs.getClient().getLocatedBlocks(file.toString(), 0);
|
|
|
|
+ LocatedStripedBlock block = (LocatedStripedBlock) blks.getLastLocatedBlock();
|
|
|
|
+ for (DatanodeInfo dn : block.getLocations()) {
|
|
|
|
+ Assert.assertFalse(dn.getHostName().equals("host1"));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|