|
@@ -20,10 +20,11 @@ package org.apache.hadoop.hdfs;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.fs.BlockLocation;
|
|
|
-import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
@@ -43,13 +44,21 @@ import java.io.FileOutputStream;
|
|
|
import java.io.IOException;
|
|
|
import java.nio.ByteBuffer;
|
|
|
|
|
|
-import static org.apache.hadoop.hdfs.StripedFileTestUtil.*;
|
|
|
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize;
|
|
|
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.cellSize;
|
|
|
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.dataBlocks;
|
|
|
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs;
|
|
|
+import static org.apache.hadoop.hdfs.StripedFileTestUtil.parityBlocks;
|
|
|
|
|
|
public class TestReadStripedFileWithDecoding {
|
|
|
static final Log LOG = LogFactory.getLog(TestReadStripedFileWithDecoding.class);
|
|
|
|
|
|
private MiniDFSCluster cluster;
|
|
|
private DistributedFileSystem fs;
|
|
|
+ private final int smallFileLength = blockSize * dataBlocks - 123;
|
|
|
+ private final int largeFileLength = blockSize * dataBlocks + 123;
|
|
|
+ private final int[] fileLengths = {smallFileLength, largeFileLength};
|
|
|
+ private final int[] dnFailureNums = {1, 2, 3};
|
|
|
|
|
|
@Before
|
|
|
public void setup() throws IOException {
|
|
@@ -67,82 +76,64 @@ public class TestReadStripedFileWithDecoding {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
- public void testReadWithDNFailure1() throws IOException {
|
|
|
- testReadWithDNFailure("/foo", cellSize * (dataBlocks + 2), 0);
|
|
|
- }
|
|
|
-
|
|
|
- @Test
|
|
|
- public void testReadWithDNFailure2() throws IOException {
|
|
|
- testReadWithDNFailure("/foo", cellSize * (dataBlocks + 2), cellSize * 5);
|
|
|
- }
|
|
|
-
|
|
|
- @Test
|
|
|
- public void testReadWithDNFailure3() throws IOException {
|
|
|
- testReadWithDNFailure("/foo", cellSize * dataBlocks, 0);
|
|
|
+ /**
|
|
|
+ * Shutdown tolerable number of Datanode before reading.
|
|
|
+ * Verify the decoding works correctly.
|
|
|
+ */
|
|
|
+ @Test(timeout=300000)
|
|
|
+ public void testReadWithDNFailure() throws IOException {
|
|
|
+ for (int fileLength : fileLengths) {
|
|
|
+ for (int dnFailureNum : dnFailureNums) {
|
|
|
+ try {
|
|
|
+ // setup a new cluster with no dead datanode
|
|
|
+ setup();
|
|
|
+ testReadWithDNFailure(fileLength, dnFailureNum);
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ String fileType = fileLength < (blockSize * dataBlocks) ?
|
|
|
+ "smallFile" : "largeFile";
|
|
|
+ LOG.error("Failed to read file with DN failure:"
|
|
|
+ + " fileType = "+ fileType
|
|
|
+ + ", dnFailureNum = " + dnFailureNum);
|
|
|
+ } finally {
|
|
|
+ // tear down the cluster
|
|
|
+ tearDown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Delete a data block before reading. Verify the decoding works correctly.
|
|
|
+ * Corrupt tolerable number of block before reading.
|
|
|
+ * Verify the decoding works correctly.
|
|
|
*/
|
|
|
- @Test
|
|
|
+ @Test(timeout=300000)
|
|
|
public void testReadCorruptedData() throws IOException {
|
|
|
- // create file
|
|
|
- final Path file = new Path("/partially_deleted");
|
|
|
- final int length = cellSize * dataBlocks * 2;
|
|
|
- final byte[] bytes = StripedFileTestUtil.generateBytes(length);
|
|
|
- DFSTestUtil.writeFile(fs, file, bytes);
|
|
|
-
|
|
|
- // corrupt the first data block
|
|
|
- // find the corresponding data node
|
|
|
- int dnIndex = findFirstDataNode(file, cellSize * dataBlocks);
|
|
|
- Assert.assertNotEquals(-1, dnIndex);
|
|
|
- // find the target block
|
|
|
- LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient()
|
|
|
- .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0);
|
|
|
- final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb,
|
|
|
- cellSize, dataBlocks, parityBlocks);
|
|
|
- // find the target block file
|
|
|
- File storageDir = cluster.getInstanceStorageDir(dnIndex, 0);
|
|
|
- File blkFile = MiniDFSCluster.getBlockFile(storageDir, blks[0].getBlock());
|
|
|
- Assert.assertTrue("Block file does not exist", blkFile.exists());
|
|
|
- // delete the block file
|
|
|
- LOG.info("Deliberately removing file " + blkFile.getName());
|
|
|
- Assert.assertTrue("Cannot remove file", blkFile.delete());
|
|
|
- verifyRead(file, length, bytes);
|
|
|
+ for (int fileLength : fileLengths) {
|
|
|
+ for (int dataDelNum = 1; dataDelNum < 4; dataDelNum++) {
|
|
|
+ for (int parityDelNum = 0; (dataDelNum+parityDelNum) < 4; parityDelNum++) {
|
|
|
+ String src = "/corrupted_" + dataDelNum + "_" + parityDelNum;
|
|
|
+ testReadWithBlockCorrupted(src, fileLength,
|
|
|
+ dataDelNum, parityDelNum, false);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Corrupt the content of the data block before reading.
|
|
|
+ * Delete tolerable number of block before reading.
|
|
|
+ * Verify the decoding works correctly.
|
|
|
*/
|
|
|
- @Test
|
|
|
- public void testReadCorruptedData2() throws IOException {
|
|
|
- // create file
|
|
|
- final Path file = new Path("/partially_corrupted");
|
|
|
- final int length = cellSize * dataBlocks * 2;
|
|
|
- final byte[] bytes = StripedFileTestUtil.generateBytes(length);
|
|
|
- DFSTestUtil.writeFile(fs, file, bytes);
|
|
|
-
|
|
|
- // corrupt the first data block
|
|
|
- // find the first data node
|
|
|
- int dnIndex = findFirstDataNode(file, cellSize * dataBlocks);
|
|
|
- Assert.assertNotEquals(-1, dnIndex);
|
|
|
- // find the first data block
|
|
|
- LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient()
|
|
|
- .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0);
|
|
|
- final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb,
|
|
|
- cellSize, dataBlocks, parityBlocks);
|
|
|
- // find the first block file
|
|
|
- File storageDir = cluster.getInstanceStorageDir(dnIndex, 0);
|
|
|
- File blkFile = MiniDFSCluster.getBlockFile(storageDir, blks[0].getBlock());
|
|
|
- Assert.assertTrue("Block file does not exist", blkFile.exists());
|
|
|
- // corrupt the block file
|
|
|
- LOG.info("Deliberately corrupting file " + blkFile.getName());
|
|
|
- try (FileOutputStream out = new FileOutputStream(blkFile)) {
|
|
|
- out.write("corruption".getBytes());
|
|
|
+ @Test(timeout=300000)
|
|
|
+ public void testReadCorruptedDataByDeleting() throws IOException {
|
|
|
+ for (int fileLength : fileLengths) {
|
|
|
+ for (int dataDelNum = 1; dataDelNum < 4; dataDelNum++) {
|
|
|
+ for (int parityDelNum = 0; (dataDelNum+parityDelNum) < 4; parityDelNum++) {
|
|
|
+ String src = "/deleted_" + dataDelNum + "_" + parityDelNum;
|
|
|
+ testReadWithBlockCorrupted(src, fileLength,
|
|
|
+ dataDelNum, parityDelNum, true);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
-
|
|
|
- verifyRead(file, length, bytes);
|
|
|
}
|
|
|
|
|
|
private int findFirstDataNode(Path file, long length) throws IOException {
|
|
@@ -159,87 +150,45 @@ public class TestReadStripedFileWithDecoding {
|
|
|
return -1;
|
|
|
}
|
|
|
|
|
|
- private void verifyRead(Path file, int length, byte[] expected)
|
|
|
+ private void verifyRead(Path testPath, int length, byte[] expected)
|
|
|
throws IOException {
|
|
|
- // pread
|
|
|
- try (FSDataInputStream fsdis = fs.open(file)) {
|
|
|
- byte[] buf = new byte[length];
|
|
|
- int readLen = fsdis.read(0, buf, 0, buf.length);
|
|
|
- Assert.assertEquals("The fileSize of file should be the same to write size",
|
|
|
- length, readLen);
|
|
|
- Assert.assertArrayEquals(expected, buf);
|
|
|
- }
|
|
|
-
|
|
|
- // stateful read
|
|
|
- ByteBuffer result = ByteBuffer.allocate(length);
|
|
|
- ByteBuffer buf = ByteBuffer.allocate(1024);
|
|
|
- int readLen = 0;
|
|
|
- int ret;
|
|
|
- try (FSDataInputStream in = fs.open(file)) {
|
|
|
- while ((ret = in.read(buf)) >= 0) {
|
|
|
- readLen += ret;
|
|
|
- buf.flip();
|
|
|
- result.put(buf);
|
|
|
- buf.clear();
|
|
|
- }
|
|
|
- }
|
|
|
- Assert.assertEquals("The length of file should be the same to write size",
|
|
|
- length, readLen);
|
|
|
- Assert.assertArrayEquals(expected, result.array());
|
|
|
+ byte[] buffer = new byte[length + 100];
|
|
|
+ StripedFileTestUtil.verifyLength(fs, testPath, length);
|
|
|
+ StripedFileTestUtil.verifyPread(fs, testPath, length, expected, buffer);
|
|
|
+ StripedFileTestUtil.verifyStatefulRead(fs, testPath, length, expected, buffer);
|
|
|
+ StripedFileTestUtil.verifyStatefulRead(fs, testPath, length, expected,
|
|
|
+ ByteBuffer.allocate(length + 100));
|
|
|
+ StripedFileTestUtil.verifySeek(fs, testPath, length);
|
|
|
}
|
|
|
|
|
|
- private void testReadWithDNFailure(String file, int fileSize,
|
|
|
- int startOffsetInFile) throws IOException {
|
|
|
- final int failedDNIdx = 2;
|
|
|
- Path testPath = new Path(file);
|
|
|
- final byte[] bytes = StripedFileTestUtil.generateBytes(fileSize);
|
|
|
+ private void testReadWithDNFailure(int fileLength, int dnFailureNum)
|
|
|
+ throws IOException {
|
|
|
+ String fileType = fileLength < (blockSize * dataBlocks) ?
|
|
|
+ "smallFile" : "largeFile";
|
|
|
+ String src = "/dnFailure_" + dnFailureNum + "_" + fileType;
|
|
|
+ LOG.info("testReadWithDNFailure: file = " + src
|
|
|
+ + ", fileSize = " + fileLength
|
|
|
+ + ", dnFailureNum = " + dnFailureNum);
|
|
|
+
|
|
|
+ Path testPath = new Path(src);
|
|
|
+ final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
|
|
|
DFSTestUtil.writeFile(fs, testPath, bytes);
|
|
|
|
|
|
// shut down the DN that holds an internal data block
|
|
|
BlockLocation[] locs = fs.getFileBlockLocations(testPath, cellSize * 5,
|
|
|
cellSize);
|
|
|
- String name = (locs[0].getNames())[failedDNIdx];
|
|
|
- for (DataNode dn : cluster.getDataNodes()) {
|
|
|
- int port = dn.getXferPort();
|
|
|
- if (name.contains(Integer.toString(port))) {
|
|
|
- dn.shutdown();
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // pread
|
|
|
- try (FSDataInputStream fsdis = fs.open(testPath)) {
|
|
|
- byte[] buf = new byte[fileSize];
|
|
|
- int readLen = fsdis.read(startOffsetInFile, buf, 0, buf.length);
|
|
|
- Assert.assertEquals("The fileSize of file should be the same to write size",
|
|
|
- fileSize - startOffsetInFile, readLen);
|
|
|
-
|
|
|
- byte[] expected = new byte[readLen];
|
|
|
- System.arraycopy(bytes, startOffsetInFile, expected, 0,
|
|
|
- fileSize - startOffsetInFile);
|
|
|
-
|
|
|
- for (int i = startOffsetInFile; i < fileSize; i++) {
|
|
|
- Assert.assertEquals("Byte at " + i + " should be the same",
|
|
|
- expected[i - startOffsetInFile], buf[i - startOffsetInFile]);
|
|
|
+ for (int failedDnIdx = 0; failedDnIdx < dnFailureNum; failedDnIdx++) {
|
|
|
+ String name = (locs[0].getNames())[failedDnIdx];
|
|
|
+ for (DataNode dn : cluster.getDataNodes()) {
|
|
|
+ int port = dn.getXferPort();
|
|
|
+ if (name.contains(Integer.toString(port))) {
|
|
|
+ dn.shutdown();
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- // stateful read
|
|
|
- ByteBuffer result = ByteBuffer.allocate(fileSize);
|
|
|
- ByteBuffer buf = ByteBuffer.allocate(1024);
|
|
|
- int readLen = 0;
|
|
|
- int ret;
|
|
|
- try (FSDataInputStream in = fs.open(testPath)) {
|
|
|
- while ((ret = in.read(buf)) >= 0) {
|
|
|
- readLen += ret;
|
|
|
- buf.flip();
|
|
|
- result.put(buf);
|
|
|
- buf.clear();
|
|
|
- }
|
|
|
- }
|
|
|
- Assert.assertEquals("The length of file should be the same to write size",
|
|
|
- fileSize, readLen);
|
|
|
- Assert.assertArrayEquals(bytes, result.array());
|
|
|
+ // check file length, pread, stateful read and seek
|
|
|
+ verifyRead(testPath, fileLength, bytes);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -279,21 +228,8 @@ public class TestReadStripedFileWithDecoding {
|
|
|
|
|
|
try {
|
|
|
// do stateful read
|
|
|
- ByteBuffer result = ByteBuffer.allocate(length);
|
|
|
- ByteBuffer buf = ByteBuffer.allocate(1024);
|
|
|
- int readLen = 0;
|
|
|
- int ret;
|
|
|
- try (FSDataInputStream in = fs.open(file)) {
|
|
|
- while ((ret = in.read(buf)) >= 0) {
|
|
|
- readLen += ret;
|
|
|
- buf.flip();
|
|
|
- result.put(buf);
|
|
|
- buf.clear();
|
|
|
- }
|
|
|
- }
|
|
|
- Assert.assertEquals("The length of file should be the same to write size",
|
|
|
- length, readLen);
|
|
|
- Assert.assertArrayEquals(bytes, result.array());
|
|
|
+ StripedFileTestUtil.verifyStatefulRead(fs, file, length, bytes,
|
|
|
+ ByteBuffer.allocate(1024));
|
|
|
|
|
|
// check whether the corruption has been reported to the NameNode
|
|
|
final FSNamesystem ns = cluster.getNamesystem();
|
|
@@ -341,4 +277,82 @@ public class TestReadStripedFileWithDecoding {
|
|
|
DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test reading a file with some blocks(data blocks or parity blocks or both)
|
|
|
+ * deleted or corrupted.
|
|
|
+ * @param src file path
|
|
|
+ * @param fileLength file length
|
|
|
+ * @param dataBlkDelNum the deleted or corrupted number of data blocks.
|
|
|
+ * @param parityBlkDelNum the deleted or corrupted number of parity blocks.
|
|
|
+ * @param deleteBlockFile whether block file is deleted or corrupted.
|
|
|
+ * true is to delete the block file.
|
|
|
+ * false is to corrupt the content of the block file.
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ private void testReadWithBlockCorrupted(String src, int fileLength,
|
|
|
+ int dataBlkDelNum, int parityBlkDelNum, boolean deleteBlockFile)
|
|
|
+ throws IOException {
|
|
|
+ LOG.info("testReadWithBlockCorrupted: file = " + src
|
|
|
+ + ", dataBlkDelNum = " + dataBlkDelNum
|
|
|
+ + ", parityBlkDelNum = " + parityBlkDelNum
|
|
|
+ + ", deleteBlockFile? " + deleteBlockFile);
|
|
|
+ int recoverBlkNum = dataBlkDelNum + parityBlkDelNum;
|
|
|
+ Assert.assertTrue("dataBlkDelNum and parityBlkDelNum should be positive",
|
|
|
+ dataBlkDelNum >= 0 && parityBlkDelNum >= 0);
|
|
|
+ Assert.assertTrue("The sum of dataBlkDelNum and parityBlkDelNum " +
|
|
|
+ "should be between 1 ~ " + parityBlocks, recoverBlkNum <= parityBlocks);
|
|
|
+
|
|
|
+ // write a file with the length of writeLen
|
|
|
+ Path srcPath = new Path(src);
|
|
|
+ final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
|
|
|
+ DFSTestUtil.writeFile(fs, srcPath, bytes);
|
|
|
+
|
|
|
+ // delete or corrupt some blocks
|
|
|
+ corruptBlocks(srcPath, dataBlkDelNum, parityBlkDelNum, deleteBlockFile);
|
|
|
+
|
|
|
+ // check the file can be read after some blocks were deleted
|
|
|
+ verifyRead(srcPath, fileLength, bytes);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void corruptBlocks(Path srcPath, int dataBlkDelNum,
|
|
|
+ int parityBlkDelNum, boolean deleteBlockFile) throws IOException {
|
|
|
+ int recoverBlkNum = dataBlkDelNum + parityBlkDelNum;
|
|
|
+
|
|
|
+ LocatedBlocks locatedBlocks = getLocatedBlocks(srcPath);
|
|
|
+ LocatedStripedBlock lastBlock =
|
|
|
+ (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
|
|
|
+
|
|
|
+ int[] delDataBlkIndices = StripedFileTestUtil.randomArray(0, dataBlocks,
|
|
|
+ dataBlkDelNum);
|
|
|
+ Assert.assertNotNull(delDataBlkIndices);
|
|
|
+ int[] delParityBlkIndices = StripedFileTestUtil.randomArray(dataBlocks,
|
|
|
+ dataBlocks + parityBlocks, parityBlkDelNum);
|
|
|
+ Assert.assertNotNull(delParityBlkIndices);
|
|
|
+
|
|
|
+ int[] delBlkIndices = new int[recoverBlkNum];
|
|
|
+ System.arraycopy(delDataBlkIndices, 0,
|
|
|
+ delBlkIndices, 0, delDataBlkIndices.length);
|
|
|
+ System.arraycopy(delParityBlkIndices, 0,
|
|
|
+ delBlkIndices, delDataBlkIndices.length, delParityBlkIndices.length);
|
|
|
+
|
|
|
+ ExtendedBlock[] delBlocks = new ExtendedBlock[recoverBlkNum];
|
|
|
+ for (int i = 0; i < recoverBlkNum; i++) {
|
|
|
+ delBlocks[i] = StripedBlockUtil
|
|
|
+ .constructInternalBlock(lastBlock.getBlock(),
|
|
|
+ cellSize, dataBlocks, delBlkIndices[i]);
|
|
|
+ if (deleteBlockFile) {
|
|
|
+ // delete the block file
|
|
|
+ cluster.corruptBlockOnDataNodesByDeletingBlockFile(delBlocks[i]);
|
|
|
+ } else {
|
|
|
+ // corrupt the block file
|
|
|
+ cluster.corruptBlockOnDataNodes(delBlocks[i]);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private LocatedBlocks getLocatedBlocks(Path filePath) throws IOException {
|
|
|
+ return fs.getClient().getLocatedBlocks(filePath.toString(),
|
|
|
+ 0, Long.MAX_VALUE);
|
|
|
+ }
|
|
|
}
|