|
@@ -32,6 +32,8 @@ import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.net.unix.DomainSocket;
|
|
|
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
|
@@ -92,22 +94,35 @@ public class TestBlockReaderLocal {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static interface BlockReaderLocalTest {
|
|
|
- final int TEST_LENGTH = 12345;
|
|
|
+ private static class BlockReaderLocalTest {
|
|
|
+ final static int TEST_LENGTH = 12345;
|
|
|
+ final static int BYTES_PER_CHECKSUM = 512;
|
|
|
+
|
|
|
+ public void setConfiguration(HdfsConfiguration conf) {
|
|
|
+ // default: no-op
|
|
|
+ }
|
|
|
public void setup(File blockFile, boolean usingChecksums)
|
|
|
- throws IOException;
|
|
|
+ throws IOException {
|
|
|
+ // default: no-op
|
|
|
+ }
|
|
|
public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
- throws IOException;
|
|
|
+ throws IOException {
|
|
|
+ // default: no-op
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
public void runBlockReaderLocalTest(BlockReaderLocalTest test,
|
|
|
- boolean checksum) throws IOException {
|
|
|
+ boolean checksum, long readahead) throws IOException {
|
|
|
MiniDFSCluster cluster = null;
|
|
|
HdfsConfiguration conf = new HdfsConfiguration();
|
|
|
conf.setBoolean(DFSConfigKeys.
|
|
|
DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, !checksum);
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY,
|
|
|
+ BlockReaderLocalTest.BYTES_PER_CHECKSUM);
|
|
|
conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "CRC32C");
|
|
|
- FileInputStream dataIn = null, checkIn = null;
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_CLIENT_CACHE_READAHEAD, readahead);
|
|
|
+ test.setConfiguration(conf);
|
|
|
+ FileInputStream dataIn = null, metaIn = null;
|
|
|
final Path TEST_PATH = new Path("/a");
|
|
|
final long RANDOM_SEED = 4567L;
|
|
|
BlockReaderLocal blockReaderLocal = null;
|
|
@@ -143,45 +158,51 @@ public class TestBlockReaderLocal {
|
|
|
cluster.shutdown();
|
|
|
cluster = null;
|
|
|
test.setup(dataFile, checksum);
|
|
|
- dataIn = new FileInputStream(dataFile);
|
|
|
- checkIn = new FileInputStream(metaFile);
|
|
|
- blockReaderLocal = new BlockReaderLocal(new DFSClient.Conf(conf),
|
|
|
- TEST_PATH.getName(), block, 0, -1,
|
|
|
- dataIn, checkIn, datanodeID, checksum, null);
|
|
|
+ FileInputStream streams[] = {
|
|
|
+ new FileInputStream(dataFile),
|
|
|
+ new FileInputStream(metaFile)
|
|
|
+ };
|
|
|
+ dataIn = streams[0];
|
|
|
+ metaIn = streams[1];
|
|
|
+ blockReaderLocal = new BlockReaderLocal.Builder(
|
|
|
+ new DFSClient.Conf(conf)).
|
|
|
+ setFilename(TEST_PATH.getName()).
|
|
|
+ setBlock(block).
|
|
|
+ setStreams(streams).
|
|
|
+ setDatanodeID(datanodeID).
|
|
|
+ setCachingStrategy(new CachingStrategy(false, readahead)).
|
|
|
+ setVerifyChecksum(checksum).
|
|
|
+ setBlockMetadataHeader(BlockMetadataHeader.preadHeader(
|
|
|
+ metaIn.getChannel())).
|
|
|
+ build();
|
|
|
dataIn = null;
|
|
|
- checkIn = null;
|
|
|
+ metaIn = null;
|
|
|
test.doTest(blockReaderLocal, original);
|
|
|
+ // BlockReaderLocal should not alter the file position.
|
|
|
+ Assert.assertEquals(0, streams[0].getChannel().position());
|
|
|
+ Assert.assertEquals(0, streams[1].getChannel().position());
|
|
|
} finally {
|
|
|
if (fsIn != null) fsIn.close();
|
|
|
if (fs != null) fs.close();
|
|
|
if (cluster != null) cluster.shutdown();
|
|
|
if (dataIn != null) dataIn.close();
|
|
|
- if (checkIn != null) checkIn.close();
|
|
|
+ if (metaIn != null) metaIn.close();
|
|
|
if (blockReaderLocal != null) blockReaderLocal.close();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
private static class TestBlockReaderLocalImmediateClose
|
|
|
- implements BlockReaderLocalTest {
|
|
|
- @Override
|
|
|
- public void setup(File blockFile, boolean usingChecksums)
|
|
|
- throws IOException { }
|
|
|
- @Override
|
|
|
- public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
- throws IOException { }
|
|
|
+ extends BlockReaderLocalTest {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testBlockReaderLocalImmediateClose() throws IOException {
|
|
|
- runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), true);
|
|
|
- runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), false);
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), true, 0);
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), false, 0);
|
|
|
}
|
|
|
|
|
|
private static class TestBlockReaderSimpleReads
|
|
|
- implements BlockReaderLocalTest {
|
|
|
- @Override
|
|
|
- public void setup(File blockFile, boolean usingChecksums)
|
|
|
- throws IOException { }
|
|
|
+ extends BlockReaderLocalTest {
|
|
|
@Override
|
|
|
public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
throws IOException {
|
|
@@ -194,24 +215,43 @@ public class TestBlockReaderLocal {
|
|
|
assertArrayRegionsEqual(original, 1024, buf, 1024, 513);
|
|
|
reader.readFully(buf, 1537, 514);
|
|
|
assertArrayRegionsEqual(original, 1537, buf, 1537, 514);
|
|
|
+ // Readahead is always at least the size of one chunk in this test.
|
|
|
+ Assert.assertTrue(reader.getMaxReadaheadLength() >=
|
|
|
+ BlockReaderLocalTest.BYTES_PER_CHECKSUM);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testBlockReaderSimpleReads() throws IOException {
|
|
|
- runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true);
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true,
|
|
|
+ DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderSimpleReadsShortReadahead() throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true,
|
|
|
+ BlockReaderLocalTest.BYTES_PER_CHECKSUM - 1);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testBlockReaderSimpleReadsNoChecksum() throws IOException {
|
|
|
- runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), false);
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), false,
|
|
|
+ DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderSimpleReadsNoReadahead() throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true, 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderSimpleReadsNoChecksumNoReadahead()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), false, 0);
|
|
|
}
|
|
|
|
|
|
private static class TestBlockReaderLocalArrayReads2
|
|
|
- implements BlockReaderLocalTest {
|
|
|
- @Override
|
|
|
- public void setup(File blockFile, boolean usingChecksums)
|
|
|
- throws IOException { }
|
|
|
+ extends BlockReaderLocalTest {
|
|
|
@Override
|
|
|
public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
throws IOException {
|
|
@@ -234,21 +274,30 @@ public class TestBlockReaderLocal {
|
|
|
@Test
|
|
|
public void testBlockReaderLocalArrayReads2() throws IOException {
|
|
|
runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
|
|
|
- true);
|
|
|
+ true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testBlockReaderLocalArrayReads2NoChecksum()
|
|
|
throws IOException {
|
|
|
runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
|
|
|
- false);
|
|
|
+ false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalArrayReads2NoReadahead()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(), true, 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalArrayReads2NoChecksumNoReadahead()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(), false, 0);
|
|
|
}
|
|
|
|
|
|
private static class TestBlockReaderLocalByteBufferReads
|
|
|
- implements BlockReaderLocalTest {
|
|
|
- @Override
|
|
|
- public void setup(File blockFile, boolean usingChecksums)
|
|
|
- throws IOException { }
|
|
|
+ extends BlockReaderLocalTest {
|
|
|
@Override
|
|
|
public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
throws IOException {
|
|
@@ -268,19 +317,105 @@ public class TestBlockReaderLocal {
|
|
|
@Test
|
|
|
public void testBlockReaderLocalByteBufferReads()
|
|
|
throws IOException {
|
|
|
- runBlockReaderLocalTest(
|
|
|
- new TestBlockReaderLocalByteBufferReads(), true);
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferReads(),
|
|
|
+ true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testBlockReaderLocalByteBufferReadsNoChecksum()
|
|
|
throws IOException {
|
|
|
runBlockReaderLocalTest(
|
|
|
- new TestBlockReaderLocalByteBufferReads(), false);
|
|
|
+ new TestBlockReaderLocalByteBufferReads(),
|
|
|
+ false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalByteBufferReadsNoReadahead()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferReads(),
|
|
|
+ true, 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalByteBufferReadsNoChecksumNoReadahead()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferReads(),
|
|
|
+ false, 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test reads that bypass the bounce buffer (because they are aligned
|
|
|
+ * and bigger than the readahead).
|
|
|
+ */
|
|
|
+ private static class TestBlockReaderLocalByteBufferFastLaneReads
|
|
|
+ extends BlockReaderLocalTest {
|
|
|
+ @Override
|
|
|
+ public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
+ throws IOException {
|
|
|
+ ByteBuffer buf = ByteBuffer.allocateDirect(TEST_LENGTH);
|
|
|
+ readFully(reader, buf, 0, 5120);
|
|
|
+ buf.flip();
|
|
|
+ assertArrayRegionsEqual(original, 0,
|
|
|
+ DFSTestUtil.asArray(buf), 0,
|
|
|
+ 5120);
|
|
|
+ reader.skip(1537);
|
|
|
+ readFully(reader, buf, 0, 1);
|
|
|
+ buf.flip();
|
|
|
+ assertArrayRegionsEqual(original, 6657,
|
|
|
+ DFSTestUtil.asArray(buf), 0,
|
|
|
+ 1);
|
|
|
+ reader.setMlocked(true);
|
|
|
+ readFully(reader, buf, 0, 5120);
|
|
|
+ buf.flip();
|
|
|
+ assertArrayRegionsEqual(original, 6658,
|
|
|
+ DFSTestUtil.asArray(buf), 0,
|
|
|
+ 5120);
|
|
|
+ reader.setMlocked(false);
|
|
|
+ readFully(reader, buf, 0, 513);
|
|
|
+ buf.flip();
|
|
|
+ assertArrayRegionsEqual(original, 11778,
|
|
|
+ DFSTestUtil.asArray(buf), 0,
|
|
|
+ 513);
|
|
|
+ reader.skip(3);
|
|
|
+ readFully(reader, buf, 0, 50);
|
|
|
+ buf.flip();
|
|
|
+ assertArrayRegionsEqual(original, 12294,
|
|
|
+ DFSTestUtil.asArray(buf), 0,
|
|
|
+ 50);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalByteBufferFastLaneReads()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferFastLaneReads(),
|
|
|
+ true, 2 * BlockReaderLocalTest.BYTES_PER_CHECKSUM);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalByteBufferFastLaneReadsNoChecksum()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(
|
|
|
+ new TestBlockReaderLocalByteBufferFastLaneReads(),
|
|
|
+ false, 2 * BlockReaderLocalTest.BYTES_PER_CHECKSUM);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalByteBufferFastLaneReadsNoReadahead()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferFastLaneReads(),
|
|
|
+ true, 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalByteBufferFastLaneReadsNoChecksumNoReadahead()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferFastLaneReads(),
|
|
|
+ false, 0);
|
|
|
+ }
|
|
|
+
|
|
|
private static class TestBlockReaderLocalReadCorruptStart
|
|
|
- implements BlockReaderLocalTest {
|
|
|
+ extends BlockReaderLocalTest {
|
|
|
boolean usingChecksums = false;
|
|
|
@Override
|
|
|
public void setup(File blockFile, boolean usingChecksums)
|
|
@@ -314,11 +449,12 @@ public class TestBlockReaderLocal {
|
|
|
@Test
|
|
|
public void testBlockReaderLocalReadCorruptStart()
|
|
|
throws IOException {
|
|
|
- runBlockReaderLocalTest(new TestBlockReaderLocalReadCorruptStart(), true);
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalReadCorruptStart(), true,
|
|
|
+ DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
}
|
|
|
|
|
|
private static class TestBlockReaderLocalReadCorrupt
|
|
|
- implements BlockReaderLocalTest {
|
|
|
+ extends BlockReaderLocalTest {
|
|
|
boolean usingChecksums = false;
|
|
|
@Override
|
|
|
public void setup(File blockFile, boolean usingChecksums)
|
|
@@ -364,8 +500,136 @@ public class TestBlockReaderLocal {
|
|
|
@Test
|
|
|
public void testBlockReaderLocalReadCorrupt()
|
|
|
throws IOException {
|
|
|
- runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), true);
|
|
|
- runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), false);
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), true,
|
|
|
+ DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalReadCorruptNoChecksum()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), false,
|
|
|
+ DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalReadCorruptNoReadahead()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), true, 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalReadCorruptNoChecksumNoReadahead()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), false, 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ private static class TestBlockReaderLocalWithMlockChanges
|
|
|
+ extends BlockReaderLocalTest {
|
|
|
+ @Override
|
|
|
+ public void setup(File blockFile, boolean usingChecksums)
|
|
|
+ throws IOException {
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
+ throws IOException {
|
|
|
+ ByteBuffer buf = ByteBuffer.wrap(new byte[TEST_LENGTH]);
|
|
|
+ reader.skip(1);
|
|
|
+ readFully(reader, buf, 1, 9);
|
|
|
+ assertArrayRegionsEqual(original, 1, buf.array(), 1, 9);
|
|
|
+ readFully(reader, buf, 10, 100);
|
|
|
+ assertArrayRegionsEqual(original, 10, buf.array(), 10, 100);
|
|
|
+ reader.setMlocked(true);
|
|
|
+ readFully(reader, buf, 110, 700);
|
|
|
+ assertArrayRegionsEqual(original, 110, buf.array(), 110, 700);
|
|
|
+ reader.setMlocked(false);
|
|
|
+ reader.skip(1); // skip from offset 810 to offset 811
|
|
|
+ readFully(reader, buf, 811, 5);
|
|
|
+ assertArrayRegionsEqual(original, 811, buf.array(), 811, 5);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalWithMlockChanges()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalWithMlockChanges(),
|
|
|
+ true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalWithMlockChangesNoChecksum()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalWithMlockChanges(),
|
|
|
+ false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalWithMlockChangesNoReadahead()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalWithMlockChanges(),
|
|
|
+ true, 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalWithMlockChangesNoChecksumNoReadahead()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalWithMlockChanges(),
|
|
|
+ false, 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ private static class TestBlockReaderLocalOnFileWithoutChecksum
|
|
|
+ extends BlockReaderLocalTest {
|
|
|
+ @Override
|
|
|
+ public void setConfiguration(HdfsConfiguration conf) {
|
|
|
+ conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "NULL");
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void doTest(BlockReaderLocal reader, byte original[])
|
|
|
+ throws IOException {
|
|
|
+ Assert.assertTrue(!reader.getVerifyChecksum());
|
|
|
+ ByteBuffer buf = ByteBuffer.wrap(new byte[TEST_LENGTH]);
|
|
|
+ reader.skip(1);
|
|
|
+ readFully(reader, buf, 1, 9);
|
|
|
+ assertArrayRegionsEqual(original, 1, buf.array(), 1, 9);
|
|
|
+ readFully(reader, buf, 10, 100);
|
|
|
+ assertArrayRegionsEqual(original, 10, buf.array(), 10, 100);
|
|
|
+ reader.setMlocked(true);
|
|
|
+ readFully(reader, buf, 110, 700);
|
|
|
+ assertArrayRegionsEqual(original, 110, buf.array(), 110, 700);
|
|
|
+ reader.setMlocked(false);
|
|
|
+ reader.skip(1); // skip from offset 810 to offset 811
|
|
|
+ readFully(reader, buf, 811, 5);
|
|
|
+ assertArrayRegionsEqual(original, 811, buf.array(), 811, 5);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalOnFileWithoutChecksum()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
|
|
|
+ true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalOnFileWithoutChecksumNoChecksum()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
|
|
|
+ false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalOnFileWithoutChecksumNoReadahead()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
|
|
|
+ true, 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockReaderLocalOnFileWithoutChecksumNoChecksumNoReadahead()
|
|
|
+ throws IOException {
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
|
|
|
+ false, 0);
|
|
|
}
|
|
|
|
|
|
@Test(timeout=60000)
|