diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java index 57c775721384e..ab4838618dabe 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java @@ -97,6 +97,13 @@ public final class StreamStatisticNames { */ public static final String STREAM_READ_OPENED = "stream_read_opened"; + /** + * Total count of times an analytics input stream was opened. + * + * Value: {@value}. + */ + public static final String STREAM_READ_ANALYTICS_OPENED = "stream_read_analytics_opened"; + /** * Count of exceptions raised during input stream reads. * Value: {@value}. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ConfigurationHelper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ConfigurationHelper.java index db39bb363238b..2078779138f3d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ConfigurationHelper.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ConfigurationHelper.java @@ -22,6 +22,7 @@ import java.util.HashMap; import java.util.Locale; import java.util.Map; +import java.util.function.Function; import java.util.stream.Collectors; import org.apache.hadoop.classification.InterfaceAudience; @@ -123,4 +124,39 @@ public static > Map mapEnumNamesToValues( return mapping; } + /** + * Look up an enum from the configuration option and map it to + * a value in the supplied enum class. + * If no value is supplied or there is no match for the supplied value, + * the fallback function is invoked, passing in the trimmed and possibly + * empty string of the value. + * Extends {link {@link Configuration#getEnum(String, Enum)}} + * by adding case independence and a lambda expression for fallback, + * rather than a default value. + * @param conf configuration + * @param name property name + * @param enumClass classname to resolve + * @param fallback fallback supplier + * @param enumeration type. + * @return an enum value + * @throws IllegalArgumentException If mapping is illegal for the type provided + */ + public static > E resolveEnum( + Configuration conf, + String name, + Class enumClass, + Function fallback) { + + final String val = conf.getTrimmed(name, ""); + + // build a map of lower case string to enum values. + final Map mapping = mapEnumNamesToValues("", enumClass); + final E mapped = mapping.get(val.toLowerCase(Locale.ROOT)); + if (mapped != null) { + return mapped; + } else { + // fallback handles it + return fallback.apply(val); + } + } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java index 16482915bdf7f..4c4514b249c28 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java @@ -507,7 +507,7 @@ public void testMultipartUpload() throws Exception { @Test public void testMultipartUploadEmptyPart() throws Exception { FileSystem fs = getFileSystem(); - Path file = path("testMultipartUpload"); + Path file = path("testMultipartUploadEmptyPart"); try (MultipartUploader uploader = fs.createMultipartUploader(file).build()) { UploadHandle uploadHandle = uploader.startUpload(file).get(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestConfigurationHelper.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestConfigurationHelper.java index 529d231572dda..cddf7d52704e8 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestConfigurationHelper.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestConfigurationHelper.java @@ -31,6 +31,7 @@ import static org.apache.hadoop.util.ConfigurationHelper.ERROR_MULTIPLE_ELEMENTS_MATCHING_TO_LOWER_CASE_VALUE; import static org.apache.hadoop.util.ConfigurationHelper.mapEnumNamesToValues; import static org.apache.hadoop.util.ConfigurationHelper.parseEnumSet; +import static org.apache.hadoop.util.ConfigurationHelper.resolveEnum; /** * Test for {@link ConfigurationHelper}. @@ -43,6 +44,12 @@ public class TestConfigurationHelper extends AbstractHadoopTestBase { */ private enum SimpleEnum { a, b, c, i } + /** + * Upper case version of SimpleEnum. + * "i" is included for case tests, as it is special in turkey. + */ + private enum UppercaseEnum { A, B, C, I } + /** * Special case: an enum with no values. @@ -171,4 +178,69 @@ public void testDuplicateValues() { .containsExactly(SimpleEnum.a, SimpleEnum.b, SimpleEnum.c); } + @Test + public void testResolveEnumGood() throws Throwable { + assertEnumResolution("c", SimpleEnum.c); + } + + @Test + public void testResolveEnumTrimmed() throws Throwable { + // strings are trimmed at each end + assertEnumResolution("\n i \n ", SimpleEnum.i); + } + + @Test + public void testResolveEnumCaseConversion() throws Throwable { + assertEnumResolution("C", SimpleEnum.c); + } + + @Test + public void testResolveEnumNoMatch() throws Throwable { + assertEnumResolution("other", null); + } + + @Test + public void testResolveEnumEmpty() throws Throwable { + assertEnumResolution("", null); + } + + @Test + public void testResolveEnumUpperCaseConversion() throws Throwable { + assertUpperEnumResolution("C", UppercaseEnum.C); + } + + @Test + public void testResolveLowerToUpperCaseConversion() throws Throwable { + assertUpperEnumResolution("i", UppercaseEnum.I); + } + + /** + * Assert that a string value in a configuration resolves to the expected + * value. + * @param value value to set + * @param expected expected outcome, set to null for no resolution. + */ + private void assertEnumResolution(final String value, final SimpleEnum expected) { + Assertions.assertThat(resolveEnum(confWithKey(value), + "key", + SimpleEnum.class, + (v) -> null)) + .describedAs("Resolution of %s", value) + .isEqualTo(expected); + } + + /** + * Equivalent for Uppercase Enum. + * @param value value to set + * @param expected expected outcome, set to null for no resolution. + */ + private void assertUpperEnumResolution(final String value, UppercaseEnum expected) { + Assertions.assertThat(resolveEnum(confWithKey(value), + "key", + UppercaseEnum.class, + (v) -> null)) + .describedAs("Resolution of %s", value) + .isEqualTo(expected); + } + } diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index b9469bc752372..cf0b4f591afe8 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -206,6 +206,7 @@ 1.12.720 2.25.53 3.1.1 + 0.0.3 1.0.1 2.7.1 1.11.2 @@ -1192,6 +1193,11 @@ + + software.amazon.s3.analyticsaccelerator + analyticsaccelerator-s3 + ${amazon-s3-analyticsaccelerator-s3.version} + org.apache.mina mina-core diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml index 39a9e51ac8125..11410a24c1ad2 100644 --- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml +++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml @@ -30,7 +30,7 @@ - + diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 1d04107ff5b48..4b9ab9bca4e21 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -48,9 +48,9 @@ 200000 - - - unset + + classic + 00 @@ -122,8 +122,8 @@ ${fs.s3a.scale.test.huge.filesize} ${fs.s3a.scale.test.huge.partitionsize} ${fs.s3a.scale.test.timeout} - - ${fs.s3a.prefetch.enabled} + + ${stream} @@ -161,8 +161,8 @@ ${fs.s3a.scale.test.huge.partitionsize} ${fs.s3a.scale.test.timeout} ${test.integration.timeout} - - ${fs.s3a.prefetch.enabled} + + ${stream} ${root.tests.enabled} @@ -212,8 +212,8 @@ ${fs.s3a.scale.test.huge.filesize} ${fs.s3a.scale.test.huge.partitionsize} ${fs.s3a.scale.test.timeout} - - ${fs.s3a.prefetch.enabled} + + ${stream} ${root.tests.enabled} job-${job.id} @@ -273,8 +273,8 @@ ${fs.s3a.scale.test.enabled} ${fs.s3a.scale.test.huge.filesize} ${fs.s3a.scale.test.timeout} - - ${fs.s3a.prefetch.enabled} + + ${stream} job-${job.id} ${fs.s3a.scale.test.timeout} @@ -308,7 +308,20 @@ - true + prefetch + + + + + + analytics + + + analytics + + + + analytics @@ -472,6 +485,11 @@ amazon-s3-encryption-client-java provided + + software.amazon.s3.analyticsaccelerator + analyticsaccelerator-s3 + compile + org.assertj assertj-core diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index e695e918c953d..b2843d24c8a1e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -21,6 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.Options; +import org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import java.time.Duration; @@ -1580,14 +1581,60 @@ private Constants() { */ public static final String AWS_AUTH_CLASS_PREFIX = "com.amazonaws.auth"; + /** + * Input stream type: {@value}. + */ + public static final String INPUT_STREAM_TYPE = "fs.s3a.input.stream.type"; + + /** + * The classic input stream: {@value}. + */ + public static final String INPUT_STREAM_TYPE_CLASSIC = + StreamIntegration.CLASSIC; + + /** + * The prefetching input stream: {@value}. + */ + public static final String INPUT_STREAM_TYPE_PREFETCH = StreamIntegration.PREFETCH; + + /** + * The analytics input stream: {@value}. + */ + public static final String INPUT_STREAM_TYPE_ANALYTICS = + StreamIntegration.ANALYTICS; + + /** + * Request the default input stream, + * whatever it is for this release: {@value}. + */ + public static final String INPUT_STREAM_TYPE_DEFAULT = StreamIntegration.DEFAULT; + + /** + * The custom input stream type: {@value}". + * If set, the classname is loaded from + * {@link #INPUT_STREAM_CUSTOM_FACTORY}. + *

+ * This option is primarily for testing as it can + * be used to generated failures. + */ + public static final String INPUT_STREAM_TYPE_CUSTOM = + StreamIntegration.CUSTOM; + + /** + * Classname of the factory to instantiate for custom streams: {@value}. + */ + public static final String INPUT_STREAM_CUSTOM_FACTORY = "fs.s3a.input.stream.custom.factory"; + /** * Controls whether the prefetching input stream is enabled. */ + @Deprecated public static final String PREFETCH_ENABLED_KEY = "fs.s3a.prefetch.enabled"; /** * Default option as to whether the prefetching input stream is enabled. */ + @Deprecated public static final boolean PREFETCH_ENABLED_DEFAULT = false; // If the default values are used, each file opened for reading will consume @@ -1780,4 +1827,12 @@ private Constants() { * Value: {@value}. */ public static final String S3A_IO_RATE_LIMIT = "fs.s3a.io.rate.limit"; + + + /** + * Prefix to configure Analytics Accelerator Library. + */ + public static final String ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX = + "fs.s3a.analytics.accelerator"; + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index 556bd3510752e..a19e5f155f00b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java @@ -166,7 +166,7 @@ public S3AsyncClient createS3AsyncClient( .httpClientBuilder(httpClientBuilder); // multipart upload pending with HADOOP-19326. - if (!parameters.isClientSideEncryptionEnabled()) { + if (!parameters.isClientSideEncryptionEnabled() && !parameters.isAnalyticsAcceleratorEnabled()) { s3AsyncClientBuilder.multipartConfiguration(multipartConfiguration) .multipartEnabled(parameters.isMultipartCopy()); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 14031ed007ef8..6d2234362ea37 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -51,14 +51,11 @@ import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.Nullable; -import software.amazon.awssdk.core.ResponseInputStream; import software.amazon.awssdk.core.exception.SdkException; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse; import software.amazon.awssdk.services.s3.model.GetBucketLocationRequest; -import software.amazon.awssdk.services.s3.model.GetObjectRequest; -import software.amazon.awssdk.services.s3.model.GetObjectResponse; import software.amazon.awssdk.services.s3.model.HeadBucketRequest; import software.amazon.awssdk.services.s3.model.HeadBucketResponse; import software.amazon.awssdk.services.s3.model.MultipartUpload; @@ -109,6 +106,7 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.s3a.api.PerformanceFlagEnum; import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; +import org.apache.hadoop.fs.s3a.audit.AuditorFlags; import org.apache.hadoop.fs.s3a.auth.SignerManager; import org.apache.hadoop.fs.s3a.auth.delegation.DelegationOperations; import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider; @@ -125,6 +123,7 @@ import org.apache.hadoop.fs.s3a.impl.ContextAccessors; import org.apache.hadoop.fs.s3a.impl.CopyFromLocalOperation; import org.apache.hadoop.fs.s3a.impl.CreateFileBuilder; +import org.apache.hadoop.fs.s3a.impl.InputStreamCallbacksImpl; import org.apache.hadoop.fs.s3a.impl.S3AFileSystemOperations; import org.apache.hadoop.fs.s3a.impl.CSEV1CompatibleS3AFileSystemOperations; import org.apache.hadoop.fs.s3a.impl.CSEMaterials; @@ -148,7 +147,11 @@ import org.apache.hadoop.fs.s3a.impl.StoreContextFactory; import org.apache.hadoop.fs.s3a.impl.UploadContentProviders; import org.apache.hadoop.fs.s3a.impl.CSEUtils; -import org.apache.hadoop.fs.s3a.prefetch.S3APrefetchingInputStream; +import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectReadParameters; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; +import org.apache.hadoop.fs.s3a.impl.streams.StreamFactoryRequirements; +import org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl; import org.apache.hadoop.fs.statistics.DurationTracker; @@ -157,7 +160,6 @@ import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.fs.statistics.IOStatisticsContext; -import org.apache.hadoop.fs.statistics.StreamStatisticNames; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.fs.store.audit.AuditEntryPoint; @@ -301,9 +303,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private String username; - /** - * Store back end. - */ private S3AStore store; /** @@ -334,18 +333,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private ExecutorService boundedThreadPool; private ThreadPoolExecutor unboundedThreadPool; - // S3 reads are prefetched asynchronously using this future pool. + /** + * Future pool built on the bounded thread pool. + * S3 reads are prefetched asynchronously using this future pool if the + * Stream Factory requests it. + */ private ExecutorServiceFuturePool futurePool; - // If true, the prefetching input stream is used for reads. - private boolean prefetchEnabled; - - // Size in bytes of a single prefetch block. - private int prefetchBlockSize; - - // Size of prefetch queue (in number of blocks). - private int prefetchBlockCount; - private int executorCapacity; private long multiPartThreshold; public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class); @@ -353,7 +347,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, /** Log to warn of storage class configuration problems. */ private static final LogExactlyOnce STORAGE_CLASS_WARNING = new LogExactlyOnce(LOG); - private LocalDirAllocator directoryAllocator; private String cannedACL; /** @@ -381,12 +374,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, /** Vectored IO context. */ private VectoredIOContext vectoredIOContext; - /** - * Maximum number of active range read operation a single - * input stream can have. - */ - private int vectoredActiveRangeReads; - private long readAhead; private ChangeDetectionPolicy changeDetectionPolicy; private final AtomicBoolean closed = new AtomicBoolean(false); @@ -451,6 +438,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, */ private boolean isCSEEnabled; + /** + * Is this S3A FS instance using analytics accelerator? + */ + private boolean isAnalyticsAccelaratorEnabled; + /** * Bucket AccessPoint. */ @@ -540,7 +532,24 @@ private static void addDeprecatedKeys() { addDeprecatedKeys(); } - /** Called after a new FileSystem instance is constructed. + /** + * Initialize the filesystem. + *

+ * This is called after a new FileSystem instance is constructed -but + * within the filesystem cache creation process. + * A slow start here while multiple threads are calling + * will result in multiple + * instances of the filesystem being created -and all but one deleted. + * Keep this as fast as possible, and avoid network IO. + *

+ * This performs the majority of the filesystem setup, and as things + * are intermixed the ordering of operations is very sensitive. + * Be very careful when moving things. + *

+ * To help identify where filesystem instances are created, + * the full stack is logged at TRACE. + *

+ * Also, ignore checkstyle complaints about method length. * @param name a uri whose authority section names the host, port, etc. * for this FileSystem * @param originalConf the configuration to use for the FS. The @@ -623,6 +632,8 @@ public void initialize(URI name, Configuration originalConf) // If encryption method is set to CSE-KMS or CSE-CUSTOM then CSE is enabled. isCSEEnabled = CSEUtils.isCSEEnabled(getS3EncryptionAlgorithm().getMethod()); + isAnalyticsAccelaratorEnabled = StreamIntegration.determineInputStreamType(conf).equals(InputStreamType.Analytics); + // Create the appropriate fsHandler instance using a factory method fsHandler = createFileSystemHandler(); fsHandler.setCSEGauge((IOStatisticsStore) getIOStatistics()); @@ -661,22 +672,11 @@ public void initialize(URI name, Configuration originalConf) dirOperationsPurgeUploads = conf.getBoolean(DIRECTORY_OPERATIONS_PURGE_UPLOADS, s3ExpressStore); - this.prefetchEnabled = conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT); - long prefetchBlockSizeLong = - longBytesOption(conf, PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE, 1); - if (prefetchBlockSizeLong > (long) Integer.MAX_VALUE) { - throw new IOException("S3A prefatch block size exceeds int limit"); - } - this.prefetchBlockSize = (int) prefetchBlockSizeLong; - this.prefetchBlockCount = - intOption(conf, PREFETCH_BLOCK_COUNT_KEY, PREFETCH_BLOCK_DEFAULT_COUNT, 1); this.isMultipartUploadEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED, DEFAULT_MULTIPART_UPLOAD_ENABLED); // multipart copy and upload are the same; this just makes it explicit this.isMultipartCopyEnabled = isMultipartUploadEnabled; - initThreadPools(conf); - int listVersion = conf.getInt(LIST_VERSION, DEFAULT_LIST_VERSION); if (listVersion < 1 || listVersion > 2) { LOG.warn("Configured fs.s3a.list.version {} is invalid, forcing " + @@ -692,13 +692,6 @@ public void initialize(URI name, Configuration originalConf) signerManager = new SignerManager(bucket, this, conf, owner); signerManager.initCustomSigners(); - // start auditing - initializeAuditService(); - - // create the requestFactory. - // requires the audit manager to be initialized. - requestFactory = createRequestFactory(); - // create an initial span for all other operations. span = createSpan(INITIALIZE_SPAN, bucket, null); @@ -778,9 +771,6 @@ public void initialize(URI name, Configuration originalConf) longBytesOption(conf, ASYNC_DRAIN_THRESHOLD, DEFAULT_ASYNC_DRAIN_THRESHOLD, 0), inputPolicy); - vectoredActiveRangeReads = intOption(conf, - AWS_S3_VECTOR_ACTIVE_RANGE_READS, DEFAULT_AWS_S3_VECTOR_ACTIVE_RANGE_READS, 1); - vectoredIOContext = populateVectoredIOContext(conf); scheme = (this.uri != null && this.uri.getScheme() != null) ? this.uri.getScheme() : FS_S3A; optimizedCopyFromLocal = conf.getBoolean(OPTIMIZED_COPY_FROM_LOCAL, OPTIMIZED_COPY_FROM_LOCAL_DEFAULT); @@ -788,12 +778,41 @@ public void initialize(URI name, Configuration originalConf) s3AccessGrantsEnabled = conf.getBoolean(AWS_S3_ACCESS_GRANTS_ENABLED, false); int rateLimitCapacity = intOption(conf, S3A_IO_RATE_LIMIT, DEFAULT_S3A_IO_RATE_LIMIT, 0); - // now create the store + + // start auditing. + // extra configuration will be passed down later. + initializeAuditService(); + + // create the requestFactory. + // requires the audit manager to be initialized. + requestFactory = createRequestFactory(); + + // now create and initialize the store store = createS3AStore(clientManager, rateLimitCapacity); // the s3 client is created through the store, rather than // directly through the client manager. // this is to aid mocking. - s3Client = store.getOrCreateS3Client(); + s3Client = getStore().getOrCreateS3Client(); + + // get the input stream factory requirements. + final StreamFactoryRequirements factoryRequirements = + getStore().factoryRequirements(); + + // If the input stream can issue get requests outside spans, + // the auditor is forced to disable rejection of unaudited requests. + final EnumSet flags = EnumSet.noneOf(AuditorFlags.class); + if (factoryRequirements.requires( + StreamFactoryRequirements.Requirements.ExpectUnauditedGetRequests)) { + flags.add(AuditorFlags.PermitOutOfBandOperations); + } + getAuditManager().setAuditFlags(flags); + // get the vector IO context from the factory.o + vectoredIOContext = factoryRequirements.vectoredIOContext(); + + // thread pool init requires store to be created and + // the stream factory requirements to include its own requirements. + initThreadPools(); + // The filesystem is now ready to perform operations against // S3 // This initiates a probe against S3 for the bucket existing. @@ -836,7 +855,7 @@ private S3AFileSystemOperations createFileSystemHandler() { /** - * Create the S3AStore instance. + * Create and start the S3AStore instance. * This is protected so that tests can override it. * @param clientManager client manager * @param rateLimitCapacity rate limit @@ -845,7 +864,7 @@ private S3AFileSystemOperations createFileSystemHandler() { @VisibleForTesting protected S3AStore createS3AStore(final ClientManager clientManager, final int rateLimitCapacity) { - return new S3AStoreBuilder() + final S3AStore st = new S3AStoreBuilder() .withAuditSpanSource(getAuditManager()) .withClientManager(clientManager) .withDurationTrackerFactory(getDurationTrackerFactory()) @@ -857,23 +876,9 @@ protected S3AStore createS3AStore(final ClientManager clientManager, .withReadRateLimiter(unlimitedRate()) .withWriteRateLimiter(RateLimitingFactory.create(rateLimitCapacity)) .build(); - } - - /** - * Populates the configurations related to vectored IO operation - * in the context which has to passed down to input streams. - * @param conf configuration object. - * @return VectoredIOContext. - */ - private VectoredIOContext populateVectoredIOContext(Configuration conf) { - final int minSeekVectored = (int) longBytesOption(conf, AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, - DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, 0); - final int maxReadSizeVectored = (int) longBytesOption(conf, AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, - DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, 0); - return new VectoredIOContext() - .setMinSeekForVectoredReads(minSeekVectored) - .setMaxReadSizeForVectoredReads(maxReadSizeVectored) - .build(); + st.init(getConf()); + st.start(); + return st; } /** @@ -939,12 +944,16 @@ public Statistics getInstanceStatistics() { } /** - * Initialize the thread pool. + * Initialize the thread pools. + *

* This must be re-invoked after replacing the S3Client during test * runs. + *

+ * It requires the S3Store to have been instantiated. * @param conf configuration. */ - private void initThreadPools(Configuration conf) { + private void initThreadPools() { + Configuration conf = getConf(); final String name = "s3a-transfer-" + getBucket(); int maxThreads = conf.getInt(MAX_THREADS, DEFAULT_MAX_THREADS); if (maxThreads < 2) { @@ -960,7 +969,9 @@ private void initThreadPools(Configuration conf) { TimeUnit.SECONDS, Duration.ZERO).getSeconds(); - int numPrefetchThreads = this.prefetchEnabled ? this.prefetchBlockCount : 0; + final StreamFactoryRequirements factoryRequirements = + getStore().factoryRequirements(); + int numPrefetchThreads = factoryRequirements.sharedThreads(); int activeTasksForBoundedThreadPool = maxThreads; int waitingTasksForBoundedThreadPool = maxThreads + totalTasks + numPrefetchThreads; @@ -978,7 +989,8 @@ private void initThreadPools(Configuration conf) { unboundedThreadPool.allowCoreThreadTimeOut(true); executorCapacity = intOption(conf, EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1); - if (prefetchEnabled) { + if (factoryRequirements.requiresFuturePool()) { + // create a future pool. final S3AInputStreamStatistics s3AInputStreamStatistics = statisticsContext.newInputStreamStatistics(); futurePool = new ExecutorServiceFuturePool( @@ -1149,6 +1161,7 @@ private ClientManager createClientManager(URI fsURI, boolean dtEnabled) throws I conf.getBoolean(CHECKSUM_VALIDATION, CHECKSUM_VALIDATION_DEFAULT)) .withClientSideEncryptionEnabled(isCSEEnabled) .withClientSideEncryptionMaterials(cseMaterials) + .withAnalyticsAcceleratorEnabled(isAnalyticsAccelaratorEnabled) .withKMSRegion(conf.get(S3_ENCRYPTION_CSE_KMS_REGION)); // this is where clients and the transfer manager are created on demand. @@ -1329,6 +1342,14 @@ public FlagSet getPerformanceFlags() { return performanceFlags; } + /** + * Get the store for low-level operations. + * @return the store the S3A FS is working through. + */ + private S3AStore getStore() { + return store; + } + /** * Implementation of all operations used by delegation tokens. */ @@ -1534,7 +1555,7 @@ public S3Client getAmazonS3Client(String reason) { @Override public S3AStore getStore() { - return store; + return S3AFileSystem.this.getStore(); } /** @@ -1663,28 +1684,8 @@ public S3AEncryptionMethods getS3EncryptionAlgorithm() { */ File createTmpFileForWrite(String pathStr, long size, Configuration conf) throws IOException { - initLocalDirAllocatorIfNotInitialized(conf); - Path path = directoryAllocator.getLocalPathForWrite(pathStr, - size, conf); - File dir = new File(path.getParent().toUri().getPath()); - String prefix = path.getName(); - // create a temp file on this directory - return File.createTempFile(prefix, null, dir); - } - /** - * Initialize dir allocator if not already initialized. - * - * @param conf The Configuration object. - */ - private void initLocalDirAllocatorIfNotInitialized(Configuration conf) { - if (directoryAllocator == null) { - synchronized (this) { - String bufferDir = conf.get(BUFFER_DIR) != null - ? BUFFER_DIR : HADOOP_TMP_DIR; - directoryAllocator = new LocalDirAllocator(bufferDir); - } - } + return getS3AInternals().getStore().createTemporaryFileForWriting(pathStr, size, conf); } /** @@ -1877,100 +1878,41 @@ private FSDataInputStream executeOpen( fileInformation.applyOptions(readContext); LOG.debug("Opening '{}'", readContext); - if (this.prefetchEnabled) { - Configuration configuration = getConf(); - initLocalDirAllocatorIfNotInitialized(configuration); - return new FSDataInputStream( - new S3APrefetchingInputStream( - readContext.build(), - createObjectAttributes(path, fileStatus), - createInputStreamCallbacks(auditSpan), - inputStreamStats, - configuration, - directoryAllocator)); - } else { - return new FSDataInputStream( - new S3AInputStream( - readContext.build(), - createObjectAttributes(path, fileStatus), - createInputStreamCallbacks(auditSpan), - inputStreamStats, - new SemaphoredDelegatingExecutor( - boundedThreadPool, - vectoredActiveRangeReads, - true, - inputStreamStats))); - } - } - - /** - * Override point: create the callbacks for S3AInputStream. - * @return an implementation of the InputStreamCallbacks, - */ - private S3AInputStream.InputStreamCallbacks createInputStreamCallbacks( + // what does the stream need + final StreamFactoryRequirements requirements = + getStore().factoryRequirements(); + + // calculate the permit count. + final int permitCount = requirements.streamThreads() + + requirements.vectoredIOContext().getVectoredActiveRangeReads(); + // create an executor which is a subset of the + // bounded thread pool. + final SemaphoredDelegatingExecutor pool = new SemaphoredDelegatingExecutor( + boundedThreadPool, + permitCount, + true, + inputStreamStats); + + // do not validate() the parameters as the store + // completes this. + ObjectReadParameters parameters = new ObjectReadParameters() + .withBoundedThreadPool(pool) + .withCallbacks(createInputStreamCallbacks(auditSpan)) + .withContext(readContext.build()) + .withObjectAttributes(createObjectAttributes(path, fileStatus)) + .withStreamStatistics(inputStreamStats); + return new FSDataInputStream(getStore().readObject(parameters)); + } + + /** + * Override point: create the callbacks for ObjectInputStream. + * @return an implementation of ObjectInputStreamCallbacks. + */ + private ObjectInputStreamCallbacks createInputStreamCallbacks( final AuditSpan auditSpan) { - return new InputStreamCallbacksImpl(auditSpan); + return new InputStreamCallbacksImpl(auditSpan, getStore(), fsHandler, unboundedThreadPool); } - /** - * Operations needed by S3AInputStream to read data. - */ - private final class InputStreamCallbacksImpl implements - S3AInputStream.InputStreamCallbacks { - - /** - * Audit span to activate before each call. - */ - private final AuditSpan auditSpan; - - /** - * Create. - * @param auditSpan Audit span to activate before each call. - */ - private InputStreamCallbacksImpl(final AuditSpan auditSpan) { - this.auditSpan = requireNonNull(auditSpan); - } - - /** - * Closes the audit span. - */ - @Override - public void close() { - auditSpan.close(); - } - - @Override - public GetObjectRequest.Builder newGetRequestBuilder(final String key) { - // active the audit span used for the operation - try (AuditSpan span = auditSpan.activate()) { - return getRequestFactory().newGetObjectRequestBuilder(key); - } - } - - @Override - public ResponseInputStream getObject(GetObjectRequest request) throws - IOException { - // active the audit span used for the operation - try (AuditSpan span = auditSpan.activate()) { - return fsHandler.getObject(store, request, getRequestFactory()); - } - } - - @Override - public CompletableFuture submit(final CallableRaisingIOE operation) { - CompletableFuture result = new CompletableFuture<>(); - unboundedThreadPool.submit(() -> - LambdaUtils.eval(result, () -> { - LOG.debug("Starting submitted operation in {}", auditSpan.getSpanId()); - try (AuditSpan span = auditSpan.activate()) { - return operation.apply(); - } finally { - LOG.debug("Completed submitted operation in {}", auditSpan.getSpanId()); - } - })); - return result; - } - } /** * Callbacks for WriteOperationHelper. @@ -1982,7 +1924,7 @@ private final class WriteOperationHelperCallbacksImpl @Retries.OnceRaw public CompleteMultipartUploadResponse completeMultipartUpload( CompleteMultipartUploadRequest request) { - return store.completeMultipartUpload(request); + return getStore().completeMultipartUpload(request); } @Override @@ -1992,7 +1934,7 @@ public UploadPartResponse uploadPart( final RequestBody body, final DurationTrackerFactory durationTrackerFactory) throws AwsServiceException, UncheckedIOException { - return store.uploadPart(request, body, durationTrackerFactory); + return getStore().uploadPart(request, body, durationTrackerFactory); } } @@ -2016,9 +1958,7 @@ protected S3AReadOpContext createReadContext( fileStatus, vectoredIOContext, IOStatisticsContext.getCurrentIOStatisticsContext().getAggregator(), - futurePool, - prefetchBlockSize, - prefetchBlockCount) + futurePool) .withAuditSpan(auditSpan); openFileHelper.applyDefaultOptions(roc); return roc.build(); @@ -2756,7 +2696,7 @@ public long getDefaultBlockSize(Path path) { */ @Override public long getObjectSize(S3Object s3Object) throws IOException { - return fsHandler.getS3ObjectSize(s3Object.key(), s3Object.size(), store, null); + return fsHandler.getS3ObjectSize(s3Object.key(), s3Object.size(), getStore(), null); } @Override @@ -2987,7 +2927,7 @@ protected DurationTrackerFactory getDurationTrackerFactory() { */ protected DurationTrackerFactory nonNullDurationTrackerFactory( DurationTrackerFactory factory) { - return store.nonNullDurationTrackerFactory(factory); + return getStore().nonNullDurationTrackerFactory(factory); } /** @@ -3025,7 +2965,7 @@ protected HeadObjectResponse getObjectMetadata(String key, ChangeTracker changeTracker, Invoker changeInvoker, String operation) throws IOException { - return store.headObject(key, changeTracker, changeInvoker, fsHandler, operation); + return getStore().headObject(key, changeTracker, changeInvoker, fsHandler, operation); } /** @@ -3173,7 +3113,7 @@ public void incrementWriteOperations() { protected void deleteObject(String key) throws SdkException, IOException { incrementWriteOperations(); - store.deleteObject(getRequestFactory() + getStore().deleteObject(getRequestFactory() .newDeleteObjectRequestBuilder(key) .build()); } @@ -3227,7 +3167,7 @@ void deleteObjectAtPath(Path f, private DeleteObjectsResponse deleteObjects(DeleteObjectsRequest deleteRequest) throws MultiObjectDeleteException, SdkException, IOException { incrementWriteOperations(); - DeleteObjectsResponse response = store.deleteObjects(deleteRequest).getValue(); + DeleteObjectsResponse response = getStore().deleteObjects(deleteRequest).getValue(); if (!response.errors().isEmpty()) { throw new MultiObjectDeleteException(response.errors()); } @@ -3270,7 +3210,7 @@ public PutObjectRequest.Builder newPutObjectRequestBuilder(String key, @Retries.OnceRaw public UploadInfo putObject(PutObjectRequest putObjectRequest, File file, ProgressableProgressListener listener) throws IOException { - return store.putObject(putObjectRequest, file, listener); + return getStore().putObject(putObjectRequest, file, listener); } /** @@ -3369,7 +3309,7 @@ UploadPartResponse uploadPart(UploadPartRequest request, RequestBody body, * @param bytes bytes in the request. */ protected void incrementPutStartStatistics(long bytes) { - store.incrementPutStartStatistics(bytes); + getStore().incrementPutStartStatistics(bytes); } /** @@ -3380,7 +3320,7 @@ protected void incrementPutStartStatistics(long bytes) { * @param bytes bytes in the request. */ protected void incrementPutCompletedStatistics(boolean success, long bytes) { - store.incrementPutCompletedStatistics(success, bytes); + getStore().incrementPutCompletedStatistics(success, bytes); } /** @@ -3391,7 +3331,7 @@ protected void incrementPutCompletedStatistics(boolean success, long bytes) { * @param bytes bytes successfully uploaded. */ protected void incrementPutProgressStatistics(String key, long bytes) { - store.incrementPutProgressStatistics(key, bytes); + getStore().incrementPutProgressStatistics(key, bytes); } /** @@ -4259,7 +4199,7 @@ PutObjectResponse executePut( ProgressableProgressListener listener = new ProgressableProgressListener(store, key, progress); UploadInfo info = putObject(putObjectRequest, file, listener); - PutObjectResponse result = store.waitForUploadCompletion(key, info).response(); + PutObjectResponse result = getStore().waitForUploadCompletion(key, info).response(); listener.uploadCompleted(info.getFileUpload()); return result; } @@ -4354,22 +4294,24 @@ public void close() throws IOException { protected synchronized void stopAllServices() { try { trackDuration(getDurationTrackerFactory(), FILESYSTEM_CLOSE.getSymbol(), () -> { - closeAutocloseables(LOG, store); + closeAutocloseables(LOG, getStore()); store = null; s3Client = null; // At this point the S3A client is shut down, // now the executor pools are closed + + // shut future pool first as it wraps the bounded thread pool + if (futurePool != null) { + futurePool.shutdown(LOG, THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); + futurePool = null; + } HadoopExecutors.shutdown(boundedThreadPool, LOG, THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); boundedThreadPool = null; HadoopExecutors.shutdown(unboundedThreadPool, LOG, THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); unboundedThreadPool = null; - if (futurePool != null) { - futurePool.shutdown(LOG, THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); - futurePool = null; - } // other services are shutdown. cleanupWithLogger(LOG, delegationTokens.orElse(null), @@ -4575,7 +4517,7 @@ private CopyObjectResponse copyFile(String srcKey, String dstKey, long size, () -> { incrementStatistic(OBJECT_COPY_REQUESTS); - Copy copy = store.getOrCreateTransferManager().copy( + Copy copy = getStore().getOrCreateTransferManager().copy( CopyRequest.builder() .copyObjectRequest(copyRequest) .build()); @@ -5434,15 +5376,18 @@ public boolean hasPathCapability(final Path path, final String capability) case AWS_S3_ACCESS_GRANTS_ENABLED: return s3AccessGrantsEnabled; - // stream leak detection. - case StreamStatisticNames.STREAM_LEAKS: - return !prefetchEnabled; - default: // is it a performance flag? if (performanceFlags.hasCapability(capability)) { return true; } + + // ask the store for what capabilities it offers + // this may include input and output capabilites -and more + if (getStore() != null && getStore().hasPathCapability(path, capability)) { + return true; + } + // fall through } @@ -5703,7 +5648,7 @@ public BulkDelete createBulkDelete(final Path path) */ protected BulkDeleteOperation.BulkDeleteOperationCallbacks createBulkDeleteCallbacks( Path path, int pageSize, AuditSpanS3A span) { - return new BulkDeleteOperationCallbacksImpl(store, pathToKey(path), pageSize, span); + return new BulkDeleteOperationCallbacksImpl(getStore(), pathToKey(path), pageSize, span); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java index c620ca042dc82..4d7de5733c0d8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java @@ -19,7 +19,6 @@ package org.apache.hadoop.fs.s3a; import javax.annotation.Nullable; -import java.io.Closeable; import java.io.EOFException; import java.io.IOException; import java.io.InputStream; @@ -30,7 +29,6 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.IntFunction; @@ -41,7 +39,9 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.impl.LeakReporter; -import org.apache.hadoop.fs.statistics.StreamStatisticNames; +import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStream; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectReadParameters; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -49,7 +49,6 @@ import org.apache.hadoop.fs.CanSetReadahead; import org.apache.hadoop.fs.CanUnbuffer; import org.apache.hadoop.fs.FSExceptionMessages; -import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.FileRange; import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.impl.CombinedFileRange; @@ -57,17 +56,11 @@ import org.apache.hadoop.fs.s3a.impl.ChangeTracker; import org.apache.hadoop.fs.s3a.impl.InternalConstants; import org.apache.hadoop.fs.s3a.impl.SDKStreamDrainer; -import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; import org.apache.hadoop.fs.statistics.DurationTracker; -import org.apache.hadoop.fs.statistics.IOStatistics; -import org.apache.hadoop.fs.statistics.IOStatisticsAggregator; import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.util.functional.CallableRaisingIOE; -import static java.util.Objects.requireNonNull; -import static org.apache.commons.lang3.StringUtils.isNotEmpty; import static org.apache.hadoop.fs.VectoredReadUtils.isOrderedDisjoint; import static org.apache.hadoop.fs.VectoredReadUtils.mergeSortedRanges; import static org.apache.hadoop.fs.VectoredReadUtils.validateAndSortRanges; @@ -94,7 +87,7 @@ */ @InterfaceAudience.Private @InterfaceStability.Evolving -public class S3AInputStream extends FSInputStream implements CanSetReadahead, +public class S3AInputStream extends ObjectInputStream implements CanSetReadahead, CanUnbuffer, StreamCapabilities, IOStatisticsSource { public static final String E_NEGATIVE_READAHEAD_VALUE @@ -134,6 +127,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, * and returned in {@link #getPos()}. */ private long pos; + /** * Closed bit. Volatile so reads are non-blocking. * Updates must be in a synchronized block to guarantee an atomic check and @@ -144,35 +138,14 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, * Input stream returned by a getObject call. */ private ResponseInputStream wrappedStream; - private final S3AReadOpContext context; - private final InputStreamCallbacks client; - - /** - * Thread pool used for vectored IO operation. - */ - private final ExecutorService boundedThreadPool; - private final String bucket; - private final String key; - private final String pathStr; - - /** - * Content length from HEAD or openFile option. - */ - private final long contentLength; /** * Content length in format for vector IO. */ private final Optional fileLength; - private final String uri; - private final S3AInputStreamStatistics streamStatistics; - private S3AInputPolicy inputPolicy; private long readahead = Constants.DEFAULT_READAHEAD_RANGE; - /** Vectored IO context. */ - private final VectoredIOContext vectoredIOContext; - /** * This is the actual position within the object, used by * lazy seek to decide whether to seek on the next read or not. @@ -193,96 +166,32 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, /** change tracker. */ private final ChangeTracker changeTracker; - /** - * IOStatistics report. - */ - private final IOStatistics ioStatistics; - /** * Threshold for stream reads to switch to * asynchronous draining. */ - private long asyncDrainThreshold; - - /** Aggregator used to aggregate per thread IOStatistics. */ - private final IOStatisticsAggregator threadIOStatistics; - - /** - * Report of leaks. - * with report and abort unclosed streams in finalize(). - */ - private final LeakReporter leakReporter; + private final long asyncDrainThreshold; /** * Create the stream. * This does not attempt to open it; that is only done on the first * actual read() operation. - * @param ctx operation context - * @param s3Attributes object attributes - * @param client S3 client to use - * @param streamStatistics stream io stats. - * @param boundedThreadPool thread pool to use. - */ - public S3AInputStream(S3AReadOpContext ctx, - S3ObjectAttributes s3Attributes, - InputStreamCallbacks client, - S3AInputStreamStatistics streamStatistics, - ExecutorService boundedThreadPool) { - Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()), - "No Bucket"); - Preconditions.checkArgument(isNotEmpty(s3Attributes.getKey()), "No Key"); - long l = s3Attributes.getLen(); - Preconditions.checkArgument(l >= 0, "Negative content length"); - this.context = ctx; - this.bucket = s3Attributes.getBucket(); - this.key = s3Attributes.getKey(); - this.pathStr = s3Attributes.getPath().toString(); - this.contentLength = l; - this.fileLength = Optional.of(contentLength); - this.client = client; - this.uri = "s3a://" + this.bucket + "/" + this.key; - this.streamStatistics = streamStatistics; - this.ioStatistics = streamStatistics.getIOStatistics(); - this.changeTracker = new ChangeTracker(uri, - ctx.getChangeDetectionPolicy(), - streamStatistics.getChangeTrackerStatistics(), - s3Attributes); - setInputPolicy(ctx.getInputPolicy()); - setReadahead(ctx.getReadahead()); - this.asyncDrainThreshold = ctx.getAsyncDrainThreshold(); - this.boundedThreadPool = boundedThreadPool; - this.vectoredIOContext = context.getVectoredIOContext(); - this.threadIOStatistics = requireNonNull(ctx.getIOStatisticsAggregator()); - // build the leak reporter - this.leakReporter = new LeakReporter( - "Stream not closed while reading " + uri, - this::isStreamOpen, - () -> abortInFinalizer()); - } - - /** - * Finalizer. - *

- * Verify that the inner stream is closed. - *

- * If it is not, it means streams are being leaked in application code. - * Log a warning, including the stack trace of the caller, - * then abort the stream. - *

- * This does not attempt to invoke {@link #close()} as that is - * a more complex operation, and this method is being executed - * during a GC finalization phase. - *

- * Applications MUST close their streams; this is a defensive - * operation to return http connections and warn the end users - * that their applications are at risk of running out of connections. * - * {@inheritDoc} + * @param parameters creation parameters. */ - @Override - protected void finalize() throws Throwable { - leakReporter.close(); - super.finalize(); + public S3AInputStream(ObjectReadParameters parameters) { + + super(InputStreamType.Classic, parameters); + + + this.fileLength = Optional.of(getContentLength()); + S3AReadOpContext context = getContext(); + this.changeTracker = new ChangeTracker(getUri(), + context.getChangeDetectionPolicy(), + getS3AStreamStatistics().getChangeTrackerStatistics(), + getObjectAttributes()); + setReadahead(context.getReadahead()); + this.asyncDrainThreshold = context.getAsyncDrainThreshold(); } /** @@ -290,7 +199,8 @@ protected void finalize() throws Throwable { * Not synchronized; the flag is volatile. * @return true if the stream is still open. */ - private boolean isStreamOpen() { + @Override + protected boolean isStreamOpen() { return !closed; } @@ -298,10 +208,11 @@ private boolean isStreamOpen() { * Brute force stream close; invoked by {@link LeakReporter}. * All exceptions raised are ignored. */ - private void abortInFinalizer() { + @Override + protected void abortInFinalizer() { try { // stream was leaked: update statistic - streamStatistics.streamLeaked(); + getS3AStreamStatistics().streamLeaked(); // abort the stream. This merges statistics into the filesystem. closeStream("finalize()", true, true).get(); } catch (InterruptedException | ExecutionException ignroed) { @@ -309,32 +220,12 @@ private void abortInFinalizer() { } } - /** - * Set/update the input policy of the stream. - * This updates the stream statistics. - * @param inputPolicy new input policy. - */ - private void setInputPolicy(S3AInputPolicy inputPolicy) { - LOG.debug("Switching to input policy {}", inputPolicy); - this.inputPolicy = inputPolicy; - streamStatistics.inputPolicySet(inputPolicy.ordinal()); - } - - /** - * Get the current input policy. - * @return input policy. - */ - @VisibleForTesting - public S3AInputPolicy getInputPolicy() { - return inputPolicy; - } - /** * If the stream is in Adaptive mode, switch to random IO at this * point. Unsynchronized. */ private void maybeSwitchToRandomIO() { - if (inputPolicy.isAdaptive()) { + if (getInputPolicy().isAdaptive()) { setInputPolicy(S3AInputPolicy.Random); } } @@ -355,24 +246,24 @@ private synchronized void reopen(String reason, long targetPos, long length, closeStream("reopen(" + reason + ")", forceAbort, false); } - contentRangeFinish = calculateRequestLimit(inputPolicy, targetPos, - length, contentLength, readahead); + contentRangeFinish = calculateRequestLimit(getInputPolicy(), targetPos, + length, getContentLength(), readahead); LOG.debug("reopen({}) for {} range[{}-{}], length={}," + " streamPosition={}, nextReadPosition={}, policy={}", - uri, reason, targetPos, contentRangeFinish, length, pos, nextReadPos, - inputPolicy); + getUri(), reason, targetPos, contentRangeFinish, length, pos, nextReadPos, + getInputPolicy()); - GetObjectRequest request = client.newGetRequestBuilder(key) + GetObjectRequest request = getCallbacks().newGetRequestBuilder(getKey()) .range(S3AUtils.formatRange(targetPos, contentRangeFinish - 1)) .applyMutation(changeTracker::maybeApplyConstraint) .build(); - long opencount = streamStatistics.streamOpened(); + long opencount = getS3AStreamStatistics().streamOpened(); String operation = opencount == 0 ? OPERATION_OPEN : OPERATION_REOPEN; String text = String.format("%s %s at %d", - operation, uri, targetPos); - wrappedStream = onceTrackingDuration(text, uri, - streamStatistics.initiateGetRequest(), () -> - client.getObject(request)); + operation, getUri(), targetPos); + wrappedStream = onceTrackingDuration(text, getUri(), + getS3AStreamStatistics().initiateGetRequest(), () -> + getCallbacks().getObject(request)); changeTracker.processResponse(wrappedStream.response(), operation, targetPos); @@ -396,7 +287,7 @@ public synchronized void seek(long targetPos) throws IOException { + " " + targetPos); } - if (this.contentLength <= 0) { + if (this.getContentLength() <= 0) { return; } @@ -414,7 +305,7 @@ private void seekQuietly(long positiveTargetPos) { seek(positiveTargetPos); } catch (IOException ioe) { LOG.debug("Ignoring IOE on seek of {} to {}", - uri, positiveTargetPos, ioe); + getUri(), positiveTargetPos, ioe); } } @@ -449,12 +340,12 @@ private void seekInStream(long targetPos, long length) throws IOException { && diff < forwardSeekLimit; if (skipForward) { // the forward seek range is within the limits - LOG.debug("Forward seek on {}, of {} bytes", uri, diff); + LOG.debug("Forward seek on {}, of {} bytes", getUri(), diff); long skipped = wrappedStream.skip(diff); if (skipped > 0) { pos += skipped; } - streamStatistics.seekForwards(diff, skipped); + getS3AStreamStatistics().seekForwards(diff, skipped); if (pos == targetPos) { // all is well @@ -464,15 +355,15 @@ private void seekInStream(long targetPos, long length) throws IOException { } else { // log a warning; continue to attempt to re-open LOG.warn("Failed to seek on {} to {}. Current position {}", - uri, targetPos, pos); + getUri(), targetPos, pos); } } else { // not attempting to read any bytes from the stream - streamStatistics.seekForwards(diff, 0); + getS3AStreamStatistics().seekForwards(diff, 0); } } else if (diff < 0) { // backwards seek - streamStatistics.seekBackwards(diff); + getS3AStreamStatistics().seekBackwards(diff); // if the stream is in "Normal" mode, switch to random IO at this // point, as it is indicative of columnar format IO maybeSwitchToRandomIO(); @@ -513,8 +404,8 @@ public boolean seekToNewSource(long targetPos) throws IOException { @Retries.RetryTranslated private void lazySeek(long targetPos, long len) throws IOException { - Invoker invoker = context.getReadInvoker(); - invoker.retry("lazySeek to " + targetPos, pathStr, true, + Invoker invoker = getContext().getReadInvoker(); + invoker.retry("lazySeek to " + targetPos, getPathStr(), true, () -> { //For lazy seek seekInStream(targetPos, len); @@ -532,9 +423,9 @@ private void lazySeek(long targetPos, long len) throws IOException { * @param bytesRead number of bytes read */ private void incrementBytesRead(long bytesRead) { - streamStatistics.bytesRead(bytesRead); - if (context.stats != null && bytesRead > 0) { - context.stats.incrementBytesRead(bytesRead); + getS3AStreamStatistics().bytesRead(bytesRead); + if (getContext().stats != null && bytesRead > 0) { + getContext().stats.incrementBytesRead(bytesRead); } } @@ -542,7 +433,7 @@ private void incrementBytesRead(long bytesRead) { @Retries.RetryTranslated public synchronized int read() throws IOException { checkNotClosed(); - if (this.contentLength == 0 || (nextReadPos >= contentLength)) { + if (this.getContentLength() == 0 || (nextReadPos >= getContentLength())) { return -1; } @@ -554,8 +445,8 @@ public synchronized int read() throws IOException { return -1; } - Invoker invoker = context.getReadInvoker(); - int byteRead = invoker.retry("read", pathStr, true, + Invoker invoker = getContext().getReadInvoker(); + int byteRead = invoker.retry("read", getPathStr(), true, () -> { int b; // When exception happens before re-setting wrappedStream in "reopen" called @@ -597,13 +488,13 @@ private void onReadFailure(IOException ioe, boolean forceAbort) { if (LOG.isDebugEnabled()) { LOG.debug("Got exception while trying to read from stream {}, " + "client: {} object: {}, trying to recover: ", - uri, client, objectResponse, ioe); + getUri(), getCallbacks(), objectResponse, ioe); } else { LOG.info("Got exception while trying to read from stream {}, " + "client: {} object: {}, trying to recover: " + ioe, - uri, client, objectResponse); + getUri(), getCallbacks(), objectResponse); } - streamStatistics.readException(); + getS3AStreamStatistics().readException(); closeStream("failure recovery", forceAbort, false); } @@ -638,7 +529,7 @@ public synchronized int read(byte[] buf, int off, int len) return 0; } - if (this.contentLength == 0 || (nextReadPos >= contentLength)) { + if (this.getContentLength() == 0 || (nextReadPos >= getContentLength())) { return -1; } @@ -649,10 +540,10 @@ public synchronized int read(byte[] buf, int off, int len) return -1; } - Invoker invoker = context.getReadInvoker(); + Invoker invoker = getContext().getReadInvoker(); - streamStatistics.readOperationStarted(nextReadPos, len); - int bytesRead = invoker.retry("read", pathStr, true, + getS3AStreamStatistics().readOperationStarted(nextReadPos, len); + int bytesRead = invoker.retry("read", getPathStr(), true, () -> { int bytes; // When exception happens before re-setting wrappedStream in "reopen" called @@ -685,7 +576,7 @@ public synchronized int read(byte[] buf, int off, int len) } else { streamReadResultNegative(); } - streamStatistics.readOperationCompleted(len, bytesRead); + getS3AStreamStatistics().readOperationCompleted(len, bytesRead); return bytesRead; } @@ -696,7 +587,7 @@ public synchronized int read(byte[] buf, int off, int len) */ private void checkNotClosed() throws IOException { if (closed) { - throw new IOException(uri + ": " + FSExceptionMessages.STREAM_IS_CLOSED); + throw new IOException(getUri() + ": " + FSExceptionMessages.STREAM_IS_CLOSED); } } @@ -717,28 +608,14 @@ public synchronized void close() throws IOException { // close or abort the stream; blocking closeStream("close() operation", false, true); // end the client+audit span. - client.close(); - // this is actually a no-op - super.close(); + getCallbacks().close(); + } finally { - // merge the statistics back into the FS statistics. - streamStatistics.close(); - // Collect ThreadLevel IOStats - mergeThreadIOStatistics(streamStatistics.getIOStatistics()); + super.close(); } } } - /** - * Merging the current thread's IOStatistics with the current IOStatistics - * context. - * - * @param streamIOStats Stream statistics to be merged into thread - * statistics aggregator. - */ - private void mergeThreadIOStatistics(IOStatistics streamIOStats) { - threadIOStatistics.aggregate(streamIOStats); - } /** * Close a stream: decide whether to abort or close, based on @@ -776,11 +653,11 @@ private CompletableFuture closeStream( boolean shouldAbort = forceAbort || remaining > readahead; CompletableFuture operation; SDKStreamDrainer> drainer = new SDKStreamDrainer<>( - uri, + getUri(), wrappedStream, shouldAbort, (int) remaining, - streamStatistics, + getS3AStreamStatistics(), reason); if (blocking || shouldAbort || remaining <= asyncDrainThreshold) { @@ -792,7 +669,7 @@ private CompletableFuture closeStream( } else { LOG.debug("initiating asynchronous drain of {} bytes", remaining); // schedule an async drain/abort - operation = client.submit(drainer); + operation = getCallbacks().submit(drainer); } // either the stream is closed in the blocking call or the async call is @@ -817,7 +694,7 @@ private CompletableFuture closeStream( @InterfaceStability.Unstable public synchronized boolean resetConnection() throws IOException { checkNotClosed(); - LOG.info("Forcing reset of connection to {}", uri); + LOG.info("Forcing reset of connection to {}", getUri()); return awaitFuture(closeStream("reset()", true, true)); } @@ -839,7 +716,7 @@ public synchronized int available() throws IOException { @InterfaceAudience.Private @InterfaceStability.Unstable public synchronized long remainingInFile() { - return this.contentLength - this.pos; + return this.getContentLength() - this.pos; } /** @@ -879,23 +756,24 @@ public boolean markSupported() { @Override @InterfaceStability.Unstable public String toString() { - String s = streamStatistics.toString(); + String s = getS3AStreamStatistics().toString(); synchronized (this) { final StringBuilder sb = new StringBuilder( "S3AInputStream{"); - sb.append(uri); + sb.append(super.toString()).append(" "); + sb.append(getUri()); sb.append(" wrappedStream=") .append(isObjectStreamOpen() ? "open" : "closed"); - sb.append(" read policy=").append(inputPolicy); + sb.append(" read policy=").append(getInputPolicy()); sb.append(" pos=").append(pos); sb.append(" nextReadPos=").append(nextReadPos); - sb.append(" contentLength=").append(contentLength); + sb.append(" contentLength=").append(getContentLength()); sb.append(" contentRangeStart=").append(contentRangeStart); sb.append(" contentRangeFinish=").append(contentRangeFinish); sb.append(" remainingInCurrentRequest=") .append(remainingInCurrentRequest()); sb.append(" ").append(changeTracker); - sb.append(" ").append(vectoredIOContext); + sb.append(" ").append(getVectoredIOContext()); sb.append('\n').append(s); sb.append('}'); return sb.toString(); @@ -920,7 +798,7 @@ public void readFully(long position, byte[] buffer, int offset, int length) throws IOException { checkNotClosed(); validatePositionedReadArgs(position, buffer, offset, length); - streamStatistics.readFullyOperationStarted(position, length); + getS3AStreamStatistics().readFullyOperationStarted(position, length); if (length == 0) { return; } @@ -945,22 +823,6 @@ public void readFully(long position, byte[] buffer, int offset, int length) } } - /** - * {@inheritDoc}. - */ - @Override - public int minSeekForVectorReads() { - return vectoredIOContext.getMinSeekForVectorReads(); - } - - /** - * {@inheritDoc}. - */ - @Override - public int maxReadSizeForVectorReads() { - return vectoredIOContext.getMaxReadSizeForVectorReads(); - } - /** * {@inheritDoc} * Vectored read implementation for S3AInputStream. @@ -971,10 +833,10 @@ public int maxReadSizeForVectorReads() { @Override public synchronized void readVectored(List ranges, IntFunction allocate) throws IOException { - LOG.debug("Starting vectored read on path {} for ranges {} ", pathStr, ranges); + LOG.debug("Starting vectored read on path {} for ranges {} ", getPathStr(), ranges); checkNotClosed(); if (stopVectoredIOOperations.getAndSet(false)) { - LOG.debug("Reinstating vectored read operation for path {} ", pathStr); + LOG.debug("Reinstating vectored read operation for path {} ", getPathStr()); } // prepare to read @@ -992,26 +854,28 @@ public synchronized void readVectored(List ranges, if (isOrderedDisjoint(sortedRanges, 1, minSeekForVectorReads())) { LOG.debug("Not merging the ranges as they are disjoint"); - streamStatistics.readVectoredOperationStarted(sortedRanges.size(), sortedRanges.size()); + getS3AStreamStatistics().readVectoredOperationStarted(sortedRanges.size(), + sortedRanges.size()); for (FileRange range: sortedRanges) { ByteBuffer buffer = allocate.apply(range.getLength()); - boundedThreadPool.submit(() -> readSingleRange(range, buffer)); + getBoundedThreadPool().submit(() -> readSingleRange(range, buffer)); } } else { LOG.debug("Trying to merge the ranges as they are not disjoint"); List combinedFileRanges = mergeSortedRanges(sortedRanges, 1, minSeekForVectorReads(), maxReadSizeForVectorReads()); - streamStatistics.readVectoredOperationStarted(sortedRanges.size(), combinedFileRanges.size()); + getS3AStreamStatistics().readVectoredOperationStarted(sortedRanges.size(), + combinedFileRanges.size()); LOG.debug("Number of original ranges size {} , Number of combined ranges {} ", ranges.size(), combinedFileRanges.size()); for (CombinedFileRange combinedFileRange: combinedFileRanges) { - boundedThreadPool.submit( + getBoundedThreadPool().submit( () -> readCombinedRangeAndUpdateChildren(combinedFileRange, allocate)); } } LOG.debug("Finished submitting vectored read to threadpool" + - " on path {} for ranges {} ", pathStr, ranges); + " on path {} for ranges {} ", getPathStr(), ranges); } /** @@ -1022,7 +886,7 @@ public synchronized void readVectored(List ranges, */ private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRange, IntFunction allocate) { - LOG.debug("Start reading {} from path {} ", combinedFileRange, pathStr); + LOG.debug("Start reading {} from path {} ", combinedFileRange, getPathStr()); ResponseInputStream rangeContent = null; try { rangeContent = getS3ObjectInputStream("readCombinedFileRange", @@ -1030,7 +894,7 @@ private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRa combinedFileRange.getLength()); populateChildBuffers(combinedFileRange, rangeContent, allocate); } catch (Exception ex) { - LOG.debug("Exception while reading {} from path {} ", combinedFileRange, pathStr, ex); + LOG.debug("Exception while reading {} from path {} ", combinedFileRange, getPathStr(), ex); // complete exception all the underlying ranges which have not already // finished. for(FileRange child : combinedFileRange.getUnderlying()) { @@ -1041,7 +905,7 @@ private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRa } finally { IOUtils.cleanupWithLogger(LOG, rangeContent); } - LOG.debug("Finished reading {} from path {} ", combinedFileRange, pathStr); + LOG.debug("Finished reading {} from path {} ", combinedFileRange, getPathStr()); } /** @@ -1129,7 +993,7 @@ private void drainUnnecessaryData( remaining -= readCount; } } finally { - streamStatistics.readVectoredBytesDiscarded(drainBytes); + getS3AStreamStatistics().readVectoredBytesDiscarded(drainBytes); LOG.debug("{} bytes drained from stream ", drainBytes); } } @@ -1140,7 +1004,7 @@ private void drainUnnecessaryData( * @param buffer buffer to fill. */ private void readSingleRange(FileRange range, ByteBuffer buffer) { - LOG.debug("Start reading {} from {} ", range, pathStr); + LOG.debug("Start reading {} from {} ", range, getPathStr()); if (range.getLength() == 0) { // a zero byte read. buffer.flip(); @@ -1155,12 +1019,12 @@ private void readSingleRange(FileRange range, ByteBuffer buffer) { populateBuffer(range, buffer, objectRange); range.getData().complete(buffer); } catch (Exception ex) { - LOG.warn("Exception while reading a range {} from path {} ", range, pathStr, ex); + LOG.warn("Exception while reading a range {} from path {} ", range, getPathStr(), ex); range.getData().completeExceptionally(ex); } finally { IOUtils.cleanupWithLogger(LOG, objectRange); } - LOG.debug("Finished reading range {} from path {} ", range, pathStr); + LOG.debug("Finished reading range {} from path {} ", range, getPathStr()); } /** @@ -1274,18 +1138,18 @@ private ResponseInputStream getS3Object(String operationName, long position, int length) throws IOException { - final GetObjectRequest request = client.newGetRequestBuilder(key) + final GetObjectRequest request = getCallbacks().newGetRequestBuilder(getKey()) .range(S3AUtils.formatRange(position, position + length - 1)) .applyMutation(changeTracker::maybeApplyConstraint) .build(); - DurationTracker tracker = streamStatistics.initiateGetRequest(); + DurationTracker tracker = getS3AStreamStatistics().initiateGetRequest(); ResponseInputStream objectRange; - Invoker invoker = context.getReadInvoker(); + Invoker invoker = getContext().getReadInvoker(); try { - objectRange = invoker.retry(operationName, pathStr, true, + objectRange = invoker.retry(operationName, getPathStr(), true, () -> { checkIfVectoredIOStopped(); - return client.getObject(request); + return getCallbacks().getObject(request); }); } catch (IOException ex) { @@ -1312,18 +1176,6 @@ private void checkIfVectoredIOStopped() throws InterruptedIOException { } } - /** - * Access the input stream statistics. - * This is for internal testing and may be removed without warning. - * @return the statistics for this input stream - */ - @InterfaceAudience.Private - @InterfaceStability.Unstable - @VisibleForTesting - public S3AInputStreamStatistics getS3AStreamStatistics() { - return streamStatistics; - } - @Override public synchronized void setReadahead(Long readahead) { this.readahead = validateReadahead(readahead); @@ -1409,8 +1261,8 @@ public synchronized void unbuffer() { stopVectoredIOOperations.set(true); closeStream("unbuffer()", false, false); } finally { - streamStatistics.unbuffered(); - if (inputPolicy.isAdaptive()) { + getS3AStreamStatistics().unbuffered(); + if (getInputPolicy().isAdaptive()) { S3AInputPolicy policy = S3AInputPolicy.Random; setInputPolicy(policy); } @@ -1420,15 +1272,12 @@ public synchronized void unbuffer() { @Override public boolean hasCapability(String capability) { switch (toLowerCase(capability)) { - case StreamCapabilities.IOSTATISTICS: case StreamCapabilities.IOSTATISTICS_CONTEXT: - case StreamStatisticNames.STREAM_LEAKS: case StreamCapabilities.READAHEAD: case StreamCapabilities.UNBUFFER: - case StreamCapabilities.VECTOREDIO: return true; default: - return false; + return super.hasCapability(capability); } } @@ -1441,11 +1290,6 @@ public boolean isObjectStreamOpen() { return wrappedStream != null; } - @Override - public IOStatistics getIOStatistics() { - return ioStatistics; - } - /** * Get the wrapped stream. * This is for testing only. @@ -1457,38 +1301,4 @@ public ResponseInputStream getWrappedStream() { return wrappedStream; } - /** - * Callbacks for input stream IO. - */ - public interface InputStreamCallbacks extends Closeable { - - /** - * Create a GET request builder. - * @param key object key - * @return the request builder - */ - GetObjectRequest.Builder newGetRequestBuilder(String key); - - /** - * Execute the request. - * When CSE is enabled with reading of unencrypted data, The object is checked if it is - * encrypted and if so, the request is made with encrypted S3 client. If the object is - * not encrypted, the request is made with unencrypted s3 client. - * @param request the request - * @return the response - * @throws IOException on any failure. - */ - @Retries.OnceRaw - ResponseInputStream getObject(GetObjectRequest request) throws IOException; - - /** - * Submit some asynchronous work, for example, draining a stream. - * @param operation operation to invoke - * @param return type - * @return a future. - */ - CompletableFuture submit(CallableRaisingIOE operation); - - } - } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index b84f19fcd8773..1eef0b404bc64 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -28,6 +28,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.impl.WeakRefMetricsSource; +import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType; import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; @@ -840,6 +841,7 @@ private final class InputStreamStatistics private final AtomicLong closed; private final AtomicLong forwardSeekOperations; private final AtomicLong openOperations; + private final AtomicLong analyticsStreamOpenOperations; private final AtomicLong readExceptions; private final AtomicLong readsIncomplete; private final AtomicLong readOperations; @@ -888,7 +890,8 @@ private InputStreamStatistics( StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS, StreamStatisticNames.STREAM_READ_VECTORED_READ_BYTES_DISCARDED, StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES, - StreamStatisticNames.STREAM_EVICT_BLOCKS_FROM_FILE_CACHE) + StreamStatisticNames.STREAM_EVICT_BLOCKS_FROM_FILE_CACHE, + StreamStatisticNames.STREAM_READ_ANALYTICS_OPENED) .withGauges(STREAM_READ_GAUGE_INPUT_POLICY, STREAM_READ_BLOCKS_IN_FILE_CACHE.getSymbol(), STREAM_READ_ACTIVE_PREFETCH_OPERATIONS.getSymbol(), @@ -927,6 +930,9 @@ private InputStreamStatistics( StreamStatisticNames.STREAM_READ_SEEK_FORWARD_OPERATIONS); openOperations = st.getCounterReference( StreamStatisticNames.STREAM_READ_OPENED); + analyticsStreamOpenOperations = st.getCounterReference( + StreamStatisticNames.STREAM_READ_ANALYTICS_OPENED + ); readExceptions = st.getCounterReference( StreamStatisticNames.STREAM_READ_EXCEPTIONS); readsIncomplete = st.getCounterReference( @@ -1030,6 +1036,16 @@ public long streamOpened() { return openOperations.getAndIncrement(); } + @Override + public long streamOpened(InputStreamType type) { + switch (type) { + case Analytics: + return analyticsStreamOpenOperations.getAndIncrement(); + default: + return openOperations.getAndIncrement(); + } + } + /** * {@inheritDoc}. * If the connection was aborted, increment {@link #aborted} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java index 55351f0c81396..0895e6a6c1155 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java @@ -34,7 +34,7 @@ import static java.util.Objects.requireNonNull; /** - * Read-specific operation context struct. + * Read-specific operation context structure. */ public class S3AReadOpContext extends S3AOpContext { @@ -75,15 +75,11 @@ public class S3AReadOpContext extends S3AOpContext { /** Thread-level IOStatistics aggregator. **/ private final IOStatisticsAggregator ioStatisticsAggregator; - // S3 reads are prefetched asynchronously using this future pool. + /** + * Pool for any future IO. + */ private ExecutorServiceFuturePool futurePool; - // Size in bytes of a single prefetch block. - private final int prefetchBlockSize; - - // Size of prefetch queue (in number of blocks). - private final int prefetchBlockCount; - /** * Instantiate. * @param path path of read @@ -93,9 +89,7 @@ public class S3AReadOpContext extends S3AOpContext { * @param dstFileStatus target file status * @param vectoredIOContext context for vectored read operation. * @param ioStatisticsAggregator IOStatistics aggregator for each thread. - * @param futurePool the ExecutorServiceFuturePool instance used by async prefetches. - * @param prefetchBlockSize the size (in number of bytes) of each prefetched block. - * @param prefetchBlockCount maximum number of prefetched blocks. + * @param futurePool Pool for any future IO */ public S3AReadOpContext( final Path path, @@ -105,9 +99,7 @@ public S3AReadOpContext( FileStatus dstFileStatus, VectoredIOContext vectoredIOContext, IOStatisticsAggregator ioStatisticsAggregator, - ExecutorServiceFuturePool futurePool, - int prefetchBlockSize, - int prefetchBlockCount) { + ExecutorServiceFuturePool futurePool) { super(invoker, stats, instrumentation, dstFileStatus); @@ -115,12 +107,7 @@ public S3AReadOpContext( this.vectoredIOContext = requireNonNull(vectoredIOContext, "vectoredIOContext"); this.ioStatisticsAggregator = ioStatisticsAggregator; this.futurePool = futurePool; - Preconditions.checkArgument( - prefetchBlockSize > 0, "invalid prefetchBlockSize %d", prefetchBlockSize); - this.prefetchBlockSize = prefetchBlockSize; - Preconditions.checkArgument( - prefetchBlockCount > 0, "invalid prefetchBlockCount %d", prefetchBlockCount); - this.prefetchBlockCount = prefetchBlockCount; + } /** @@ -265,23 +252,6 @@ public ExecutorServiceFuturePool getFuturePool() { return this.futurePool; } - /** - * Gets the size in bytes of a single prefetch block. - * - * @return the size in bytes of a single prefetch block. - */ - public int getPrefetchBlockSize() { - return this.prefetchBlockSize; - } - - /** - * Gets the size of prefetch queue (in number of blocks). - * - * @return the size of prefetch queue (in number of blocks). - */ - public int getPrefetchBlockCount() { - return this.prefetchBlockCount; - } @Override public String toString() { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java index ab8785e01dafd..f5c89a32a3813 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java @@ -45,15 +45,21 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.PathCapabilities; import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.impl.ChangeTracker; import org.apache.hadoop.fs.s3a.impl.ClientManager; import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteException; import org.apache.hadoop.fs.s3a.impl.S3AFileSystemOperations; import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamFactory; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.service.Service; /** * Interface for the S3A Store; @@ -63,10 +69,19 @@ * The {@link ClientManager} interface is used to create the AWS clients; * the base implementation forwards to the implementation of this interface * passed in at construction time. + *

+ * The interface extends the Hadoop {@link Service} interface + * and follows its lifecycle: it MUST NOT be used until + * {@link Service#init(Configuration)} has been invoked. */ @InterfaceAudience.LimitedPrivate("Extensions") @InterfaceStability.Unstable -public interface S3AStore extends IOStatisticsSource, ClientManager { +public interface S3AStore extends + ClientManager, + IOStatisticsSource, + ObjectInputStreamFactory, + PathCapabilities, + Service { /** * Acquire write capacity for operations. @@ -302,4 +317,54 @@ CompletedFileUpload waitForUploadCompletion(String key, UploadInfo uploadInfo) @Retries.OnceRaw CompleteMultipartUploadResponse completeMultipartUpload( CompleteMultipartUploadRequest request); + + /** + * Get the directory allocator. + * @return the directory allocator + */ + LocalDirAllocator getDirectoryAllocator(); + + /** + * Demand create the directory allocator, then create a temporary file. + * This does not mark the file for deletion when a process exits. + * Pass in a file size of {@link LocalDirAllocator#SIZE_UNKNOWN} if the + * size is unknown. + * {@link LocalDirAllocator#createTmpFileForWrite(String, long, Configuration)}. + * @param pathStr prefix for the temporary file + * @param size the size of the file that is going to be written + * @param conf the Configuration object + * @return a unique temporary file + * @throws IOException IO problems + */ + File createTemporaryFileForWriting(String pathStr, + long size, + Configuration conf) throws IOException; + + + /* + =============== BEGIN ObjectInputStreamFactory =============== + */ + + /** + * Return the capabilities of input streams created + * through the store. + * @param capability string to query the stream support for. + * @return capabilities declared supported in streams. + */ + boolean inputStreamHasCapability(String capability); + + /** + * The StreamCapabilities is part of ObjectInputStreamFactory. + * To avoid confusion with any other streams which may + * be added here: always return false. + * @param capability string to query the stream support for. + * @return false, always. + */ + default boolean hasCapability(String capability) { + return false; + } + + /* + =============== END ObjectInputStreamFactory =============== + */ } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java index cda09b622eaff..3a9425de1114e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java @@ -202,6 +202,11 @@ final class S3ClientCreationParameters { */ private boolean fipsEnabled; + /** + * Is analytics accelerator enabled + */ + private boolean isAnalyticsAcceleratorEnabled; + /** * List of execution interceptors to include in the chain * of interceptors in the SDK. @@ -457,6 +462,17 @@ public S3ClientCreationParameters withClientSideEncryptionEnabled(final boolean return this; } + /** + * Set the analytics accelerator enabled flag. + * + * @param value new value + * @return the builder + */ + public S3ClientCreationParameters withAnalyticsAcceleratorEnabled(final boolean value) { + this.isAnalyticsAcceleratorEnabled = value; + return this; + } + /** * Set the KMS client region. * This is required for CSE-KMS @@ -477,6 +493,14 @@ public boolean isClientSideEncryptionEnabled() { return this.isCSEEnabled; } + /** + * Get the analytics accelerator enabled flag. + * @return analytics accelerator enabled flag. + */ + public boolean isAnalyticsAcceleratorEnabled() { + return this.isAnalyticsAcceleratorEnabled; + } + /** * Set the client side encryption materials. * diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/VectoredIOContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/VectoredIOContext.java index 31f0ae4cb5515..eb87aec2dea06 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/VectoredIOContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/VectoredIOContext.java @@ -18,14 +18,12 @@ package org.apache.hadoop.fs.s3a; -import java.util.List; -import java.util.function.IntFunction; +import static org.apache.hadoop.util.Preconditions.checkState; /** - * Context related to vectored IO operation. - * See {@link S3AInputStream#readVectored(List, IntFunction)}. + * Configuration information for vectored IO. */ -public class VectoredIOContext { +public final class VectoredIOContext { /** * What is the smallest reasonable seek that we should group @@ -36,43 +34,118 @@ public class VectoredIOContext { /** * What is the largest size that we should group ranges * together during vectored read operation. - * Setting this value 0 will disable merging of ranges. + * Setting this value to 0 will disable merging of ranges. */ private int maxReadSizeForVectorReads; + /** + * Maximum number of active range read operation a single + * input stream can have. + */ + private int vectoredActiveRangeReads; + + /** + * Can this instance be updated? + */ + private boolean immutable = false; + /** * Default no arg constructor. */ public VectoredIOContext() { } - public VectoredIOContext setMinSeekForVectoredReads(int minSeek) { - this.minSeekForVectorReads = minSeek; + /** + * Make immutable. + * @return this instance. + */ + public VectoredIOContext build() { + immutable = true; return this; } - public VectoredIOContext setMaxReadSizeForVectoredReads(int maxSize) { - this.maxReadSizeForVectorReads = maxSize; - return this; + /** + * Verify this object is still mutable. + * @throws IllegalStateException if not. + */ + private void checkMutable() { + checkState(!immutable, "Instance is immutable"); } - public VectoredIOContext build() { + /** + * What is the threshold at which a seek() to a new location + * is initiated, rather than merging ranges? + * Set to zero to disable range merging entirely. + * @param minSeek minimum amount of data to skip. + */ + public VectoredIOContext setMinSeekForVectoredReads(int minSeek) { + checkMutable(); + checkState(minSeek >= 0); + this.minSeekForVectorReads = minSeek; return this; } + /** + * What is the threshold at which a seek() to a new location + * is initiated, rather than merging ranges? + * @return a number >= 0 + */ public int getMinSeekForVectorReads() { return minSeekForVectorReads; } + /** + * What is the largest size that we should group ranges + * together during vectored read operation? + * @param maxSize maximum size + * @return this instance. + */ + public VectoredIOContext setMaxReadSizeForVectoredReads(int maxSize) { + checkMutable(); + checkState(maxSize >= 0); + this.maxReadSizeForVectorReads = maxSize; + return this; + } + + /** + * The largest size that we should group ranges + * together during vectored read operation + * @return a number >= 0 + */ public int getMaxReadSizeForVectorReads() { return maxReadSizeForVectorReads; } + /** + * Maximum number of active range read operation a single + * input stream can have. + * @return number of extra threads for reading, or zero. + */ + public int getVectoredActiveRangeReads() { + return vectoredActiveRangeReads; + } + + /** + * Maximum number of active range read operation a single + * input stream can have. + * @return this instance. + * number of extra threads for reading, or zero. + * @param activeReads number of extra threads for reading, or zero. + */ + public VectoredIOContext setVectoredActiveRangeReads( + final int activeReads) { + checkMutable(); + checkState(activeReads >= 0); + this.vectoredActiveRangeReads = activeReads; + return this; + } + @Override public String toString() { return "VectoredIOContext{" + "minSeekForVectorReads=" + minSeekForVectorReads + ", maxReadSizeForVectorReads=" + maxReadSizeForVectorReads + + ", vectoredActiveRangeReads=" + vectoredActiveRangeReads + '}'; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java index bb7f94cfc206d..672e28a1b28a6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a.audit; import java.io.IOException; +import java.util.EnumSet; import java.util.List; import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; @@ -90,4 +91,10 @@ public interface AuditManagerS3A extends Service, */ boolean checkAccess(Path path, S3AFileStatus status, FsAction mode) throws IOException; + + /** + * Update audit flags, especially the out of span rejection option. + * @param flags audit flags. + */ + void setAuditFlags(final EnumSet flags); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditorFlags.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditorFlags.java new file mode 100644 index 0000000000000..b071f466e7835 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditorFlags.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit; + +/** + * Flags which can be passed down during initialization, or after it. + */ +public enum AuditorFlags { + /** + * Are out of band operations allowed? + */ + PermitOutOfBandOperations +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java index 10c8f871cad32..383c44bb21dbf 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/OperationAuditor.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a.audit; import java.io.IOException; +import java.util.EnumSet; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -41,6 +42,12 @@ public interface OperationAuditor extends Service, */ void init(OperationAuditorOptions options); + /** + * Update audit flags, especially the out of span rejection option. + * @param flags audit flags. + */ + void setAuditFlags(final EnumSet flags); + /** * Get the unbonded span to use after deactivating an active * span. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java index c5ce1a2c9e4b8..d41c5697c1d10 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/AbstractOperationAuditor.java @@ -18,9 +18,12 @@ package org.apache.hadoop.fs.s3a.audit.impl; +import java.util.EnumSet; import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import org.apache.hadoop.fs.s3a.audit.AuditorFlags; import org.apache.hadoop.fs.s3a.audit.OperationAuditor; import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; @@ -60,6 +63,11 @@ public abstract class AbstractOperationAuditor extends AbstractService */ private OperationAuditorOptions options; + /** + * Should out of span requests be rejected? + */ + private AtomicBoolean rejectOutOfSpan = new AtomicBoolean(false); + /** * Auditor ID as a UUID. */ @@ -120,4 +128,9 @@ protected final String createSpanID() { return String.format("%s-%08d", auditorID, SPAN_ID_COUNTER.incrementAndGet()); } + + @Override + public void setAuditFlags(final EnumSet flags) { + /* no-op */ + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java index e8e989efaa141..fbbc7112bfd6e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.lang.reflect.Constructor; import java.util.ArrayList; +import java.util.EnumSet; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; @@ -35,6 +36,7 @@ import software.amazon.awssdk.transfer.s3.progress.TransferListener; import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.fs.s3a.audit.AuditorFlags; import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -392,6 +394,11 @@ public AuditSpanS3A createSpan(final String operation, operation, path1, path2)); } + @Override + public void setAuditFlags(final EnumSet flags) { + auditor.setAuditFlags(flags); + } + /** * Return a list of execution interceptors for the AWS SDK which * relays to this class. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java index 16bae4b816457..739257c0d15e6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java @@ -21,6 +21,7 @@ import javax.annotation.Nullable; import java.io.IOException; import java.util.Collection; +import java.util.EnumSet; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -46,6 +47,7 @@ import org.apache.hadoop.fs.s3a.audit.AuditFailureException; import org.apache.hadoop.fs.s3a.audit.AuditOperationRejectedException; import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; +import org.apache.hadoop.fs.s3a.audit.AuditorFlags; import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.fs.store.audit.HttpReferrerAuditHeader; import org.apache.hadoop.security.UserGroupInformation; @@ -215,6 +217,15 @@ public AuditSpanS3A createSpan(final String operation, return span; } + @Override + public void setAuditFlags(final EnumSet flags) { + // if out of band operations are allowed, configuration settings are overridden + if (flags.contains(AuditorFlags.PermitOutOfBandOperations)) { + LOG.debug("Out of span operations are required by the stream factory"); + rejectOutOfSpan = false; + } + } + /** * Get/Prepare the active context for a span. * @return the common audit context. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManagerS3A.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManagerS3A.java index e58c906460daa..729706cb3d81b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManagerS3A.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManagerS3A.java @@ -21,6 +21,7 @@ import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; +import java.util.EnumSet; import java.util.List; import java.util.UUID; @@ -34,6 +35,7 @@ import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.audit.AuditManagerS3A; import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; +import org.apache.hadoop.fs.s3a.audit.AuditorFlags; import org.apache.hadoop.fs.s3a.audit.OperationAuditor; import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions; import org.apache.hadoop.service.CompositeService; @@ -169,4 +171,9 @@ public static AuditSpanS3A createNewSpan( final String path2) { return NoopSpan.INSTANCE; } + + @Override + public void setAuditFlags(final EnumSet flags) { + /* no-op */ + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManager.java index ad7afc732387f..b60551088824c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManager.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManager.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.s3a.impl; -import java.io.Closeable; import java.io.IOException; import java.io.UncheckedIOException; @@ -26,11 +25,13 @@ import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.transfer.s3.S3TransferManager; +import org.apache.hadoop.service.Service; + /** * Interface for on-demand/async creation of AWS clients * and extension services. */ -public interface ClientManager extends Closeable { +public interface ClientManager extends Service { /** * Get the transfer manager, creating it and any dependencies if needed. @@ -76,8 +77,4 @@ S3TransferManager getOrCreateTransferManager() */ S3Client getOrCreateAsyncS3ClientUnchecked() throws UncheckedIOException; - /** - * Close operation is required to not raise exceptions. - */ - void close(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManagerImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManagerImpl.java index 44383e381248f..350d9f09d16b7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManagerImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClientManagerImpl.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicBoolean; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,6 +34,7 @@ import org.apache.hadoop.fs.s3a.S3ClientFactory; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.util.functional.CallableRaisingIOE; import org.apache.hadoop.util.functional.LazyAutoCloseableReference; @@ -49,11 +49,13 @@ /** * Client manager for on-demand creation of S3 clients, - * with parallelized close of them in {@link #close()}. + * with parallelized close of them in {@link #serviceStop()}. * Updates {@link org.apache.hadoop.fs.s3a.Statistic#STORE_CLIENT_CREATION} * to track count and duration of client creation. */ -public class ClientManagerImpl implements ClientManager { +public class ClientManagerImpl + extends AbstractService + implements ClientManager { public static final Logger LOG = LoggerFactory.getLogger(ClientManagerImpl.class); @@ -67,11 +69,6 @@ public class ClientManagerImpl implements ClientManager { */ private final S3ClientFactory unencryptedClientFactory; - /** - * Closed flag. - */ - private final AtomicBoolean closed = new AtomicBoolean(false); - /** * Parameters to create sync/async clients. */ @@ -115,12 +112,13 @@ public ClientManagerImpl( final S3ClientFactory unencryptedClientFactory, final S3ClientFactory.S3ClientCreationParameters clientCreationParameters, final DurationTrackerFactory durationTrackerFactory) { + super("ClientManager"); this.clientFactory = requireNonNull(clientFactory); this.unencryptedClientFactory = unencryptedClientFactory; this.clientCreationParameters = requireNonNull(clientCreationParameters); this.durationTrackerFactory = requireNonNull(durationTrackerFactory); this.s3Client = new LazyAutoCloseableReference<>(createS3Client()); - this.s3AsyncClient = new LazyAutoCloseableReference<>(createAyncClient()); + this.s3AsyncClient = new LazyAutoCloseableReference<>(createAsyncClient()); this.unencryptedS3Client = new LazyAutoCloseableReference<>(createUnencryptedS3Client()); this.transferManager = new LazyAutoCloseableReference<>(createTransferManager()); @@ -143,7 +141,7 @@ private CallableRaisingIOE createS3Client() { * Create the function to create the S3 Async client. * @return a callable which will create the client. */ - private CallableRaisingIOE createAyncClient() { + private CallableRaisingIOE createAsyncClient() { return trackDurationOfOperation( durationTrackerFactory, STORE_CLIENT_CREATION.getSymbol(), @@ -226,26 +224,8 @@ public synchronized S3TransferManager getOrCreateTransferManager() throws IOExce return transferManager.eval(); } - /** - * Check that the client manager is not closed. - * @throws IllegalStateException if it is closed. - */ - private void checkNotClosed() { - checkState(!closed.get(), "Client manager is closed"); - } - - /** - * Close() is synchronized to avoid race conditions between - * slow client creation and this close operation. - *

- * The objects are all deleted in parallel - */ @Override - public synchronized void close() { - if (closed.getAndSet(true)) { - // re-entrant close. - return; - } + protected void serviceStop() throws Exception { // queue the closures. List> l = new ArrayList<>(); l.add(closeAsync(transferManager)); @@ -253,14 +233,18 @@ public synchronized void close() { l.add(closeAsync(s3Client)); l.add(closeAsync(unencryptedS3Client)); - // once all are queued, await their completion - // and swallow any exception. - try { - awaitAllFutures(l); - } catch (Exception e) { - // should never happen. - LOG.warn("Exception in close", e); - } + // once all are queued, await their completion; + // exceptions will be swallowed. + awaitAllFutures(l); + super.serviceStop(); + } + + /** + * Check that the client manager is not closed. + * @throws IllegalStateException if it is closed. + */ + private void checkNotClosed() { + checkState(!isInState(STATE.STOPPED), "Client manager is closed"); } /** @@ -297,7 +281,7 @@ private CompletableFuture closeAsync( @Override public String toString() { return "ClientManagerImpl{" + - "closed=" + closed.get() + + "state=" + getServiceState() + ", s3Client=" + s3Client + ", s3AsyncClient=" + s3AsyncClient + ", unencryptedS3Client=" + unencryptedS3Client + diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InputStreamCallbacksImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InputStreamCallbacksImpl.java new file mode 100644 index 0000000000000..fde21f7a9b920 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InputStreamCallbacksImpl.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ThreadPoolExecutor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; + +import org.apache.hadoop.fs.s3a.S3AStore; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; +import org.apache.hadoop.fs.store.audit.AuditSpan; +import org.apache.hadoop.util.functional.CallableRaisingIOE; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.util.LambdaUtils.eval; + +/** + * Callbacks for object stream operations. + */ +public class InputStreamCallbacksImpl implements ObjectInputStreamCallbacks { + + private static final Logger LOG = LoggerFactory.getLogger(InputStreamCallbacksImpl.class); + + /** + * Audit span to activate before each call. + */ + private final AuditSpan auditSpan; + + /** + * store operations. + */ + private final S3AStore store; + + /** + * crypto FS operations. + */ + private final S3AFileSystemOperations fsOperations; + + /** + * A (restricted) thread pool for asynchronous operations. + */ + private final ThreadPoolExecutor threadPool; + + /** + * Create. + * @param auditSpan Audit span to activate before each call. + * @param store store operations + * @param fsOperations crypto FS operations. + * @param threadPool thread pool for async operations. + */ + public InputStreamCallbacksImpl( + final AuditSpan auditSpan, + final S3AStore store, + final S3AFileSystemOperations fsOperations, + final ThreadPoolExecutor threadPool) { + this.auditSpan = requireNonNull(auditSpan); + this.store = requireNonNull(store); + this.fsOperations = requireNonNull(fsOperations); + this.threadPool = requireNonNull(threadPool); + } + + /** + * Closes the audit span. + */ + @Override + public void close() { + auditSpan.close(); + } + + @Override + public GetObjectRequest.Builder newGetRequestBuilder(final String key) { + // active the audit span used for the operation + try (AuditSpan span = auditSpan.activate()) { + return store.getRequestFactory().newGetObjectRequestBuilder(key); + } + } + + @Override + public ResponseInputStream getObject(GetObjectRequest request) throws + IOException { + // active the audit span used for the operation + try (AuditSpan span = auditSpan.activate()) { + return fsOperations.getObject(store, request, store.getRequestFactory()); + } + } + + @Override + public CompletableFuture submit(final CallableRaisingIOE operation) { + CompletableFuture result = new CompletableFuture<>(); + threadPool.submit(() -> + eval(result, () -> { + LOG.debug("Starting submitted operation in {}", auditSpan.getSpanId()); + try (AuditSpan span = auditSpan.activate()) { + return operation.apply(); + } finally { + LOG.debug("Completed submitted operation in {}", auditSpan.getSpanId()); + } + })); + return result; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java index 00d9368aa58f1..cf237c5386b6e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java @@ -115,7 +115,7 @@ public class RequestFactoryImpl implements RequestFactory { /** * Callback to prepare requests. */ - private final PrepareRequest requestPreparer; + private PrepareRequest requestPreparer; /** * Content encoding (null for none). diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java index db07881345500..1a7868dd044df 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java @@ -57,7 +57,11 @@ import software.amazon.awssdk.transfer.s3.model.FileUpload; import software.amazon.awssdk.transfer.s3.model.UploadFileRequest; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.ProgressableProgressListener; import org.apache.hadoop.fs.s3a.Retries; @@ -69,16 +73,26 @@ import org.apache.hadoop.fs.s3a.UploadInfo; import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; +import org.apache.hadoop.fs.s3a.impl.streams.FactoryBindingParameters; +import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStream; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamFactory; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectReadParameters; +import org.apache.hadoop.fs.s3a.impl.streams.StreamFactoryRequirements; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.statistics.DurationTracker; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.store.audit.AuditSpanSource; +import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.util.RateLimiting; import org.apache.hadoop.util.functional.Tuples; import static java.util.Objects.requireNonNull; +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.fs.s3a.S3AUtils.extractException; import static org.apache.hadoop.fs.s3a.S3AUtils.getPutRequestLength; import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException; @@ -99,17 +113,21 @@ import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLE_RATE; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT; +import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.factoryFromConfig; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier; import static org.apache.hadoop.util.Preconditions.checkArgument; +import static org.apache.hadoop.util.StringUtils.toLowerCase; /** * Store Layer. * This is where lower level storage operations are intended * to move. */ -public class S3AStoreImpl implements S3AStore { +public class S3AStoreImpl + extends CompositeService + implements S3AStore, ObjectInputStreamFactory { private static final Logger LOG = LoggerFactory.getLogger(S3AStoreImpl.class); @@ -165,7 +183,20 @@ public class S3AStoreImpl implements S3AStore { */ private final FileSystem.Statistics fsStatistics; - /** Constructor to create S3A store. */ + /** + * Allocator of local FS storage. + */ + private LocalDirAllocator directoryAllocator; + + /** + * Factory for input streams. + */ + private ObjectInputStreamFactory objectInputStreamFactory; + + /** + * Constructor to create S3A store. + * Package private, as {@link S3AStoreBuilder} creates them. + * */ S3AStoreImpl(StoreContextFactory storeContextFactory, ClientManager clientManager, DurationTrackerFactory durationTrackerFactory, @@ -176,25 +207,90 @@ public class S3AStoreImpl implements S3AStore { RateLimiting writeRateLimiter, AuditSpanSource auditSpanSource, @Nullable FileSystem.Statistics fsStatistics) { - this.storeContextFactory = requireNonNull(storeContextFactory); + super("S3AStore"); + this.auditSpanSource = requireNonNull(auditSpanSource); this.clientManager = requireNonNull(clientManager); this.durationTrackerFactory = requireNonNull(durationTrackerFactory); + this.fsStatistics = fsStatistics; this.instrumentation = requireNonNull(instrumentation); this.statisticsContext = requireNonNull(statisticsContext); + this.storeContextFactory = requireNonNull(storeContextFactory); this.storageStatistics = requireNonNull(storageStatistics); this.readRateLimiter = requireNonNull(readRateLimiter); this.writeRateLimiter = requireNonNull(writeRateLimiter); - this.auditSpanSource = requireNonNull(auditSpanSource); this.storeContext = requireNonNull(storeContextFactory.createStoreContext()); - this.fsStatistics = fsStatistics; - this.invoker = storeContext.getInvoker(); - this.bucket = storeContext.getBucket(); - this.requestFactory = storeContext.getRequestFactory(); + + this.invoker = requireNonNull(storeContext.getInvoker()); + this.bucket = requireNonNull(storeContext.getBucket()); + this.requestFactory = requireNonNull(storeContext.getRequestFactory()); + addService(clientManager); + } + + /** + * Create and initialize any subsidiary services, including the input stream factory. + * @param conf configuration + */ + @Override + protected void serviceInit(final Configuration conf) throws Exception { + + // create and register the stream factory, which will + // then follow the service lifecycle + objectInputStreamFactory = factoryFromConfig(conf); + addService(objectInputStreamFactory); + + // init all child services, including the stream factory + super.serviceInit(conf); + + // pass down extra information to the stream factory. + finishStreamFactoryInit(); + } + + @Override + protected void serviceStart() throws Exception { + super.serviceStart(); + initLocalDirAllocator(); + } + + /** + * Return the store path capabilities. + * If the object stream factory is non-null, hands off the + * query to that factory if not handled here. + * @param path path to query the capability of. + * @param capability non-null, non-empty string to query the path for support. + * @return known capabilities + */ + @Override + public boolean hasPathCapability(final Path path, final String capability) { + switch (toLowerCase(capability)) { + case StreamCapabilities.IOSTATISTICS: + return true; + default: + return inputStreamHasCapability(capability); + } } + /** + * Return the capabilities of input streams created + * through the store. + * @param capability string to query the stream support for. + * @return capabilities declared supported in streams. + */ @Override - public void close() { - clientManager.close(); + public boolean inputStreamHasCapability(final String capability) { + if (objectInputStreamFactory != null) { + return objectInputStreamFactory.hasCapability(capability); + } + return false; + } + + /** + * Initialize dir allocator if not already initialized. + */ + private void initLocalDirAllocator() { + String bufferDir = getConfig().get(BUFFER_DIR) != null + ? BUFFER_DIR + : HADOOP_TMP_DIR; + directoryAllocator = new LocalDirAllocator(bufferDir); } /** Acquire write capacity for rate limiting {@inheritDoc}. */ @@ -808,4 +904,101 @@ public CompleteMultipartUploadResponse completeMultipartUpload( return getS3Client().completeMultipartUpload(request); } + /** + * Get the directory allocator. + * @return the directory allocator + */ + @Override + public LocalDirAllocator getDirectoryAllocator() { + return directoryAllocator; + } + + /** + * Demand create the directory allocator, then create a temporary file. + * This does not mark the file for deletion when a process exits. + * Pass in a file size of {@link LocalDirAllocator#SIZE_UNKNOWN} if the + * size is unknown. + * {@link LocalDirAllocator#createTmpFileForWrite(String, long, Configuration)}. + * @param pathStr prefix for the temporary file + * @param size the size of the file that is going to be written + * @param conf the Configuration object + * @return a unique temporary file + * @throws IOException IO problems + */ + @Override + public File createTemporaryFileForWriting(String pathStr, + long size, + Configuration conf) throws IOException { + requireNonNull(directoryAllocator, "directory allocator not initialized"); + Path path = directoryAllocator.getLocalPathForWrite(pathStr, + size, conf); + File dir = new File(path.getParent().toUri().getPath()); + String prefix = path.getName(); + // create a temp file on this directory + return File.createTempFile(prefix, null, dir); + } + + /* + =============== BEGIN ObjectInputStreamFactory =============== + */ + + /** + * All stream factory initialization required after {@code Service.init()}, + * after all other services have themselves been initialized. + */ + private void finishStreamFactoryInit() throws IOException { + // must be on be invoked during service initialization + Preconditions.checkState(isInState(STATE.INITED), + "Store is in wrong state: %s", getServiceState()); + Preconditions.checkState(clientManager.isInState(STATE.INITED), + "Client Manager is in wrong state: %s", clientManager.getServiceState()); + + // finish initialization and pass down callbacks to self + objectInputStreamFactory.bind(new FactoryBindingParameters(new FactoryCallbacks())); + } + + @Override /* ObjectInputStreamFactory */ + public ObjectInputStream readObject(ObjectReadParameters parameters) + throws IOException { + parameters.withDirectoryAllocator(getDirectoryAllocator()); + return objectInputStreamFactory.readObject(parameters.validate()); + } + + @Override /* ObjectInputStreamFactory */ + public StreamFactoryRequirements factoryRequirements() { + return objectInputStreamFactory.factoryRequirements(); + } + + /** + * This operation is not implemented, as + * is this class which invokes it on the actual factory. + * @param factoryBindingParameters ignored + * @throws UnsupportedOperationException always + */ + @Override /* ObjectInputStreamFactory */ + public void bind(final FactoryBindingParameters factoryBindingParameters) { + throw new UnsupportedOperationException("Not supported"); + } + + @Override /* ObjectInputStreamFactory */ + public InputStreamType streamType() { + return objectInputStreamFactory.streamType(); + } + + /** + * Callbacks from {@link ObjectInputStreamFactory} instances. + */ + private class FactoryCallbacks implements StreamFactoryCallbacks { + + @Override + public S3AsyncClient getOrCreateAsyncClient(final boolean requireCRT) throws IOException { + // Needs support of the CRT before the requireCRT can be used + LOG.debug("Stream factory requested async client"); + return clientManager().getOrCreateAsyncClient(); + } + } + + /* + =============== END ObjectInputStreamFactory =============== + */ } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AbstractObjectInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AbstractObjectInputStreamFactory.java new file mode 100644 index 0000000000000..205a4aaa858e7 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AbstractObjectInputStreamFactory.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.streams; + +import java.io.IOException; + +import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.util.Preconditions; + +import static org.apache.hadoop.util.StringUtils.toLowerCase; + +/** + * Base implementation of {@link ObjectInputStreamFactory}. + */ +public abstract class AbstractObjectInputStreamFactory extends AbstractService + implements ObjectInputStreamFactory { + + /** + * Parameters passed down in + * {@link #bind(FactoryBindingParameters)}. + */ + private FactoryBindingParameters bindingParameters; + + /** + * Callbacks. + */ + private StreamFactoryCallbacks callbacks; + + /** + * Constructor. + * @param name service name. + */ + protected AbstractObjectInputStreamFactory(final String name) { + super(name); + } + + /** + * Bind to the callbacks. + *

+ * The base class checks service state then stores + * the callback interface. + * @param factoryBindingParameters parameters for the factory binding + */ + @Override + public void bind(final FactoryBindingParameters factoryBindingParameters) throws IOException { + // must be on be invoked during service initialization + Preconditions.checkState(isInState(STATE.INITED), + "Input Stream factory %s is in wrong state: %s", + this, getServiceState()); + bindingParameters = factoryBindingParameters; + callbacks = bindingParameters.callbacks(); + } + + /** + * Return base capabilities of all stream factories, + * defining what the base ObjectInputStream class does. + * This also includes the probe for stream type capability. + * @param capability string to query the stream support for. + * @return true if implemented + */ + @Override + public boolean hasCapability(final String capability) { + switch (toLowerCase(capability)) { + case StreamCapabilities.IOSTATISTICS: + case StreamStatisticNames.STREAM_LEAKS: + return true; + default: + // dynamic probe for the name of this stream + return streamType().capability().equals(capability); + } + } + + /** + * Get the factory callbacks. + * @return callbacks. + */ + protected StreamFactoryCallbacks callbacks() { + return callbacks; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStream.java new file mode 100644 index 0000000000000..9e6f98c555373 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStream.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.streams; + +import java.io.EOFException; +import java.io.IOException; + +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import software.amazon.s3.analyticsaccelerator.S3SeekableInputStream; +import software.amazon.s3.analyticsaccelerator.util.S3URI; + +/** + * Analytics stream creates a stream using aws-analytics-accelerator-s3. This stream supports + * parquet specific optimisations such as parquet-aware prefetching. For more details, see + * https://github.com/awslabs/analytics-accelerator-s3. + */ +public class AnalyticsStream extends ObjectInputStream implements StreamCapabilities { + + private S3SeekableInputStream inputStream; + private long lastReadCurrentPos = 0; + private volatile boolean closed; + + public static final Logger LOG = LoggerFactory.getLogger(AnalyticsStream.class); + + public AnalyticsStream(final ObjectReadParameters parameters, final S3SeekableInputStreamFactory s3SeekableInputStreamFactory) throws IOException { + super(InputStreamType.Analytics, parameters); + S3ObjectAttributes s3Attributes = parameters.getObjectAttributes(); + this.inputStream = s3SeekableInputStreamFactory.createStream(S3URI.of(s3Attributes.getBucket(), s3Attributes.getKey())); + getS3AStreamStatistics().streamOpened(InputStreamType.Analytics); + } + + /** + * Indicates whether the given {@code capability} is supported by this stream. + * + * @param capability the capability to check. + * @return true if the given {@code capability} is supported by this stream, false otherwise. + */ + @Override + public boolean hasCapability(String capability) { + return false; + } + + @Override + public int read() throws IOException { + throwIfClosed(); + int bytesRead; + try { + bytesRead = inputStream.read(); + } catch (IOException ioe) { + onReadFailure(ioe); + throw ioe; + } + return bytesRead; + } + + @Override + public void seek(long pos) throws IOException { + throwIfClosed(); + if (pos < 0) { + throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK + + " " + pos); + } + inputStream.seek(pos); + } + + + @Override + public synchronized long getPos() { + if (!closed) { + lastReadCurrentPos = inputStream.getPos(); + } + return lastReadCurrentPos; + } + + + /** + * Reads the last n bytes from the stream into a byte buffer. Blocks until end of stream is + * reached. Leaves the position of the stream unaltered. + * + * @param buf buffer to read data into + * @param off start position in buffer at which data is written + * @param len the number of bytes to read; the n-th byte should be the last byte of the stream. + * @return the total number of bytes read into the buffer + * @throws IOException if an I/O error occurs + */ + public int readTail(byte[] buf, int off, int len) throws IOException { + throwIfClosed(); + int bytesRead; + try { + bytesRead = inputStream.readTail(buf, off, len); + } catch (IOException ioe) { + onReadFailure(ioe); + throw ioe; + } + return bytesRead; + } + + @Override + public int read(byte[] buf, int off, int len) throws IOException { + throwIfClosed(); + int bytesRead; + try { + bytesRead = inputStream.read(buf, off, len); + } catch (IOException ioe) { + onReadFailure(ioe); + throw ioe; + } + return bytesRead; + } + + + @Override + public boolean seekToNewSource(long l) throws IOException { + return false; + } + + @Override + public int available() throws IOException { + throwIfClosed(); + return super.available(); + } + + @Override + protected boolean isStreamOpen() { + return !isClosed(); + } + + protected boolean isClosed() { + return inputStream == null; + } + + @Override + protected void abortInFinalizer() { + try { + close(); + } catch (IOException ignored) { + + } + } + + @Override + public synchronized void close() throws IOException { + if(!closed) { + closed = true; + try { + inputStream.close(); + inputStream = null; + super.close(); + } catch (IOException ioe) { + LOG.debug("Failure closing stream {}: ", getKey()); + throw ioe; + } + } + } + + /** + * Close the stream on read failure. + * No attempt to recover from failure + * + * @param ioe exception caught. + */ + @Retries.OnceTranslated + private void onReadFailure(IOException ioe) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("Got exception while trying to read from stream {}, " + + "not trying to recover:", + getKey(), ioe); + } else { + LOG.info("Got exception while trying to read from stream {}, " + + "not trying to recover:", + getKey(), ioe); + } + this.close(); + } + + + protected void throwIfClosed() throws IOException { + if (closed) { + throw new IOException(getKey() + ": " + FSExceptionMessages.STREAM_IS_CLOSED); + } + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStreamFactory.java new file mode 100644 index 0000000000000..0ec6713d9cac7 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStreamFactory.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.streams; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.VectoredIOContext; + +import software.amazon.s3.analyticsaccelerator.S3SdkObjectClient; +import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamConfiguration; +import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamFactory; +import software.amazon.s3.analyticsaccelerator.common.ConnectorConfiguration; + +import java.io.IOException; + +import static org.apache.hadoop.fs.s3a.Constants.*; +import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.populateVectoredIOContext; + +/** + * A factory for {@link AnalyticsStream}. This class is instantiated during initialization of + * {@code S3AStore}, if fs.s3a.input.stream.type is set to Analytics. + */ +public class AnalyticsStreamFactory extends AbstractObjectInputStreamFactory { + + private S3SeekableInputStreamConfiguration seekableInputStreamConfiguration; + private S3SeekableInputStreamFactory s3SeekableInputStreamFactory; + private boolean requireCrt; + + public AnalyticsStreamFactory() { + super("AnalyticsStreamFactory"); + } + + @Override + protected void serviceInit(final Configuration conf) throws Exception { + super.serviceInit(conf); + ConnectorConfiguration configuration = new ConnectorConfiguration(conf, + ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); + this.seekableInputStreamConfiguration = + S3SeekableInputStreamConfiguration.fromConfiguration(configuration); + this.requireCrt = false; + } + + @Override + public void bind(final FactoryBindingParameters factoryBindingParameters) throws IOException { + super.bind(factoryBindingParameters); + this.s3SeekableInputStreamFactory = new S3SeekableInputStreamFactory( + new S3SdkObjectClient(callbacks().getOrCreateAsyncClient(requireCrt)), + seekableInputStreamConfiguration); + } + + @Override + public ObjectInputStream readObject(final ObjectReadParameters parameters) throws IOException { + return new AnalyticsStream( + parameters, + s3SeekableInputStreamFactory); + } + + + + @Override + public InputStreamType streamType() { + return InputStreamType.Analytics; + } + + /** + * Calculate Return StreamFactoryRequirements + * @return a positive thread count. + */ + @Override + public StreamFactoryRequirements factoryRequirements() { + // fill in the vector context + final VectoredIOContext vectorContext = populateVectoredIOContext(getConfig()); + // and then disable range merging. + // this ensures that no reads are made for data which is then discarded... + // so the prefetch and block read code doesn't ever do wasteful fetches. + vectorContext.setMinSeekForVectoredReads(0); + + return new StreamFactoryRequirements(0, + 0, vectorContext); + } + + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ClassicObjectInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ClassicObjectInputStreamFactory.java new file mode 100644 index 0000000000000..13ceb1dbc4feb --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ClassicObjectInputStreamFactory.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.streams; + +import java.io.IOException; + +import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.s3a.S3AInputStream; + +import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.populateVectoredIOContext; +import static org.apache.hadoop.util.StringUtils.toLowerCase; + +/** + * Factory of classic {@link S3AInputStream} instances. + */ +public class ClassicObjectInputStreamFactory extends AbstractObjectInputStreamFactory { + + public ClassicObjectInputStreamFactory() { + super("ClassicObjectInputStreamFactory"); + } + + @Override + public ObjectInputStream readObject(final ObjectReadParameters parameters) + throws IOException { + return new S3AInputStream(parameters); + } + + @Override + public boolean hasCapability(final String capability) { + + switch (toLowerCase(capability)) { + case StreamCapabilities.IOSTATISTICS_CONTEXT: + case StreamCapabilities.READAHEAD: + case StreamCapabilities.UNBUFFER: + case StreamCapabilities.VECTOREDIO: + return true; + default: + return super.hasCapability(capability); + } + } + + @Override + public InputStreamType streamType() { + return InputStreamType.Classic; + } + + /** + * Get the number of background threads required for this factory. + * @return the count of background threads. + */ + @Override + public StreamFactoryRequirements factoryRequirements() { + return new StreamFactoryRequirements(0, 0, + populateVectoredIOContext(getConfig())); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/FactoryBindingParameters.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/FactoryBindingParameters.java new file mode 100644 index 0000000000000..2a6bbd5c8b6b6 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/FactoryBindingParameters.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.streams; + +import static java.util.Objects.requireNonNull; + +/** + * Parameters passed down to {@link ObjectInputStreamFactory#bind}. + */ +public class FactoryBindingParameters { + + /** + * Callbacks which may be invoked by a stream factory directly. + */ + private final ObjectInputStreamFactory.StreamFactoryCallbacks callbacks; + + /** + * @param callbacks callback implementation. + */ + public FactoryBindingParameters(final ObjectInputStreamFactory.StreamFactoryCallbacks callbacks) { + this.callbacks = requireNonNull(callbacks); + } + + /** + * Callbacks which may be invoked by a stream factory directly. + */ + ObjectInputStreamFactory.StreamFactoryCallbacks callbacks() { + return callbacks; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/InputStreamType.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/InputStreamType.java new file mode 100644 index 0000000000000..a9c33a60fc61c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/InputStreamType.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.streams; + +import java.util.function.Function; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.prefetch.PrefetchingInputStreamFactory; + +import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE; +import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.loadCustomFactory; + +/** + * Enum of input stream types. + *

+ * Each enum value contains the factory function actually used to create + * the factory. + */ +public enum InputStreamType { + + /** + * The classic input stream. + */ + Classic(StreamIntegration.CLASSIC, 1, c -> + new ClassicObjectInputStreamFactory()), + + /** + * The prefetching input stream. + */ + Prefetch(StreamIntegration.PREFETCH, 2, c -> + new PrefetchingInputStreamFactory()), + /** + * The analytics input stream. + */ + Analytics(StreamIntegration.ANALYTICS, 3, c -> + new AnalyticsStreamFactory()), + + /** + * The a custom input stream. + */ + Custom(StreamIntegration.CUSTOM, 4, c -> { + return loadCustomFactory(c); + }); + + /** + * Name. + */ + private final String name; + + /** + * Stream ID. + */ + private final int streamID; + + /** + * Factory lambda-expression. + */ + private final Function factory; + + /** + * String name. + * @return the name + */ + public String getName() { + return name; + } + + /** + * Constructor. + * @param name name, used in configuration binding and capability. + * @param id ID + * @param factory factory factory function. "metafactory", really. + */ + InputStreamType(final String name, + final int id, + final Function factory) { + this.name = name; + this.streamID = id; + this.factory = factory; + } + + /** + * Get the ID of this stream. + * Isolated from the enum ID in case it ever needs to be tuned. + * @return the numeric ID of the stream. + */ + public int streamID() { + return streamID; + } + + /** + * Get the capability string for this stream type. + * @return the name of a string to probe for. + */ + public String capability() { + return INPUT_STREAM_TYPE + "." + getName(); + } + + /** + * Factory constructor. + * @return the factory function associated with this stream type. + */ + public Function factory() { + return factory; + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStream.java new file mode 100644 index 0000000000000..0a30467a4c950 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStream.java @@ -0,0 +1,377 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.streams; + +import java.io.IOException; +import java.util.concurrent.ExecutorService; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.impl.LeakReporter; +import org.apache.hadoop.fs.s3a.S3AInputPolicy; +import org.apache.hadoop.fs.s3a.S3AReadOpContext; +import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.VectoredIOContext; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsAggregator; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; + +import static java.util.Objects.requireNonNull; +import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.util.Preconditions.checkArgument; +import static org.apache.hadoop.util.StringUtils.toLowerCase; + +/** + * A stream of data from an S3 object. + *

+ * The base class includes common methods, stores + * common data and incorporates leak tracking. + */ +public abstract class ObjectInputStream extends FSInputStream + implements StreamCapabilities, IOStatisticsSource { + + private static final Logger LOG = + LoggerFactory.getLogger(ObjectInputStream.class); + + /** + * IOStatistics report. + */ + private final IOStatistics ioStatistics; + + /** + * Read-specific operation context structure. + */ + private final S3AReadOpContext context; + + /** + * Callbacks for reading input stream data from the S3 Store. + */ + private final ObjectInputStreamCallbacks callbacks; + + /** + * Thread pool used for vectored IO operation. + */ + private final ExecutorService boundedThreadPool; + + /** + * URI of path. + */ + private final String uri; + + /** + * Store bucket. + */ + private final String bucket; + + /** + * Store key. + */ + private final String key; + + /** + * Path URI as a string. + */ + private final String pathStr; + + /** + * Content length from HEAD or openFile option. + */ + private final long contentLength; + + /** + * Attributes of the remote object. + */ + private final S3ObjectAttributes objectAttributes; + + /** + * Stream statistics. + */ + private final S3AInputStreamStatistics streamStatistics; + + /** Aggregator used to aggregate per thread IOStatistics. */ + private final IOStatisticsAggregator threadIOStatistics; + + /** + * Report of leaks. + * with report and abort unclosed streams in finalize(). + */ + private final LeakReporter leakReporter; + + /** + * Stream type. + */ + private final InputStreamType streamType; + + /** + * Requested input policy. + */ + private S3AInputPolicy inputPolicy; + + + /** Vectored IO context. */ + private final VectoredIOContext vectoredIOContext; + + /** + * Constructor. + * @param streamType stream type enum. + * @param parameters extensible parameter list. + */ + protected ObjectInputStream( + final InputStreamType streamType, + final ObjectReadParameters parameters) { + + this.streamType = requireNonNull(streamType); + this.objectAttributes = parameters.getObjectAttributes(); + checkArgument(isNotEmpty(objectAttributes.getBucket()), + "No Bucket"); + checkArgument(isNotEmpty(objectAttributes.getKey()), "No Key"); + long l = objectAttributes.getLen(); + checkArgument(l >= 0, "Negative content length"); + this.context = parameters.getContext(); + this.contentLength = l; + + this.bucket = objectAttributes.getBucket(); + this.key = objectAttributes.getKey(); + this.pathStr = objectAttributes.getPath().toString(); + this.callbacks = parameters.getCallbacks(); + this.uri = "s3a://" + bucket + "/" + key; + this.streamStatistics = parameters.getStreamStatistics(); + this.ioStatistics = streamStatistics.getIOStatistics(); + this.inputPolicy = context.getInputPolicy(); + streamStatistics.inputPolicySet(inputPolicy.ordinal()); + this.boundedThreadPool = parameters.getBoundedThreadPool(); + this.threadIOStatistics = requireNonNull(context.getIOStatisticsAggregator()); + // build the leak reporter + this.leakReporter = new LeakReporter( + "Stream not closed while reading " + uri, + this::isStreamOpen, + this::abortInFinalizer); + this.vectoredIOContext = getContext().getVectoredIOContext(); + } + + /** + * Probe for stream being open. + * Not synchronized; the flag is volatile. + * @return true if the stream is still open. + */ + protected abstract boolean isStreamOpen(); + + /** + * Brute force stream close; invoked by {@link LeakReporter}. + * All exceptions raised are ignored. + */ + protected abstract void abortInFinalizer(); + + /** + * Close the stream. + * This triggers publishing of the stream statistics back to the filesystem + * statistics. + * This operation is synchronized, so that only one thread can attempt to + * @throws IOException on any problem + */ + @Override + public synchronized void close() throws IOException { + // end the client+audit span. + callbacks.close(); + // merge the statistics back into the FS statistics. + streamStatistics.close(); + // Collect ThreadLevel IOStats + mergeThreadIOStatistics(streamStatistics.getIOStatistics()); + } + + /** + * Merging the current thread's IOStatistics with the current IOStatistics + * context. + * @param streamIOStats Stream statistics to be merged into thread + * statistics aggregator. + */ + protected void mergeThreadIOStatistics(IOStatistics streamIOStats) { + threadIOStatistics.aggregate(streamIOStats); + } + + /** + * Finalizer. + *

+ * Verify that the inner stream is closed. + *

+ * If it is not, it means streams are being leaked in application code. + * Log a warning, including the stack trace of the caller, + * then abort the stream. + *

+ * This does not attempt to invoke {@link #close()} as that is + * a more complex operation, and this method is being executed + * during a GC finalization phase. + *

+ * Applications MUST close their streams; this is a defensive + * operation to return http connections and warn the end users + * that their applications are at risk of running out of connections. + * + * {@inheritDoc} + */ + @Override + protected void finalize() throws Throwable { + leakReporter.close(); + super.finalize(); + } + + /** + * Get the current input policy. + * @return input policy. + */ + @VisibleForTesting + public S3AInputPolicy getInputPolicy() { + return inputPolicy; + } + + /** + * Set/update the input policy of the stream. + * This updates the stream statistics. + * @param inputPolicy new input policy. + */ + protected void setInputPolicy(S3AInputPolicy inputPolicy) { + LOG.debug("Switching to input policy {}", inputPolicy); + this.inputPolicy = inputPolicy; + streamStatistics.inputPolicySet(inputPolicy.ordinal()); + } + + /** + * Access the input stream statistics. + * This is for internal testing and may be removed without warning. + * @return the statistics for this input stream + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + @VisibleForTesting + public S3AInputStreamStatistics getS3AStreamStatistics() { + return streamStatistics; + } + + @Override + public IOStatistics getIOStatistics() { + return ioStatistics; + } + + /** + * Declare the base capabilities implemented by this class and so by + * all subclasses. + *

+ * Subclasses MUST override this if they add more capabilities, + * or actually remove any of these. + * @param capability string to query the stream support for. + * @return true if all implementations are known to have the specific + * capability. + */ + @Override + public boolean hasCapability(String capability) { + switch (toLowerCase(capability)) { + case StreamCapabilities.IOSTATISTICS: + case StreamStatisticNames.STREAM_LEAKS: + return true; + default: + // dynamic probe for the name of this stream + if (streamType.capability().equals(capability)) { + return true; + } + return false; + } + } + + protected final S3AReadOpContext getContext() { + return context; + } + + protected final ObjectInputStreamCallbacks getCallbacks() { + return callbacks; + } + + protected final ExecutorService getBoundedThreadPool() { + return boundedThreadPool; + } + + protected final String getUri() { + return uri; + } + + protected final String getBucket() { + return bucket; + } + + protected final String getKey() { + return key; + } + + protected final String getPathStr() { + return pathStr; + } + + protected final long getContentLength() { + return contentLength; + } + + protected final IOStatisticsAggregator getThreadIOStatistics() { + return threadIOStatistics; + } + + protected final S3ObjectAttributes getObjectAttributes() { + return objectAttributes; + } + + protected VectoredIOContext getVectoredIOContext() { + return vectoredIOContext; + } + + /** + * {@inheritDoc}. + */ + @Override + public int minSeekForVectorReads() { + return vectoredIOContext.getMinSeekForVectorReads(); + } + + /** + * {@inheritDoc}. + */ + @Override + public int maxReadSizeForVectorReads() { + return vectoredIOContext.getMaxReadSizeForVectorReads(); + } + + public InputStreamType streamType() { + return streamType; + } + + @Override + public String toString() { + return "ObjectInputStream{" + + "streamType=" + streamType + + ", uri='" + uri + '\'' + + ", contentLength=" + contentLength + + ", inputPolicy=" + inputPolicy + + ", vectoredIOContext=" + vectoredIOContext + + "} " + super.toString(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamCallbacks.java new file mode 100644 index 0000000000000..f4ed384ed977a --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamCallbacks.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.streams; + +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.CompletableFuture; + +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; + +import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.util.functional.CallableRaisingIOE; + +/** + * Callbacks for reading object data from the S3 Store. + */ +public interface ObjectInputStreamCallbacks extends Closeable { + + /** + * Create a GET request builder. + * @param key object key + * @return the request builder + */ + GetObjectRequest.Builder newGetRequestBuilder(String key); + + /** + * Execute the request. + * When CSE is enabled with reading of unencrypted data, The object is checked if it is + * encrypted and if so, the request is made with encrypted S3 client. If the object is + * not encrypted, the request is made with unencrypted s3 client. + * @param request the request + * @return the response + * @throws IOException on any failure. + */ + @Retries.OnceRaw + ResponseInputStream getObject(GetObjectRequest request) throws IOException; + + /** + * Submit some asynchronous work, for example, draining a stream. + * @param operation operation to invoke + * @param return type + * @return a future. + */ + CompletableFuture submit(CallableRaisingIOE operation); + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamFactory.java new file mode 100644 index 0000000000000..ce168bea88308 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamFactory.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.streams; + +import java.io.IOException; + +import software.amazon.awssdk.services.s3.S3AsyncClient; + +import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.service.Service; + +/** + * A Factory for {@link ObjectInputStream} streams. + *

+ * This class is instantiated during initialization of + * {@code S3AStore}, it then follows the same service + * lifecycle. + *

+ * Note for maintainers: do try and keep this mostly stable. + * If new parameters need to be added, expand the + * {@link ObjectReadParameters} class, rather than change the + * interface signature. + */ +public interface ObjectInputStreamFactory + extends Service, StreamCapabilities { + + /** + * Set extra initialization parameters. + * This MUST ONLY be invoked between {@code init()} + * and {@code start()}. + * @param factoryBindingParameters parameters for the factory binding + * @throws IOException if IO problems. + */ + void bind(FactoryBindingParameters factoryBindingParameters) throws IOException; + + /** + * Create a new input stream. + * There is no requirement to actually contact the store; this is generally done + * lazily. + * @param parameters parameters. + * @return the input stream + * @throws IOException problem creating the stream. + */ + ObjectInputStream readObject(ObjectReadParameters parameters) + throws IOException; + + /** + * Get requirements from the factory which then tune behavior + * elsewhere in the system. + * @return the count of background threads. + */ + StreamFactoryRequirements factoryRequirements(); + + /** + * Get the input stream type. + * @return the specific stream type this factory produces. + */ + InputStreamType streamType(); + + /** + * Callbacks for stream factories. + */ + interface StreamFactoryCallbacks { + + /** + * Get the Async S3Client, raising a failure to create as an IOException. + * @param requireCRT is the CRT required. + * @return the Async S3 client + * @throws IOException failure to create the client. + */ + S3AsyncClient getOrCreateAsyncClient(boolean requireCRT) throws IOException; + } +} + diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectReadParameters.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectReadParameters.java new file mode 100644 index 0000000000000..32a2487d3f7e7 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectReadParameters.java @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.streams; + +import java.util.concurrent.ExecutorService; + +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.s3a.S3AReadOpContext; +import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; + +import static java.util.Objects.requireNonNull; + +/** + * Parameters for object input streams created through + * {@link ObjectInputStreamFactory}. + * It is designed to support extra parameters added + * in future. + *

Note that the {@link #validate()} + * operation does not freeze the parameters -instead it simply + * verifies that all required values are set. + */ +public final class ObjectReadParameters { + + /** + * Read operation context. + */ + private S3AReadOpContext context; + + /** + * Attributes of the object. + */ + private S3ObjectAttributes objectAttributes; + + /** + * Callbacks to the store. + */ + private ObjectInputStreamCallbacks callbacks; + + /** + * Stream statistics. + */ + private S3AInputStreamStatistics streamStatistics; + + /** + * Bounded thread pool for submitting asynchronous + * work. + */ + private ExecutorService boundedThreadPool; + + /** + * Allocator of local FS storage. + */ + private LocalDirAllocator directoryAllocator; + + /** + * @return Read operation context. + */ + public S3AReadOpContext getContext() { + return context; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public ObjectReadParameters withContext(S3AReadOpContext value) { + context = value; + return this; + } + + /** + * @return Attributes of the object. + */ + public S3ObjectAttributes getObjectAttributes() { + return objectAttributes; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public ObjectReadParameters withObjectAttributes(S3ObjectAttributes value) { + objectAttributes = value; + return this; + } + + /** + * @return callbacks to the store. + */ + public ObjectInputStreamCallbacks getCallbacks() { + return callbacks; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public ObjectReadParameters withCallbacks(ObjectInputStreamCallbacks value) { + callbacks = value; + return this; + } + + /** + * @return Stream statistics. + */ + public S3AInputStreamStatistics getStreamStatistics() { + return streamStatistics; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public ObjectReadParameters withStreamStatistics(S3AInputStreamStatistics value) { + streamStatistics = value; + return this; + } + + /** + * @return Bounded thread pool for submitting asynchronous work. + */ + public ExecutorService getBoundedThreadPool() { + return boundedThreadPool; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public ObjectReadParameters withBoundedThreadPool(ExecutorService value) { + boundedThreadPool = value; + return this; + } + + public LocalDirAllocator getDirectoryAllocator() { + return directoryAllocator; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public ObjectReadParameters withDirectoryAllocator(final LocalDirAllocator value) { + directoryAllocator = value; + return this; + } + + /** + * Validate that all attributes are as expected. + * Mock tests can skip this if required. + * @return the object. + */ + public ObjectReadParameters validate() { + // please keep in alphabetical order. + requireNonNull(boundedThreadPool, "boundedThreadPool"); + requireNonNull(callbacks, "callbacks"); + requireNonNull(context, "context"); + requireNonNull(directoryAllocator, "directoryAllocator"); + requireNonNull(objectAttributes, "objectAttributes"); + requireNonNull(streamStatistics, "streamStatistics"); + return this; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamFactoryRequirements.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamFactoryRequirements.java new file mode 100644 index 0000000000000..ecf34c35e0ed8 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamFactoryRequirements.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.streams; + +import java.util.Arrays; +import java.util.EnumSet; + +import org.apache.hadoop.fs.s3a.VectoredIOContext; + +/** + * Requirements for requirements for streams from this factory, + * including threading and vector IO. + */ +public class StreamFactoryRequirements { + + /** + * Number of shared threads to included in the bounded pool. + */ + private final int sharedThreads; + + /** + * How many threads per stream, ignoring vector IO requirements? + */ + private final int streamThreads; + + /** + * VectoredIO behaviour. + */ + private final VectoredIOContext vectoredIOContext; + + /** + * Set of requirement flags. + */ + private final EnumSet requirementFlags; + + /** + * Create the thread options. + * @param sharedThreads Number of shared threads to included in the bounded pool. + * @param streamThreads How many threads per stream, ignoring vector IO requirements. + * @param vectoredIOContext vector IO settings -made immutable if not already done. + * @param requirements requirement flags of the factory and stream. + */ + public StreamFactoryRequirements( + final int sharedThreads, + final int streamThreads, + final VectoredIOContext vectoredIOContext, + final Requirements...requirements) { + this.sharedThreads = sharedThreads; + this.streamThreads = streamThreads; + this.vectoredIOContext = vectoredIOContext.build(); + if (requirements.length == 0) { + this.requirementFlags = EnumSet.noneOf(Requirements.class); + } else { + this.requirementFlags = EnumSet.copyOf((Arrays.asList(requirements))); + } + } + + /** + * Number of shared threads to included in the bounded pool. + */ + public int sharedThreads() { + return sharedThreads; + } + + /** + * How many threads per stream, ignoring vector IO requirements? + */ + public int streamThreads() { + return streamThreads; + } + + /** + * Should the future pool be created? + * @return true if the future pool is required. + */ + public boolean requiresFuturePool() { + return requires(Requirements.RequiresFuturePool); + } + + /** + * The VectorIO requirements of streams. + * @return vector IO context. + */ + public VectoredIOContext vectoredIOContext() { + return vectoredIOContext; + } + + /** + * Does this factory have this requirement? + * @param r requirement to probe for. + * @return true if this is a requirement. + */ + public boolean requires(Requirements r) { + return requirementFlags.contains(r); + } + + @Override + public String toString() { + return "StreamFactoryRequirements{" + + "sharedThreads=" + sharedThreads + + ", streamThreads=" + streamThreads + + ", requirementFlags=" + requirementFlags + + ", vectoredIOContext=" + vectoredIOContext + + '}'; + } + + /** + * An enum of options. + */ + public enum Requirements { + + /** + * Expect Unaudited GETs. + * Disables auditor warning/errors about GET requests being + * issued outside an audit span. + */ + ExpectUnauditedGetRequests, + + /** + * Requires a future pool bound to the thread pool. + */ + RequiresFuturePool + + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamIntegration.java new file mode 100644 index 0000000000000..bb35a0580a20e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamIntegration.java @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.streams; + +import java.lang.reflect.Constructor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.ConfigurationHelper; +import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.fs.s3a.VectoredIOContext; +import org.apache.hadoop.fs.store.LogExactlyOnce; + +import static org.apache.commons.lang3.StringUtils.isEmpty; +import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_VECTOR_ACTIVE_RANGE_READS; +import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE; +import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_AWS_S3_VECTOR_ACTIVE_RANGE_READS; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_CUSTOM_FACTORY; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE; +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; +import static org.apache.hadoop.fs.s3a.S3AUtils.intOption; +import static org.apache.hadoop.fs.s3a.S3AUtils.longBytesOption; +import static org.apache.hadoop.util.Preconditions.checkArgument; + +/** + * Stream integration, including factory construction. + */ +public final class StreamIntegration { + + /** + * Enum/config name of a classic S3AInputStream: {@value}. + */ + public static final String CLASSIC = "classic"; + + /** + * Enum/config name of of the Pinterest S3APrefetchingInputStream: {@value}. + */ + public static final String PREFETCH = "prefetch"; + + /** + * Enum/config name of the analytics input stream: {@value}. + */ + public static final String ANALYTICS = "analytics"; + + /** + * Reads in a classname : {@value}. + */ + public static final String CUSTOM = "custom"; + + /** + * Special string for configuration only; is + * mapped to the default stream type: {@value}. + */ + public static final String DEFAULT = "default"; + + /** + * What is the default type? + */ + public static final InputStreamType DEFAULT_STREAM_TYPE = InputStreamType.Classic; + + /** + * Configuration deprecation log for warning about use of the + * now deprecated {@code "fs.s3a.prefetch.enabled"} option.. + */ + private static final Logger LOG_DEPRECATION = + LoggerFactory.getLogger( + "org.apache.hadoop.conf.Configuration.deprecation"); + + /** + * Warn once on use of prefetch configuration option. + */ + private static final LogExactlyOnce WARN_PREFETCH_KEY = new LogExactlyOnce(LOG_DEPRECATION); + + public static final String E_EMPTY_CUSTOM_CLASSNAME = + "Configuration option " + INPUT_STREAM_CUSTOM_FACTORY + + " is required when the input stream type is \"custom\""; + + public static final String E_INVALID_STREAM_TYPE = "Invalid stream type:"; + + private StreamIntegration() { + } + + /** + * Create the input stream factory the configuration asks for. + *

+ * This does not initialize the factory. + *

+ * See {@link #determineInputStreamType(Configuration)} for the + * resolution algorithm. + * @param conf configuration + * @return a stream factory. + * @throws RuntimeException any binding/loading/instantiation problem + */ + public static ObjectInputStreamFactory factoryFromConfig(final Configuration conf) { + // Construct the factory. + return determineInputStreamType(conf) + .factory() + .apply(conf); + } + + /** + * Determine the input stream type for the supplied configuration. + *

+ * This does not perform any instantiation. + *

+ * If the option {@code "fs.s3a.prefetch.enabled"} is set, the + * prefetch stream is selected, after printing a + * warning the first time this happens. + *

+ * If the input stream type is declared as "default", then whatever + * the current default stream type is returned, as defined by + * {@link #DEFAULT_STREAM_TYPE}. + * @param conf configuration + * @return a stream factory. + */ + public static InputStreamType determineInputStreamType(final Configuration conf) { + // work out the default stream; this includes looking at the + // deprecated prefetch enabled key to see if it is set. + if (conf.getBoolean(PREFETCH_ENABLED_KEY, false)) { + // prefetch enabled, warn (once) then change it to be the default. + WARN_PREFETCH_KEY.info("Using {} is deprecated: choose the appropriate stream in {}", + PREFETCH_ENABLED_KEY, INPUT_STREAM_TYPE); + return InputStreamType.Prefetch; + } + + // retrieve the enum value, returning the configured value or + // the (calculated) default + return ConfigurationHelper.resolveEnum(conf, + INPUT_STREAM_TYPE, + InputStreamType.class, + s -> { + if (isEmpty(s) || DEFAULT.equalsIgnoreCase(s)) { + // return default type. + return DEFAULT_STREAM_TYPE; + } else { + // any other value + throw new IllegalArgumentException(E_INVALID_STREAM_TYPE + + " \"" + s + "\""); + } + }); + } + + /** + * Load the input stream factory defined in the option + * {@link Constants#INPUT_STREAM_CUSTOM_FACTORY}. + * @param conf configuration to use + * @return the custom factory + * @throws RuntimeException any binding/loading/instantiation problem + */ + static ObjectInputStreamFactory loadCustomFactory(Configuration conf) { + + // make sure the classname option is actually set + final String name = conf.getTrimmed(INPUT_STREAM_CUSTOM_FACTORY, ""); + checkArgument(!isEmpty(name), E_EMPTY_CUSTOM_CLASSNAME); + + final Class factoryClass = + conf.getClass(INPUT_STREAM_CUSTOM_FACTORY, + null, + ObjectInputStreamFactory.class); + + try { + final Constructor ctor = + factoryClass.getConstructor(); + return ctor.newInstance(); + } catch (Exception e) { + throw new RuntimeException("Failed to instantiate custom class " + + name + " " + e, e); + } + } + + /** + * Populates the configurations related to vectored IO operations. + * The context is still mutable at this point. + * @param conf configuration object. + * @return VectoredIOContext. + */ + public static VectoredIOContext populateVectoredIOContext(Configuration conf) { + final int minSeekVectored = (int) longBytesOption(conf, AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, + DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, 0); + final int maxReadSizeVectored = + (int) longBytesOption(conf, AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, + DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, 0); + final int vectoredActiveRangeReads = intOption(conf, + AWS_S3_VECTOR_ACTIVE_RANGE_READS, DEFAULT_AWS_S3_VECTOR_ACTIVE_RANGE_READS, 1); + return new VectoredIOContext() + .setMinSeekForVectoredReads(minSeekVectored) + .setMaxReadSizeForVectoredReads(maxReadSizeVectored) + .setVectoredActiveRangeReads(vectoredActiveRangeReads); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/package-info.java new file mode 100644 index 0000000000000..514e8dbd70376 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/package-info.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Input and Output stream support. + *

+ * A lot of the existing stream work is elsewhere, + * this module is where ongoing work should take place. + */ + +@InterfaceAudience.Private +package org.apache.hadoop.fs.s3a.impl.streams; + +import org.apache.hadoop.classification.InterfaceAudience; \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchOptions.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchOptions.java new file mode 100644 index 0000000000000..ee584418d61ac --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchOptions.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.prefetch; + +import static org.apache.hadoop.util.Preconditions.checkArgument; + +/** + * Options for the prefetch stream which are built up in {@link PrefetchingInputStreamFactory} + * and passed down. + */ +public class PrefetchOptions { + + /** Size in bytes of a single prefetch block. */ + private final int prefetchBlockSize; + + /** Size of prefetch queue (in number of blocks). */ + private final int prefetchBlockCount; + + /** + * Constructor. + * @param prefetchBlockSize the size (in number of bytes) of each prefetched block. + * @param prefetchBlockCount maximum number of prefetched blocks. + */ + public PrefetchOptions(final int prefetchBlockSize, final int prefetchBlockCount) { + + checkArgument( + prefetchBlockSize > 0, "invalid prefetchBlockSize %d", prefetchBlockSize); + this.prefetchBlockSize = prefetchBlockSize; + checkArgument( + prefetchBlockCount > 0, "invalid prefetchBlockCount %d", prefetchBlockCount); + this.prefetchBlockCount = prefetchBlockCount; + } + + /** + * Gets the size in bytes of a single prefetch block. + * + * @return the size in bytes of a single prefetch block. + */ + public int getPrefetchBlockSize() { + return prefetchBlockSize; + } + + /** + * Gets the size of prefetch queue (in number of blocks). + * + * @return the size of prefetch queue (in number of blocks). + */ + public int getPrefetchBlockCount() { + return prefetchBlockCount; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchingInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchingInputStreamFactory.java new file mode 100644 index 0000000000000..bb5f8754da694 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchingInputStreamFactory.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.prefetch; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.VectoredIOContext; +import org.apache.hadoop.fs.s3a.impl.streams.AbstractObjectInputStreamFactory; +import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStream; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectReadParameters; +import org.apache.hadoop.fs.s3a.impl.streams.StreamFactoryRequirements; + +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_COUNT_KEY; +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_DEFAULT_COUNT; +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.S3AUtils.intOption; +import static org.apache.hadoop.fs.s3a.S3AUtils.longBytesOption; +import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.populateVectoredIOContext; +import static org.apache.hadoop.util.Preconditions.checkState; + +/** + * Factory for prefetching streams. + *

+ * Reads and validates prefetch configuration options during service init. + */ +public class PrefetchingInputStreamFactory extends AbstractObjectInputStreamFactory { + + /** Size in bytes of a single prefetch block. */ + private int prefetchBlockSize; + + /** Size of prefetch queue (in number of blocks). */ + private int prefetchBlockCount; + + /** + * Shared prefetch options. + */ + private PrefetchOptions prefetchOptions; + + public PrefetchingInputStreamFactory() { + super("PrefetchingInputStreamFactory"); + } + + @Override + public InputStreamType streamType() { + return InputStreamType.Prefetch; + } + + @Override + protected void serviceInit(final Configuration conf) throws Exception { + super.serviceInit(conf); + long prefetchBlockSizeLong = + longBytesOption(conf, PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE, 1); + checkState(prefetchBlockSizeLong < Integer.MAX_VALUE, + "S3A prefetch block size exceeds int limit"); + prefetchBlockSize = (int) prefetchBlockSizeLong; + prefetchBlockCount = + intOption(conf, PREFETCH_BLOCK_COUNT_KEY, PREFETCH_BLOCK_DEFAULT_COUNT, 1); + + prefetchOptions = new PrefetchOptions( + prefetchBlockSize, + prefetchBlockCount); + } + + @Override + public ObjectInputStream readObject(final ObjectReadParameters parameters) throws IOException { + return new S3APrefetchingInputStream(parameters, + getConfig(), + prefetchOptions); + } + + /** + * Calculate Return StreamFactoryRequirements. + * @return thread count a vector minimum seek of 0. + */ + @Override + public StreamFactoryRequirements factoryRequirements() { + // fill in the vector context + // and then disable range merging. + // this ensures that no reads are made for data which is then discarded... + // so the prefetch and block read code doesn't ever do wasteful fetches. + final VectoredIOContext vectorContext = populateVectoredIOContext(getConfig()) + .setMinSeekForVectoredReads(0); + + return new StreamFactoryRequirements(prefetchBlockCount, + 0, + vectorContext, + StreamFactoryRequirements.Requirements.RequiresFuturePool); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java index e05ad7e38b5b8..a183b9d1003b1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java @@ -32,10 +32,10 @@ import org.apache.hadoop.fs.impl.prefetch.BlockManagerParameters; import org.apache.hadoop.fs.impl.prefetch.BufferData; import org.apache.hadoop.fs.impl.prefetch.FilePosition; -import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.S3AReadOpContext; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_PREFETCH_MAX_BLOCKS_COUNT; import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_MAX_BLOCKS_COUNT; @@ -63,25 +63,25 @@ public class S3ACachingInputStream extends S3ARemoteInputStream { * Initializes a new instance of the {@code S3ACachingInputStream} class. * * @param context read-specific operation context. - * @param s3Attributes attributes of the S3 object being read. + * @param prefetchOptions prefetch stream specific options + * @param s3Attributes attributes of the S3a object being read. * @param client callbacks used for interacting with the underlying S3 client. * @param streamStatistics statistics for this stream. * @param conf the configuration. * @param localDirAllocator the local dir allocator instance. - * @throws IllegalArgumentException if context is null. - * @throws IllegalArgumentException if s3Attributes is null. - * @throws IllegalArgumentException if client is null. + * @throws NullPointerException if a required parameter is null. */ public S3ACachingInputStream( S3AReadOpContext context, + PrefetchOptions prefetchOptions, S3ObjectAttributes s3Attributes, - S3AInputStream.InputStreamCallbacks client, + ObjectInputStreamCallbacks client, S3AInputStreamStatistics streamStatistics, Configuration conf, LocalDirAllocator localDirAllocator) { - super(context, s3Attributes, client, streamStatistics); + super(context, prefetchOptions, s3Attributes, client, streamStatistics); - this.numBlocksToPrefetch = this.getContext().getPrefetchBlockCount(); + this.numBlocksToPrefetch = prefetchOptions.getPrefetchBlockCount(); int bufferPoolSize = this.numBlocksToPrefetch + 1; BlockManagerParameters blockManagerParamsBuilder = new BlockManagerParameters() diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java index e8bfe946f4abf..73d0b8c102d47 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java @@ -27,10 +27,10 @@ import org.apache.hadoop.fs.impl.prefetch.BufferData; import org.apache.hadoop.fs.impl.prefetch.FilePosition; -import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.S3AReadOpContext; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; /** * Provides an {@code InputStream} that allows reading from an S3 file. @@ -50,6 +50,7 @@ public class S3AInMemoryInputStream extends S3ARemoteInputStream { * Initializes a new instance of the {@code S3AInMemoryInputStream} class. * * @param context read-specific operation context. + * @param prefetchOptions prefetching options. * @param s3Attributes attributes of the S3 object being read. * @param client callbacks used for interacting with the underlying S3 client. * @param streamStatistics statistics for this stream. @@ -60,10 +61,11 @@ public class S3AInMemoryInputStream extends S3ARemoteInputStream { */ public S3AInMemoryInputStream( S3AReadOpContext context, + PrefetchOptions prefetchOptions, S3ObjectAttributes s3Attributes, - S3AInputStream.InputStreamCallbacks client, + ObjectInputStreamCallbacks client, S3AInputStreamStatistics streamStatistics) { - super(context, s3Attributes, client, streamStatistics); + super(context, prefetchOptions, s3Attributes, client, streamStatistics); int fileSize = (int) s3Attributes.getLen(); this.buffer = ByteBuffer.allocate(fileSize); LOG.debug("Created in-memory input stream for {} (size = {})", diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java index 9b9ee12ad7502..36a07bf1684a7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java @@ -30,17 +30,21 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CanSetReadahead; import org.apache.hadoop.fs.FSExceptionMessages; -import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.impl.prefetch.Validate; -import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.S3AReadOpContext; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStream; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectReadParameters; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import static org.apache.hadoop.util.StringUtils.toLowerCase; + /** * Enhanced {@code InputStream} for reading from S3. * @@ -48,7 +52,7 @@ * blocks of configurable size from the underlying S3 file. */ public class S3APrefetchingInputStream - extends FSInputStream + extends ObjectInputStream implements CanSetReadahead, StreamCapabilities, IOStatisticsSource { private static final Logger LOG = LoggerFactory.getLogger( @@ -74,26 +78,24 @@ public class S3APrefetchingInputStream */ private S3AInputStreamStatistics inputStreamStatistics = null; + /** * Initializes a new instance of the {@code S3APrefetchingInputStream} class. - * - * @param context read-specific operation context. - * @param s3Attributes attributes of the S3 object being read. - * @param client callbacks used for interacting with the underlying S3 client. - * @param streamStatistics statistics for this stream. + * @param parameters creation parameters. * @param conf the configuration. - * @param localDirAllocator the local dir allocator instance retrieved from S3A FS. - * @throws IllegalArgumentException if context is null. - * @throws IllegalArgumentException if s3Attributes is null. - * @throws IllegalArgumentException if client is null. + * @param prefetchOptions prefetch stream specific options + * @throws NullPointerException if a required parameter is null. */ public S3APrefetchingInputStream( - S3AReadOpContext context, - S3ObjectAttributes s3Attributes, - S3AInputStream.InputStreamCallbacks client, - S3AInputStreamStatistics streamStatistics, - Configuration conf, - LocalDirAllocator localDirAllocator) { + final ObjectReadParameters parameters, + final Configuration conf, + final PrefetchOptions prefetchOptions) { + super(InputStreamType.Prefetch, parameters); + S3ObjectAttributes s3Attributes = parameters.getObjectAttributes(); + ObjectInputStreamCallbacks client = parameters.getCallbacks(); + S3AInputStreamStatistics streamStatistics = parameters.getStreamStatistics(); + final S3AReadOpContext context = parameters.getContext(); + LocalDirAllocator localDirAllocator = parameters.getDirectoryAllocator(); Validate.checkNotNull(context, "context"); Validate.checkNotNull(s3Attributes, "s3Attributes"); @@ -106,10 +108,11 @@ public S3APrefetchingInputStream( Validate.checkNotNull(streamStatistics, "streamStatistics"); long fileSize = s3Attributes.getLen(); - if (fileSize <= context.getPrefetchBlockSize()) { + if (fileSize <= prefetchOptions.getPrefetchBlockSize()) { LOG.debug("Creating in memory input stream for {}", context.getPath()); this.inputStream = new S3AInMemoryInputStream( context, + prefetchOptions, s3Attributes, client, streamStatistics); @@ -117,6 +120,7 @@ public S3APrefetchingInputStream( LOG.debug("Creating in caching input stream for {}", context.getPath()); this.inputStream = new S3ACachingInputStream( context, + prefetchOptions, s3Attributes, client, streamStatistics, @@ -198,6 +202,22 @@ public synchronized void close() throws IOException { } } + + @Override + protected boolean isStreamOpen() { + return !isClosed(); + } + + @Override + protected void abortInFinalizer() { + getS3AStreamStatistics().streamLeaked(); + try { + close(); + } catch (IOException ignored) { + + } + } + /** * Updates internal data such that the next read will take place at the given {@code pos}. * @@ -230,11 +250,12 @@ public synchronized void setReadahead(Long readahead) { */ @Override public boolean hasCapability(String capability) { - if (!isClosed()) { - return inputStream.hasCapability(capability); + switch (toLowerCase(capability)) { + case StreamCapabilities.READAHEAD: + return true; + default: + return super.hasCapability(capability); } - - return false; } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java index 38d740bd74f94..5ce0b7de074fd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java @@ -36,11 +36,11 @@ import org.apache.hadoop.fs.impl.prefetch.FilePosition; import org.apache.hadoop.fs.impl.prefetch.Validate; import org.apache.hadoop.fs.s3a.S3AInputPolicy; -import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.S3AReadOpContext; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; import org.apache.hadoop.fs.s3a.impl.ChangeTracker; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; @@ -98,7 +98,7 @@ public abstract class S3ARemoteInputStream private S3ObjectAttributes s3Attributes; /** Callbacks used for interacting with the underlying S3 client. */ - private S3AInputStream.InputStreamCallbacks client; + private ObjectInputStreamCallbacks client; /** Used for reporting input stream access statistics. */ private final S3AInputStreamStatistics streamStatistics; @@ -113,18 +113,18 @@ public abstract class S3ARemoteInputStream * Initializes a new instance of the {@code S3ARemoteInputStream} class. * * @param context read-specific operation context. + * @param prefetchOptions prefetch stream specific options * @param s3Attributes attributes of the S3 object being read. * @param client callbacks used for interacting with the underlying S3 client. * @param streamStatistics statistics for this stream. * - * @throws IllegalArgumentException if context is null. - * @throws IllegalArgumentException if s3Attributes is null. - * @throws IllegalArgumentException if client is null. + * @throws NullPointerException if a required parameter is null. */ public S3ARemoteInputStream( S3AReadOpContext context, + PrefetchOptions prefetchOptions, S3ObjectAttributes s3Attributes, - S3AInputStream.InputStreamCallbacks client, + ObjectInputStreamCallbacks client, S3AInputStreamStatistics streamStatistics) { this.context = requireNonNull(context); @@ -143,7 +143,7 @@ public S3ARemoteInputStream( setReadahead(context.getReadahead()); long fileSize = s3Attributes.getLen(); - int bufferSize = context.getPrefetchBlockSize(); + int bufferSize = prefetchOptions.getPrefetchBlockSize(); this.blockData = new BlockData(fileSize, bufferSize); this.fpos = new FilePosition(fileSize, bufferSize); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java index ec6e3700226e0..8a624c40c9afd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java @@ -36,6 +36,7 @@ import org.apache.hadoop.fs.s3a.impl.ChangeTracker; import org.apache.hadoop.fs.s3a.impl.SDKStreamDrainer; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; import org.apache.hadoop.fs.statistics.DurationTracker; @@ -60,7 +61,7 @@ public class S3ARemoteObject { /** * Callbacks used for interacting with the underlying S3 client. */ - private final S3AInputStream.InputStreamCallbacks client; + private final ObjectInputStreamCallbacks client; /** * Used for reporting input stream access statistics. @@ -100,7 +101,7 @@ public class S3ARemoteObject { public S3ARemoteObject( S3AReadOpContext context, S3ObjectAttributes s3Attributes, - S3AInputStream.InputStreamCallbacks client, + ObjectInputStreamCallbacks client, S3AInputStreamStatistics streamStatistics, ChangeTracker changeTracker) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java index 4e691ff10009f..fabe61aac7bdd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java @@ -72,6 +72,7 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterConstants.FILESYSTEM_TEMP_PATH; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.S3A_DYNAMIC_CAPABILITIES; +import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.DEFAULT_STREAM_TYPE; import static org.apache.hadoop.fs.s3a.select.SelectConstants.SELECT_UNSUPPORTED; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; @@ -454,12 +455,17 @@ public int run(String[] args, PrintStream out) String encryption = printOption(out, "\tEncryption", Constants.S3_ENCRYPTION_ALGORITHM, "none"); - printOption(out, "\tInput seek policy", INPUT_FADVISE, + + // stream input + printOption(out, "\tInput seek policy", INPUT_STREAM_TYPE, + DEFAULT_STREAM_TYPE.getName()); + printOption(out, "\tInput seek policy", INPUT_STREAM_TYPE, Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT); printOption(out, "\tChange Detection Source", CHANGE_DETECT_SOURCE, CHANGE_DETECT_SOURCE_DEFAULT); printOption(out, "\tChange Detection Mode", CHANGE_DETECT_MODE, CHANGE_DETECT_MODE_DEFAULT); + // committers println(out, "%nS3A Committers"); boolean magic = fs.hasPathCapability( diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java index 1e28a27e7088c..7ad7cf75367e2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a.statistics; import org.apache.hadoop.fs.impl.prefetch.PrefetchingStatistics; +import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType; import org.apache.hadoop.fs.statistics.DurationTracker; /** @@ -53,6 +54,13 @@ public interface S3AInputStreamStatistics extends AutoCloseable, */ long streamOpened(); + /** + * A stream of the given type was opened. + * @param type type of input stream + * @return the previous count or zero if this is the first opening. + */ + long streamOpened(InputStreamType type); + /** * The inner stream was closed. * @param abortedConnection flag to indicate the stream was aborted, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java index e47656efd1800..a5d20095ba5cc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java @@ -22,6 +22,7 @@ import java.time.Duration; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType; import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; @@ -164,6 +165,11 @@ public long streamOpened() { return 0; } + @Override + public long streamOpened(InputStreamType type) { + return 0; + } + @Override public void streamClose(final boolean abortedConnection, final long remainingInCurrentRequest) { diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md index 8bb85008e3624..354db3a705fd9 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md @@ -39,7 +39,8 @@ Multiple blocks may be read in parallel. |Property |Meaning |Default | |---|---|---| -|`fs.s3a.prefetch.enabled` |Enable the prefetch input stream |`false` | +| `fs.s3a.input.stream.type` |Uses the prefetch input stream when set to `prefetch` |`classic` | +|(deprecated) `fs.s3a.prefetch.enabled` |Enable the prefetch input stream |`false` | |`fs.s3a.prefetch.block.size` |Size of a block |`8M` | |`fs.s3a.prefetch.block.count` |Number of blocks to prefetch |`8` | @@ -47,9 +48,18 @@ The default size of a block is 8MB, and the minimum allowed block size is 1 byte Decreasing block size will increase the number of blocks to be read for a file. A smaller block size may negatively impact performance as the number of prefetches required will increase. +The original option to enable prefetching was the boolean option `fs.s3a.prefetch.enabled`. + +This has been superseded by the option `fs.s3a.input.stream.type` which now takes an enumeration of values; `prefetch` selects the prefetching stream. + +1. The original option is deprecated. +2. It is supported *provided the option `fs.s3a.input.stream.type` is unset. +3. The first time a stream created through the `fs.s3a.input.stream.type` option, + a warning message is printed. + ### Key Components -`S3PrefetchingInputStream` - When prefetching is enabled, S3AFileSystem will return an instance of +`S3PrefetchingInputStream` - When the prefetch stream is used, S3AFileSystem will return an instance of this class as the input stream. Depending on the remote file size, it will either use the `S3InMemoryInputStream` or the `S3CachingInputStream` as the underlying input stream. diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/reading.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/reading.md new file mode 100644 index 0000000000000..528f36d31ef47 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/reading.md @@ -0,0 +1,112 @@ +t + +# Reading Data From S3 Storage. + +One of the most important --and performance sensitive-- parts +of the S3A connector is reading data from storage. +This is always evolving, based on experience, and benchmarking, +and in collaboration with other projects. + +## Key concepts + +* Data is read from S3 through an instance of an `ObjectInputStream`. +* There are different implementations of this in the codebase: + `classic`, `prefetch` and `analytics`; these are called "stream types" +* The choice of which stream type to use is made in the hadoop configuration. + +Configuration Options + + +| Property | Permitted Values | Default | Meaning | +|----------------------------|---------------------------------------------------------|-----------|----------------------------| +| `fs.s3a.input.stream.type` | `default`, `classic`, `prefetch`, `analytics`, `custom` | `classic` | name of stream type to use | + + +## Vector IO and Stream Types + +All streams support VectorIO to some degree. + +| Stream | Support | +|-------------|-------------------------------------------------------------| +| `classic` | Parallel issuing of GET request with range coalescing | +| `prefetch` | Sequential reads, using prefetched blocks as appropriate | +| `analytics` | Sequential reads, using prefetched blocks as where possible | + +Because the analytics streams is doing parquet-aware RowGroup prefetch + + +## Developer Topics + +### Stream IOStatistics + +Some of the streams support detailed IOStatistics, which will get aggregated into +the filesystem IOStatistics when the stream is closed(), or possibly after `unbuffer()`. + +The filesystem aggregation can be displayed when the instance is closed, which happens +in process termination, if not earlier: +```xml + + fs.thread.level.iostatistics.enabled + true + +``` + +### Capabilities Probe for stream type and features. + +`StreamCapabilities.hasCapability()` can be used to probe for the active +stream type and its capabilities. + +### Unbuffer() support + +The `unbuffer()` operation requires the stream to release all client-side +resources: buffer, connections to remote servers, cached files etc. +This is used in some query engines, including Apache Impala, to keep +streams open for rapid re-use, avoiding the overhead of re-opening files. + +Only the classic stream supports `CanUnbuffer.unbuffer()`; +the other streams must be closed rather than kept open for an extended +period of time. + +### Stream Leak alerts + +All input streams MUST be closed via a `close()` call once no-longer needed +-this is the only way to guarantee a timely release of HTTP connections +and local resources. + +Some applications/libraries neglect to close the stram + +### Custom Stream Types + +There is a special stream type `custom`. +This is primarily used internally for testing, however it may also be used by +anyone who wishes to experiment with alternative input stream implementations. + +If it is requested, then the name of the _factory_ for streams must be set in the +property `fs.s3a.input.stream.custom.factory`. + +This must be a classname to an implementation of the factory service, +`org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamFactory`. +Consult the source and javadocs of package `org.apache.hadoop.fs.s3a.impl.streams` for +details. + +*Note* this is very much internal code and unstable: any use of this should be considered +experimental, unstable -and is not recommended for production use. + + + +| Property | Permitted Values | Meaning | +|--------------------------------------|----------------------------------------|-----------------------------| +| `fs.s3a.input.stream.custom.factory` | name of factory class on the classpath | classname of custom factory | + diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java index 033c2d94c7bf8..3e87a2f7e5f7d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java @@ -31,9 +31,8 @@ import static org.apache.hadoop.fs.s3a.S3ATestConstants.KEY_PERFORMANCE_TESTS_ENABLED; import static org.apache.hadoop.fs.s3a.Constants.CONNECTION_EXPECT_CONTINUE; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.setPerformanceFlags; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfNotEnabled; + +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; /** * S3A contract tests creating files. @@ -93,6 +92,17 @@ protected Configuration createConfiguration() { return conf; } + @Override + public void testOverwriteExistingFile() throws Throwable { + // Currently analytics accelerator does not support reading of files that have been overwritten. + // This is because the analytics accelerator library caches metadata, and when a file is + // overwritten, the old metadata continues to be used, until it is removed from the cache over + // time. This will be fixed in https://github.com/awslabs/analytics-accelerator-s3/issues/218. + skipIfAnalyticsAcceleratorEnabled(this.createConfiguration(), + "Analytics Accelerator currently does not support reading of over written files"); + super.testOverwriteExistingFile(); + } + @Override public void testOverwriteNonEmptyDirectory() throws Throwable { try { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java index e761e0d14bf83..9c269d3ed443b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestConstants.SCALE_TEST_TIMEOUT_MILLIS; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.StorageStatistics; @@ -78,6 +79,19 @@ public void testNonDirectWrite() throws Exception { getRenameOperationCount() - renames); } + @Override + public void testDistCpUpdateCheckFileSkip() throws Exception { + // Currently analytics accelerator does not support reading of files that have been overwritten. + // This is because the analytics accelerator library caches metadata and data, and when a file is + // overwritten, the old data continues to be used, until it is removed from the cache over + // time. This will be fixed in https://github.com/awslabs/analytics-accelerator-s3/issues/218. + // In this test case, the remote file is created, read, then deleted, and then created again + // with different contents, and read again, which leads to assertions failing. + skipIfAnalyticsAcceleratorEnabled(createConfiguration(), + "Analytics Accelerator Library does not support update to existing files"); + super.testDistCpUpdateCheckFileSkip(); + } + private long getRenameOperationCount() { return getFileSystem().getStorageStatistics() .getLong(StorageStatistics.CommonStatisticNames.OP_RENAME); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMultipartUploader.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMultipartUploader.java index 5a92066e06db4..54aca14798b97 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMultipartUploader.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractMultipartUploader.java @@ -32,6 +32,7 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeNotS3ExpressFileSystem; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBytes; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.s3a.scale.AbstractSTestS3AHugeFiles.DEFAULT_HUGE_PARTITION_SIZE; /** @@ -127,6 +128,12 @@ public void testMultipartUploadReverseOrderNonContiguousPartNumbers() throws Exc @Override public void testConcurrentUploads() throws Throwable { assumeNotS3ExpressFileSystem(getFileSystem()); + // Currently analytics accelerator does not support reading of files that have been overwritten. + // This is because the analytics accelerator library caches metadata and data, and when a file is + // overwritten, the old data continues to be used, until it is removed from the cache over + // time. This will be fixed in https://github.com/awslabs/analytics-accelerator-s3/issues/218. + skipIfAnalyticsAcceleratorEnabled(getContract().getConf(), + "Analytics Accelerator currently does not support reading of over written files"); super.testConcurrentUploads(); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java index dd41583de3fe4..60e3d66317a4c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java @@ -84,7 +84,7 @@ public class ITestS3AContractSeek extends AbstractContractSeekTest { * which S3A Supports. * @return a list of seek policies to test. */ - @Parameterized.Parameters + @Parameterized.Parameters(name="policy={0}") public static Collection params() { return Arrays.asList(new Object[][]{ {FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL, Default_JSSE}, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java index fbb6d5a04d27a..87e3b23cd710d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java @@ -63,6 +63,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult; import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE; import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; import static org.apache.hadoop.io.Sizes.S_1M; @@ -88,6 +89,17 @@ protected AbstractFSContract createContract(Configuration conf) { return new S3AContract(conf); } + /** + * Analytics Accelerator Library for Amazon S3 does not support Vectored Reads. + * @throws Exception + */ + @Override + public void setup() throws Exception { + skipIfAnalyticsAcceleratorEnabled(createConfiguration(), + "Analytics Accelerator does not support vectored reads"); + super.setup(); + } + /** * Verify response to a vector read request which is beyond the * real length of the file. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAnalyticsAcceleratorStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAnalyticsAcceleratorStream.java new file mode 100644 index 0000000000000..040a141586e2c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAnalyticsAcceleratorStream.java @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.UUID; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathHandle; +import org.apache.hadoop.fs.s3a.impl.AwsSdkWorkarounds; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.test.GenericTestUtils; + +import static org.apache.commons.io.FileUtils.ONE_KB; +import static org.apache.hadoop.fs.s3a.Constants.*; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.enableAnalyticsAccelerator; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.enablePrefetching; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.getExternalData; +import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.isUsingDefaultExternalDataFile; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_ANALYTICS_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.test.GenericTestUtils.LogCapturer.captureLogs; + +import org.assertj.core.api.Assertions; + +import org.slf4j.LoggerFactory; +import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamConfiguration; +import software.amazon.s3.analyticsaccelerator.common.ConnectorConfiguration; +import software.amazon.s3.analyticsaccelerator.io.logical.parquet.ParquetMetadataParsingTask; +import software.amazon.s3.analyticsaccelerator.util.PrefetchMode; + +public class ITestS3AAnalyticsAcceleratorStream extends AbstractS3ATestBase { + + private static final String PHYSICAL_IO_PREFIX = "physicalio"; + private static final String LOGICAL_IO_PREFIX = "logicalio"; + + + private Configuration conf; + private Path testFile; + + @Before + public void setUp() throws Exception { + super.setup(); + conf = createConfiguration(); + testFile = getExternalData(conf); + } + + @Override + public Configuration createConfiguration() { + Configuration configuration = super.createConfiguration(); + if (isUsingDefaultExternalDataFile(configuration)) { + S3ATestUtils.removeBaseAndBucketOverrides(configuration, + ENDPOINT); + } + enableAnalyticsAccelerator(configuration); + return configuration; + } + + @Test + public void testConnectorFrameWorkIntegration() throws IOException { + describe("Verify S3 connector framework integration"); + + S3AFileSystem fs = + (S3AFileSystem) FileSystem.get(testFile.toUri(), conf); + byte[] buffer = new byte[500]; + IOStatistics ioStats; + + try (FSDataInputStream inputStream = fs.open(testFile)) { + ioStats = inputStream.getIOStatistics(); + inputStream.seek(5); + inputStream.read(buffer, 0, 500); + } + verifyStatisticCounterValue(ioStats, STREAM_READ_ANALYTICS_OPENED, 1); + } + + @Test + public void testMalformedParquetFooter() throws IOException { + describe("Reading a malformed parquet file should not throw an exception"); + + // File with malformed footer take from https://github.com/apache/parquet-testing/blob/master/bad_data/PARQUET-1481.parquet. + // This test ensures AAL does not throw exceptions if footer parsing fails. It will only emit a WARN log, + // "Unable to parse parquet footer for test/malformedFooter.parquet, parquet prefetch optimisations will be disabled for this key." + Path dest = path("malformed_footer.parquet"); + + File file = new File("src/test/resources/malformed_footer.parquet"); + Path sourcePath = new Path(file.toURI().getPath()); + getFileSystem().copyFromLocalFile(false, true, sourcePath, dest); + + byte[] buffer = new byte[500]; + IOStatistics ioStats; + + try (FSDataInputStream inputStream = getFileSystem().open(dest)) { + ioStats = inputStream.getIOStatistics(); + inputStream.seek(5); + inputStream.read(buffer, 0, 500); + } + + verifyStatisticCounterValue(ioStats, STREAM_READ_ANALYTICS_OPENED, 1); + } + + @Test + public void testMultiRowGroupParquet() throws IOException { + describe("A parquet file is read successfully"); + + Path dest = path("multi_row_group.parquet"); + + File file = new File("src/test/resources/multi_row_group.parquet"); + Path sourcePath = new Path(file.toURI().getPath()); + getFileSystem().copyFromLocalFile(false, true, sourcePath, dest); + + FileStatus fileStatus = getFileSystem().getFileStatus(dest); + + byte[] buffer = new byte[3000]; + IOStatistics ioStats; + + try (FSDataInputStream inputStream = getFileSystem().open(dest)) { + ioStats = inputStream.getIOStatistics(); + inputStream.readFully(buffer, 0, (int) fileStatus.getLen()); + } + + verifyStatisticCounterValue(ioStats, STREAM_READ_ANALYTICS_OPENED, 1); + } + + @Test + public void testConnectorFrameworkConfigurable() { + describe("Verify S3 connector framework reads configuration"); + + Configuration conf = getConfiguration(); + removeBaseAndBucketOverrides(conf); + + //Disable Predictive Prefetching + conf.set(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + + "." + LOGICAL_IO_PREFIX + ".prefetching.mode", "all"); + + //Set Blobstore Capacity + conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", 1); + + ConnectorConfiguration connectorConfiguration = + new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); + + S3SeekableInputStreamConfiguration configuration = + S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration); + + Assertions.assertThat(configuration.getLogicalIOConfiguration().getPrefetchingMode()) + .as("AnalyticsStream configuration is not set to expected value") + .isSameAs(PrefetchMode.ALL); + + Assertions.assertThat(configuration.getPhysicalIOConfiguration().getBlobStoreCapacity()) + .as("AnalyticsStream configuration is not set to expected value") + .isEqualTo(1); + } + + @Test + public void testInvalidConfigurationThrows() throws Exception { + describe("Verify S3 connector framework throws with invalid configuration"); + + Configuration conf = getConfiguration(); + removeBaseAndBucketOverrides(conf); + //Disable Sequential Prefetching + conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", -1); + + ConnectorConfiguration connectorConfiguration = + new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); + Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> + S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration)); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java index 24115177f35a2..97af80e70a542 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java @@ -58,6 +58,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.alias.CredentialProvider; import org.apache.hadoop.security.alias.CredentialProviderFactory; +import org.apache.hadoop.test.AbstractHadoopTestBase; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.VersionInfo; import org.apache.http.HttpStatus; @@ -75,7 +76,7 @@ /** * S3A tests for configuration, especially credentials. */ -public class ITestS3AConfiguration { +public class ITestS3AConfiguration extends AbstractHadoopTestBase { private static final String EXAMPLE_ID = "AKASOMEACCESSKEY"; private static final String EXAMPLE_KEY = "RGV0cm9pdCBSZ/WQgY2xl/YW5lZCB1cAEXAMPLE"; @@ -487,11 +488,20 @@ public void testDirectoryAllocatorDefval() throws Throwable { conf = new Configuration(); conf.unset(Constants.BUFFER_DIR); fs = S3ATestUtils.createTestFileSystem(conf); - File tmp = fs.createTmpFileForWrite("out-", 1024, conf); + File tmp = createTemporaryFileForWriting(); assertTrue("not found: " + tmp, tmp.exists()); tmp.delete(); } + /** + * Create a temporary file for writing; requires the FS to have been created/initialized. + * @return a temporary file + * @throws IOException creation issues. + */ + private File createTemporaryFileForWriting() throws IOException { + return fs.getS3AInternals().getStore().createTemporaryFileForWriting("out-", 1024, conf); + } + @Test public void testDirectoryAllocatorRR() throws Throwable { File dir1 = GenericTestUtils.getRandomizedTestDir(); @@ -501,9 +511,9 @@ public void testDirectoryAllocatorRR() throws Throwable { conf = new Configuration(); conf.set(Constants.BUFFER_DIR, dir1 + ", " + dir2); fs = S3ATestUtils.createTestFileSystem(conf); - File tmp1 = fs.createTmpFileForWrite("out-", 1024, conf); + File tmp1 = createTemporaryFileForWriting(); tmp1.delete(); - File tmp2 = fs.createTmpFileForWrite("out-", 1024, conf); + File tmp2 = createTemporaryFileForWriting(); tmp2.delete(); assertNotEquals("round robin not working", tmp1.getParent(), tmp2.getParent()); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java index d22de3b06d81b..169898df829cc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java @@ -38,9 +38,7 @@ import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_KEY; import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM; import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_KEY; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeSkipRootTests; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** @@ -93,6 +91,8 @@ protected Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Analytics Accelerator currently does not support SSE-C"); assumeEnabled(); // although not a root dir test, this confuses paths enough it shouldn't be run in // parallel with other jobs diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFSMainOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFSMainOperations.java index 0281c57f5cbce..b056dac5a85ae 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFSMainOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFSMainOperations.java @@ -29,9 +29,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.s3a.S3AContract; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.createTestPath; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.isCreatePerformanceEnabled; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.setPerformanceFlags; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; /** * S3A Test suite for the FSMainOperationsBaseTest tests. @@ -78,6 +76,28 @@ public void testCopyToLocalWithUseRawLocalFileSystemOption() throws Exception { } + @Override + public void testWriteReadAndDeleteOneAndAHalfBlocks() throws Exception { + // Currently analytics accelerator does not support reading of files that have been overwritten. + // This is because the analytics accelerator library caches metadata, and when a file is + // overwritten, the old metadata continues to be used, until it is removed from the cache over + // time. This will be fixed in https://github.com/awslabs/analytics-accelerator-s3/issues/218. + skipIfAnalyticsAcceleratorEnabled(this.contract.getConf(), + "Analytics Accelerator currently does not support reading of over written files"); + super.testWriteReadAndDeleteOneAndAHalfBlocks(); + } + + @Override + public void testWriteReadAndDeleteTwoBlocks() throws Exception { + // Currently analytics accelerator does not support reading of files that have been overwritten. + // This is because the analytics accelerator library caches metadata, and when a file is + // overwritten, the old metadata continues to be used, until it is removed from the cache over + // time. This will be fixed in https://github.com/awslabs/analytics-accelerator-s3/issues/218. + skipIfAnalyticsAcceleratorEnabled(this.contract.getConf(), + "Analytics Accelerator currently does not support reading of over written files"); + super.testWriteReadAndDeleteTwoBlocks(); + } + @Override public void testOverwrite() throws IOException { boolean createPerformance = isCreatePerformanceEnabled(fSys); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java index 4808145765822..2034ea274e7aa 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java @@ -34,8 +34,7 @@ import org.apache.hadoop.fs.Path; import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.isCreatePerformanceEnabled; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.setPerformanceFlags; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.junit.Assume.*; import static org.junit.Assert.*; @@ -160,4 +159,15 @@ public void testOverwrite() throws IOException { } } } + + @Override + public void testOverWriteAndRead() throws Exception { + // Currently analytics accelerator does not support reading of files that have been overwritten. + // This is because the analytics accelerator library caches metadata, and when a file is + // overwritten, the old metadata continues to be used, until it is removed from the cache over + // time. This will be fixed in https://github.com/awslabs/analytics-accelerator-s3/issues/218. + skipIfAnalyticsAcceleratorEnabled(fs.getConf(), + "Analytics Accelerator currently does not support reading of over written files"); + super.testOverWriteAndRead(); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AIOStatisticsContext.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AIOStatisticsContext.java index 70dc5ee476c47..daf5306dc399e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AIOStatisticsContext.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AIOStatisticsContext.java @@ -43,6 +43,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; import static org.apache.hadoop.fs.s3a.S3ATestUtils.disablePrefetching; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES; @@ -77,6 +78,11 @@ protected Configuration createConfiguration() { public void setup() throws Exception { super.setup(); executor = HadoopExecutors.newFixedThreadPool(SMALL_THREADS); + // Analytics accelerator currently does not support IOStatisticsContext, this will be added as + // part of https://issues.apache.org/jira/browse/HADOOP-19364 + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Analytics Accelerator currently does not support IOStatisticsContext"); + } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInputStreamLeakage.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInputStreamLeakage.java index 4b871c6a197db..f26a585776a21 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInputStreamLeakage.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInputStreamLeakage.java @@ -29,11 +29,14 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStream; +import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.test.GenericTestUtils; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_LEAKS; import static org.apache.hadoop.test.GenericTestUtils.LogCapturer.captureLogs; @@ -58,7 +61,7 @@ public class ITestS3AInputStreamLeakage extends AbstractS3ATestBase { @Override public void setup() throws Exception { super.setup(); - assume("Stream leak detection not avaialable", + assume("Stream leak detection not available", getFileSystem().hasCapability(STREAM_LEAKS)); } @@ -81,14 +84,15 @@ public void setup() throws Exception { *

* The stream leak counter of the FileSystem is also updated; this * is verified. - *

- * Note: if the stream under test is not an S3AInputStream (i.e. is a prefetching one, - * this test is skipped. If/when the prefetching stream adds the same code, - * this check can be removed. */ @Test public void testFinalizer() throws Throwable { Path path = methodPath(); + // Analytics accelerator currently does not support stream leak detection. This work is tracked + // in https://issues.apache.org/jira/browse/HADOOP-19451 + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Analytics Accelerator currently does not support leak detection"); + final S3AFileSystem fs = getFileSystem(); ContractTestUtils.createFile(fs, path, true, DATASET); @@ -100,7 +104,7 @@ public void testFinalizer() throws Throwable { try { Assertions.assertThat(in.hasCapability(STREAM_LEAKS)) - .describedAs("Stream leak detection not supported in: " + in.getClass()) + .describedAs("Stream leak detection not supported in: %s", in.getWrappedStream()) .isTrue(); Assertions.assertThat(in.read()) @@ -108,12 +112,12 @@ public void testFinalizer() throws Throwable { .isEqualTo(DATASET[0]); // get a weak ref so that after a GC we can look for it and verify it is gone - Assertions.assertThat(((S3AInputStream) in.getWrappedStream()).isObjectStreamOpen()) - .describedAs("stream http connection status") - .isTrue(); - // weak reference to track GC progress - WeakReference wrs = - new WeakReference<>((S3AInputStream) in.getWrappedStream()); + WeakReference wrs = + new WeakReference<>((ObjectInputStream) in.getWrappedStream()); + + boolean isClassicStream = wrs.get() instanceof S3AInputStream; + final IOStatistics fsStats = fs.getIOStatistics(); + final long leaks = fsStats.counters().getOrDefault(STREAM_LEAKS, 0L); // Capture the logs GenericTestUtils.LogCapturer logs = @@ -125,7 +129,7 @@ public void testFinalizer() throws Throwable { in = null; // force the gc. System.gc(); - // make sure the GC removed the S3AInputStream. + // make sure the GC removed the Stream. Assertions.assertThat(wrs.get()) .describedAs("weak stream reference wasn't GC'd") .isNull(); @@ -144,14 +148,26 @@ public void testFinalizer() throws Throwable { LOG.info("output of leak log is {}", output); Assertions.assertThat(output) .describedAs("output from the logs during GC") - .contains("drain or abort reason finalize()") // stream release + .contains("Stream not closed") // stream release .contains(path.toUri().toString()) // path .contains(Thread.currentThread().getName()) // thread .contains("testFinalizer"); // stack - // verify that leakages are added to the FS statistics - assertThatStatisticCounter(fs.getIOStatistics(), STREAM_LEAKS) - .isEqualTo(1); + + // for classic stream the counter is 1, but for prefetching + // the count is greater -the inner streams can also + // get finalized while open so increment the leak counter + // multiple times. + assertThatStatisticCounter(fsStats, STREAM_LEAKS) + .isGreaterThanOrEqualTo(leaks + 1); + if (isClassicStream) { + Assertions.assertThat(output) + .describedAs("output from the logs during GC") + .contains("drain or abort reason finalize()"); // stream release + assertThatStatisticCounter(fsStats, STREAM_LEAKS) + .isEqualTo(leaks + 1); + } + } finally { if (in != null) { IOUtils.cleanupWithLogger(LOG, in); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java index 3bfe69c2bca91..4ec579ce4f649 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.io.InputStream; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; /** @@ -51,6 +52,11 @@ public void testMetricsRegister() @Test public void testStreamStatistics() throws IOException { + // Analytics accelerator currently does not support IOStatistics, this will be added as + // part of https://issues.apache.org/jira/browse/HADOOP-19364 + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Analytics Accelerator currently does not support stream statistics"); + S3AFileSystem fs = getFileSystem(); Path file = path("testStreamStatistics"); byte[] data = "abcdefghijklmnopqrstuvwxyz".getBytes(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java index e7c9921824c84..39acaa717a7f2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java @@ -40,7 +40,7 @@ import static org.apache.hadoop.fs.s3a.Constants.BUFFER_DIR; import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT; 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.S3ATestUtils.enablePrefetching; import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.getExternalData; import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.isUsingDefaultExternalDataFile; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; @@ -84,10 +84,9 @@ public Configuration createConfiguration() { Configuration configuration = super.createConfiguration(); if (isUsingDefaultExternalDataFile(configuration)) { S3ATestUtils.removeBaseAndBucketOverrides(configuration, - PREFETCH_ENABLED_KEY, ENDPOINT); } - configuration.setBoolean(PREFETCH_ENABLED_KEY, true); + enablePrefetching(configuration); // use a small block size unless explicitly set in the test config. configuration.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); // patch buffer dir with a unique path for test isolation. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java index 28c854194656b..d894adb66c738 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java @@ -35,7 +35,7 @@ import org.apache.hadoop.test.LambdaTestUtils; 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.S3ATestUtils.enablePrefetching; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMaximum; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue; @@ -73,10 +73,8 @@ public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest { @Override public Configuration createConfiguration() { - Configuration conf = super.createConfiguration(); - S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY); + Configuration conf = enablePrefetching(super.createConfiguration()); S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_BLOCK_SIZE_KEY); - conf.setBoolean(PREFETCH_ENABLED_KEY, true); conf.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); return conf; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java index 7375105909ba7..5cc948a044dbb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java @@ -43,8 +43,8 @@ import org.apache.hadoop.test.LambdaTestUtils; 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_MAX_BLOCKS_COUNT; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.enablePrefetching; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValues; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue; @@ -85,11 +85,10 @@ public ITestS3APrefetchingLruEviction(final String maxBlocks) { @Override public Configuration createConfiguration() { - Configuration conf = super.createConfiguration(); - S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY); - S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_MAX_BLOCKS_COUNT); - S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_BLOCK_SIZE_KEY); - conf.setBoolean(PREFETCH_ENABLED_KEY, true); + Configuration conf = enablePrefetching(super.createConfiguration()); + S3ATestUtils.removeBaseAndBucketOverrides(conf, + PREFETCH_MAX_BLOCKS_COUNT, + PREFETCH_BLOCK_SIZE_KEY); conf.setInt(PREFETCH_MAX_BLOCKS_COUNT, Integer.parseInt(maxBlocks)); conf.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); return conf; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index b88d0b4aab617..725e54c7d8557 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -31,9 +31,9 @@ import org.apache.hadoop.fs.statistics.StreamStatisticNames; import static org.apache.hadoop.fs.s3a.Constants.ALLOW_REQUESTER_PAYS; -import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_DEFAULT; -import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.streamType; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** @@ -60,6 +60,11 @@ protected Configuration createConfiguration() { public void testRequesterPaysOptionSuccess() throws Throwable { describe("Test requester pays enabled case by reading last then first byte"); skipIfClientSideEncryption(); + // Analytics accelerator currently does not support IOStatistics which leads to the + // STREAM_READ_OPENED assertion to fail, this will be added as part of + // https://issues.apache.org/jira/browse/HADOOP-19364 + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Analytics Accelerator currently does not support IOStatistics"); Configuration conf = this.createConfiguration(); conf.setBoolean(ALLOW_REQUESTER_PAYS, true); // Enable bucket exists check, the first failure point people may encounter @@ -80,15 +85,20 @@ public void testRequesterPaysOptionSuccess() throws Throwable { inputStream.seek(0); inputStream.readByte(); - if (conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT)) { - // For S3APrefetchingInputStream, verify a call was made - IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(), - StreamStatisticNames.STREAM_READ_OPENED).isEqualTo(1); - } else { + switch (streamType(getFileSystem())) { + case Classic: // For S3AInputStream, verify > 1 call was made, // so we're sure it is correctly configured for each request IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(), StreamStatisticNames.STREAM_READ_OPENED).isGreaterThan(1); + break; + case Prefetch: + // For S3APrefetchingInputStream, verify a call was made + IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(), + StreamStatisticNames.STREAM_READ_OPENED).isEqualTo(1); + break; + default: + break; } // Check list calls work without error diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index 1f779ab7ca38c..bf80441513381 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -45,6 +45,7 @@ import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder; +import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType; import org.apache.hadoop.fs.s3a.prefetch.S3APrefetchingInputStream; import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; @@ -104,6 +105,9 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.impl.FlagSet.createFlagSet; +import static org.apache.hadoop.fs.s3a.impl.streams.InputStreamType.Analytics; +import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.DEFAULT_STREAM_TYPE; +import static org.apache.hadoop.fs.s3a.impl.streams.InputStreamType.Prefetch; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; import static org.apache.hadoop.fs.s3a.impl.S3ExpressStorage.STORE_CAPABILITY_S3_EXPRESS_STORAGE; @@ -118,6 +122,7 @@ import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3AUtils.buildEncryptionSecrets; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.apache.hadoop.util.functional.FunctionalIO.uncheckIOExceptions; import static org.apache.hadoop.util.functional.RemoteIterators.mappingRemoteIterator; import static org.apache.hadoop.util.functional.RemoteIterators.toList; import static org.junit.Assert.*; @@ -574,6 +579,21 @@ public static boolean isS3ExpressTestBucket(final Configuration conf) { return S3ExpressStorage.isS3ExpressStore(getTestBucketName(conf), ""); } + /** + * Skip a test if the Analytics Accelerator Library for Amazon S3 is enabled. + * @param configuration configuration to probe + */ + public static void skipIfAnalyticsAcceleratorEnabled( + Configuration configuration, String message) { + assume(message, + !isAnalyticsAcceleratorEnabled(configuration)); + } + + public static boolean isAnalyticsAcceleratorEnabled(final Configuration conf) { + return conf.getEnum(INPUT_STREAM_TYPE, + InputStreamType.Classic) == InputStreamType.Analytics; + } + /** * Skip a test if the filesystem lacks a required capability. * @param fs filesystem @@ -722,10 +742,8 @@ public static Configuration prepareTestConfiguration(final Configuration conf) { } conf.set(BUFFER_DIR, tmpDir); - boolean prefetchEnabled = - getTestPropertyBool(conf, PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT); - conf.setBoolean(PREFETCH_ENABLED_KEY, prefetchEnabled); - + conf.set(INPUT_STREAM_TYPE, + getTestProperty(conf, INPUT_STREAM_TYPE, INPUT_STREAM_TYPE_DEFAULT)); return conf; } @@ -1780,11 +1798,62 @@ public static void assertStreamIsNotChecksummed(final S3AInputStream wrappedS3A) /** * Disable Prefetching streams from S3AFileSystem in tests. * @param conf Configuration to remove the prefetch property from. + * @return patched config + */ + public static Configuration disablePrefetching(Configuration conf) { + removeBaseAndBucketOverrides(conf, + PREFETCH_ENABLED_KEY, + INPUT_STREAM_TYPE); + return conf; + } + + + /** + *Enable Prefetching streams from S3AFileSystem in tests. + * @param conf Configuration to update + * @return patched config */ - public static void disablePrefetching(Configuration conf) { - removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY); + public static Configuration enablePrefetching(Configuration conf) { + removeBaseAndBucketOverrides(conf, + PREFETCH_ENABLED_KEY, + INPUT_STREAM_TYPE); + conf.setEnum(INPUT_STREAM_TYPE, Prefetch); + return conf; + } + + /** + * Enable analytics stream for S3A S3AFileSystem in tests. + * @param conf Configuration to update + * @return patched config + */ + public static Configuration enableAnalyticsAccelerator(Configuration conf) { + removeBaseAndBucketOverrides(conf, + INPUT_STREAM_TYPE); + conf.setEnum(INPUT_STREAM_TYPE, Analytics); + return conf; + } + + /** + * Probe for a filesystem having a specific stream type; + * this is done through filesystem capabilities. + * @param fs filesystem + * @param type stream type + * @return true if the fs has the specific type. + */ + public static boolean hasInputStreamType(FileSystem fs, InputStreamType type) { + return uncheckIOExceptions(() -> + fs.hasPathCapability(new Path("/"), + type.capability())); } + /** + * What is the stream type of this filesystem? + * @param fs filesystem to probe + * @return the stream type + */ + public static InputStreamType streamType(S3AFileSystem fs) { + return fs.getS3AInternals().getStore().streamType(); + } /** * Skip root tests if the system properties/config says so. * @param conf configuration to check diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java index 6eccdc23dd5d5..07f6550d83eee 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java @@ -40,6 +40,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.audit.impl.NoopSpan; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectReadParameters; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; import org.apache.hadoop.util.functional.CallableRaisingIOE; import org.apache.http.NoHttpResponseException; @@ -164,7 +166,7 @@ private static void assertReadValueMatchesOffset( * @return a stream. */ private S3AInputStream getMockedS3AInputStream( - S3AInputStream.InputStreamCallbacks streamCallback) { + ObjectInputStreamCallbacks streamCallback) { Path path = new Path("test-path"); String eTag = "test-etag"; String versionId = "test-version-id"; @@ -187,12 +189,15 @@ private S3AInputStream getMockedS3AInputStream( s3AFileStatus, NoopSpan.INSTANCE); - return new S3AInputStream( - s3AReadOpContext, - s3ObjectAttributes, - streamCallback, - s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics(), - null); + ObjectReadParameters parameters = new ObjectReadParameters() + .withCallbacks(streamCallback) + .withObjectAttributes(s3ObjectAttributes) + .withContext(s3AReadOpContext) + .withStreamStatistics( + s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics()) + .withBoundedThreadPool(null); + + return new S3AInputStream(parameters); } /** @@ -203,7 +208,7 @@ private S3AInputStream getMockedS3AInputStream( * @param ex exception to raise on failure * @return mocked object. */ - private S3AInputStream.InputStreamCallbacks failingInputStreamCallbacks( + private ObjectInputStreamCallbacks failingInputStreamCallbacks( final RuntimeException ex) { GetObjectResponse objectResponse = GetObjectResponse.builder() @@ -238,7 +243,7 @@ private S3AInputStream.InputStreamCallbacks failingInputStreamCallbacks( * @param ex exception to raise on failure * @return mocked object. */ - private S3AInputStream.InputStreamCallbacks maybeFailInGetCallback( + private ObjectInputStreamCallbacks maybeFailInGetCallback( final RuntimeException ex, final Function failurePredicate) { GetObjectResponse objectResponse = GetObjectResponse.builder() @@ -259,13 +264,13 @@ private S3AInputStream.InputStreamCallbacks maybeFailInGetCallback( * @param streamFactory factory for the stream to return on the given attempt. * @return mocked object. */ - private S3AInputStream.InputStreamCallbacks mockInputStreamCallback( + private ObjectInputStreamCallbacks mockInputStreamCallback( final RuntimeException ex, final Function failurePredicate, final Function> streamFactory) { - return new S3AInputStream.InputStreamCallbacks() { + return new ObjectInputStreamCallbacks() { private int attempt = 0; @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java index e453fd3caad57..dee18804d809d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java @@ -18,7 +18,10 @@ package org.apache.hadoop.fs.s3a.audit; +import org.assertj.core.api.Assumptions; + import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditManagerS3A; import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor; @@ -122,6 +125,8 @@ public static IOStatisticsStore createIOStatisticsStoreForAuditing() { /** * Remove all overridden values for * the test bucket/global in the given config. + * Note that the rejection flag may be overridden by the + * requirements returned by the output stream factory. * @param conf configuration to patch * @return the configuration. */ @@ -134,4 +139,23 @@ public static Configuration resetAuditOptions(Configuration conf) { AUDIT_ENABLED); return conf; } + + /** + * Skip a test if the filesystem's audit manager has had them disabled. + * @param fs filesystem + */ + public static void requireOutOfSpanOperationsRejected(final S3AFileSystem fs) { + Assumptions.assumeThat(outOfSpanOperationAreRejected(fs)) + .describedAs("Out of span operations rejected") + .isTrue(); + } + + /** + * Are Out of Span operations rejected by the filesystem's audit manager? + * @param fs filesystem + * @return true if out of span calls raise exceptions + */ + private static boolean outOfSpanOperationAreRejected(final S3AFileSystem fs) { + return fs.getAuditManager().getConfig().getBoolean(REJECT_OUT_OF_SPAN_OPERATIONS, false); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java index 11b434cae17ed..824fd514626c0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java @@ -32,6 +32,7 @@ import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_FAILURE; import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_REQUEST_EXECUTION; import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.enableLoggingAuditor; +import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.requireOutOfSpanOperationsRejected; import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.resetAuditOptions; import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_EXECUTION_INTERCEPTORS; import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS; @@ -78,6 +79,9 @@ private IOStatistics iostats() { public void testInvokeOutOfSpanRejected() throws Throwable { describe("Operations against S3 will be rejected outside of a span"); final S3AFileSystem fs = getFileSystem(); + + requireOutOfSpanOperationsRejected(fs); + final long failures0 = lookupCounterStatistic(iostats(), AUDIT_FAILURE.getSymbol()); final long exec0 = lookupCounterStatistic(iostats(), diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java index 8132b44cdb438..02f0251e8f055 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java @@ -40,6 +40,7 @@ import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; import org.apache.hadoop.fs.statistics.IOStatisticsLogging; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.s3a.Statistic.ACTION_HTTP_GET_REQUEST; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MAGIC_FILES_CREATED; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MAGIC_MARKER_PUT; @@ -167,7 +168,12 @@ private void abortActiveStream() throws IOException { @Test public void testCostOfCreatingMagicFile() throws Throwable { - describe("Files created under magic paths skip existence checks"); + describe("Files created under magic paths skip existence checks and marker deletes"); + + // Analytics accelerator currently does not support IOStatistics, this will be added as + // part of https://issues.apache.org/jira/browse/HADOOP-19364 + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Analytics Accelerator currently does not support stream statistics"); S3AFileSystem fs = getFileSystem(); Path destFile = methodSubPath("file.txt"); fs.delete(destFile.getParent(), true); @@ -245,6 +251,10 @@ public void testCostOfCreatingMagicFile() throws Throwable { public void testCostOfSavingLoadingPendingFile() throws Throwable { describe("Verify costs of saving .pending file under a magic path"); + // Analytics accelerator currently does not support IOStatistics, this will be added as + // part of https://issues.apache.org/jira/browse/HADOOP-19364 + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Analytics Accelerator currently does not support stream statistics"); S3AFileSystem fs = getFileSystem(); Path partDir = methodSubPath("file.pending"); Path destFile = new Path(partDir, "file.pending"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java index 1724006a83198..5b489c1c39c7a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java @@ -30,6 +30,8 @@ import org.junit.Assert; import org.junit.Before; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; + /** * S3a implementation of FCStatisticsBaseTest. */ @@ -44,6 +46,10 @@ public class ITestS3AFileContextStatistics extends FCStatisticsBaseTest { @Before public void setUp() throws Exception { conf = new Configuration(); + // Analytics accelerator currently does not support IOStatistics, this will be added as + // part of https://issues.apache.org/jira/browse/HADOOP-19364 + skipIfAnalyticsAcceleratorEnabled(conf, + "Analytics Accelerator currently does not support stream statistics"); fc = S3ATestUtils.createTestFileContext(conf); testRootPath = fileContextTestHelper.getTestRootPath(fc, "test"); fc.mkdir(testRootPath, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestConnectionTimeouts.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestConnectionTimeouts.java index a4cc5cadc5da0..aeb9629b3a6d1 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestConnectionTimeouts.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestConnectionTimeouts.java @@ -55,10 +55,10 @@ import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS; import static org.apache.hadoop.fs.s3a.Constants.MAX_ERROR_RETRIES; import static org.apache.hadoop.fs.s3a.Constants.PART_UPLOAD_TIMEOUT; -import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; import static org.apache.hadoop.fs.s3a.Constants.REQUEST_TIMEOUT; import static org.apache.hadoop.fs.s3a.Constants.RETRY_LIMIT; import static org.apache.hadoop.fs.s3a.Constants.SOCKET_TIMEOUT; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disablePrefetching; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC_PATH_PREFIX; import static org.apache.hadoop.fs.s3a.impl.ConfigurationHelper.setDurationAsMillis; @@ -104,7 +104,7 @@ protected Configuration createConfiguration() { * @return a configuration to use for the brittle FS. */ private Configuration timingOutConfiguration() { - Configuration conf = new Configuration(getConfiguration()); + Configuration conf = disablePrefetching(new Configuration(getConfiguration())); removeBaseAndBucketOverrides(conf, CONNECTION_TTL, CONNECTION_ACQUISITION_TIMEOUT, @@ -113,7 +113,6 @@ private Configuration timingOutConfiguration() { MAX_ERROR_RETRIES, MAXIMUM_CONNECTIONS, PART_UPLOAD_TIMEOUT, - PREFETCH_ENABLED_KEY, REQUEST_TIMEOUT, SOCKET_TIMEOUT, FS_S3A_CREATE_PERFORMANCE, @@ -125,7 +124,7 @@ private Configuration timingOutConfiguration() { conf.setInt(MAX_ERROR_RETRIES, 0); // needed to ensure that streams are kept open. // without this the tests is unreliable in batch runs. - conf.setBoolean(PREFETCH_ENABLED_KEY, false); + disablePrefetching(conf); conf.setInt(RETRY_LIMIT, 0); conf.setBoolean(FS_S3A_CREATE_PERFORMANCE, true); final Duration ms10 = Duration.ofMillis(10); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/streams/TestStreamFactories.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/streams/TestStreamFactories.java new file mode 100644 index 0000000000000..b42aaefbbde53 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/streams/TestStreamFactories.java @@ -0,0 +1,333 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl.streams; + +import java.io.IOException; +import java.io.UncheckedIOException; + +import org.assertj.core.api.AbstractBooleanAssert; +import org.assertj.core.api.Assertions; +import org.junit.Test; +import software.amazon.awssdk.services.s3.S3AsyncClient; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.VectoredIOContext; +import org.apache.hadoop.fs.s3a.prefetch.PrefetchingInputStreamFactory; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_CUSTOM_FACTORY; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_CLASSIC; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_CUSTOM; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_DEFAULT; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_PREFETCH; +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; +import static org.apache.hadoop.fs.s3a.impl.streams.StreamFactoryRequirements.Requirements.ExpectUnauditedGetRequests; +import static org.apache.hadoop.fs.s3a.impl.streams.StreamFactoryRequirements.Requirements.RequiresFuturePool; +import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.DEFAULT_STREAM_TYPE; +import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.E_EMPTY_CUSTOM_CLASSNAME; +import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.E_INVALID_STREAM_TYPE; +import static org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.factoryFromConfig; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Unit test for stream factory creation. + * Verifies mapping of name to type, default handling, + * legacy prefetch switch and failure handling. + */ +public class TestStreamFactories extends AbstractHadoopTestBase { + + /** + * The empty string and "default" both map to the classic stream. + */ + @Test + public void testDefaultFactoryCreation() throws Throwable { + load("", DEFAULT_STREAM_TYPE, + ClassicObjectInputStreamFactory.class); + load(INPUT_STREAM_TYPE_DEFAULT, DEFAULT_STREAM_TYPE, + ClassicObjectInputStreamFactory.class); + } + + /** + * Classic factory. + */ + @Test + public void testClassicFactoryCreation() throws Throwable { + final ClassicObjectInputStreamFactory f = + load(INPUT_STREAM_TYPE_CLASSIC, DEFAULT_STREAM_TYPE, + ClassicObjectInputStreamFactory.class); + final StreamFactoryRequirements requirements = f.factoryRequirements(); + Assertions.assertThat(requirements.requiresFuturePool()) + .describedAs("requires future pool of %s", requirements) + .isFalse(); + expectRequirement(requirements, + ExpectUnauditedGetRequests, + false); + } + + private static AbstractBooleanAssert expectRequirement(final StreamFactoryRequirements requirements, + final StreamFactoryRequirements.Requirements r, + final boolean expected) { + return Assertions.assertThat(requirements.requires(r)) + .describedAs("%s of %s", r, requirements) + .isEqualTo(expected); + } + + /** + * Prefetch factory. + */ + @Test + public void testPrefetchFactoryCreation() throws Throwable { + // load from config option + final PrefetchingInputStreamFactory f = load(INPUT_STREAM_TYPE_PREFETCH, + InputStreamType.Prefetch, + PrefetchingInputStreamFactory.class); + final StreamFactoryRequirements requirements = f.factoryRequirements(); + Assertions.assertThat(requirements.requiresFuturePool()) + .describedAs("requires future pool of %s", requirements) + .isTrue(); + expectRequirement(requirements, + ExpectUnauditedGetRequests, + false); + expectRequirement(requirements, + RequiresFuturePool, + true); + } + + /** + * Prefetch factory via the prefect enabled flag. + * This is returned before any attempt is made to instantiate + * the stream type option. + */ + + @Test + public void testPrefetchEnabledFlag() throws Throwable { + + // request an analytics stream + final Configuration conf = configWithStream("undefined"); + // but then set the prefetch key + conf.setBoolean(PREFETCH_ENABLED_KEY, true); + assertFactorySatisfies(factoryFromConfig(conf), + INPUT_STREAM_TYPE_PREFETCH, + InputStreamType.Prefetch, + PrefetchingInputStreamFactory.class); + } + + @Test + public void testRequirementFlagsNoElements() throws Throwable { + VectoredIOContext vertex = new VectoredIOContext(); + + // no elements + final StreamFactoryRequirements r1 = + new StreamFactoryRequirements(1, 2, vertex); + expectRequirement(r1, ExpectUnauditedGetRequests, false); + expectRequirement(r1, RequiresFuturePool, false); + Assertions.assertThat(r1.requiresFuturePool()) + .describedAs("requiresFuturePool() %s", r1) + .isFalse(); + Assertions.assertThat(r1) + .describedAs("%s", r1) + .matches(r -> !r.requiresFuturePool(), "requiresFuturePool") + .satisfies(r -> + Assertions.assertThat(r.sharedThreads()).isEqualTo(1)) + .satisfies(r -> + Assertions.assertThat(r.streamThreads()).isEqualTo(2)); + } + + @Test + public void testRequirementFlagsFutures() throws Throwable { + VectoredIOContext vertex = new VectoredIOContext(); + + final StreamFactoryRequirements r1 = + new StreamFactoryRequirements(1, 2, vertex, RequiresFuturePool); + expectRequirement(r1, ExpectUnauditedGetRequests, false); + expectRequirement(r1, RequiresFuturePool, true); + Assertions.assertThat(r1.requiresFuturePool()) + .describedAs("requiresFuturePool() %s", r1) + .isTrue(); + } + + @Test + public void testRequirementFlagsUnaudited() throws Throwable { + VectoredIOContext vertex = new VectoredIOContext(); + + final StreamFactoryRequirements r1 = + new StreamFactoryRequirements(1, 2, vertex, ExpectUnauditedGetRequests); + expectRequirement(r1, ExpectUnauditedGetRequests, true); + expectRequirement(r1, RequiresFuturePool, false); + } + + + /** + * Create a factory, assert that it satisfies the requirements. + * @param name name: only used for assertion messages. + * @param type expected stream type. + * @param clazz expected class. + * @param class to expect + */ + private static T load( + String name, + InputStreamType type, + Class clazz) throws IOException { + + final ObjectInputStreamFactory factory = factory(name); + assertFactorySatisfies(factory, name, type, clazz); + factory.init(new Configuration(false)); + factory.bind(new FactoryBindingParameters(new Callbacks())); + return (T)factory; + } + + /** + * Assert that a factory satisfies the requirements. + * @param factory factory + * @param name name: only used for assertion messages. + * @param type expected stream type. + * @param clazz expected class. + * @param class to expect + */ + private static void assertFactorySatisfies( + final ObjectInputStreamFactory factory, + final String name, + final InputStreamType type, + final Class clazz) { + assertThat(factory) + .describedAs("Factory for stream %s", name) + .isInstanceOf(clazz) + .satisfies(f -> + assertThat(factory.streamType()).isEqualTo(type)); + } + + /** + * When an unknown stream type is passed in, it is rejected. + */ + @Test + public void testUnknownStreamType() throws Throwable { + final String name = "unknown"; + intercept(IllegalArgumentException.class, E_INVALID_STREAM_TYPE, + () -> factory(name)); + } + + /** + * Create a factory, using the given name as the configuration option. + * @param name stream name. + * @return the factory + */ + private static ObjectInputStreamFactory factory(final String name) { + return factoryFromConfig(configWithStream(name)); + } + + /** + * Create a configuration with the given name declared as the input + * stream. + * @param name stream name. + * @return the prepared configuration. + */ + private static Configuration configWithStream(final String name) { + final Configuration conf = new Configuration(false); + conf.set(INPUT_STREAM_TYPE, name); + return conf; + } + + /** + * Custom factory loading: the good path. + */ + @Test + public void testCustomFactoryLoad() throws Throwable { + final Configuration conf = configWithStream(INPUT_STREAM_TYPE_CUSTOM); + conf.set(INPUT_STREAM_CUSTOM_FACTORY, CustomFactory.class.getName()); + final ObjectInputStreamFactory factory = factoryFromConfig(conf); + assertThat(factory.streamType()) + .isEqualTo(InputStreamType.Custom); + assertThat(factory) + .isInstanceOf(CustomFactory.class); + } + + /** + * A custom factory must have a classname. + */ + @Test + public void testCustomFactoryUndefined() throws Throwable { + intercept(IllegalArgumentException.class, E_EMPTY_CUSTOM_CLASSNAME, + () -> factory(INPUT_STREAM_TYPE_CUSTOM)); + } + + /** + * Constructor failures are passed in, deeply wrapped though. + */ + @Test + public void testCustomConstructorFailure() throws Throwable { + final Configuration conf = configWithStream(INPUT_STREAM_TYPE_CUSTOM); + conf.set(INPUT_STREAM_CUSTOM_FACTORY, FactoryFailsToInstantiate.class.getName()); + final RuntimeException ex = + intercept(RuntimeException.class, "InvocationTargetException", + () -> factoryFromConfig(conf)); + assertThat(ex.getCause().getCause()) + .describedAs("innermost exception") + .isInstanceOf(UncheckedIOException.class); + } + + /** + * Simple factory. + */ + public static class CustomFactory extends AbstractObjectInputStreamFactory { + + public CustomFactory() { + super("custom"); + } + + @Override + public InputStreamType streamType() { + return InputStreamType.Custom; + } + + @Override + public ObjectInputStream readObject(final ObjectReadParameters parameters) throws IOException { + return null; + } + + @Override + public StreamFactoryRequirements factoryRequirements() { + return null; + } + } + + /** + * Factory which raises an exception during construction. + */ + public static final class FactoryFailsToInstantiate extends CustomFactory { + + public FactoryFailsToInstantiate() { + throw new UncheckedIOException("failed to instantiate", new IOException()); + } + + } + + /** + * Callbacks from {@link ObjectInputStreamFactory} instances. + */ + private static final class Callbacks implements ObjectInputStreamFactory.StreamFactoryCallbacks { + + @Override + public S3AsyncClient getOrCreateAsyncClient(final boolean requireCRT) throws IOException { + throw new UnsupportedOperationException("not implemented"); + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java index fdafce3c2eb6f..75689211ba8e2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java @@ -40,6 +40,7 @@ import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType; import org.apache.hadoop.fs.statistics.IOStatistics; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FOOTER_CACHE; @@ -52,12 +53,7 @@ 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.Constants.PREFETCH_ENABLED_DEFAULT; -import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; -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.S3ATestUtils.*; 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; @@ -110,6 +106,10 @@ public Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); + // Analytics accelerator currently does not support IOStatistics, this will be added as + // part of https://issues.apache.org/jira/browse/HADOOP-19364 + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Analytics Accelerator currently does not support stream statistics"); S3AFileSystem fs = getFileSystem(); testFile = methodPath(); @@ -388,7 +388,6 @@ public void testPositionedReadableReadPastEOF() throws Throwable { describe("PositionedReadable.read() past the end of the file"); assumeNoPrefetching(); - verifyMetrics(() -> { try (FSDataInputStream in = openFile(longLen, FS_OPTION_OPENFILE_READ_POLICY_RANDOM)) { @@ -452,8 +451,7 @@ public void testVectorReadPastEOF() throws Throwable { * @return true if the fs has prefetching enabled. */ private boolean prefetching() { - return getFileSystem().getConf().getBoolean( - PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT); + return InputStreamType.Prefetch == streamType(getFileSystem()); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestUnbufferDraining.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestUnbufferDraining.java index be210003da0d8..e68ea9a031521 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestUnbufferDraining.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestUnbufferDraining.java @@ -48,11 +48,11 @@ import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS; import static org.apache.hadoop.fs.s3a.Constants.MAX_ERROR_RETRIES; -import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE; import static org.apache.hadoop.fs.s3a.Constants.REQUEST_TIMEOUT; import static org.apache.hadoop.fs.s3a.Constants.RETRY_LIMIT; import static org.apache.hadoop.fs.s3a.Constants.SOCKET_TIMEOUT; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disablePrefetching; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import static org.apache.hadoop.fs.s3a.impl.ConfigurationHelper.setDurationAsSeconds; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; @@ -98,7 +98,7 @@ public class ITestUnbufferDraining extends AbstractS3ACostTest { @Override public Configuration createConfiguration() { - Configuration conf = super.createConfiguration(); + Configuration conf = disablePrefetching(super.createConfiguration()); removeBaseAndBucketOverrides(conf, ASYNC_DRAIN_THRESHOLD, CHECKSUM_VALIDATION, @@ -106,7 +106,6 @@ public Configuration createConfiguration() { INPUT_FADVISE, MAX_ERROR_RETRIES, MAXIMUM_CONNECTIONS, - PREFETCH_ENABLED_KEY, READAHEAD_RANGE, REQUEST_TIMEOUT, RETRY_LIMIT, @@ -118,7 +117,6 @@ public Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); - // now create a new FS with minimal http capacity and recovery // a separate one is used to avoid test teardown suffering // from the lack of http connections and short timeouts. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java index 5fbbc3a127997..63b80cff97906 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java @@ -29,8 +29,8 @@ import software.amazon.awssdk.services.s3.model.GetObjectResponse; import org.apache.hadoop.fs.impl.prefetch.Validate; -import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; import org.apache.hadoop.util.functional.CallableRaisingIOE; /** @@ -54,7 +54,7 @@ class MockS3ARemoteObject extends S3ARemoteObject { MockS3ARemoteObject(int size, boolean throwExceptionOnOpen) { super( - S3APrefetchFakes.createReadContext(null, KEY, size, 1, 1), + S3APrefetchFakes.createReadContext(null, KEY, size), S3APrefetchFakes.createObjectAttributes(BUCKET, KEY, size), S3APrefetchFakes.createInputStreamCallbacks(BUCKET), EmptyS3AStatisticsContext.EMPTY_INPUT_STREAM_STATISTICS, @@ -95,8 +95,8 @@ public static byte byteAtOffset(int offset) { return (byte) (offset % 128); } - public static S3AInputStream.InputStreamCallbacks createClient(String bucketName) { - return new S3AInputStream.InputStreamCallbacks() { + public static ObjectInputStreamCallbacks createClient(String bucketName) { + return new ObjectInputStreamCallbacks() { @Override public ResponseInputStream getObject( GetObjectRequest request) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java index 1c509702188f3..ac42f7ba095bc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java @@ -52,7 +52,6 @@ import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AInputPolicy; -import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.S3AReadOpContext; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; import org.apache.hadoop.fs.s3a.VectoredIOContext; @@ -62,6 +61,7 @@ import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.s3a.statistics.impl.CountingChangeTracker; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryPolicy; @@ -133,9 +133,7 @@ public static S3ObjectAttributes createObjectAttributes( public static S3AReadOpContext createReadContext( ExecutorServiceFuturePool futurePool, String key, - int fileSize, - int prefetchBlockSize, - int prefetchBlockCount) { + int fileSize) { S3AFileStatus fileStatus = createFileStatus(key, fileSize); org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key); @@ -156,9 +154,8 @@ public static S3AReadOpContext createReadContext( .setMaxReadSizeForVectoredReads(1) .build(), emptyStatisticsStore(), - futurePool, - prefetchBlockSize, - prefetchBlockCount) + futurePool + ) .withChangeDetectionPolicy( ChangeDetectionPolicy.createPolicy(ChangeDetectionPolicy.Mode.None, ChangeDetectionPolicy.Source.ETag, false)) @@ -187,7 +184,7 @@ public static ResponseInputStream createS3ObjectInputStream( AbortableInputStream.create(new ByteArrayInputStream(buffer), () -> {})); } - public static S3AInputStream.InputStreamCallbacks createInputStreamCallbacks( + public static ObjectInputStreamCallbacks createInputStreamCallbacks( String bucket) { GetObjectResponse objectResponse = GetObjectResponse.builder() @@ -197,7 +194,7 @@ public static S3AInputStream.InputStreamCallbacks createInputStreamCallbacks( ResponseInputStream responseInputStream = createS3ObjectInputStream(objectResponse, new byte[8]); - return new S3AInputStream.InputStreamCallbacks() { + return new ObjectInputStreamCallbacks() { @Override public ResponseInputStream getObject(GetObjectRequest request) { return responseInputStream; @@ -234,21 +231,25 @@ public static S3ARemoteInputStream createInputStream( S3AReadOpContext s3AReadOpContext = createReadContext( futurePool, key, - fileSize, - prefetchBlockSize, - prefetchBlockCount); + fileSize + ); - S3AInputStream.InputStreamCallbacks callbacks = + ObjectInputStreamCallbacks callbacks = createInputStreamCallbacks(bucket); S3AInputStreamStatistics stats = s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics(); + final PrefetchOptions options = + new PrefetchOptions(prefetchBlockSize, prefetchBlockCount); if (clazz == FakeS3AInMemoryInputStream.class) { - return new FakeS3AInMemoryInputStream(s3AReadOpContext, + return new FakeS3AInMemoryInputStream(s3AReadOpContext, options, s3ObjectAttributes, callbacks, stats); } else if (clazz == FakeS3ACachingInputStream.class) { - return new FakeS3ACachingInputStream(s3AReadOpContext, s3ObjectAttributes, - callbacks, stats); + return new FakeS3ACachingInputStream(s3AReadOpContext, + options, + s3ObjectAttributes, + callbacks, + stats); } throw new RuntimeException("Unsupported class: " + clazz); @@ -288,10 +289,11 @@ public static class FakeS3AInMemoryInputStream public FakeS3AInMemoryInputStream( S3AReadOpContext context, + PrefetchOptions prefetchOptions, S3ObjectAttributes s3Attributes, - S3AInputStream.InputStreamCallbacks client, + ObjectInputStreamCallbacks client, S3AInputStreamStatistics streamStatistics) { - super(context, s3Attributes, client, streamStatistics); + super(context, prefetchOptions, s3Attributes, client, streamStatistics); } @Override @@ -390,10 +392,11 @@ public static class FakeS3ACachingInputStream extends S3ACachingInputStream { public FakeS3ACachingInputStream( S3AReadOpContext context, + PrefetchOptions prefetchOptions, S3ObjectAttributes s3Attributes, - S3AInputStream.InputStreamCallbacks client, + ObjectInputStreamCallbacks client, S3AInputStreamStatistics streamStatistics) { - super(context, s3Attributes, client, streamStatistics, CONF, + super(context, prefetchOptions, s3Attributes, client, streamStatistics, CONF, new LocalDirAllocator( CONF.get(BUFFER_DIR) != null ? BUFFER_DIR : HADOOP_TMP_DIR)); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java index 8ce26033c1182..4d94a58b2c060 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java @@ -31,11 +31,11 @@ import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.fs.impl.prefetch.ExceptionAsserts; import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool; -import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.S3AReadOpContext; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; import org.apache.hadoop.test.AbstractHadoopTestBase; import static org.assertj.core.api.Assertions.assertThat; @@ -53,13 +53,14 @@ public class TestS3ARemoteInputStream extends AbstractHadoopTestBase { private final ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(threadPool); - private final S3AInputStream.InputStreamCallbacks client = + private final ObjectInputStreamCallbacks client = MockS3ARemoteObject.createClient("bucket"); @Test public void testArgChecks() throws Exception { S3AReadOpContext readContext = - S3APrefetchFakes.createReadContext(futurePool, "key", 10, 10, 1); + S3APrefetchFakes.createReadContext(futurePool, "key", 10); + PrefetchOptions prefetchOptions = new PrefetchOptions(10, 1); S3ObjectAttributes attrs = S3APrefetchFakes.createObjectAttributes("bucket", "key", 10); S3AInputStreamStatistics stats = @@ -67,23 +68,25 @@ public void testArgChecks() throws Exception { Configuration conf = S3ATestUtils.prepareTestConfiguration(new Configuration()); // Should not throw. - new S3ACachingInputStream(readContext, attrs, client, stats, conf, null); + new S3ACachingInputStream(readContext, prefetchOptions, attrs, client, stats, conf, null); ExceptionAsserts.assertThrows( NullPointerException.class, - () -> new S3ACachingInputStream(null, attrs, client, stats, conf, null)); + () -> new S3ACachingInputStream(null, null, attrs, client, stats, conf, null)); ExceptionAsserts.assertThrows( NullPointerException.class, - () -> new S3ACachingInputStream(readContext, null, client, stats, conf, null)); + () -> new S3ACachingInputStream(readContext, null, null, client, stats, conf, null)); ExceptionAsserts.assertThrows( NullPointerException.class, - () -> new S3ACachingInputStream(readContext, attrs, null, stats, conf, null)); + () -> new S3ACachingInputStream(readContext, prefetchOptions, attrs, null, stats, conf, + null)); ExceptionAsserts.assertThrows( NullPointerException.class, - () -> new S3ACachingInputStream(readContext, attrs, client, null, conf, null)); + () -> new S3ACachingInputStream(readContext, prefetchOptions, attrs, client, null, conf, + null)); } @Test diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java index b3788aac80834..9e1e979eeced4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java @@ -26,11 +26,11 @@ import org.apache.hadoop.fs.impl.prefetch.ExceptionAsserts; import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool; -import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.S3AReadOpContext; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; import org.apache.hadoop.fs.s3a.impl.ChangeTracker; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; import org.apache.hadoop.test.AbstractHadoopTestBase; public class TestS3ARemoteObject extends AbstractHadoopTestBase { @@ -40,13 +40,13 @@ public class TestS3ARemoteObject extends AbstractHadoopTestBase { private final ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(threadPool); - private final S3AInputStream.InputStreamCallbacks client = + private final ObjectInputStreamCallbacks client = MockS3ARemoteObject.createClient("bucket"); @Test public void testArgChecks() throws Exception { S3AReadOpContext readContext = - S3APrefetchFakes.createReadContext(futurePool, "key", 10, 10, 1); + S3APrefetchFakes.createReadContext(futurePool, "key", 10); S3ObjectAttributes attrs = S3APrefetchFakes.createObjectAttributes("bucket", "key", 10); S3AInputStreamStatistics stats = diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesSSECDiskBlocks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesSSECDiskBlocks.java index 6f19ba15c1c9a..0203b00caab69 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesSSECDiskBlocks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesSSECDiskBlocks.java @@ -31,6 +31,7 @@ import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM; import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_KEY; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfEncryptionTestsDisabled; /** @@ -54,6 +55,8 @@ public class ITestS3AHugeFilesSSECDiskBlocks public void setup() throws Exception { try { super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Analytics Accelerator currently does not support SSE-C"); } catch (AccessDeniedException | AWSUnsupportedFeatureException e) { skip("Bucket does not allow " + S3AEncryptionMethods.SSE_C + " encryption method"); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java index a787f52bd4d40..e0c71136e8023 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java @@ -58,6 +58,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.*; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disablePrefetching; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getInputStreamStatistics; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getS3AInputStream; import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.isUsingDefaultExternalDataFile; @@ -72,6 +73,7 @@ import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN; +import static org.apache.hadoop.io.Sizes.*; import static org.apache.hadoop.util.functional.FutureIO.awaitFuture; /** @@ -80,15 +82,15 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase { private static final Logger LOG = LoggerFactory.getLogger( ITestS3AInputStreamPerformance.class); - private static final int READAHEAD_128K = 128 * _1KB; + private static final int READAHEAD_128K = S_128K; private S3AFileSystem s3aFS; private Path testData; private FileStatus testDataStatus; private FSDataInputStream in; private S3AInputStreamStatistics streamStatistics; - public static final int BLOCK_SIZE = 32 * 1024; - public static final int BIG_BLOCK_SIZE = 256 * 1024; + public static final int BLOCK_SIZE = S_32K; + public static final int BIG_BLOCK_SIZE = S_256K; private static final IOStatisticsSnapshot IOSTATS = snapshotIOStatistics(); @@ -99,15 +101,12 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase { @Override protected Configuration createScaleConfiguration() { - Configuration conf = super.createScaleConfiguration(); - S3ATestUtils.removeBaseAndBucketOverrides(conf, - PREFETCH_ENABLED_KEY); + Configuration conf = disablePrefetching(super.createScaleConfiguration()); if (isUsingDefaultExternalDataFile(conf)) { S3ATestUtils.removeBaseAndBucketOverrides( conf, ENDPOINT); } - conf.setBoolean(PREFETCH_ENABLED_KEY, false); return conf; } @@ -118,8 +117,8 @@ protected Configuration createScaleConfiguration() { @Before public void openFS() throws IOException { Configuration conf = getConf(); - conf.setInt(SOCKET_SEND_BUFFER, 16 * 1024); - conf.setInt(SOCKET_RECV_BUFFER, 16 * 1024); + conf.setInt(SOCKET_SEND_BUFFER, S_16K); + conf.setInt(SOCKET_RECV_BUFFER, S_16K); // look up the test file, no requirement to be set. String testFile = conf.getTrimmed(KEY_CSVTEST_FILE, PublicDatasetTestUtils.DEFAULT_EXTERNAL_FILE); @@ -283,7 +282,7 @@ public void testTimeToOpenAndReadWholeFileBlocks() throws Throwable { // implicitly rounding down here long blockCount = len / blockSize; long totalToRead = blockCount * blockSize; - long minimumBandwidth = 128 * 1024; + long minimumBandwidth = S_128K; int maxResetCount = 4; int resetCount = 0; for (long i = 0; i < blockCount; i++) { @@ -473,22 +472,10 @@ protected void executeSeekReadSequence(long blockSize, logStreamStatistics(); } - public static final int _4K = 4 * 1024; - public static final int _8K = 8 * 1024; - public static final int _16K = 16 * 1024; - public static final int _32K = 32 * 1024; - public static final int _64K = 64 * 1024; - public static final int _128K = 128 * 1024; - public static final int _256K = 256 * 1024; - public static final int _1MB = 1024 * 1024; - public static final int _2MB = 2 * _1MB; - public static final int _10MB = _1MB * 10; - public static final int _5MB = _1MB * 5; - private static final int[][] RANDOM_IO_SEQUENCE = { - {_2MB, _128K}, - {_128K, _128K}, - {_5MB, _64K}, + {S_2M, S_128K}, + {S_128K, S_128K}, + {S_5M, S_64K}, {_1MB, _1MB}, }; @@ -536,7 +523,7 @@ private ContractTestUtils.NanoTimer executeRandomIO(S3AInputPolicy policy, long expectedOpenCount) throws IOException { describe("Random IO with policy \"%s\"", policy); - byte[] buffer = new byte[_1MB]; + byte[] buffer = new byte[S_1M]; long totalBytesRead = 0; final long len = testDataStatus.getLen(); in = openTestFile(policy, 0); @@ -588,15 +575,15 @@ S3AInputStream getS3aStream() { public void testRandomReadOverBuffer() throws Throwable { describe("read over a buffer, making sure that the requests" + " spans readahead ranges"); - int datasetLen = _32K; + int datasetLen = S_32K; S3AFileSystem fs = getFileSystem(); Path dataFile = path("testReadOverBuffer.bin"); byte[] sourceData = dataset(datasetLen, 0, 64); // relies on the field 'fs' referring to the R/W FS - writeDataset(fs, dataFile, sourceData, datasetLen, _16K, true); + writeDataset(fs, dataFile, sourceData, datasetLen, S_16K, true); byte[] buffer = new byte[datasetLen]; - int readahead = _8K; - int halfReadahead = _4K; + int readahead = S_8K; + int halfReadahead = S_4K; in = openDataFile(fs, dataFile, S3AInputPolicy.Random, readahead, datasetLen); LOG.info("Starting initial reads"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java index 0f6b69cd54d89..3c7644f23230b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.contract.s3a.S3AContract; import org.apache.hadoop.fs.statistics.StreamStatisticNames; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.statistics.StreamStatisticNames.*; /** @@ -78,4 +79,12 @@ public List outputStreamStatisticKeys() { STREAM_WRITE_EXCEPTIONS); } + @Override + public void testInputStreamStatisticRead() throws Throwable { + // Analytics accelerator currently does not support IOStatistics, this will be added as + // part of https://issues.apache.org/jira/browse/HADOOP-19364 + skipIfAnalyticsAcceleratorEnabled(createConfiguration(), + "Analytics Accelerator currently does not support stream statistics"); + super.testInputStreamStatisticRead(); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java index 0d5d2a789a02a..8cb0de1b83fca 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java @@ -31,6 +31,8 @@ import org.apache.hadoop.fs.statistics.IOStatisticAssertions; import org.apache.hadoop.fs.statistics.StreamStatisticNames; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; + public class ITestS3AFileSystemStatistic extends AbstractS3ATestBase { private static final int ONE_KB = 1024; @@ -42,6 +44,10 @@ public class ITestS3AFileSystemStatistic extends AbstractS3ATestBase { */ @Test public void testBytesReadWithStream() throws IOException { + // Analytics accelerator currently does not support IOStatistics, this will be added as + // part of https://issues.apache.org/jira/browse/HADOOP-19364 + skipIfAnalyticsAcceleratorEnabled(createConfiguration(), + "Analytics Accelerator currently does not support stream statistics"); S3AFileSystem fs = getFileSystem(); Path filePath = path(getMethodName()); byte[] oneKbBuf = new byte[ONE_KB]; diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties index 7442a357f9777..7b8dd3c11fcdc 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties @@ -98,3 +98,7 @@ log4j.logger.org.apache.hadoop.fs.s3a.S3AStorageStatistics=INFO # uncomment this to get S3 Delete requests to return the list of deleted objects # log4.logger.org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl=TRACE +# debug service lifecycle of components such as S3AStore and +# services it launches itself. +# log4.logger.org.apache.hadoop.service=DEBUG + diff --git a/hadoop-tools/hadoop-aws/src/test/resources/malformed_footer.parquet b/hadoop-tools/hadoop-aws/src/test/resources/malformed_footer.parquet new file mode 100644 index 0000000000000..614912f630bce Binary files /dev/null and b/hadoop-tools/hadoop-aws/src/test/resources/malformed_footer.parquet differ diff --git a/hadoop-tools/hadoop-aws/src/test/resources/multi_row_group.parquet b/hadoop-tools/hadoop-aws/src/test/resources/multi_row_group.parquet new file mode 100644 index 0000000000000..b8d6a95cd4418 Binary files /dev/null and b/hadoop-tools/hadoop-aws/src/test/resources/multi_row_group.parquet differ