|
@@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.StripedFileTestUtil;
|
|
|
+import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
@@ -45,6 +46,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
|
|
|
|
|
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
|
import org.slf4j.Logger;
|
|
@@ -84,6 +86,7 @@ public class TestReconstructStripedBlocks {
|
|
|
// chooseUnderReplicatedBlocks at once.
|
|
|
conf.setInt(
|
|
|
DFSConfigKeys.DFS_NAMENODE_REPLICATION_WORK_MULTIPLIER_PER_ITERATION, 5);
|
|
|
+
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -430,4 +433,86 @@ public class TestReconstructStripedBlocks {
|
|
|
dfsCluster.shutdown();
|
|
|
}
|
|
|
}
|
|
|
+ private byte[] writeStripedFile(DistributedFileSystem fs, Path ecFile,
|
|
|
+ int writeBytes) throws Exception {
|
|
|
+ byte[] bytes = StripedFileTestUtil.generateBytes(writeBytes);
|
|
|
+ DFSTestUtil.writeFile(fs, ecFile, new String(bytes));
|
|
|
+ StripedFileTestUtil.waitBlockGroupsReported(fs, ecFile.toString());
|
|
|
+
|
|
|
+ return bytes;
|
|
|
+ }
|
|
|
+ @Test
|
|
|
+ public void testReconstrutionWithBusyBlock1() throws Exception {
|
|
|
+ //When the index of busy block is smaller than the missing block
|
|
|
+ //[0(busy),1(busy),3,4,5,6,7,8]
|
|
|
+ int busyNodeIndex1 = 0;
|
|
|
+ int busyNodeIndex2 = 1;
|
|
|
+ int deadNodeIndex = 2;
|
|
|
+ final Path ecDir = new Path(GenericTestUtils.getRandomizedTempPath());
|
|
|
+ final Path ecFile = new Path(ecDir, "testReconstrutionWithBusyBlock1");
|
|
|
+ int writeBytes = cellSize * dataBlocks;
|
|
|
+ HdfsConfiguration conf = new HdfsConfiguration();
|
|
|
+ initConf(conf);
|
|
|
+ conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, false);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
|
|
|
+ 2000);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
|
|
|
+ 1000);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY,
|
|
|
+ 4);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
|
|
|
+ 1);
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(groupSize + 5)
|
|
|
+ .build();
|
|
|
+ cluster.waitActive();
|
|
|
+ DistributedFileSystem dfs = cluster.getFileSystem(0);
|
|
|
+ dfs.enableErasureCodingPolicy(
|
|
|
+ StripedFileTestUtil.getDefaultECPolicy().getName());
|
|
|
+ dfs.mkdirs(ecDir);
|
|
|
+ dfs.setErasureCodingPolicy(ecDir,
|
|
|
+ StripedFileTestUtil.getDefaultECPolicy().getName());
|
|
|
+ byte[] originBytesArray = writeStripedFile(dfs, ecFile, writeBytes);
|
|
|
+ List<LocatedBlock> lbs = ((HdfsDataInputStream) dfs.open(ecFile))
|
|
|
+ .getAllBlocks();
|
|
|
+ LocatedStripedBlock lsb = (LocatedStripedBlock) lbs.get(0);
|
|
|
+ DatanodeInfo[] dnList = lsb.getLocations();
|
|
|
+ BlockManager bm = cluster.getNamesystem().getBlockManager();
|
|
|
+ BlockInfoStriped blockInfo =
|
|
|
+ (BlockInfoStriped) bm.getStoredBlock(
|
|
|
+ new Block(lsb.getBlock().getBlockId()));
|
|
|
+
|
|
|
+ //1.Make nodes busy
|
|
|
+ DatanodeDescriptor busyNode = bm.getDatanodeManager()
|
|
|
+ .getDatanode(dnList[busyNodeIndex1].getDatanodeUuid());
|
|
|
+ for (int j = 0; j < maxReplicationStreams; j++) {
|
|
|
+ busyNode.incrementPendingReplicationWithoutTargets();
|
|
|
+ }
|
|
|
+ DatanodeDescriptor busyNode2 = bm.getDatanodeManager()
|
|
|
+ .getDatanode(dnList[busyNodeIndex2].getDatanodeUuid());
|
|
|
+ for (int j = 0; j < maxReplicationStreams; j++) {
|
|
|
+ busyNode2.incrementPendingReplicationWithoutTargets();
|
|
|
+ }
|
|
|
+
|
|
|
+ //2.Make a node missing
|
|
|
+ DataNode dn = cluster.getDataNode(dnList[deadNodeIndex].getIpcPort());
|
|
|
+ cluster.stopDataNode(dnList[deadNodeIndex].getXferAddr());
|
|
|
+ cluster.setDataNodeDead(dn.getDatanodeId());
|
|
|
+
|
|
|
+ //3.Whether there is excess replicas or not during the recovery?
|
|
|
+ assertEquals(8, bm.countNodes(blockInfo).liveReplicas());
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(
|
|
|
+ () -> {
|
|
|
+ return bm.countNodes(blockInfo).liveReplicas() == 9||
|
|
|
+ bm.countNodes(blockInfo).excessReplicas() >= 1||
|
|
|
+ bm.countNodes(blockInfo).redundantInternalBlocks() >= 1;
|
|
|
+ },
|
|
|
+ 10, 100000);
|
|
|
+
|
|
|
+ assertEquals(0, bm.countNodes(blockInfo).excessReplicas());
|
|
|
+ assertEquals(9, bm.countNodes(blockInfo).liveReplicas());
|
|
|
+ }
|
|
|
+
|
|
|
}
|