|
@@ -26,13 +26,18 @@ import java.util.concurrent.Executors;
|
|
|
|
|
|
import org.junit.Test;
|
|
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.BlockData;
|
|
import org.apache.hadoop.fs.impl.prefetch.BufferData;
|
|
import org.apache.hadoop.fs.impl.prefetch.BufferData;
|
|
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
|
|
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
|
|
|
|
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
|
|
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
|
|
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
|
|
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
|
|
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
|
|
import org.apache.hadoop.test.AbstractHadoopTestBase;
|
|
import org.apache.hadoop.test.AbstractHadoopTestBase;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.fs.s3a.Constants.BUFFER_DIR;
|
|
|
|
+import static org.apache.hadoop.fs.s3a.Constants.HADOOP_TMP_DIR;
|
|
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
|
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertEquals;
|
|
|
|
|
|
@@ -59,44 +64,45 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
|
|
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
|
|
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
|
|
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
|
|
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
|
|
|
|
|
|
|
|
+ Configuration conf = new Configuration();
|
|
// Should not throw.
|
|
// Should not throw.
|
|
S3ACachingBlockManager blockManager =
|
|
S3ACachingBlockManager blockManager =
|
|
new S3ACachingBlockManager(futurePool, reader, blockData, POOL_SIZE,
|
|
new S3ACachingBlockManager(futurePool, reader, blockData, POOL_SIZE,
|
|
- streamStatistics);
|
|
|
|
|
|
+ streamStatistics, conf, null);
|
|
|
|
|
|
// Verify it throws correctly.
|
|
// Verify it throws correctly.
|
|
intercept(
|
|
intercept(
|
|
NullPointerException.class,
|
|
NullPointerException.class,
|
|
() -> new S3ACachingBlockManager(null, reader, blockData, POOL_SIZE,
|
|
() -> new S3ACachingBlockManager(null, reader, blockData, POOL_SIZE,
|
|
- streamStatistics));
|
|
|
|
|
|
+ streamStatistics, conf, null));
|
|
|
|
|
|
intercept(
|
|
intercept(
|
|
IllegalArgumentException.class,
|
|
IllegalArgumentException.class,
|
|
"'reader' must not be null",
|
|
"'reader' must not be null",
|
|
() -> new S3ACachingBlockManager(futurePool, null, blockData, POOL_SIZE,
|
|
() -> new S3ACachingBlockManager(futurePool, null, blockData, POOL_SIZE,
|
|
- streamStatistics));
|
|
|
|
|
|
+ streamStatistics, conf, null));
|
|
|
|
|
|
intercept(
|
|
intercept(
|
|
IllegalArgumentException.class,
|
|
IllegalArgumentException.class,
|
|
"'blockData' must not be null",
|
|
"'blockData' must not be null",
|
|
() -> new S3ACachingBlockManager(futurePool, reader, null, POOL_SIZE,
|
|
() -> new S3ACachingBlockManager(futurePool, reader, null, POOL_SIZE,
|
|
- streamStatistics));
|
|
|
|
|
|
+ streamStatistics, conf, null));
|
|
|
|
|
|
intercept(
|
|
intercept(
|
|
IllegalArgumentException.class,
|
|
IllegalArgumentException.class,
|
|
"'bufferPoolSize' must be a positive integer",
|
|
"'bufferPoolSize' must be a positive integer",
|
|
() -> new S3ACachingBlockManager(futurePool, reader, blockData, 0,
|
|
() -> new S3ACachingBlockManager(futurePool, reader, blockData, 0,
|
|
- streamStatistics));
|
|
|
|
|
|
+ streamStatistics, conf, null));
|
|
|
|
|
|
intercept(
|
|
intercept(
|
|
IllegalArgumentException.class,
|
|
IllegalArgumentException.class,
|
|
"'bufferPoolSize' must be a positive integer",
|
|
"'bufferPoolSize' must be a positive integer",
|
|
() -> new S3ACachingBlockManager(futurePool, reader, blockData, -1,
|
|
() -> new S3ACachingBlockManager(futurePool, reader, blockData, -1,
|
|
- streamStatistics));
|
|
|
|
|
|
+ streamStatistics, conf, null));
|
|
|
|
|
|
intercept(NullPointerException.class,
|
|
intercept(NullPointerException.class,
|
|
() -> new S3ACachingBlockManager(futurePool, reader, blockData,
|
|
() -> new S3ACachingBlockManager(futurePool, reader, blockData,
|
|
- POOL_SIZE, null));
|
|
|
|
|
|
+ POOL_SIZE, null, conf, null));
|
|
|
|
|
|
intercept(
|
|
intercept(
|
|
IllegalArgumentException.class,
|
|
IllegalArgumentException.class,
|
|
@@ -125,13 +131,17 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
|
|
private static final class BlockManagerForTesting
|
|
private static final class BlockManagerForTesting
|
|
extends S3ACachingBlockManager {
|
|
extends S3ACachingBlockManager {
|
|
|
|
|
|
|
|
+ private static final Configuration CONF =
|
|
|
|
+ S3ATestUtils.prepareTestConfiguration(new Configuration());
|
|
|
|
+
|
|
BlockManagerForTesting(
|
|
BlockManagerForTesting(
|
|
ExecutorServiceFuturePool futurePool,
|
|
ExecutorServiceFuturePool futurePool,
|
|
S3ARemoteObjectReader reader,
|
|
S3ARemoteObjectReader reader,
|
|
BlockData blockData,
|
|
BlockData blockData,
|
|
int bufferPoolSize,
|
|
int bufferPoolSize,
|
|
S3AInputStreamStatistics streamStatistics) {
|
|
S3AInputStreamStatistics streamStatistics) {
|
|
- super(futurePool, reader, blockData, bufferPoolSize, streamStatistics);
|
|
|
|
|
|
+ super(futurePool, reader, blockData, bufferPoolSize, streamStatistics, CONF,
|
|
|
|
+ new LocalDirAllocator(HADOOP_TMP_DIR));
|
|
}
|
|
}
|
|
|
|
|
|
// If true, forces the next read operation to fail.
|
|
// If true, forces the next read operation to fail.
|
|
@@ -154,8 +164,8 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
|
|
private boolean forceNextCachePutToFail;
|
|
private boolean forceNextCachePutToFail;
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- protected void cachePut(int blockNumber, ByteBuffer buffer)
|
|
|
|
- throws IOException {
|
|
|
|
|
|
+ protected void cachePut(int blockNumber,
|
|
|
|
+ ByteBuffer buffer) throws IOException {
|
|
if (forceNextCachePutToFail) {
|
|
if (forceNextCachePutToFail) {
|
|
forceNextCachePutToFail = false;
|
|
forceNextCachePutToFail = false;
|
|
throw new RuntimeException("bar");
|
|
throw new RuntimeException("bar");
|
|
@@ -262,9 +272,11 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
|
|
throws IOException, InterruptedException {
|
|
throws IOException, InterruptedException {
|
|
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
|
|
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
|
|
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
|
|
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
|
|
|
|
+ Configuration conf = new Configuration();
|
|
S3ACachingBlockManager blockManager =
|
|
S3ACachingBlockManager blockManager =
|
|
new S3ACachingBlockManager(futurePool, reader, blockData, POOL_SIZE,
|
|
new S3ACachingBlockManager(futurePool, reader, blockData, POOL_SIZE,
|
|
- streamStatistics);
|
|
|
|
|
|
+ streamStatistics, conf, new LocalDirAllocator(
|
|
|
|
+ conf.get(BUFFER_DIR) != null ? BUFFER_DIR : HADOOP_TMP_DIR));
|
|
assertInitialState(blockManager);
|
|
assertInitialState(blockManager);
|
|
|
|
|
|
for (int b = 0; b < blockData.getNumBlocks(); b++) {
|
|
for (int b = 0; b < blockData.getNumBlocks(); b++) {
|