|
@@ -20,9 +20,7 @@ package org.apache.hadoop.hdfs;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
-import java.io.ByteArrayOutputStream;
|
|
|
import java.io.File;
|
|
|
-import java.io.FileInputStream;
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.BitSet;
|
|
@@ -41,11 +39,9 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
|
|
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo;
|
|
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
|
@@ -80,9 +76,12 @@ public class TestRecoverStripedFile {
|
|
|
public void setup() throws IOException {
|
|
|
conf = new Configuration();
|
|
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
|
|
|
- conf.setInt(DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY, cellSize - 1);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY,
|
|
|
+ cellSize - 1);
|
|
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
|
|
|
- cluster = new MiniDFSCluster.Builder(conf).numDataNodes(dnNum).build();;
|
|
|
+ conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY,
|
|
|
+ false);
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(dnNum).build();
|
|
|
cluster.waitActive();
|
|
|
|
|
|
fs = cluster.getFileSystem();
|
|
@@ -251,82 +250,56 @@ public class TestRecoverStripedFile {
|
|
|
lastBlock.getBlockSize(), cellSize, dataBlkNum, indices[toDead[i]]));
|
|
|
assertTrue(metadatas[i].getName().
|
|
|
endsWith(blocks[i].getGenerationStamp() + ".meta"));
|
|
|
- replicaContents[i] = readReplica(replicas[i]);
|
|
|
+ replicaContents[i] = DFSTestUtil.readFileAsBytes(replicas[i]);
|
|
|
}
|
|
|
|
|
|
int cellsNum = (fileLen - 1) / cellSize + 1;
|
|
|
int groupSize = Math.min(cellsNum, dataBlkNum) + parityBlkNum;
|
|
|
|
|
|
- try {
|
|
|
- DatanodeID[] dnIDs = new DatanodeID[toRecoverBlockNum];
|
|
|
- for (int i = 0; i < toRecoverBlockNum; i++) {
|
|
|
- /*
|
|
|
- * Kill the datanode which contains one replica
|
|
|
- * We need to make sure it dead in namenode: clear its update time and
|
|
|
- * trigger NN to check heartbeat.
|
|
|
- */
|
|
|
- DataNode dn = cluster.getDataNodes().get(deadDnIndices[i]);
|
|
|
- dn.shutdown();
|
|
|
- dnIDs[i] = dn.getDatanodeId();
|
|
|
- }
|
|
|
- setDataNodesDead(dnIDs);
|
|
|
-
|
|
|
- // Check the locatedBlocks of the file again
|
|
|
- locatedBlocks = getLocatedBlocks(file);
|
|
|
- lastBlock = (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
|
|
|
- storageInfos = lastBlock.getLocations();
|
|
|
- assertEquals(storageInfos.length, groupSize - toRecoverBlockNum);
|
|
|
-
|
|
|
- int[] targetDNs = new int[dnNum - groupSize];
|
|
|
- n = 0;
|
|
|
- for (int i = 0; i < dnNum; i++) {
|
|
|
- if (!bitset.get(i)) { // not contain replica of the block.
|
|
|
- targetDNs[n++] = i;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- waitForRecoveryFinished(file, groupSize);
|
|
|
-
|
|
|
- targetDNs = sortTargetsByReplicas(blocks, targetDNs);
|
|
|
-
|
|
|
- // Check the replica on the new target node.
|
|
|
- for (int i = 0; i < toRecoverBlockNum; i++) {
|
|
|
- File replicaAfterRecovery = cluster.getBlockFile(targetDNs[i], blocks[i]);
|
|
|
- File metadataAfterRecovery =
|
|
|
- cluster.getBlockMetadataFile(targetDNs[i], blocks[i]);
|
|
|
- assertEquals(replicaAfterRecovery.length(), replicas[i].length());
|
|
|
- assertTrue(metadataAfterRecovery.getName().
|
|
|
- endsWith(blocks[i].getGenerationStamp() + ".meta"));
|
|
|
- byte[] replicaContentAfterRecovery = readReplica(replicaAfterRecovery);
|
|
|
-
|
|
|
- Assert.assertArrayEquals(replicaContents[i], replicaContentAfterRecovery);
|
|
|
- }
|
|
|
- } finally {
|
|
|
- for (int i = 0; i < toRecoverBlockNum; i++) {
|
|
|
- restartDataNode(toDead[i]);
|
|
|
- }
|
|
|
- cluster.waitActive();
|
|
|
+ for (int i = 0; i < toRecoverBlockNum; i++) {
|
|
|
+ /*
|
|
|
+ * Kill the datanode which contains one replica
|
|
|
+ * We need to make sure it dead in namenode: clear its update time and
|
|
|
+ * trigger NN to check heartbeat.
|
|
|
+ */
|
|
|
+ DataNode dn = cluster.getDataNodes().get(deadDnIndices[i]);
|
|
|
+ dn.shutdown();
|
|
|
+ cluster.setDataNodeDead(dn.getDatanodeId());
|
|
|
}
|
|
|
- fs.delete(file, true);
|
|
|
- }
|
|
|
-
|
|
|
- private void setDataNodesDead(DatanodeID[] dnIDs) throws IOException {
|
|
|
- for (DatanodeID dn : dnIDs) {
|
|
|
- DatanodeDescriptor dnd =
|
|
|
- NameNodeAdapter.getDatanode(cluster.getNamesystem(), dn);
|
|
|
- DFSTestUtil.setDatanodeDead(dnd);
|
|
|
+
|
|
|
+ // Check the locatedBlocks of the file again
|
|
|
+ locatedBlocks = getLocatedBlocks(file);
|
|
|
+ lastBlock = (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
|
|
|
+ storageInfos = lastBlock.getLocations();
|
|
|
+ assertEquals(storageInfos.length, groupSize - toRecoverBlockNum);
|
|
|
+
|
|
|
+ int[] targetDNs = new int[dnNum - groupSize];
|
|
|
+ n = 0;
|
|
|
+ for (int i = 0; i < dnNum; i++) {
|
|
|
+ if (!bitset.get(i)) { // not contain replica of the block.
|
|
|
+ targetDNs[n++] = i;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- BlockManagerTestUtil.checkHeartbeat(cluster.getNamesystem().getBlockManager());
|
|
|
- }
|
|
|
-
|
|
|
- private void restartDataNode(int dn) {
|
|
|
- try {
|
|
|
- cluster.restartDataNode(dn, true, true);
|
|
|
- } catch (IOException e) {
|
|
|
+ waitForRecoveryFinished(file, groupSize);
|
|
|
+
|
|
|
+ targetDNs = sortTargetsByReplicas(blocks, targetDNs);
|
|
|
+
|
|
|
+ // Check the replica on the new target node.
|
|
|
+ for (int i = 0; i < toRecoverBlockNum; i++) {
|
|
|
+ File replicaAfterRecovery = cluster.getBlockFile(targetDNs[i], blocks[i]);
|
|
|
+ File metadataAfterRecovery =
|
|
|
+ cluster.getBlockMetadataFile(targetDNs[i], blocks[i]);
|
|
|
+ assertEquals(replicaAfterRecovery.length(), replicas[i].length());
|
|
|
+ assertTrue(metadataAfterRecovery.getName().
|
|
|
+ endsWith(blocks[i].getGenerationStamp() + ".meta"));
|
|
|
+ byte[] replicaContentAfterRecovery =
|
|
|
+ DFSTestUtil.readFileAsBytes(replicaAfterRecovery);
|
|
|
+
|
|
|
+ Assert.assertArrayEquals(replicaContents[i], replicaContentAfterRecovery);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private int[] sortTargetsByReplicas(ExtendedBlock[] blocks, int[] targetDNs) {
|
|
|
int[] result = new int[blocks.length];
|
|
|
for (int i = 0; i < blocks.length; i++) {
|
|
@@ -347,31 +320,7 @@ public class TestRecoverStripedFile {
|
|
|
}
|
|
|
return result;
|
|
|
}
|
|
|
-
|
|
|
- private byte[] readReplica(File replica) throws IOException {
|
|
|
- int length = (int)replica.length();
|
|
|
- ByteArrayOutputStream content = new ByteArrayOutputStream(length);
|
|
|
- FileInputStream in = new FileInputStream(replica);
|
|
|
- try {
|
|
|
- byte[] buffer = new byte[1024];
|
|
|
- int total = 0;
|
|
|
- while (total < length) {
|
|
|
- int n = in.read(buffer);
|
|
|
- if (n <= 0) {
|
|
|
- break;
|
|
|
- }
|
|
|
- content.write(buffer, 0, n);
|
|
|
- total += n;
|
|
|
- }
|
|
|
- if (total < length) {
|
|
|
- Assert.fail("Failed to read all content of replica");
|
|
|
- }
|
|
|
- return content.toByteArray();
|
|
|
- } finally {
|
|
|
- in.close();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
+
|
|
|
private LocatedBlocks waitForRecoveryFinished(Path file, int groupSize)
|
|
|
throws Exception {
|
|
|
final int ATTEMPTS = 60;
|