|
@@ -25,6 +25,7 @@ import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.BitSet;
|
|
|
+import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
@@ -44,6 +45,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
|
|
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
|
@@ -81,6 +83,7 @@ public class TestReconstructStripedFile {
|
|
|
Any
|
|
|
}
|
|
|
|
|
|
+ private Configuration conf;
|
|
|
private MiniDFSCluster cluster;
|
|
|
private DistributedFileSystem fs;
|
|
|
// Map: DatanodeID -> datanode index in cluster
|
|
@@ -89,7 +92,7 @@ public class TestReconstructStripedFile {
|
|
|
|
|
|
@Before
|
|
|
public void setup() throws IOException {
|
|
|
- final Configuration conf = new Configuration();
|
|
|
+ conf = new Configuration();
|
|
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
|
|
|
conf.setInt(
|
|
|
DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_READ_BUFFER_SIZE_KEY,
|
|
@@ -263,6 +266,14 @@ public class TestReconstructStripedFile {
|
|
|
return stoppedDNs;
|
|
|
}
|
|
|
|
|
|
+ private static void writeFile(DistributedFileSystem fs, String fileName,
|
|
|
+ int fileLen) throws Exception {
|
|
|
+ final byte[] data = new byte[fileLen];
|
|
|
+ Arrays.fill(data, (byte) 1);
|
|
|
+ DFSTestUtil.writeFile(fs, new Path(fileName), data);
|
|
|
+ StripedFileTestUtil.waitBlockGroupsReported(fs, fileName);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Test the file blocks reconstruction.
|
|
|
* 1. Check the replica is reconstructed in the target datanode,
|
|
@@ -278,10 +289,7 @@ public class TestReconstructStripedFile {
|
|
|
|
|
|
Path file = new Path(fileName);
|
|
|
|
|
|
- final byte[] data = new byte[fileLen];
|
|
|
- Arrays.fill(data, (byte) 1);
|
|
|
- DFSTestUtil.writeFile(fs, file, data);
|
|
|
- StripedFileTestUtil.waitBlockGroupsReported(fs, fileName);
|
|
|
+ writeFile(fs, fileName, fileLen);
|
|
|
|
|
|
LocatedBlocks locatedBlocks =
|
|
|
StripedFileTestUtil.getLocatedBlocks(file, fs);
|
|
@@ -424,4 +432,60 @@ public class TestReconstructStripedFile {
|
|
|
ecTasks.add(invalidECInfo);
|
|
|
dataNode.getErasureCodingWorker().processErasureCodingTasks(ecTasks);
|
|
|
}
|
|
|
+
|
|
|
+ // HDFS-12044
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testNNSendsErasureCodingTasks() throws Exception {
|
|
|
+ testNNSendsErasureCodingTasks(1);
|
|
|
+ testNNSendsErasureCodingTasks(2);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void testNNSendsErasureCodingTasks(int deadDN) throws Exception {
|
|
|
+ cluster.shutdown();
|
|
|
+
|
|
|
+ final int numDataNodes = dnNum + 1;
|
|
|
+ conf.setInt(
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_KEY, 10);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 20);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_STRIPED_BLK_THREADS_KEY,
|
|
|
+ 2);
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .numDataNodes(numDataNodes).build();
|
|
|
+ cluster.waitActive();
|
|
|
+ fs = cluster.getFileSystem();
|
|
|
+ ErasureCodingPolicy policy = StripedFileTestUtil.getDefaultECPolicy();
|
|
|
+ fs.getClient().setErasureCodingPolicy("/", policy.getName());
|
|
|
+
|
|
|
+ final int fileLen = cellSize * ecPolicy.getNumDataUnits() * 2;
|
|
|
+ for (int i = 0; i < 100; i++) {
|
|
|
+ writeFile(fs, "/ec-file-" + i, fileLen);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Inject data-loss by tear down desired number of DataNodes.
|
|
|
+ assertTrue(policy.getNumParityUnits() >= deadDN);
|
|
|
+ List<DataNode> dataNodes = new ArrayList<>(cluster.getDataNodes());
|
|
|
+ Collections.shuffle(dataNodes);
|
|
|
+ for (DataNode dn : dataNodes.subList(0, deadDN)) {
|
|
|
+ shutdownDataNode(dn);
|
|
|
+ }
|
|
|
+
|
|
|
+ final FSNamesystem ns = cluster.getNamesystem();
|
|
|
+ GenericTestUtils.waitFor(() -> ns.getPendingDeletionBlocks() == 0,
|
|
|
+ 500, 30000);
|
|
|
+
|
|
|
+ // Make sure that all pending reconstruction tasks can be processed.
|
|
|
+ while (ns.getPendingReconstructionBlocks() > 0) {
|
|
|
+ long timeoutPending = ns.getNumTimedOutPendingReconstructions();
|
|
|
+ assertTrue(String.format("Found %d timeout pending reconstruction tasks",
|
|
|
+ timeoutPending), timeoutPending == 0);
|
|
|
+ Thread.sleep(1000);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Verify all DN reaches zero xmitsInProgress.
|
|
|
+ GenericTestUtils.waitFor(() ->
|
|
|
+ cluster.getDataNodes().stream().mapToInt(
|
|
|
+ DataNode::getXmitsInProgress).sum() == 0,
|
|
|
+ 500, 30000
|
|
|
+ );
|
|
|
+ }
|
|
|
}
|