|
@@ -22,7 +22,9 @@ import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.BlockLocation;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
@@ -40,6 +42,7 @@ public class TestReadStripedFileWithMissingBlocks {
|
|
|
.getLog(TestReadStripedFileWithMissingBlocks.class);
|
|
|
private MiniDFSCluster cluster;
|
|
|
private DistributedFileSystem fs;
|
|
|
+ private DFSClient dfsClient;
|
|
|
private Configuration conf = new HdfsConfiguration();
|
|
|
private final ErasureCodingPolicy ecPolicy =
|
|
|
StripedFileTestUtil.getDefaultECPolicy();
|
|
@@ -49,7 +52,9 @@ public class TestReadStripedFileWithMissingBlocks {
|
|
|
private final int stripPerBlock = 4;
|
|
|
private final int blockSize = stripPerBlock * cellSize;
|
|
|
private final int blockGroupSize = blockSize * dataBlocks;
|
|
|
- private final int numDNs = dataBlocks + parityBlocks;
|
|
|
+ // Starting with two more datanodes, minimum 9 should be up for
|
|
|
+ // test to pass.
|
|
|
+ private final int numDNs = dataBlocks + parityBlocks + 2;
|
|
|
private final int fileLength = blockSize * dataBlocks + 123;
|
|
|
|
|
|
@Rule
|
|
@@ -63,6 +68,8 @@ public class TestReadStripedFileWithMissingBlocks {
|
|
|
"/", ecPolicy.getName());
|
|
|
fs = cluster.getFileSystem();
|
|
|
fs.enableErasureCodingPolicy(ecPolicy.getName());
|
|
|
+ dfsClient = new DFSClient(cluster.getNameNode(0).getNameNodeAddress(),
|
|
|
+ conf);
|
|
|
}
|
|
|
|
|
|
public void tearDown() throws IOException {
|
|
@@ -74,29 +81,34 @@ public class TestReadStripedFileWithMissingBlocks {
|
|
|
|
|
|
@Test
|
|
|
public void testReadFileWithMissingBlocks() throws Exception {
|
|
|
- for (int missingData = 1; missingData <= dataBlocks; missingData++) {
|
|
|
- for (int missingParity = 0; missingParity <=
|
|
|
- parityBlocks - missingData; missingParity++) {
|
|
|
- try {
|
|
|
- setup();
|
|
|
- readFileWithMissingBlocks(new Path("/foo"), fileLength,
|
|
|
- missingData, missingParity);
|
|
|
- } finally {
|
|
|
- tearDown();
|
|
|
+ try {
|
|
|
+ setup();
|
|
|
+ Path srcPath = new Path("/foo");
|
|
|
+ final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
|
|
|
+ DFSTestUtil.writeFile(fs, srcPath, new String(expected));
|
|
|
+ StripedFileTestUtil
|
|
|
+ .waitBlockGroupsReported(fs, srcPath.toUri().getPath());
|
|
|
+ StripedFileTestUtil.verifyLength(fs, srcPath, fileLength);
|
|
|
+
|
|
|
+ for (int missingData = 1; missingData <= dataBlocks; missingData++) {
|
|
|
+ for (int missingParity = 0; missingParity <=
|
|
|
+ parityBlocks - missingData; missingParity++) {
|
|
|
+ readFileWithMissingBlocks(srcPath, fileLength, missingData,
|
|
|
+ missingParity, expected);
|
|
|
}
|
|
|
}
|
|
|
+ } finally {
|
|
|
+ tearDown();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+
|
|
|
private void readFileWithMissingBlocks(Path srcPath, int fileLength,
|
|
|
- int missingDataNum, int missingParityNum)
|
|
|
+ int missingDataNum, int missingParityNum, byte[] expected)
|
|
|
throws Exception {
|
|
|
LOG.info("readFileWithMissingBlocks: (" + missingDataNum + ","
|
|
|
+ missingParityNum + ")");
|
|
|
- final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
|
|
|
- DFSTestUtil.writeFile(fs, srcPath, new String(expected));
|
|
|
- StripedFileTestUtil.waitBlockGroupsReported(fs, srcPath.toUri().getPath());
|
|
|
- StripedFileTestUtil.verifyLength(fs, srcPath, fileLength);
|
|
|
+
|
|
|
int dataBlocks = (fileLength - 1) / cellSize + 1;
|
|
|
BlockLocation[] locs = fs.getFileBlockLocations(srcPath, 0, cellSize);
|
|
|
|
|
@@ -112,7 +124,8 @@ public class TestReadStripedFileWithMissingBlocks {
|
|
|
|
|
|
// make sure there are missing block locations
|
|
|
BlockLocation[] newLocs = fs.getFileBlockLocations(srcPath, 0, cellSize);
|
|
|
- Assert.assertTrue(newLocs[0].getNames().length < locs[0].getNames().length);
|
|
|
+ Assert.assertTrue(
|
|
|
+ newLocs[0].getNames().length < locs[0].getNames().length);
|
|
|
|
|
|
byte[] smallBuf = new byte[1024];
|
|
|
byte[] largeBuf = new byte[fileLength + 100];
|
|
@@ -120,10 +133,18 @@ public class TestReadStripedFileWithMissingBlocks {
|
|
|
blockGroupSize);
|
|
|
StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected,
|
|
|
smallBuf);
|
|
|
- StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf);
|
|
|
+ StripedFileTestUtil
|
|
|
+ .verifyPread(fs, srcPath, fileLength, expected, largeBuf);
|
|
|
+ restartDeadDataNodes();
|
|
|
+ }
|
|
|
|
|
|
- // delete the file
|
|
|
- fs.delete(srcPath, true);
|
|
|
+ private void restartDeadDataNodes() throws IOException {
|
|
|
+ DatanodeInfo[] deadNodes = dfsClient
|
|
|
+ .datanodeReport(DatanodeReportType.DEAD);
|
|
|
+ for (DatanodeInfo dnInfo : deadNodes) {
|
|
|
+ cluster.restartDataNode(dnInfo.getXferAddr());
|
|
|
+ }
|
|
|
+ cluster.triggerHeartbeats();
|
|
|
}
|
|
|
|
|
|
private void stopDataNodes(BlockLocation[] locs, int[] datanodes)
|