|
@@ -116,7 +116,7 @@ public class TestBlockReaderLocal {
|
|
}
|
|
}
|
|
|
|
|
|
private static class BlockReaderLocalTest {
|
|
private static class BlockReaderLocalTest {
|
|
- final static int TEST_LENGTH = 12345;
|
|
|
|
|
|
+ final static int TEST_LENGTH = 1234567;
|
|
final static int BYTES_PER_CHECKSUM = 512;
|
|
final static int BYTES_PER_CHECKSUM = 512;
|
|
|
|
|
|
public void setConfiguration(HdfsConfiguration conf) {
|
|
public void setConfiguration(HdfsConfiguration conf) {
|
|
@@ -130,10 +130,14 @@ public class TestBlockReaderLocal {
|
|
throws IOException {
|
|
throws IOException {
|
|
// default: no-op
|
|
// default: no-op
|
|
}
|
|
}
|
|
- }
|
|
|
|
|
|
+ public void doTest(BlockReaderLocal reader, byte[] original, int shift)
|
|
|
|
+ throws IOException {
|
|
|
|
+ // default: no-op
|
|
|
|
+ } }
|
|
|
|
|
|
public void runBlockReaderLocalTest(BlockReaderLocalTest test,
|
|
public void runBlockReaderLocalTest(BlockReaderLocalTest test,
|
|
- boolean checksum, long readahead) throws IOException {
|
|
|
|
|
|
+ boolean checksum, long readahead, int shortCircuitCachesNum)
|
|
|
|
+ throws IOException {
|
|
Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
|
|
Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
|
|
MiniDFSCluster cluster = null;
|
|
MiniDFSCluster cluster = null;
|
|
HdfsConfiguration conf = new HdfsConfiguration();
|
|
HdfsConfiguration conf = new HdfsConfiguration();
|
|
@@ -143,10 +147,13 @@ public class TestBlockReaderLocal {
|
|
BlockReaderLocalTest.BYTES_PER_CHECKSUM);
|
|
BlockReaderLocalTest.BYTES_PER_CHECKSUM);
|
|
conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "CRC32C");
|
|
conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "CRC32C");
|
|
conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_CACHE_READAHEAD, readahead);
|
|
conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_CACHE_READAHEAD, readahead);
|
|
|
|
+ conf.setInt(HdfsClientConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_NUM,
|
|
|
|
+ shortCircuitCachesNum);
|
|
test.setConfiguration(conf);
|
|
test.setConfiguration(conf);
|
|
FileInputStream dataIn = null, metaIn = null;
|
|
FileInputStream dataIn = null, metaIn = null;
|
|
final Path TEST_PATH = new Path("/a");
|
|
final Path TEST_PATH = new Path("/a");
|
|
final long RANDOM_SEED = 4567L;
|
|
final long RANDOM_SEED = 4567L;
|
|
|
|
+ final int blockSize = 10 * 1024;
|
|
BlockReaderLocal blockReaderLocal = null;
|
|
BlockReaderLocal blockReaderLocal = null;
|
|
FSDataInputStream fsIn = null;
|
|
FSDataInputStream fsIn = null;
|
|
byte original[] = new byte[BlockReaderLocalTest.TEST_LENGTH];
|
|
byte original[] = new byte[BlockReaderLocalTest.TEST_LENGTH];
|
|
@@ -158,8 +165,8 @@ public class TestBlockReaderLocal {
|
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
|
cluster.waitActive();
|
|
cluster.waitActive();
|
|
fs = cluster.getFileSystem();
|
|
fs = cluster.getFileSystem();
|
|
- DFSTestUtil.createFile(fs, TEST_PATH,
|
|
|
|
- BlockReaderLocalTest.TEST_LENGTH, (short)1, RANDOM_SEED);
|
|
|
|
|
|
+ DFSTestUtil.createFile(fs, TEST_PATH, 1024,
|
|
|
|
+ BlockReaderLocalTest.TEST_LENGTH, blockSize, (short)1, RANDOM_SEED);
|
|
try {
|
|
try {
|
|
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
|
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
|
} catch (InterruptedException e) {
|
|
} catch (InterruptedException e) {
|
|
@@ -174,47 +181,52 @@ public class TestBlockReaderLocal {
|
|
BlockReaderLocalTest.TEST_LENGTH);
|
|
BlockReaderLocalTest.TEST_LENGTH);
|
|
fsIn.close();
|
|
fsIn.close();
|
|
fsIn = null;
|
|
fsIn = null;
|
|
- ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, TEST_PATH);
|
|
|
|
- File dataFile = cluster.getBlockFile(0, block);
|
|
|
|
- File metaFile = cluster.getBlockMetadataFile(0, block);
|
|
|
|
-
|
|
|
|
- ShortCircuitCache shortCircuitCache =
|
|
|
|
- ClientContext.getFromConf(conf).getShortCircuitCache();
|
|
|
|
|
|
+ for (int i = 0; i < shortCircuitCachesNum; i++) {
|
|
|
|
+ ExtendedBlock block = DFSTestUtil.getAllBlocks(
|
|
|
|
+ fs, TEST_PATH).get(i).getBlock();
|
|
|
|
+ File dataFile = cluster.getBlockFile(0, block);
|
|
|
|
+ File metaFile = cluster.getBlockMetadataFile(0, block);
|
|
|
|
+
|
|
|
|
+ ShortCircuitCache shortCircuitCache =
|
|
|
|
+ ClientContext.getFromConf(conf).getShortCircuitCache(
|
|
|
|
+ block.getBlockId());
|
|
|
|
+ test.setup(dataFile, checksum);
|
|
|
|
+ FileInputStream[] streams = {
|
|
|
|
+ new FileInputStream(dataFile),
|
|
|
|
+ new FileInputStream(metaFile)
|
|
|
|
+ };
|
|
|
|
+ dataIn = streams[0];
|
|
|
|
+ metaIn = streams[1];
|
|
|
|
+ ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(),
|
|
|
|
+ block.getBlockPoolId());
|
|
|
|
+ raf = new RandomAccessFile(
|
|
|
|
+ new File(sockDir.getDir().getAbsolutePath(),
|
|
|
|
+ UUID.randomUUID().toString()), "rw");
|
|
|
|
+ raf.setLength(8192);
|
|
|
|
+ FileInputStream shmStream = new FileInputStream(raf.getFD());
|
|
|
|
+ shm = new ShortCircuitShm(ShmId.createRandom(), shmStream);
|
|
|
|
+ ShortCircuitReplica replica =
|
|
|
|
+ new ShortCircuitReplica(key, dataIn, metaIn, shortCircuitCache,
|
|
|
|
+ Time.now(), shm.allocAndRegisterSlot(
|
|
|
|
+ ExtendedBlockId.fromExtendedBlock(block)));
|
|
|
|
+ blockReaderLocal = new BlockReaderLocal.Builder(
|
|
|
|
+ new DfsClientConf.ShortCircuitConf(conf)).
|
|
|
|
+ setFilename(TEST_PATH.getName()).
|
|
|
|
+ setBlock(block).
|
|
|
|
+ setShortCircuitReplica(replica).
|
|
|
|
+ setCachingStrategy(new CachingStrategy(false, readahead)).
|
|
|
|
+ setVerifyChecksum(checksum).
|
|
|
|
+ build();
|
|
|
|
+ dataIn = null;
|
|
|
|
+ metaIn = null;
|
|
|
|
+ test.doTest(blockReaderLocal, original, i * blockSize);
|
|
|
|
+ // BlockReaderLocal should not alter the file position.
|
|
|
|
+ Assert.assertEquals(0, streams[0].getChannel().position());
|
|
|
|
+ Assert.assertEquals(0, streams[1].getChannel().position());
|
|
|
|
+ }
|
|
cluster.shutdown();
|
|
cluster.shutdown();
|
|
cluster = null;
|
|
cluster = null;
|
|
- test.setup(dataFile, checksum);
|
|
|
|
- FileInputStream streams[] = {
|
|
|
|
- new FileInputStream(dataFile),
|
|
|
|
- new FileInputStream(metaFile)
|
|
|
|
- };
|
|
|
|
- dataIn = streams[0];
|
|
|
|
- metaIn = streams[1];
|
|
|
|
- ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(),
|
|
|
|
- block.getBlockPoolId());
|
|
|
|
- raf = new RandomAccessFile(
|
|
|
|
- new File(sockDir.getDir().getAbsolutePath(),
|
|
|
|
- UUID.randomUUID().toString()), "rw");
|
|
|
|
- raf.setLength(8192);
|
|
|
|
- FileInputStream shmStream = new FileInputStream(raf.getFD());
|
|
|
|
- shm = new ShortCircuitShm(ShmId.createRandom(), shmStream);
|
|
|
|
- ShortCircuitReplica replica =
|
|
|
|
- new ShortCircuitReplica(key, dataIn, metaIn, shortCircuitCache,
|
|
|
|
- Time.now(), shm.allocAndRegisterSlot(
|
|
|
|
- ExtendedBlockId.fromExtendedBlock(block)));
|
|
|
|
- blockReaderLocal = new BlockReaderLocal.Builder(
|
|
|
|
- new DfsClientConf.ShortCircuitConf(conf)).
|
|
|
|
- setFilename(TEST_PATH.getName()).
|
|
|
|
- setBlock(block).
|
|
|
|
- setShortCircuitReplica(replica).
|
|
|
|
- setCachingStrategy(new CachingStrategy(false, readahead)).
|
|
|
|
- setVerifyChecksum(checksum).
|
|
|
|
- build();
|
|
|
|
- dataIn = 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 {
|
|
} finally {
|
|
if (fsIn != null) fsIn.close();
|
|
if (fsIn != null) fsIn.close();
|
|
if (fs != null) fs.close();
|
|
if (fs != null) fs.close();
|
|
@@ -227,6 +239,11 @@ public class TestBlockReaderLocal {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public void runBlockReaderLocalTest(BlockReaderLocalTest test,
|
|
|
|
+ boolean checksum, long readahead) throws IOException {
|
|
|
|
+ runBlockReaderLocalTest(test, checksum, readahead, 1);
|
|
|
|
+ }
|
|
|
|
+
|
|
private static class TestBlockReaderLocalImmediateClose
|
|
private static class TestBlockReaderLocalImmediateClose
|
|
extends BlockReaderLocalTest {
|
|
extends BlockReaderLocalTest {
|
|
}
|
|
}
|
|
@@ -242,7 +259,7 @@ public class TestBlockReaderLocal {
|
|
@Override
|
|
@Override
|
|
public void doTest(BlockReaderLocal reader, byte original[])
|
|
public void doTest(BlockReaderLocal reader, byte original[])
|
|
throws IOException {
|
|
throws IOException {
|
|
- byte buf[] = new byte[TEST_LENGTH];
|
|
|
|
|
|
+ byte[] buf = new byte[TEST_LENGTH];
|
|
reader.readFully(buf, 0, 512);
|
|
reader.readFully(buf, 0, 512);
|
|
assertArrayRegionsEqual(original, 0, buf, 0, 512);
|
|
assertArrayRegionsEqual(original, 0, buf, 0, 512);
|
|
reader.readFully(buf, 512, 512);
|
|
reader.readFully(buf, 512, 512);
|
|
@@ -291,7 +308,7 @@ public class TestBlockReaderLocal {
|
|
@Override
|
|
@Override
|
|
public void doTest(BlockReaderLocal reader, byte original[])
|
|
public void doTest(BlockReaderLocal reader, byte original[])
|
|
throws IOException {
|
|
throws IOException {
|
|
- byte buf[] = new byte[TEST_LENGTH];
|
|
|
|
|
|
+ byte[] buf = new byte[TEST_LENGTH];
|
|
reader.readFully(buf, 0, 10);
|
|
reader.readFully(buf, 0, 10);
|
|
assertArrayRegionsEqual(original, 0, buf, 0, 10);
|
|
assertArrayRegionsEqual(original, 0, buf, 0, 10);
|
|
reader.readFully(buf, 10, 100);
|
|
reader.readFully(buf, 10, 100);
|
|
@@ -369,7 +386,7 @@ public class TestBlockReaderLocal {
|
|
public void testBlockReaderLocalByteBufferReadsNoReadahead()
|
|
public void testBlockReaderLocalByteBufferReadsNoReadahead()
|
|
throws IOException {
|
|
throws IOException {
|
|
runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferReads(),
|
|
runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferReads(),
|
|
- true, 0);
|
|
|
|
|
|
+ true, 0);
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
@@ -468,7 +485,7 @@ public class TestBlockReaderLocal {
|
|
|
|
|
|
public void doTest(BlockReaderLocal reader, byte original[])
|
|
public void doTest(BlockReaderLocal reader, byte original[])
|
|
throws IOException {
|
|
throws IOException {
|
|
- byte buf[] = new byte[TEST_LENGTH];
|
|
|
|
|
|
+ byte[] buf = new byte[TEST_LENGTH];
|
|
if (usingChecksums) {
|
|
if (usingChecksums) {
|
|
try {
|
|
try {
|
|
reader.readFully(buf, 0, 10);
|
|
reader.readFully(buf, 0, 10);
|
|
@@ -508,7 +525,7 @@ public class TestBlockReaderLocal {
|
|
|
|
|
|
public void doTest(BlockReaderLocal reader, byte original[])
|
|
public void doTest(BlockReaderLocal reader, byte original[])
|
|
throws IOException {
|
|
throws IOException {
|
|
- byte buf[] = new byte[TEST_LENGTH];
|
|
|
|
|
|
+ byte[] buf = new byte[TEST_LENGTH];
|
|
try {
|
|
try {
|
|
reader.readFully(buf, 0, 10);
|
|
reader.readFully(buf, 0, 10);
|
|
assertArrayRegionsEqual(original, 0, buf, 0, 10);
|
|
assertArrayRegionsEqual(original, 0, buf, 0, 10);
|
|
@@ -845,4 +862,78 @@ public class TestBlockReaderLocal {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ private static class TestBlockReaderFiveShortCircutCachesReads
|
|
|
|
+ extends BlockReaderLocalTest {
|
|
|
|
+ @Override
|
|
|
|
+ public void doTest(BlockReaderLocal reader, byte[] original, int shift)
|
|
|
|
+ throws IOException {
|
|
|
|
+ byte[] buf = new byte[TEST_LENGTH];
|
|
|
|
+ reader.readFully(buf, 0, 512);
|
|
|
|
+ assertArrayRegionsEqual(original, shift, buf, 0, 512);
|
|
|
|
+ reader.readFully(buf, 512, 512);
|
|
|
|
+ assertArrayRegionsEqual(original, 512 + shift, buf, 512, 512);
|
|
|
|
+ reader.readFully(buf, 1024, 513);
|
|
|
|
+ assertArrayRegionsEqual(original, 1024 + shift, buf, 1024, 513);
|
|
|
|
+ reader.readFully(buf, 1537, 514);
|
|
|
|
+ assertArrayRegionsEqual(original, 1537 + shift, 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 testBlockReaderFiveShortCircutCachesReads() throws IOException {
|
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderFiveShortCircutCachesReads(),
|
|
|
|
+ true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT,
|
|
|
|
+ 5);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testBlockReaderFiveShortCircutCachesReadsShortReadahead()
|
|
|
|
+ throws IOException {
|
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderFiveShortCircutCachesReads(),
|
|
|
|
+ true, BlockReaderLocalTest.BYTES_PER_CHECKSUM - 1,
|
|
|
|
+ 5);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testBlockReaderFiveShortCircutCachesReadsNoChecksum()
|
|
|
|
+ throws IOException {
|
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderFiveShortCircutCachesReads(),
|
|
|
|
+ false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT,
|
|
|
|
+ 5);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testBlockReaderFiveShortCircutCachesReadsNoReadahead()
|
|
|
|
+ throws IOException {
|
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderFiveShortCircutCachesReads(),
|
|
|
|
+ true, 0, 5);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testBlockReaderFiveShortCircutCachesReadsNoChecksumNoReadahead()
|
|
|
|
+ throws IOException {
|
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderFiveShortCircutCachesReads(),
|
|
|
|
+ false, 0, 5);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test(expected = IllegalArgumentException.class)
|
|
|
|
+ public void testBlockReaderShortCircutCachesOutOfRangeBelow()
|
|
|
|
+ throws IOException {
|
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderFiveShortCircutCachesReads(),
|
|
|
|
+ true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT,
|
|
|
|
+ 0);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test(expected = IllegalArgumentException.class)
|
|
|
|
+ public void testBlockReaderShortCircutCachesOutOfRangeAbove()
|
|
|
|
+ throws IOException {
|
|
|
|
+ runBlockReaderLocalTest(new TestBlockReaderFiveShortCircutCachesReads(),
|
|
|
|
+ true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT,
|
|
|
|
+ 555);
|
|
|
|
+ }
|
|
|
|
+
|
|
}
|
|
}
|