|
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a.performance;
|
|
|
|
|
|
|
|
|
import java.io.EOFException;
|
|
|
+import java.io.InputStream;
|
|
|
import java.nio.ByteBuffer;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
@@ -29,6 +30,7 @@ import org.junit.Test;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FileRange;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
@@ -45,8 +47,15 @@ import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_RE
|
|
|
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_RANDOM;
|
|
|
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL;
|
|
|
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
|
|
|
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE;
|
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.readStream;
|
|
|
+import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
|
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile;
|
|
|
+import static org.apache.hadoop.fs.s3a.Constants.CHECKSUM_VALIDATION;
|
|
|
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.assertStreamIsNotChecksummed;
|
|
|
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
|
|
|
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.getS3AInputStream;
|
|
|
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
|
|
|
import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES_READ_CLOSE;
|
|
|
import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_OPENED;
|
|
|
import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_SEEK_BYTES_SKIPPED;
|
|
@@ -79,6 +88,16 @@ public class ITestS3AOpenCost extends AbstractS3ACostTest {
|
|
|
super(true);
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public Configuration createConfiguration() {
|
|
|
+ Configuration conf = super.createConfiguration();
|
|
|
+ removeBaseAndBucketOverrides(conf,
|
|
|
+ CHECKSUM_VALIDATION);
|
|
|
+ conf.setBoolean(CHECKSUM_VALIDATION, false);
|
|
|
+ disableFilesystemCaching(conf);
|
|
|
+ return conf;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Setup creates a test file, saves is status and length
|
|
|
* to fields.
|
|
@@ -139,6 +158,34 @@ public class ITestS3AOpenCost extends AbstractS3ACostTest {
|
|
|
assertEquals("bytes read from file", fileLength, readLen);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testStreamIsNotChecksummed() throws Throwable {
|
|
|
+ describe("Verify that an opened stream is not checksummed");
|
|
|
+ S3AFileSystem fs = getFileSystem();
|
|
|
+ // open the file
|
|
|
+ try (FSDataInputStream in = verifyMetrics(() ->
|
|
|
+ fs.openFile(testFile)
|
|
|
+ .must(FS_OPTION_OPENFILE_READ_POLICY,
|
|
|
+ FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE)
|
|
|
+ .mustLong(FS_OPTION_OPENFILE_LENGTH, fileLength)
|
|
|
+ .build()
|
|
|
+ .get(),
|
|
|
+ always(NO_HEAD_OR_LIST),
|
|
|
+ with(STREAM_READ_OPENED, 0))) {
|
|
|
+
|
|
|
+ // if prefetching is enabled, skip this test
|
|
|
+ final InputStream wrapped = in.getWrappedStream();
|
|
|
+ if (!(wrapped instanceof S3AInputStream)) {
|
|
|
+ skip("Not an S3AInputStream: " + wrapped);
|
|
|
+ }
|
|
|
+
|
|
|
+ // open the stream.
|
|
|
+ in.read();
|
|
|
+ // now examine the innermost stream and make sure it doesn't have a checksum
|
|
|
+ assertStreamIsNotChecksummed(getS3AInputStream(in));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testOpenFileShorterLength() throws Throwable {
|
|
|
// do a second read with the length declared as short.
|