|
@@ -17,35 +17,37 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.hdfs;
|
|
package org.apache.hadoop.hdfs;
|
|
|
|
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
|
|
|
+import org.apache.commons.logging.Log;
|
|
|
|
+import org.apache.commons.logging.LogFactory;
|
|
import org.apache.hadoop.fs.BlockLocation;
|
|
import org.apache.hadoop.fs.BlockLocation;
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
-import org.apache.hadoop.fs.FileSystem;
|
|
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
|
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
import org.junit.Assert;
|
|
import org.junit.Assert;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
|
|
|
|
+import java.io.File;
|
|
|
|
+import java.io.FileOutputStream;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.nio.ByteBuffer;
|
|
import java.nio.ByteBuffer;
|
|
|
|
|
|
-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.*;
|
|
|
|
|
|
public class TestReadStripedFileWithDecoding {
|
|
public class TestReadStripedFileWithDecoding {
|
|
|
|
+ static final Log LOG = LogFactory.getLog(TestReadStripedFileWithDecoding.class);
|
|
|
|
|
|
private MiniDFSCluster cluster;
|
|
private MiniDFSCluster cluster;
|
|
- private FileSystem fs;
|
|
|
|
|
|
+ private DistributedFileSystem fs;
|
|
|
|
|
|
@Before
|
|
@Before
|
|
public void setup() throws IOException {
|
|
public void setup() throws IOException {
|
|
- Configuration conf = new HdfsConfiguration();
|
|
|
|
- conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
|
|
|
|
- cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
|
|
|
|
|
|
+ cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())
|
|
|
|
+ .numDataNodes(numDNs).build();
|
|
cluster.getFileSystem().getClient().createErasureCodingZone("/",
|
|
cluster.getFileSystem().getClient().createErasureCodingZone("/",
|
|
null, cellSize);
|
|
null, cellSize);
|
|
fs = cluster.getFileSystem();
|
|
fs = cluster.getFileSystem();
|
|
@@ -73,6 +75,112 @@ public class TestReadStripedFileWithDecoding {
|
|
testReadWithDNFailure("/foo", cellSize * dataBlocks, 0);
|
|
testReadWithDNFailure("/foo", cellSize * dataBlocks, 0);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Delete a data block before reading. Verify the decoding works correctly.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ 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);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Corrupt the content of the data block before reading.
|
|
|
|
+ */
|
|
|
|
+ @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());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ verifyRead(file, length, bytes);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private int findFirstDataNode(Path file, long length) throws IOException {
|
|
|
|
+ BlockLocation[] locs = fs.getFileBlockLocations(file, 0, length);
|
|
|
|
+ String name = (locs[0].getNames())[0];
|
|
|
|
+ int dnIndex = 0;
|
|
|
|
+ for (DataNode dn : cluster.getDataNodes()) {
|
|
|
|
+ int port = dn.getXferPort();
|
|
|
|
+ if (name.contains(Integer.toString(port))) {
|
|
|
|
+ return dnIndex;
|
|
|
|
+ }
|
|
|
|
+ dnIndex++;
|
|
|
|
+ }
|
|
|
|
+ return -1;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void verifyRead(Path file, 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());
|
|
|
|
+ }
|
|
|
|
+
|
|
private void testReadWithDNFailure(String file, int fileSize,
|
|
private void testReadWithDNFailure(String file, int fileSize,
|
|
int startOffsetInFile) throws IOException {
|
|
int startOffsetInFile) throws IOException {
|
|
final int failedDNIdx = 2;
|
|
final int failedDNIdx = 2;
|