|
@@ -18,8 +18,6 @@
|
|
|
|
|
|
package org.apache.hadoop.fs.s3a;
|
|
package org.apache.hadoop.fs.s3a;
|
|
|
|
|
|
-import java.net.URI;
|
|
|
|
-
|
|
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -36,7 +34,6 @@ import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
|
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
|
import org.apache.hadoop.test.LambdaTestUtils;
|
|
import org.apache.hadoop.test.LambdaTestUtils;
|
|
|
|
|
|
-import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_DEFAULT_SIZE;
|
|
|
|
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY;
|
|
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY;
|
|
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
|
|
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
|
|
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMaximum;
|
|
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMaximum;
|
|
@@ -49,7 +46,6 @@ import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_A
|
|
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE;
|
|
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE;
|
|
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_OPENED;
|
|
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_OPENED;
|
|
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS;
|
|
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS;
|
|
-import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
* Test the prefetching input stream, validates that the underlying S3ACachingInputStream and
|
|
* Test the prefetching input stream, validates that the underlying S3ACachingInputStream and
|
|
@@ -64,47 +60,39 @@ public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest {
|
|
private static final Logger LOG =
|
|
private static final Logger LOG =
|
|
LoggerFactory.getLogger(ITestS3APrefetchingInputStream.class);
|
|
LoggerFactory.getLogger(ITestS3APrefetchingInputStream.class);
|
|
|
|
|
|
- private static final int S_1K = 1024;
|
|
|
|
|
|
+ private static final int S_500 = 512;
|
|
|
|
+ private static final int S_1K = S_500 * 2;
|
|
private static final int S_1M = S_1K * S_1K;
|
|
private static final int S_1M = S_1K * S_1K;
|
|
- // Path for file which should have length > block size so S3ACachingInputStream is used
|
|
|
|
- private Path largeFile;
|
|
|
|
- private FileSystem largeFileFS;
|
|
|
|
private int numBlocks;
|
|
private int numBlocks;
|
|
- private int blockSize;
|
|
|
|
|
|
+
|
|
|
|
+ // Size should be > block size so S3ACachingInputStream is used
|
|
private long largeFileSize;
|
|
private long largeFileSize;
|
|
|
|
+
|
|
// Size should be < block size so S3AInMemoryInputStream is used
|
|
// Size should be < block size so S3AInMemoryInputStream is used
|
|
- private static final int SMALL_FILE_SIZE = S_1K * 16;
|
|
|
|
|
|
+ private static final int SMALL_FILE_SIZE = S_1K * 9;
|
|
|
|
|
|
private static final int TIMEOUT_MILLIS = 5000;
|
|
private static final int TIMEOUT_MILLIS = 5000;
|
|
private static final int INTERVAL_MILLIS = 500;
|
|
private static final int INTERVAL_MILLIS = 500;
|
|
-
|
|
|
|
|
|
+ private static final int BLOCK_SIZE = S_1K * 10;
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public Configuration createConfiguration() {
|
|
public Configuration createConfiguration() {
|
|
Configuration conf = super.createConfiguration();
|
|
Configuration conf = super.createConfiguration();
|
|
S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY);
|
|
S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY);
|
|
|
|
+ S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_BLOCK_SIZE_KEY);
|
|
conf.setBoolean(PREFETCH_ENABLED_KEY, true);
|
|
conf.setBoolean(PREFETCH_ENABLED_KEY, true);
|
|
|
|
+ conf.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
return conf;
|
|
return conf;
|
|
}
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
|
- public void teardown() throws Exception {
|
|
|
|
- super.teardown();
|
|
|
|
- cleanupWithLogger(LOG, largeFileFS);
|
|
|
|
- largeFileFS = null;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void openFS() throws Exception {
|
|
|
|
- Configuration conf = getConfiguration();
|
|
|
|
- String largeFileUri = S3ATestUtils.getCSVTestFile(conf);
|
|
|
|
-
|
|
|
|
- largeFile = new Path(largeFileUri);
|
|
|
|
- blockSize = conf.getInt(PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE);
|
|
|
|
- largeFileFS = new S3AFileSystem();
|
|
|
|
- largeFileFS.initialize(new URI(largeFileUri), getConfiguration());
|
|
|
|
|
|
+ private void createLargeFile() throws Exception {
|
|
|
|
+ byte[] data = ContractTestUtils.dataset(S_1K * 72, 'x', 26);
|
|
|
|
+ Path largeFile = methodPath();
|
|
|
|
+ FileSystem largeFileFS = getFileSystem();
|
|
|
|
+ ContractTestUtils.writeDataset(getFileSystem(), largeFile, data, data.length, 16, true);
|
|
FileStatus fileStatus = largeFileFS.getFileStatus(largeFile);
|
|
FileStatus fileStatus = largeFileFS.getFileStatus(largeFile);
|
|
largeFileSize = fileStatus.getLen();
|
|
largeFileSize = fileStatus.getLen();
|
|
- numBlocks = calculateNumBlocks(largeFileSize, blockSize);
|
|
|
|
|
|
+ numBlocks = calculateNumBlocks(largeFileSize, BLOCK_SIZE);
|
|
}
|
|
}
|
|
|
|
|
|
private static int calculateNumBlocks(long largeFileSize, int blockSize) {
|
|
private static int calculateNumBlocks(long largeFileSize, int blockSize) {
|
|
@@ -119,9 +107,9 @@ public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest {
|
|
public void testReadLargeFileFully() throws Throwable {
|
|
public void testReadLargeFileFully() throws Throwable {
|
|
describe("read a large file fully, uses S3ACachingInputStream");
|
|
describe("read a large file fully, uses S3ACachingInputStream");
|
|
IOStatistics ioStats;
|
|
IOStatistics ioStats;
|
|
- openFS();
|
|
|
|
|
|
+ createLargeFile();
|
|
|
|
|
|
- try (FSDataInputStream in = largeFileFS.open(largeFile)) {
|
|
|
|
|
|
+ try (FSDataInputStream in = getFileSystem().open(methodPath())) {
|
|
ioStats = in.getIOStatistics();
|
|
ioStats = in.getIOStatistics();
|
|
|
|
|
|
byte[] buffer = new byte[S_1M * 10];
|
|
byte[] buffer = new byte[S_1M * 10];
|
|
@@ -152,9 +140,9 @@ public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest {
|
|
describe("read a large file using readFully(position,buffer,offset,length),"
|
|
describe("read a large file using readFully(position,buffer,offset,length),"
|
|
+ " uses S3ACachingInputStream");
|
|
+ " uses S3ACachingInputStream");
|
|
IOStatistics ioStats;
|
|
IOStatistics ioStats;
|
|
- openFS();
|
|
|
|
|
|
+ createLargeFile();
|
|
|
|
|
|
- try (FSDataInputStream in = largeFileFS.open(largeFile)) {
|
|
|
|
|
|
+ try (FSDataInputStream in = getFileSystem().open(methodPath())) {
|
|
ioStats = in.getIOStatistics();
|
|
ioStats = in.getIOStatistics();
|
|
|
|
|
|
byte[] buffer = new byte[S_1M * 10];
|
|
byte[] buffer = new byte[S_1M * 10];
|
|
@@ -183,25 +171,25 @@ public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest {
|
|
public void testRandomReadLargeFile() throws Throwable {
|
|
public void testRandomReadLargeFile() throws Throwable {
|
|
describe("random read on a large file, uses S3ACachingInputStream");
|
|
describe("random read on a large file, uses S3ACachingInputStream");
|
|
IOStatistics ioStats;
|
|
IOStatistics ioStats;
|
|
- openFS();
|
|
|
|
|
|
+ createLargeFile();
|
|
|
|
|
|
- try (FSDataInputStream in = largeFileFS.open(largeFile)) {
|
|
|
|
|
|
+ try (FSDataInputStream in = getFileSystem().open(methodPath())) {
|
|
ioStats = in.getIOStatistics();
|
|
ioStats = in.getIOStatistics();
|
|
|
|
|
|
- byte[] buffer = new byte[blockSize];
|
|
|
|
|
|
+ byte[] buffer = new byte[BLOCK_SIZE];
|
|
|
|
|
|
// Don't read block 0 completely so it gets cached on read after seek
|
|
// Don't read block 0 completely so it gets cached on read after seek
|
|
- in.read(buffer, 0, blockSize - S_1K * 10);
|
|
|
|
|
|
+ in.read(buffer, 0, BLOCK_SIZE - S_500 * 10);
|
|
|
|
|
|
// Seek to block 2 and read all of it
|
|
// Seek to block 2 and read all of it
|
|
- in.seek(blockSize * 2);
|
|
|
|
- in.read(buffer, 0, blockSize);
|
|
|
|
|
|
+ in.seek(BLOCK_SIZE * 2);
|
|
|
|
+ in.read(buffer, 0, BLOCK_SIZE);
|
|
|
|
|
|
// Seek to block 4 but don't read: noop.
|
|
// Seek to block 4 but don't read: noop.
|
|
- in.seek(blockSize * 4);
|
|
|
|
|
|
+ in.seek(BLOCK_SIZE * 4);
|
|
|
|
|
|
// Backwards seek, will use cached block 0
|
|
// Backwards seek, will use cached block 0
|
|
- in.seek(S_1K * 5);
|
|
|
|
|
|
+ in.seek(S_500 * 5);
|
|
in.read();
|
|
in.read();
|
|
|
|
|
|
// Expected to get block 0 (partially read), 1 (prefetch), 2 (fully read), 3 (prefetch)
|
|
// Expected to get block 0 (partially read), 1 (prefetch), 2 (fully read), 3 (prefetch)
|
|
@@ -234,9 +222,9 @@ public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest {
|
|
|
|
|
|
byte[] buffer = new byte[SMALL_FILE_SIZE];
|
|
byte[] buffer = new byte[SMALL_FILE_SIZE];
|
|
|
|
|
|
- in.read(buffer, 0, S_1K * 4);
|
|
|
|
- in.seek(S_1K * 12);
|
|
|
|
- in.read(buffer, 0, S_1K * 4);
|
|
|
|
|
|
+ in.read(buffer, 0, S_1K * 2);
|
|
|
|
+ in.seek(S_1K * 7);
|
|
|
|
+ in.read(buffer, 0, S_1K * 2);
|
|
|
|
|
|
verifyStatisticCounterValue(ioStats, ACTION_HTTP_GET_REQUEST, 1);
|
|
verifyStatisticCounterValue(ioStats, ACTION_HTTP_GET_REQUEST, 1);
|
|
verifyStatisticCounterValue(ioStats, STREAM_READ_OPENED, 1);
|
|
verifyStatisticCounterValue(ioStats, STREAM_READ_OPENED, 1);
|
|
@@ -261,9 +249,9 @@ public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest {
|
|
FSDataInputStream in = getFileSystem().open(smallFile);
|
|
FSDataInputStream in = getFileSystem().open(smallFile);
|
|
|
|
|
|
byte[] buffer = new byte[SMALL_FILE_SIZE];
|
|
byte[] buffer = new byte[SMALL_FILE_SIZE];
|
|
- in.read(buffer, 0, S_1K * 4);
|
|
|
|
- in.seek(S_1K * 12);
|
|
|
|
- in.read(buffer, 0, S_1K * 4);
|
|
|
|
|
|
+ in.read(buffer, 0, S_1K * 2);
|
|
|
|
+ in.seek(S_1K * 7);
|
|
|
|
+ in.read(buffer, 0, S_1K * 2);
|
|
|
|
|
|
long pos = in.getPos();
|
|
long pos = in.getPos();
|
|
IOStatistics ioStats = in.getIOStatistics();
|
|
IOStatistics ioStats = in.getIOStatistics();
|
|
@@ -298,7 +286,6 @@ public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest {
|
|
inputStreamStatistics, newInputStreamStatistics);
|
|
inputStreamStatistics, newInputStreamStatistics);
|
|
|
|
|
|
assertFalse("seekToNewSource() not supported with prefetch", in.seekToNewSource(10));
|
|
assertFalse("seekToNewSource() not supported with prefetch", in.seekToNewSource(10));
|
|
-
|
|
|
|
}
|
|
}
|
|
|
|
|
|
}
|
|
}
|