|
@@ -29,6 +29,7 @@ import org.junit.Test;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.LocalDirAllocator;
|
|
|
import org.apache.hadoop.fs.impl.prefetch.BlockData;
|
|
|
+import org.apache.hadoop.fs.impl.prefetch.BlockManagerParameters;
|
|
|
import org.apache.hadoop.fs.impl.prefetch.BufferData;
|
|
|
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
|
|
|
import org.apache.hadoop.fs.s3a.S3ATestUtils;
|
|
@@ -43,6 +44,9 @@ import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_MAX_BLOCKS_COUNT;
|
|
|
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
|
|
|
+/**
|
|
|
+ * Tests to perform read from S3ACachingBlockManager.
|
|
|
+ */
|
|
|
public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
|
|
|
|
|
|
static final int FILE_SIZE = 15;
|
|
@@ -61,50 +65,126 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
|
|
|
|
|
|
private final BlockData blockData = new BlockData(FILE_SIZE, BLOCK_SIZE);
|
|
|
|
|
|
+ private static final Configuration CONF =
|
|
|
+ S3ATestUtils.prepareTestConfiguration(new Configuration());
|
|
|
+
|
|
|
@Test
|
|
|
- public void testArgChecks() throws Exception {
|
|
|
+ public void testFuturePoolNull() throws Exception {
|
|
|
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
|
|
|
- S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ try (S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File)) {
|
|
|
+ BlockManagerParameters blockManagerParams =
|
|
|
+ new BlockManagerParameters()
|
|
|
+ .withBlockData(blockData)
|
|
|
+ .withBufferPoolSize(POOL_SIZE)
|
|
|
+ .withPrefetchingStatistics(streamStatistics)
|
|
|
+ .withConf(conf);
|
|
|
+
|
|
|
+ intercept(NullPointerException.class,
|
|
|
+ () -> new S3ACachingBlockManager(blockManagerParams, reader));
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testNullReader() throws Exception {
|
|
|
Configuration conf = new Configuration();
|
|
|
- // Should not throw.
|
|
|
- S3ACachingBlockManager blockManager =
|
|
|
- new S3ACachingBlockManager(futurePool, reader, blockData, POOL_SIZE,
|
|
|
- streamStatistics, conf, null);
|
|
|
+ BlockManagerParameters blockManagerParams =
|
|
|
+ new BlockManagerParameters()
|
|
|
+ .withFuturePool(futurePool)
|
|
|
+ .withBlockData(blockData)
|
|
|
+ .withBufferPoolSize(POOL_SIZE)
|
|
|
+ .withPrefetchingStatistics(streamStatistics)
|
|
|
+ .withConf(conf)
|
|
|
+ .withMaxBlocksCount(
|
|
|
+ conf.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT));
|
|
|
+
|
|
|
+ intercept(IllegalArgumentException.class, "'reader' must not be null",
|
|
|
+ () -> new S3ACachingBlockManager(blockManagerParams, null));
|
|
|
+ }
|
|
|
|
|
|
- // Verify it throws correctly.
|
|
|
- intercept(
|
|
|
- NullPointerException.class,
|
|
|
- () -> new S3ACachingBlockManager(null, reader, blockData, POOL_SIZE,
|
|
|
- streamStatistics, conf, null));
|
|
|
+ @Test
|
|
|
+ public void testNullBlockData() throws Exception {
|
|
|
+ MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ try (S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File)) {
|
|
|
+ BlockManagerParameters blockManagerParams =
|
|
|
+ new BlockManagerParameters()
|
|
|
+ .withFuturePool(futurePool)
|
|
|
+ .withBufferPoolSize(POOL_SIZE)
|
|
|
+ .withPrefetchingStatistics(streamStatistics)
|
|
|
+ .withConf(conf);
|
|
|
+
|
|
|
+ intercept(IllegalArgumentException.class, "'blockData' must not be null",
|
|
|
+ () -> new S3ACachingBlockManager(blockManagerParams, reader));
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- intercept(
|
|
|
- IllegalArgumentException.class,
|
|
|
- "'reader' must not be null",
|
|
|
- () -> new S3ACachingBlockManager(futurePool, null, blockData, POOL_SIZE,
|
|
|
- streamStatistics, conf, null));
|
|
|
+ @Test
|
|
|
+ public void testNonPositiveBufferPoolSize() throws Exception {
|
|
|
+ MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ try (S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File)) {
|
|
|
+ BlockManagerParameters blockManagerParams =
|
|
|
+ new BlockManagerParameters()
|
|
|
+ .withFuturePool(futurePool)
|
|
|
+ .withBlockData(blockData)
|
|
|
+ .withBufferPoolSize(0)
|
|
|
+ .withPrefetchingStatistics(streamStatistics)
|
|
|
+ .withConf(conf);
|
|
|
+
|
|
|
+ intercept(IllegalArgumentException.class, "'bufferPoolSize' must be a positive integer",
|
|
|
+ () -> new S3ACachingBlockManager(blockManagerParams, reader));
|
|
|
+
|
|
|
+ BlockManagerParameters blockManagerParamsWithNegativeSize =
|
|
|
+ new BlockManagerParameters()
|
|
|
+ .withFuturePool(futurePool)
|
|
|
+ .withBlockData(blockData)
|
|
|
+ .withBufferPoolSize(-1)
|
|
|
+ .withPrefetchingStatistics(streamStatistics)
|
|
|
+ .withConf(conf);
|
|
|
+
|
|
|
+ intercept(IllegalArgumentException.class, "'bufferPoolSize' must be a positive integer",
|
|
|
+ () -> new S3ACachingBlockManager(blockManagerParamsWithNegativeSize, reader));
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- intercept(
|
|
|
- IllegalArgumentException.class,
|
|
|
- "'blockData' must not be null",
|
|
|
- () -> new S3ACachingBlockManager(futurePool, reader, null, POOL_SIZE,
|
|
|
- streamStatistics, conf, null));
|
|
|
+ @Test
|
|
|
+ public void testNullPrefetchingStatistics() throws Exception {
|
|
|
+ MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ try (S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File)) {
|
|
|
|
|
|
- intercept(
|
|
|
- IllegalArgumentException.class,
|
|
|
- "'bufferPoolSize' must be a positive integer",
|
|
|
- () -> new S3ACachingBlockManager(futurePool, reader, blockData, 0,
|
|
|
- streamStatistics, conf, null));
|
|
|
+ BlockManagerParameters blockManagerParamsBuilder7 =
|
|
|
+ new BlockManagerParameters()
|
|
|
+ .withFuturePool(futurePool)
|
|
|
+ .withBlockData(blockData)
|
|
|
+ .withBufferPoolSize(POOL_SIZE)
|
|
|
+ .withConf(conf);
|
|
|
|
|
|
- intercept(
|
|
|
- IllegalArgumentException.class,
|
|
|
- "'bufferPoolSize' must be a positive integer",
|
|
|
- () -> new S3ACachingBlockManager(futurePool, reader, blockData, -1,
|
|
|
- streamStatistics, conf, null));
|
|
|
+ intercept(NullPointerException.class,
|
|
|
+ () -> new S3ACachingBlockManager(blockManagerParamsBuilder7, reader));
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- intercept(NullPointerException.class,
|
|
|
- () -> new S3ACachingBlockManager(futurePool, reader, blockData,
|
|
|
- POOL_SIZE, null, conf, null));
|
|
|
+ @Test
|
|
|
+ public void testArgChecks() throws Exception {
|
|
|
+ MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
|
|
|
+ S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
|
|
|
+
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ BlockManagerParameters blockManagerParams =
|
|
|
+ new BlockManagerParameters()
|
|
|
+ .withFuturePool(futurePool)
|
|
|
+ .withBlockData(blockData)
|
|
|
+ .withBufferPoolSize(POOL_SIZE)
|
|
|
+ .withPrefetchingStatistics(streamStatistics)
|
|
|
+ .withConf(conf)
|
|
|
+ .withMaxBlocksCount(
|
|
|
+ conf.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT));
|
|
|
+
|
|
|
+ // Should not throw.
|
|
|
+ S3ACachingBlockManager blockManager =
|
|
|
+ new S3ACachingBlockManager(blockManagerParams, reader);
|
|
|
|
|
|
intercept(
|
|
|
IllegalArgumentException.class,
|
|
@@ -133,17 +213,9 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
|
|
|
private static final class BlockManagerForTesting
|
|
|
extends S3ACachingBlockManager {
|
|
|
|
|
|
- private static final Configuration CONF =
|
|
|
- S3ATestUtils.prepareTestConfiguration(new Configuration());
|
|
|
-
|
|
|
- BlockManagerForTesting(
|
|
|
- ExecutorServiceFuturePool futurePool,
|
|
|
- S3ARemoteObjectReader reader,
|
|
|
- BlockData blockData,
|
|
|
- int bufferPoolSize,
|
|
|
- S3AInputStreamStatistics streamStatistics) {
|
|
|
- super(futurePool, reader, blockData, bufferPoolSize, streamStatistics, CONF,
|
|
|
- new LocalDirAllocator(HADOOP_TMP_DIR));
|
|
|
+ BlockManagerForTesting(BlockManagerParameters blockManagerParameters,
|
|
|
+ S3ARemoteObjectReader reader) {
|
|
|
+ super(blockManagerParameters, reader);
|
|
|
}
|
|
|
|
|
|
// If true, forces the next read operation to fail.
|
|
@@ -196,9 +268,9 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
|
|
|
private void testGetHelper(boolean forceReadFailure) throws Exception {
|
|
|
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, true);
|
|
|
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
|
|
|
+ BlockManagerParameters blockManagerParams = getBlockManagerParameters();
|
|
|
BlockManagerForTesting blockManager =
|
|
|
- new BlockManagerForTesting(futurePool, reader, blockData, POOL_SIZE,
|
|
|
- streamStatistics);
|
|
|
+ new BlockManagerForTesting(blockManagerParams, reader);
|
|
|
|
|
|
for (int b = 0; b < blockData.getNumBlocks(); b++) {
|
|
|
// We simulate caching failure for all even numbered blocks.
|
|
@@ -244,9 +316,9 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
|
|
|
throws IOException, InterruptedException {
|
|
|
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
|
|
|
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
|
|
|
+ BlockManagerParameters blockManagerParams = getBlockManagerParameters();
|
|
|
BlockManagerForTesting blockManager =
|
|
|
- new BlockManagerForTesting(futurePool, reader, blockData, POOL_SIZE,
|
|
|
- streamStatistics);
|
|
|
+ new BlockManagerForTesting(blockManagerParams, reader);
|
|
|
assertInitialState(blockManager);
|
|
|
|
|
|
int expectedNumErrors = 0;
|
|
@@ -272,6 +344,18 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
|
|
|
assertEquals(expectedNumSuccesses, blockManager.numCached());
|
|
|
}
|
|
|
|
|
|
+ private BlockManagerParameters getBlockManagerParameters() {
|
|
|
+ return new BlockManagerParameters()
|
|
|
+ .withFuturePool(futurePool)
|
|
|
+ .withBlockData(blockData)
|
|
|
+ .withBufferPoolSize(POOL_SIZE)
|
|
|
+ .withPrefetchingStatistics(streamStatistics)
|
|
|
+ .withLocalDirAllocator(new LocalDirAllocator(HADOOP_TMP_DIR))
|
|
|
+ .withConf(CONF)
|
|
|
+ .withMaxBlocksCount(
|
|
|
+ CONF.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT));
|
|
|
+ }
|
|
|
+
|
|
|
// @Ignore
|
|
|
@Test
|
|
|
public void testCachingOfPrefetched()
|
|
@@ -279,10 +363,19 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
|
|
|
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
|
|
|
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
|
|
|
Configuration conf = new Configuration();
|
|
|
+ BlockManagerParameters blockManagerParamsBuilder =
|
|
|
+ new BlockManagerParameters()
|
|
|
+ .withFuturePool(futurePool)
|
|
|
+ .withBlockData(blockData)
|
|
|
+ .withBufferPoolSize(POOL_SIZE)
|
|
|
+ .withPrefetchingStatistics(streamStatistics)
|
|
|
+ .withLocalDirAllocator(
|
|
|
+ new LocalDirAllocator(conf.get(BUFFER_DIR) != null ? BUFFER_DIR : HADOOP_TMP_DIR))
|
|
|
+ .withConf(conf)
|
|
|
+ .withMaxBlocksCount(
|
|
|
+ conf.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT));
|
|
|
S3ACachingBlockManager blockManager =
|
|
|
- new S3ACachingBlockManager(futurePool, reader, blockData, POOL_SIZE,
|
|
|
- streamStatistics, conf, new LocalDirAllocator(
|
|
|
- conf.get(BUFFER_DIR) != null ? BUFFER_DIR : HADOOP_TMP_DIR));
|
|
|
+ new S3ACachingBlockManager(blockManagerParamsBuilder, reader);
|
|
|
assertInitialState(blockManager);
|
|
|
|
|
|
for (int b = 0; b < blockData.getNumBlocks(); b++) {
|
|
@@ -316,9 +409,9 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
|
|
|
throws IOException, InterruptedException {
|
|
|
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
|
|
|
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
|
|
|
+ BlockManagerParameters blockManagerParams = getBlockManagerParameters();
|
|
|
BlockManagerForTesting blockManager =
|
|
|
- new BlockManagerForTesting(futurePool, reader, blockData, POOL_SIZE,
|
|
|
- streamStatistics);
|
|
|
+ new BlockManagerForTesting(blockManagerParams, reader);
|
|
|
assertInitialState(blockManager);
|
|
|
|
|
|
int expectedNumErrors = 0;
|