From 9c8e753209dfec18005fe61af229e7fb9402312e Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 6 Dec 2024 18:37:40 +0000 Subject: [PATCH 1/5] HADOOP-19354. S3A: Create S3 InputStreams via a factory S3 InputStreams are created by a factory class, with the choice of factory dynamically chosen by the option fs.s3a.input.stream.type Supported values: classic, prefetching, analytics. S3AStore * Manages the creation and service lifecycle of the chosen factory, as well as forwarding stream construction requests to the chosen factory. * Provides the callbacks needed by both the factories and input streams. * StreamCapabilities.hasCapability(), which is relayed to the active factory. This avoids the FS having to know what capabilities are available in the stream. --- .../dev-support/findbugs-exclude.xml | 2 +- .../org/apache/hadoop/fs/s3a/Constants.java | 30 ++ .../apache/hadoop/fs/s3a/S3AFileSystem.java | 283 +++++--------- .../apache/hadoop/fs/s3a/S3AInputStream.java | 367 +++++------------- .../hadoop/fs/s3a/S3AReadOpContext.java | 44 +-- .../org/apache/hadoop/fs/s3a/S3AStore.java | 38 +- .../hadoop/fs/s3a/impl/ClientManager.java | 9 +- .../hadoop/fs/s3a/impl/ClientManagerImpl.java | 56 +-- .../fs/s3a/impl/InputStreamCallbacksImpl.java | 123 ++++++ .../hadoop/fs/s3a/impl/S3AStoreImpl.java | 203 +++++++++- .../AbstractObjectInputStreamFactory.java | 83 ++++ .../ClassicObjectInputStreamFactory.java | 66 ++++ .../fs/s3a/impl/streams/InputStreamType.java | 106 +++++ .../s3a/impl/streams/ObjectInputStream.java | 330 ++++++++++++++++ .../streams/ObjectInputStreamCallbacks.java | 64 +++ .../streams/ObjectInputStreamFactory.java | 82 ++++ .../impl/streams/ObjectReadParameters.java | 186 +++++++++ .../s3a/impl/streams/StreamIntegration.java | 96 +++++ .../s3a/impl/streams/StreamThreadOptions.java | 77 ++++ .../fs/s3a/impl/streams/package-info.java | 29 ++ .../fs/s3a/prefetch/PrefetchOptions.java | 67 ++++ .../PrefetchingInputStreamFactory.java | 91 +++++ .../s3a/prefetch/S3ACachingInputStream.java | 16 +- .../s3a/prefetch/S3AInMemoryInputStream.java | 7 +- .../prefetch/S3APrefetchingInputStream.java | 67 ++-- .../fs/s3a/prefetch/S3ARemoteInputStream.java | 14 +- .../fs/s3a/prefetch/S3ARemoteObject.java | 5 +- .../markdown/tools/hadoop-aws/prefetching.md | 14 +- .../fs/contract/s3a/ITestS3AContractSeek.java | 2 +- .../hadoop/fs/s3a/ITestS3AConfiguration.java | 18 +- .../fs/s3a/ITestS3AInputStreamLeakage.java | 44 ++- .../fs/s3a/ITestS3APrefetchingCacheFiles.java | 5 +- .../s3a/ITestS3APrefetchingInputStream.java | 6 +- .../s3a/ITestS3APrefetchingLruEviction.java | 11 +- .../hadoop/fs/s3a/ITestS3ARequesterPays.java | 5 +- .../apache/hadoop/fs/s3a/S3ATestUtils.java | 33 +- .../fs/s3a/TestS3AInputStreamRetry.java | 27 +- .../fs/s3a/impl/ITestConnectionTimeouts.java | 7 +- .../fs/s3a/performance/ITestS3AOpenCost.java | 7 +- .../performance/ITestUnbufferDraining.java | 5 +- .../fs/s3a/prefetch/MockS3ARemoteObject.java | 8 +- .../fs/s3a/prefetch/S3APrefetchFakes.java | 43 +- .../prefetch/TestS3ARemoteInputStream.java | 19 +- .../fs/s3a/prefetch/TestS3ARemoteObject.java | 6 +- .../scale/ITestS3AInputStreamPerformance.java | 47 +-- .../src/test/resources/log4j.properties | 4 + 46 files changed, 2146 insertions(+), 706 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InputStreamCallbacksImpl.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AbstractObjectInputStreamFactory.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ClassicObjectInputStreamFactory.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/InputStreamType.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStream.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamCallbacks.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamFactory.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectReadParameters.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamIntegration.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamThreadOptions.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/package-info.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchOptions.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchingInputStreamFactory.java 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/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..ad902ad404f13 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,8 @@ 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.InputStreamType; +import org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import java.time.Duration; @@ -1580,6 +1582,34 @@ 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: "prefetch". + */ + public static final String INPUT_STREAM_TYPE_PREFETCH = StreamIntegration.PREFETCH; + + /** + * The analytics input stream: "analytics". + */ + public static final String INPUT_STREAM_TYPE_ANALYTICS = + StreamIntegration.ANALYTICS; + + /** + * The default input stream. + * Currently {@link #INPUT_STREAM_TYPE_CLASSIC} + */ + public static final String INPUT_STREAM_TYPE_DEFAULT = InputStreamType.DEFAULT_STREAM_TYPE.getName(); + /** * Controls whether the prefetching input stream is enabled. */ 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..d8a1c49f056f3 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; @@ -125,6 +122,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 +146,9 @@ 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.ObjectReadParameters; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; +import org.apache.hadoop.fs.s3a.impl.streams.StreamThreadOptions; 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 +157,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 +300,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private String username; - /** - * Store back end. - */ private S3AStore store; /** @@ -334,18 +330,11 @@ 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. + */ 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 +342,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; /** @@ -661,22 +649,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 " + @@ -788,12 +765,16 @@ 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 + // 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(); + + // thread pool init requires store to be created + initThreadPools(); + // The filesystem is now ready to perform operations against // S3 // This initiates a probe against S3 for the bucket existing. @@ -836,7 +817,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 +826,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,6 +838,9 @@ protected S3AStore createS3AStore(final ClientManager clientManager, .withReadRateLimiter(unlimitedRate()) .withWriteRateLimiter(RateLimitingFactory.create(rateLimitCapacity)) .build(); + st.init(getConf()); + st.start(); + return st; } /** @@ -939,12 +923,18 @@ 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 +950,9 @@ private void initThreadPools(Configuration conf) { TimeUnit.SECONDS, Duration.ZERO).getSeconds(); - int numPrefetchThreads = this.prefetchEnabled ? this.prefetchBlockCount : 0; + final StreamThreadOptions threadRequirements = + getStore().threadRequirements(); + int numPrefetchThreads = threadRequirements.sharedThreads(); int activeTasksForBoundedThreadPool = maxThreads; int waitingTasksForBoundedThreadPool = maxThreads + totalTasks + numPrefetchThreads; @@ -978,7 +970,8 @@ private void initThreadPools(Configuration conf) { unboundedThreadPool.allowCoreThreadTimeOut(true); executorCapacity = intOption(conf, EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1); - if (prefetchEnabled) { + if (threadRequirements.createFuturePool()) { + // create a future pool. final S3AInputStreamStatistics s3AInputStreamStatistics = statisticsContext.newInputStreamStatistics(); futurePool = new ExecutorServiceFuturePool( @@ -1329,6 +1322,15 @@ 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 +1536,7 @@ public S3Client getAmazonS3Client(String reason) { @Override public S3AStore getStore() { - return store; + return S3AFileSystem.this.getStore(); } /** @@ -1663,28 +1665,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 +1859,43 @@ 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 StreamThreadOptions requirements = + getStore().threadRequirements(); + + // calculate the permit count. + final int permitCount = requirements.streamThreads() + + (requirements.vectorSupported() + ? vectoredActiveRangeReads + : 0); + // 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 callbacks, + */ + 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 +1907,7 @@ private final class WriteOperationHelperCallbacksImpl @Retries.OnceRaw public CompleteMultipartUploadResponse completeMultipartUpload( CompleteMultipartUploadRequest request) { - return store.completeMultipartUpload(request); + return getStore().completeMultipartUpload(request); } @Override @@ -1992,7 +1917,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 +1941,8 @@ protected S3AReadOpContext createReadContext( fileStatus, vectoredIOContext, IOStatisticsContext.getCurrentIOStatisticsContext().getAggregator(), - futurePool, - prefetchBlockSize, - prefetchBlockCount) + futurePool + ) .withAuditSpan(auditSpan); openFileHelper.applyDefaultOptions(roc); return roc.build(); @@ -2756,7 +2680,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 +2911,7 @@ protected DurationTrackerFactory getDurationTrackerFactory() { */ protected DurationTrackerFactory nonNullDurationTrackerFactory( DurationTrackerFactory factory) { - return store.nonNullDurationTrackerFactory(factory); + return getStore().nonNullDurationTrackerFactory(factory); } /** @@ -3025,7 +2949,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 +3097,7 @@ public void incrementWriteOperations() { protected void deleteObject(String key) throws SdkException, IOException { incrementWriteOperations(); - store.deleteObject(getRequestFactory() + getStore().deleteObject(getRequestFactory() .newDeleteObjectRequestBuilder(key) .build()); } @@ -3227,7 +3151,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 +3194,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 +3293,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 +3304,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 +3315,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 +4183,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 +4278,25 @@ 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 +4502,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 +5361,17 @@ 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 input stream capabilities it offers + if (getStore() != null && getStore().hasCapability(capability)) { + return true; + } + // fall through } @@ -5703,7 +5632,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..e7ef62a8b4e3b 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,30 +138,12 @@ 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. */ @@ -193,96 +169,33 @@ 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(); + this.vectoredIOContext = this.getContext().getVectoredIOContext(); } /** @@ -290,7 +203,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 +212,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 +224,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 +250,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 +291,7 @@ public synchronized void seek(long targetPos) throws IOException { + " " + targetPos); } - if (this.contentLength <= 0) { + if (this.getContentLength() <= 0) { return; } @@ -414,7 +309,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 +344,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 +359,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 +408,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 +427,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 +437,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 +449,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 +492,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 +533,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 +544,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 +580,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 +591,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 +612,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 +657,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 +673,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 +698,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 +720,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,17 +760,17 @@ 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(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=") @@ -920,7 +801,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; } @@ -971,10 +852,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 +873,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 +905,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 +913,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 +924,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 +1012,7 @@ private void drainUnnecessaryData( remaining -= readCount; } } finally { - streamStatistics.readVectoredBytesDiscarded(drainBytes); + getS3AStreamStatistics().readVectoredBytesDiscarded(drainBytes); LOG.debug("{} bytes drained from stream ", drainBytes); } } @@ -1140,7 +1023,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 +1038,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 +1157,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 +1195,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 +1280,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 +1291,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 +1309,6 @@ public boolean isObjectStreamOpen() { return wrappedStream != null; } - @Override - public IOStatistics getIOStatistics() { - return ioStatistics; - } - /** * Get the wrapped stream. * This is for testing only. @@ -1457,38 +1320,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/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..8655956cf7d03 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,20 @@ 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.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 +68,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 +316,26 @@ 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; } 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..2632c0820e854 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,6 +112,7 @@ 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); @@ -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/S3AStoreImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java index db07881345500..a432c92fddec5 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,24 @@ 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.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.StreamThreadOptions; 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 +111,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.createStreamFactory; 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 +181,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 +205,93 @@ 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(); + 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 = createStreamFactory(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 hasCapability(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 hasCapability(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 +905,96 @@ 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() { + // 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 FactoryCallbacks()); + } + + @Override /* ObjectInputStreamFactory */ + public ObjectInputStream readObject(ObjectReadParameters parameters) + throws IOException { + parameters.withDirectoryAllocator(getDirectoryAllocator()); + return objectInputStreamFactory.readObject(parameters.validate()); + } + + @Override /* ObjectInputStreamFactory */ + public StreamThreadOptions threadRequirements() { + return objectInputStreamFactory.threadRequirements(); + } + + /** + * This operation is not implemented, as + * is this class which invokes it on the actual factory. + * @param callbacks factory callbacks. + * @throws UnsupportedOperationException always + */ + @Override /* ObjectInputStreamFactory */ + public void bind(final StreamFactoryCallbacks callbacks) { + throw new UnsupportedOperationException("Not supported"); + } + + /** + * 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..7c20f7d66f61b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AbstractObjectInputStreamFactory.java @@ -0,0 +1,83 @@ +/* + * 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.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 { + + protected AbstractObjectInputStreamFactory(final String name) { + super(name); + } + + /** + * Callbacks. + */ + private StreamFactoryCallbacks callbacks; + + /** + * Bind to the callbacks. + *

+ * The base class checks service state then stores + * the callback interface. + * @param factoryCallbacks callbacks needed by the factories. + */ + @Override + public void bind(final StreamFactoryCallbacks factoryCallbacks) { + // must be on be invoked during service initialization + Preconditions.checkState(isInState(STATE.INITED), + "Input Stream factory %s is in wrong state: %s", + this, getServiceState()); + this.callbacks = factoryCallbacks; + } + + /** + * Return base capabilities of all stream factories, + * defined what the base ObjectInputStream class does. + * @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: + return false; + } + } + + /** + * Get the factory callbacks. + * @return callbacks. + */ + public StreamFactoryCallbacks callbacks() { + return callbacks; + } +} 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..030e9bad01a5e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ClassicObjectInputStreamFactory.java @@ -0,0 +1,66 @@ +/* + * 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.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); + } + } + + /** + * Get the number of background threads required for this factory. + * @return the count of background threads. + */ + @Override + public StreamThreadOptions threadRequirements() { + return new StreamThreadOptions(0, 0, false, true); + } + +} 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..92df0c406f7c9 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/InputStreamType.java @@ -0,0 +1,106 @@ +/* + * 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; + +/** + * 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 -> { + throw new IllegalArgumentException("not yet supported"); + }); + + /** + * 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; + } + + 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; + } + + /** + * Factory constructor. + * @return the factory associated with this stream type. + */ + public Function factory() { + return factory; + } + + /** + * What is the default type? + */ + public static final InputStreamType DEFAULT_STREAM_TYPE = Classic; + + +} 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..eccfd060e8451 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStream.java @@ -0,0 +1,330 @@ +/* + * 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.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; + + private final InputStreamType streamType; + /** + * Requested input policy. + */ + private S3AInputPolicy inputPolicy; + + /** + * Constructor. + * @param streamType stream type enum. + * @param parameters extensible parameter list. + */ + protected ObjectInputStream( + final InputStreamType streamType, + final ObjectReadParameters parameters) { + + objectAttributes = parameters.getObjectAttributes(); + this.streamType = streamType; + 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); + } + + /** + * 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: + 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; + } +} + + 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..d8fe87f9cf7fd --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamFactory.java @@ -0,0 +1,82 @@ +/* + * 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 callbacks extra initialization parameters + */ + void bind(StreamFactoryCallbacks callbacks); + + /** + * 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 the number of background threads required for this factory. + * @return the count of background threads. + */ + StreamThreadOptions threadRequirements(); + + /** + * 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/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..d64b4326fa4b0 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamIntegration.java @@ -0,0 +1,96 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.store.LogExactlyOnce; + +import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE; +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; + +/** + * 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"; + + private StreamIntegration() { + } + + /** + * 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); + + /** + * Create the input stream factory the configuration asks for. + * If the option {@code "fs.s3a.prefetch.enabled"} is set, this + * changes the default value from classic to prefetch, after printing a + * warning the first time this happens. + *

+ * This does not initialize the factory. + * @param conf configuration + * @return a stream factory. + */ + public static ObjectInputStreamFactory createStreamFactory(final Configuration conf) { + // choose the default input stream type + + // work out the default stream; this includes looking at the + // deprecated prefetch enabled key to see if it is set. + InputStreamType defaultStream = InputStreamType.DEFAULT_STREAM_TYPE; + 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); + defaultStream = InputStreamType.Prefetch; + } + + // retrieve the enum value, returning the configured value or + // the default...then instantiate it. + return conf.getEnum(INPUT_STREAM_TYPE, defaultStream) + .factory() + .apply(conf); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamThreadOptions.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamThreadOptions.java new file mode 100644 index 0000000000000..254b3976b7c2c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamThreadOptions.java @@ -0,0 +1,77 @@ +/* + * 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; + +/** + * Options for threading on this input stream. + */ +public class StreamThreadOptions { + + /** 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; + + /** + * Flag to enable creation of a future pool around the bounded thread pool. + */ + private final boolean createFuturePool; + + /** + * Is vector IO supported (so its thread requirements + * included too)? + */ + private final boolean vectorSupported; + + /** + * 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 createFuturePool Flag to enable creation of a future pool around the + * bounded thread pool. + */ + public StreamThreadOptions(final int sharedThreads, + final int streamThreads, + final boolean createFuturePool, + final boolean vectorSupported) { + this.sharedThreads = sharedThreads; + this.streamThreads = streamThreads; + this.createFuturePool = createFuturePool; + this.vectorSupported = vectorSupported; + } + + public int sharedThreads() { + return sharedThreads; + } + + public int streamThreads() { + return streamThreads; + } + + public boolean createFuturePool() { + return createFuturePool; + } + + public boolean vectorSupported() { + return vectorSupported; + } +} 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..4109580c4ce5d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchingInputStreamFactory.java @@ -0,0 +1,91 @@ +/* + * 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.impl.streams.AbstractObjectInputStreamFactory; +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.StreamThreadOptions; + +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.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 + 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); + } + + /** + * The thread count is calculated from the configuration. + * @return a positive thread count. + */ + @Override + public StreamThreadOptions threadRequirements() { + return new StreamThreadOptions(prefetchBlockCount, 0, true, false); + } + +} 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..22de5015f9686 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. @@ -60,10 +60,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/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/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/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/ITestS3AInputStreamLeakage.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AInputStreamLeakage.java index 4b871c6a197db..72c75162c9fda 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,6 +29,8 @@ 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; @@ -58,7 +60,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,10 +83,6 @@ 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 { @@ -100,7 +98,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 +106,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 +123,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 +142,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/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..5a3bb1d6404c9 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,8 @@ 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.isPrefetchingEnabled; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** @@ -80,7 +79,7 @@ public void testRequesterPaysOptionSuccess() throws Throwable { inputStream.seek(0); inputStream.readByte(); - if (conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT)) { + if (isPrefetchingEnabled(conf)) { // For S3APrefetchingInputStream, verify a call was made IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(), StreamStatisticNames.STREAM_READ_OPENED).isEqualTo(1); 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..da21fb22e2ede 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 @@ -104,6 +104,8 @@ 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.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; @@ -1780,9 +1782,36 @@ 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 void disablePrefetching(Configuration conf) { - removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY); + 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 Configuration enablePrefetching(Configuration conf) { + removeBaseAndBucketOverrides(conf, + PREFETCH_ENABLED_KEY, + INPUT_STREAM_TYPE); + conf.setEnum(INPUT_STREAM_TYPE, Prefetch); + return conf; + } + + /** + * Probe the configuration for supporting prefetching. + * @return true if the config has prefetching enabled. + */ + public static boolean isPrefetchingEnabled(Configuration conf) { + return conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT) + || conf.getEnum(INPUT_STREAM_TYPE, DEFAULT_STREAM_TYPE) == Prefetch; } /** 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/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/performance/ITestS3AOpenCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java index fdafce3c2eb6f..37b1413ed0f5f 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 @@ -52,11 +52,10 @@ 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.isPrefetchingEnabled; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES_READ_CLOSE; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_OPENED; @@ -452,8 +451,8 @@ 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 isPrefetchingEnabled(getFileSystem().getConf()); + } /** 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..caf723b95dedd 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, 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/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/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 + From eadf0ddd71d55ee2e15b5faf0795bbc801ab836c Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 30 Jan 2025 12:30:08 +0000 Subject: [PATCH 2/5] HADOOP-19354. S3AInputStream to be created by factory under S3AStore Ability to create custom streams (type = custom), which reads class from "fs.s3a.input.stream.custom.factory". This is mainly for testing, especially CNFE and similar. Unit test TestStreamFactories for this. ObjectInputStreams save and export stream type to assist these tests too, as it enables assertions on the generated stream type. Simplified that logic related to the old prefetch enabled flag If fs.s3a.prefetch.enabled is true, the prefetch stream is returned, the stream.type option is not used at all. Simpler logic, simpler docs, fewer support calls. Parameters supplied to ObjectInputStreamFactory.bind converted to a parameter object. Allows for more parameters to be added later if ever required. ObjectInputStreamFactory returns more requirements to the store/fs. For this reason StreamThreadOptions threadRequirements(); is renamed StreamFactoryRequirements factoryRequirements() VectorIO context changes * Returned in factoryRequirements() * exiting configuration reading code moved into StreamIntegration.populateVectoredIOContext() * Streams which don't have custom vector IO, e.g. prefetching can return a minimum seek range of 0. This disables range merging on the default PositionedReadable implementation, so ensures that they will only get asked for data which will be read...leaving prefetch/cache code to know exactly what is needed. Other * Draft docs. * Stream capability declares stream type & is exported through FS too. (todo: test, document, add to bucket-info) * ConfigurationHelper.resolveEnum() supercedes Configuration.getEnum() with - case independence - fallback is a supplier rather than a simple value. Change-Id: I2e59300af48042df8173de61d0b3d6139a0ae7fe --- .../hadoop/util/ConfigurationHelper.java | 36 +++ .../org/apache/hadoop/fs/s3a/Constants.java | 29 ++- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 47 +--- .../apache/hadoop/fs/s3a/S3AInputStream.java | 23 +- .../hadoop/fs/s3a/VectoredIOContext.java | 17 ++ .../hadoop/fs/s3a/impl/S3AStoreImpl.java | 25 +- .../AbstractObjectInputStreamFactory.java | 27 +- .../ClassicObjectInputStreamFactory.java | 11 +- .../streams/FactoryBindingParameters.java | 46 ++++ .../fs/s3a/impl/streams/InputStreamType.java | 37 ++- .../s3a/impl/streams/ObjectInputStream.java | 54 +++- .../streams/ObjectInputStreamFactory.java | 15 +- ...ns.java => StreamFactoryRequirements.java} | 23 +- .../s3a/impl/streams/StreamIntegration.java | 146 ++++++++++- .../PrefetchingInputStreamFactory.java | 25 +- .../site/markdown/tools/hadoop-aws/reading.md | 112 ++++++++ .../apache/hadoop/fs/s3a/S3ATestUtils.java | 2 +- .../s3a/impl/streams/TestStreamFactories.java | 240 ++++++++++++++++++ 18 files changed, 794 insertions(+), 121 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/FactoryBindingParameters.java rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/{StreamThreadOptions.java => StreamFactoryRequirements.java} (77%) create mode 100644 hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/reading.md create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/streams/TestStreamFactories.java 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..6653eee24c0de 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 + * @return an enum value + * @param enumeration type. + * @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-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 ad902ad404f13..6796c29d34874 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,7 +21,6 @@ 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.InputStreamType; import org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; @@ -1594,30 +1593,48 @@ private Constants() { StreamIntegration.CLASSIC; /** - * The prefetching input stream: "prefetch". + * The prefetching input stream: {@value}. */ public static final String INPUT_STREAM_TYPE_PREFETCH = StreamIntegration.PREFETCH; /** - * The analytics input stream: "analytics". + * The analytics input stream: {@value}. */ public static final String INPUT_STREAM_TYPE_ANALYTICS = StreamIntegration.ANALYTICS; /** - * The default input stream. - * Currently {@link #INPUT_STREAM_TYPE_CLASSIC} + * Request the default input stream, + * whatever it is for this release: {@value}. */ - public static final String INPUT_STREAM_TYPE_DEFAULT = InputStreamType.DEFAULT_STREAM_TYPE.getName(); + 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 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 d8a1c49f056f3..19047d9ad4d54 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 @@ -148,7 +148,7 @@ import org.apache.hadoop.fs.s3a.impl.CSEUtils; 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.StreamThreadOptions; +import org.apache.hadoop.fs.s3a.impl.streams.StreamFactoryRequirements; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl; import org.apache.hadoop.fs.statistics.DurationTracker; @@ -369,12 +369,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); @@ -755,9 +749,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); @@ -772,6 +763,11 @@ public void initialize(URI name, Configuration originalConf) // this is to aid mocking. s3Client = getStore().getOrCreateS3Client(); + final StreamFactoryRequirements factoryRequirements = + getStore().factoryRequirements(); + // get the vector IO context from the factory. + vectoredIOContext = factoryRequirements.vectoredIOContext(); + // thread pool init requires store to be created initThreadPools(); @@ -843,23 +839,6 @@ protected S3AStore createS3AStore(final ClientManager clientManager, return st; } - /** - * 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(); - } - /** * Test bucket existence in S3. * When the value of {@link Constants#S3A_BUCKET_PROBE} is set to 0, @@ -950,9 +929,9 @@ private void initThreadPools() { TimeUnit.SECONDS, Duration.ZERO).getSeconds(); - final StreamThreadOptions threadRequirements = - getStore().threadRequirements(); - int numPrefetchThreads = threadRequirements.sharedThreads(); + final StreamFactoryRequirements factoryRequirements = + getStore().factoryRequirements(); + int numPrefetchThreads = factoryRequirements.sharedThreads(); int activeTasksForBoundedThreadPool = maxThreads; int waitingTasksForBoundedThreadPool = maxThreads + totalTasks + numPrefetchThreads; @@ -970,7 +949,7 @@ private void initThreadPools() { unboundedThreadPool.allowCoreThreadTimeOut(true); executorCapacity = intOption(conf, EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1); - if (threadRequirements.createFuturePool()) { + if (factoryRequirements.createFuturePool()) { // create a future pool. final S3AInputStreamStatistics s3AInputStreamStatistics = statisticsContext.newInputStreamStatistics(); @@ -1860,13 +1839,13 @@ private FSDataInputStream executeOpen( LOG.debug("Opening '{}'", readContext); // what does the stream need - final StreamThreadOptions requirements = - getStore().threadRequirements(); + final StreamFactoryRequirements requirements = + getStore().factoryRequirements(); // calculate the permit count. final int permitCount = requirements.streamThreads() + (requirements.vectorSupported() - ? vectoredActiveRangeReads + ? requirements.vectoredIOContext().getVectoredActiveRangeReads() : 0); // create an executor which is a subset of the // bounded thread pool. 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 e7ef62a8b4e3b..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 @@ -146,9 +146,6 @@ public class S3AInputStream extends ObjectInputStream implements CanSetReadahead 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. @@ -195,7 +192,6 @@ public S3AInputStream(ObjectReadParameters parameters) { getObjectAttributes()); setReadahead(context.getReadahead()); this.asyncDrainThreshold = context.getAsyncDrainThreshold(); - this.vectoredIOContext = this.getContext().getVectoredIOContext(); } /** @@ -764,6 +760,7 @@ public String toString() { synchronized (this) { final StringBuilder sb = new StringBuilder( "S3AInputStream{"); + sb.append(super.toString()).append(" "); sb.append(getUri()); sb.append(" wrappedStream=") .append(isObjectStreamOpen() ? "open" : "closed"); @@ -776,7 +773,7 @@ public String toString() { 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(); @@ -826,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. 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..a5729d77f7c8b 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 @@ -40,6 +40,12 @@ public class VectoredIOContext { */ private int maxReadSizeForVectorReads; + /** + * Maximum number of active range read operation a single + * input stream can have. + */ + private int vectoredActiveRangeReads; + /** * Default no arg constructor. */ @@ -68,11 +74,22 @@ public int getMaxReadSizeForVectorReads() { return maxReadSizeForVectorReads; } + public int getVectoredActiveRangeReads() { + return vectoredActiveRangeReads; + } + + public VectoredIOContext setVectoredActiveRangeReads( + final int vectoredActiveRangeReads) { + this.vectoredActiveRangeReads = vectoredActiveRangeReads; + 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/impl/S3AStoreImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java index a432c92fddec5..d284a098bcd3f 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 @@ -73,10 +73,12 @@ 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.StreamThreadOptions; +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; @@ -111,7 +113,7 @@ 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.createStreamFactory; +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; @@ -233,7 +235,7 @@ protected void serviceInit(final Configuration conf) throws Exception { // create and register the stream factory, which will // then follow the service lifecycle - objectInputStreamFactory = createStreamFactory(conf); + objectInputStreamFactory = factoryFromConfig(conf); addService(objectInputStreamFactory); // init all child services, including the stream factory @@ -269,7 +271,6 @@ public boolean hasPathCapability(final Path path, final String capability) { } } - /** * Return the capabilities of input streams created * through the store. @@ -955,7 +956,7 @@ private void finishStreamFactoryInit() { "Client Manager is in wrong state: %s", clientManager.getServiceState()); // finish initialization and pass down callbacks to self - objectInputStreamFactory.bind(new FactoryCallbacks()); + objectInputStreamFactory.bind(new FactoryBindingParameters(new FactoryCallbacks())); } @Override /* ObjectInputStreamFactory */ @@ -966,21 +967,25 @@ public ObjectInputStream readObject(ObjectReadParameters parameters) } @Override /* ObjectInputStreamFactory */ - public StreamThreadOptions threadRequirements() { - return objectInputStreamFactory.threadRequirements(); + public StreamFactoryRequirements factoryRequirements() { + return objectInputStreamFactory.factoryRequirements(); } /** * This operation is not implemented, as * is this class which invokes it on the actual factory. - * @param callbacks factory callbacks. - * @throws UnsupportedOperationException always + * @param factoryBindingParameters@throws UnsupportedOperationException always */ @Override /* ObjectInputStreamFactory */ - public void bind(final StreamFactoryCallbacks callbacks) { + public void bind(final FactoryBindingParameters factoryBindingParameters) { throw new UnsupportedOperationException("Not supported"); } + @Override /* ObjectInputStreamFactory */ + public InputStreamType streamType() { + return objectInputStreamFactory.streamType(); + } + /** * Callbacks from {@link ObjectInputStreamFactory} instances. */ 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 index 7c20f7d66f61b..c8f82d2abd6a3 100644 --- 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 @@ -31,34 +31,42 @@ public abstract class AbstractObjectInputStreamFactory extends AbstractService implements ObjectInputStreamFactory { - protected AbstractObjectInputStreamFactory(final String name) { - super(name); - } + /** + * Parameters passed down in + * {@link #bind(FactoryBindingParameters)}. + */ + private FactoryBindingParameters bindingParameters; /** * Callbacks. */ private StreamFactoryCallbacks callbacks; + protected AbstractObjectInputStreamFactory(final String name) { + super(name); + } + /** * Bind to the callbacks. *

* The base class checks service state then stores * the callback interface. - * @param factoryCallbacks callbacks needed by the factories. + * @param factoryBindingParameters parameters for the factory binding */ @Override - public void bind(final StreamFactoryCallbacks factoryCallbacks) { + public void bind(final FactoryBindingParameters factoryBindingParameters) { // must be on be invoked during service initialization Preconditions.checkState(isInState(STATE.INITED), "Input Stream factory %s is in wrong state: %s", this, getServiceState()); - this.callbacks = factoryCallbacks; + bindingParameters = factoryBindingParameters; + callbacks = bindingParameters.callbacks(); } /** * Return base capabilities of all stream factories, - * defined what the base ObjectInputStream class does. + * 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 */ @@ -69,7 +77,8 @@ public boolean hasCapability(final String capability) { case StreamStatisticNames.STREAM_LEAKS: return true; default: - return false; + // dynamic probe for the name of this stream + return streamType().capability().equals(capability); } } @@ -77,7 +86,7 @@ public boolean hasCapability(final String capability) { * Get the factory callbacks. * @return callbacks. */ - public StreamFactoryCallbacks callbacks() { + protected StreamFactoryCallbacks callbacks() { return callbacks; } } 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 index 030e9bad01a5e..70699d2c6966c 100644 --- 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 @@ -23,6 +23,7 @@ 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; /** @@ -54,13 +55,19 @@ public boolean hasCapability(final String 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 StreamThreadOptions threadRequirements() { - return new StreamThreadOptions(0, 0, false, true); + public StreamFactoryRequirements factoryRequirements() { + return new StreamFactoryRequirements(0, 0, false, false, + 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 index 92df0c406f7c9..1775fa5f05c25 100644 --- 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 @@ -23,12 +23,17 @@ 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. */ @@ -46,6 +51,13 @@ public enum InputStreamType { */ Analytics(StreamIntegration.ANALYTICS, 3, c -> { throw new IllegalArgumentException("not yet supported"); + }), + + /** + * The a custom input stream. + */ + Custom(StreamIntegration.CUSTOM, 4, c -> { + return loadCustomFactory(c); }); /** @@ -71,6 +83,12 @@ 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) { @@ -81,8 +99,7 @@ public String getName() { /** * Get the ID of this stream. - * Isolated from the enum ID in case it ever needs to be - * tuned. + * Isolated from the enum ID in case it ever needs to be tuned. * @return the numeric ID of the stream. */ public int streamID() { @@ -90,17 +107,19 @@ public int streamID() { } /** - * Factory constructor. - * @return the factory associated with this stream type. + * Get the capability string for this stream type. + * @return the name of a string to probe for. */ - public Function factory() { - return factory; + public String capability() { + return INPUT_STREAM_TYPE + "." + getName(); } /** - * What is the default type? + * Factory constructor. + * @return the factory function associated with this stream type. */ - public static final InputStreamType DEFAULT_STREAM_TYPE = Classic; - + 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 index eccfd060e8451..5071d537b0f5f 100644 --- 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 @@ -33,6 +33,7 @@ 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; @@ -41,6 +42,7 @@ import static java.util.Objects.requireNonNull; import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE; import static org.apache.hadoop.util.Preconditions.checkArgument; import static org.apache.hadoop.util.StringUtils.toLowerCase; @@ -120,12 +122,20 @@ public abstract class ObjectInputStream extends FSInputStream */ 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. @@ -135,8 +145,8 @@ protected ObjectInputStream( final InputStreamType streamType, final ObjectReadParameters parameters) { - objectAttributes = parameters.getObjectAttributes(); - this.streamType = streamType; + this.streamType = requireNonNull(streamType); + this.objectAttributes = parameters.getObjectAttributes(); checkArgument(isNotEmpty(objectAttributes.getBucket()), "No Bucket"); checkArgument(isNotEmpty(objectAttributes.getKey()), "No Key"); @@ -161,6 +171,7 @@ protected ObjectInputStream( "Stream not closed while reading " + uri, this::isStreamOpen, this::abortInFinalizer); + this.vectoredIOContext = getContext().getVectoredIOContext(); } /** @@ -282,6 +293,10 @@ public boolean hasCapability(String capability) { case StreamStatisticNames.STREAM_LEAKS: return true; default: + // dynamic probe for the name of this stream + if (streamType.capability().equals(capability)) { + return true; + } return false; } } @@ -325,6 +340,39 @@ protected final IOStatisticsAggregator getThreadIOStatistics() { 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/ObjectInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamFactory.java index d8fe87f9cf7fd..d1c96c7411dc2 100644 --- 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 @@ -44,9 +44,9 @@ public interface ObjectInputStreamFactory * Set extra initialization parameters. * This MUST ONLY be invoked between {@code init()} * and {@code start()}. - * @param callbacks extra initialization parameters + * @param factoryBindingParameters parameters for the factory binding */ - void bind(StreamFactoryCallbacks callbacks); + void bind(final FactoryBindingParameters factoryBindingParameters); /** * Create a new input stream. @@ -60,10 +60,17 @@ ObjectInputStream readObject(ObjectReadParameters parameters) throws IOException; /** - * Get the number of background threads required for this factory. + * Get requirements from the factory which then tune behavior + * elsewhere in the system. * @return the count of background threads. */ - StreamThreadOptions threadRequirements(); + StreamFactoryRequirements factoryRequirements(); + + /** + * Get the input stream type. + * @return the specific stream type this factory produces. + */ + InputStreamType streamType(); /** * Callbacks for stream factories. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamThreadOptions.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamFactoryRequirements.java similarity index 77% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamThreadOptions.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamFactoryRequirements.java index 254b3976b7c2c..7d5d323bdac0b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamThreadOptions.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamFactoryRequirements.java @@ -18,10 +18,13 @@ package org.apache.hadoop.fs.s3a.impl.streams; +import org.apache.hadoop.fs.s3a.VectoredIOContext; + /** - * Options for threading on this input stream. + * Options for requirements for streams from this factory, + * including threading and vector IO. */ -public class StreamThreadOptions { +public class StreamFactoryRequirements { /** Number of shared threads to included in the bounded pool. */ private final int sharedThreads; @@ -42,21 +45,29 @@ public class StreamThreadOptions { */ private final boolean vectorSupported; + /** + * VectoredIO behaviour. + * This is examined. + */ + private final VectoredIOContext vectoredIOContext; /** * 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 createFuturePool Flag to enable creation of a future pool around the * bounded thread pool. + * @param vectoredIOContext vector IO settings. */ - public StreamThreadOptions(final int sharedThreads, + public StreamFactoryRequirements(final int sharedThreads, final int streamThreads, final boolean createFuturePool, - final boolean vectorSupported) { + final boolean vectorSupported, + final VectoredIOContext vectoredIOContext) { this.sharedThreads = sharedThreads; this.streamThreads = streamThreads; this.createFuturePool = createFuturePool; this.vectorSupported = vectorSupported; + this.vectoredIOContext = vectoredIOContext; } public int sharedThreads() { @@ -74,4 +85,8 @@ public boolean createFuturePool() { public boolean vectorSupported() { return vectorSupported; } + + public VectoredIOContext vectoredIOContext() { + return vectoredIOContext; + } } 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 index d64b4326fa4b0..83c498fd0b7d0 100644 --- 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 @@ -18,14 +18,30 @@ 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. @@ -47,9 +63,23 @@ public final class StreamIntegration { */ public static final String ANALYTICS = "analytics"; - private StreamIntegration() { - } + /** + * 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.. @@ -63,34 +93,122 @@ private StreamIntegration() { */ 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. - * If the option {@code "fs.s3a.prefetch.enabled"} is set, this - * changes the default value from classic to prefetch, after printing a - * warning the first time this happens. *

* 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 createStreamFactory(final Configuration conf) { - // choose the default input stream type + 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. + */ + 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. - InputStreamType defaultStream = InputStreamType.DEFAULT_STREAM_TYPE; 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); - defaultStream = InputStreamType.Prefetch; + return InputStreamType.Prefetch; } // retrieve the enum value, returning the configured value or - // the default...then instantiate it. - return conf.getEnum(INPUT_STREAM_TYPE, defaultStream) - .factory() - .apply(conf); + // 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. + * @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) + .build(); + } + } 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 index 4109580c4ce5d..2b25f3b1395b6 100644 --- 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 @@ -21,10 +21,12 @@ 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.StreamThreadOptions; +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; @@ -32,6 +34,7 @@ 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; /** @@ -56,6 +59,11 @@ public PrefetchingInputStreamFactory() { super("PrefetchingInputStreamFactory"); } + @Override + public InputStreamType streamType() { + return InputStreamType.Prefetch; + } + @Override protected void serviceInit(final Configuration conf) throws Exception { super.serviceInit(conf); @@ -80,12 +88,21 @@ public ObjectInputStream readObject(final ObjectReadParameters parameters) throw } /** - * The thread count is calculated from the configuration. + * Calculate Return StreamFactoryRequirements * @return a positive thread count. */ @Override - public StreamThreadOptions threadRequirements() { - return new StreamThreadOptions(prefetchBlockCount, 0, true, false); + 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(prefetchBlockCount, + 0, true, false, + vectorContext); } } 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..a5bfca6d9734d --- /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/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index da21fb22e2ede..a1b3821f7fa6b 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 @@ -104,7 +104,7 @@ 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.DEFAULT_STREAM_TYPE; +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; 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..15f1266aba105 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/streams/TestStreamFactories.java @@ -0,0 +1,240 @@ +/* + * 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.junit.Test; + +import org.apache.hadoop.conf.Configuration; +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.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 { + load(INPUT_STREAM_TYPE_CLASSIC, DEFAULT_STREAM_TYPE, + ClassicObjectInputStreamFactory.class); + } + + /** + * Prefetch factory. + */ + @Test + public void testPrefetchFactoryCreation() throws Throwable { + // load from config option + load(INPUT_STREAM_TYPE_PREFETCH, + InputStreamType.Prefetch, + PrefetchingInputStreamFactory.class); + } + + /** + * 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); + } + + /** + * 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 void load( + String name, + InputStreamType type, + Class clazz) { + + assertFactorySatisfies(factory(name), name, type, clazz); + } + + /** + * 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()); + } + + } +} From 63daf564351f97804e80bfebde8a27335b3fdf63 Mon Sep 17 00:00:00 2001 From: fuatbasik Date: Wed, 18 Dec 2024 10:06:56 +0000 Subject: [PATCH 3/5] HADOOP-19348. Add initial support for Analytics Accelerator Library for Amazon S3 (#7192) --- hadoop-tools/hadoop-aws/pom.xml | 11 ++ .../org/apache/hadoop/fs/s3a/Constants.java | 34 ++++ .../apache/hadoop/fs/s3a/S3AFileSystem.java | 67 ++++++- .../hadoop/fs/s3a/S3ASeekableStream.java | 183 ++++++++++++++++++ .../contract/s3a/ITestS3AContractCreate.java | 13 +- .../contract/s3a/ITestS3AContractDistCp.java | 9 + .../contract/s3a/ITestS3AContractRename.java | 8 + .../s3a/ITestS3AContractVectoredRead.java | 12 ++ .../hadoop/fs/s3a/ITestS3ADelayedFNF.java | 3 + .../hadoop/fs/s3a/ITestS3AEncryptionSSEC.java | 6 +- .../fs/s3a/ITestS3AFSMainOperations.java | 18 +- .../fs/s3a/ITestS3AFileSystemContract.java | 11 +- .../fs/s3a/ITestS3AIOStatisticsContext.java | 5 + .../fs/s3a/ITestS3AInputStreamLeakage.java | 5 + .../apache/hadoop/fs/s3a/ITestS3AMetrics.java | 5 + .../fs/s3a/ITestS3APrefetchingCacheFiles.java | 8 +- .../s3a/ITestS3APrefetchingInputStream.java | 6 +- .../s3a/ITestS3APrefetchingLruEviction.java | 6 +- .../fs/s3a/ITestS3AS3SeekableStream.java | 130 +++++++++++++ .../apache/hadoop/fs/s3a/S3ATestUtils.java | 15 ++ .../s3a/commit/ITestCommitOperationCost.java | 7 + .../s3a/commit/ITestS3ACommitterFactory.java | 3 + .../magic/ITestMagicCommitProtocol.java | 3 + .../ITestDirectoryCommitProtocol.java | 9 + .../ITestPartitionedCommitProtocol.java | 9 + .../ITestStagingCommitProtocol.java | 4 + .../ITestStagingCommitProtocolFailure.java | 12 +- .../ITestS3AFileContextStatistics.java | 4 + .../fs/s3a/impl/ITestConnectionTimeouts.java | 11 ++ .../fs/s3a/performance/ITestS3AOpenCost.java | 11 +- .../performance/ITestUnbufferDraining.java | 4 + .../ITestS3AContractStreamIOStatistics.java | 8 + .../ITestS3AFileSystemStatistic.java | 6 + 33 files changed, 617 insertions(+), 29 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 1d04107ff5b48..28605c47f33b6 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -472,6 +472,17 @@ amazon-s3-encryption-client-java provided + + software.amazon.s3.analyticsaccelerator + analyticsaccelerator-s3 + 0.0.2 + compile + + + software.amazon.awssdk.crt + aws-crt + 0.29.10 + 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 6796c29d34874..189448fcf1c45 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 @@ -1827,4 +1827,38 @@ 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"; + + /** + * Config to enable Analytics Accelerator Library for Amazon S3. + * https://github.com/awslabs/analytics-accelerator-s3 + */ + public static final String ANALYTICS_ACCELERATOR_ENABLED_KEY = + ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + ".enabled"; + + /** + * Config to enable usage of crt client with Analytics Accelerator Library. + * It is by default true. + */ + public static final String ANALYTICS_ACCELERATOR_CRT_ENABLED = + "fs.s3a.analytics.accelerator.crt.client"; + + /** + * Default value for {@link #ANALYTICS_ACCELERATOR_ENABLED_KEY } + * Value {@value}. + */ + public static final boolean ANALYTICS_ACCELERATOR_ENABLED_DEFAULT = false; + + /** + * Default value for {@link #ANALYTICS_ACCELERATOR_CRT_ENABLED } + * Value {@value}. + */ + public static final boolean ANALYTICS_ACCELERATOR_CRT_ENABLED_DEFAULT = true; + } 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 19047d9ad4d54..1ba931ff728ce 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 @@ -52,7 +52,9 @@ import javax.annotation.Nullable; import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.internal.crt.S3CrtAsyncClient; import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse; import software.amazon.awssdk.services.s3.model.GetBucketLocationRequest; @@ -83,6 +85,11 @@ import software.amazon.awssdk.transfer.s3.model.Copy; import software.amazon.awssdk.transfer.s3.model.CopyRequest; +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 org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -309,6 +316,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, */ private S3Client s3Client; + /** + * CRT-Based S3Client created of analytics accelerator library is enabled + * and managed by the S3AStoreImpl. Analytics accelerator library can be + * enabled with {@link Constants#ANALYTICS_ACCELERATOR_ENABLED_KEY} + */ + private S3AsyncClient s3AsyncClient; + // initial callback policy is fail-once; it's there just to assist // some mock tests and other codepaths trying to call the low level // APIs on an uninitialized filesystem. @@ -335,6 +349,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, */ private ExecutorServiceFuturePool futurePool; + + // If true, S3SeekableInputStream from Analytics Accelerator for Amazon S3 will be used. + private boolean analyticsAcceleratorEnabled; + + private boolean analyticsAcceleratorCRTEnabled; + private int executorCapacity; private long multiPartThreshold; public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class); @@ -497,6 +517,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, */ private boolean s3AccessGrantsEnabled; + /** + * Factory to create S3SeekableInputStream if {@link this#analyticsAcceleratorEnabled} is true. + */ + private S3SeekableInputStreamFactory s3SeekableInputStreamFactory; + /** Add any deprecated keys. */ @SuppressWarnings("deprecation") private static void addDeprecatedKeys() { @@ -643,8 +668,21 @@ public void initialize(URI name, Configuration originalConf) dirOperationsPurgeUploads = conf.getBoolean(DIRECTORY_OPERATIONS_PURGE_UPLOADS, s3ExpressStore); + + this.analyticsAcceleratorEnabled = + conf.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, ANALYTICS_ACCELERATOR_ENABLED_DEFAULT); + this.analyticsAcceleratorCRTEnabled = + conf.getBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, + ANALYTICS_ACCELERATOR_CRT_ENABLED_DEFAULT); + this.isMultipartUploadEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED, - DEFAULT_MULTIPART_UPLOAD_ENABLED); + DEFAULT_MULTIPART_UPLOAD_ENABLED); + + if(this.analyticsAcceleratorEnabled && !analyticsAcceleratorCRTEnabled) { + // Temp change: Analytics Accelerator with S3AsyncClient do not support Multi-part upload. + this.isMultipartUploadEnabled = false; + } + // multipart copy and upload are the same; this just makes it explicit this.isMultipartCopyEnabled = isMultipartUploadEnabled; @@ -771,6 +809,26 @@ public void initialize(URI name, Configuration originalConf) // thread pool init requires store to be created initThreadPools(); + if (this.analyticsAcceleratorEnabled) { + LOG.info("Using S3SeekableInputStream"); + if(this.analyticsAcceleratorCRTEnabled) { + LOG.info("Using S3 CRT client for analytics accelerator S3"); + this.s3AsyncClient = S3CrtAsyncClient.builder().maxConcurrency(600).build(); + } else { + LOG.info("Using S3 async client for analytics accelerator S3"); + this.s3AsyncClient = store.getOrCreateAsyncClient(); + } + + ConnectorConfiguration configuration = new ConnectorConfiguration(conf, + ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); + S3SeekableInputStreamConfiguration seekableInputStreamConfiguration = + S3SeekableInputStreamConfiguration.fromConfiguration(configuration); + this.s3SeekableInputStreamFactory = + new S3SeekableInputStreamFactory( + new S3SdkObjectClient(this.s3AsyncClient), + seekableInputStreamConfiguration); + } + // The filesystem is now ready to perform operations against // S3 // This initiates a probe against S3 for the bucket existing. @@ -1822,6 +1880,7 @@ private FSDataInputStream executeOpen( final Path path, final OpenFileSupport.OpenFileInformation fileInformation) throws IOException { + // create the input stream statistics before opening // the file so that the time to prepare to open the file is included. S3AInputStreamStatistics inputStreamStats = @@ -4257,9 +4316,13 @@ public void close() throws IOException { protected synchronized void stopAllServices() { try { trackDuration(getDurationTrackerFactory(), FILESYSTEM_CLOSE.getSymbol(), () -> { - closeAutocloseables(LOG, getStore()); + + closeAutocloseables(LOG, getStore(), s3SeekableInputStreamFactory); + store = null; s3Client = null; + s3AsyncClient = null; + s3SeekableInputStreamFactory = null; // At this point the S3A client is shut down, // now the executor pools are closed diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java new file mode 100644 index 0000000000000..ef6a299081587 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java @@ -0,0 +1,183 @@ +/* + * 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.EOFException; +import java.io.IOException; + +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.StreamCapabilities; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSInputStream; + +import software.amazon.s3.analyticsaccelerator.S3SeekableInputStream; +import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamFactory; +import software.amazon.s3.analyticsaccelerator.util.S3URI; + +public class S3ASeekableStream extends FSInputStream implements StreamCapabilities { + + private S3SeekableInputStream inputStream; + private long lastReadCurrentPos = 0; + private final String key; + private volatile boolean closed; + + public static final Logger LOG = LoggerFactory.getLogger(S3ASeekableStream.class); + + public S3ASeekableStream(String bucket, String key, + S3SeekableInputStreamFactory s3SeekableInputStreamFactory) { + this.inputStream = s3SeekableInputStreamFactory.createStream(S3URI.of(bucket, key)); + this.key = key; + } + + /** + * 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 + 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 {}: ", key); + 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:", + key, ioe); + } else { + LOG.info("Got exception while trying to read from stream {}, " + + "not trying to recover:", + key, ioe); + } + this.close(); + } + + + protected void throwIfClosed() throws IOException { + if (closed) { + throw new IOException(key + ": " + FSExceptionMessages.STREAM_IS_CLOSED); + } + } +} \ No newline at end of file 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..3e5c67e5894d3 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,14 @@ protected Configuration createConfiguration() { return conf; } + @Override + public void testOverwriteExistingFile() throws Throwable { + // Will remove this when Analytics Accelerator supports overwrites + skipIfAnalyticsAcceleratorEnabled(this.createConfiguration(), + "Analytics Accelerator does not support overwrites yet"); + 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..f6127700b5f1f 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,14 @@ public void testNonDirectWrite() throws Exception { getRenameOperationCount() - renames); } + @Override + public void testDistCpUpdateCheckFileSkip() throws Exception { + //Will remove this when Analytics Accelerator supports overwrites + 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/ITestS3AContractRename.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java index d3ba7373cc944..9f5246d9ddec5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java @@ -36,6 +36,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents; import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; import static org.apache.hadoop.fs.s3a.S3ATestConstants.S3A_TEST_TIMEOUT; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; /** * S3A contract tests covering rename. @@ -45,6 +46,13 @@ public class ITestS3AContractRename extends AbstractContractRenameTest { public static final Logger LOG = LoggerFactory.getLogger( ITestS3AContractRename.class); + @Override + public void setup() throws Exception { + super.setup(); + skipIfAnalyticsAcceleratorEnabled(createConfiguration(), + "Analytics Accelerator does not support rename"); + + } @Override protected int getTestTimeoutMillis() { return S3A_TEST_TIMEOUT; 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/ITestS3ADelayedFNF.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java index ca9d185c3e9e1..4793092b717dd 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java @@ -36,6 +36,7 @@ import static org.apache.hadoop.fs.s3a.Constants.RETRY_INTERVAL; import static org.apache.hadoop.fs.s3a.Constants.RETRY_LIMIT; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; /** * Tests behavior of a FileNotFound error that happens after open(), i.e. on @@ -65,6 +66,8 @@ protected Configuration createConfiguration() { */ @Test public void testNotFoundFirstRead() throws Exception { + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Temporarily disabling to fix Exception handling on Analytics Accelerator"); S3AFileSystem fs = getFileSystem(); ChangeDetectionPolicy changeDetectionPolicy = fs.getChangeDetectionPolicy(); 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..12e5ef3841a64 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 does not support SSEC"); 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..1d3806e75e329 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,20 @@ public void testCopyToLocalWithUseRawLocalFileSystemOption() throws Exception { } + @Override + public void testWriteReadAndDeleteOneAndAHalfBlocks() throws Exception { + // Skipping this test for AnalyticsAccelerator as it is acting as an overwrite test + skipIfAnalyticsAcceleratorEnabled(this.contract.getConf(), + "Analytics Accelerator does not support overwrites"); + } + + @Override + public void testWriteReadAndDeleteTwoBlocks() throws Exception { + // Skipping this test for AnalyticsAccelerator as it is acting as an overwrite test + skipIfAnalyticsAcceleratorEnabled(this.contract.getConf(), + "Analytics Accelerator does not support overwrites"); + } + @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..32d9a511a4159 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,12 @@ public void testOverwrite() throws IOException { } } } + + @Override + public void testOverWriteAndRead() throws Exception { + //Will remove this when Analytics Accelerator supports overwrites + skipIfAnalyticsAcceleratorEnabled(fs.getConf(), + "Analytics Accelerator does not support overwrites"); + 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..005b2fbf91d41 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,10 @@ protected Configuration createConfiguration() { public void setup() throws Exception { super.setup(); executor = HadoopExecutors.newFixedThreadPool(SMALL_THREADS); + // TODO: Add IOStatistics Support to S3SeekableStream + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3SeekableStream 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 72c75162c9fda..cbd183f080d99 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 @@ -36,6 +36,7 @@ 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; @@ -87,6 +88,10 @@ public void setup() throws Exception { @Test public void testFinalizer() throws Throwable { Path path = methodPath(); + // TODO: Add Leak Detection to S3SeekableStream + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3SeekableStream does not support leak detection"); + final S3AFileSystem fs = getFileSystem(); ContractTestUtils.createFile(fs, path, true, DATASET); 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..61d159996357f 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,10 @@ public void testMetricsRegister() @Test public void testStreamStatistics() throws IOException { + // TODO: Add StreamStatistics support to S3SeekableStream + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3SeekableStream 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 39acaa717a7f2..837ecb067324c 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 @@ -37,10 +37,9 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; -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.S3ATestUtils.enablePrefetching; + +import static org.apache.hadoop.fs.s3a.Constants.*; 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; @@ -73,7 +72,6 @@ public void setUp() throws Exception { super.setup(); // Sets BUFFER_DIR by calling S3ATestUtils#prepareTestConfiguration conf = createConfiguration(); - testFile = getExternalData(conf); prefetchBlockSize = conf.getInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); fs = FileSystem.get(testFile.toUri(), conf); @@ -93,6 +91,8 @@ public Configuration createConfiguration() { final String bufferDirBase = configuration.get(BUFFER_DIR); bufferDir = bufferDirBase + "/" + UUID.randomUUID(); configuration.set(BUFFER_DIR, bufferDir); + // When both Prefetching and Analytics Accelerator enabled Analytics Accelerator is used + configuration.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, false); return configuration; } 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 d894adb66c738..5f9df4e4dbaa3 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 @@ -34,8 +34,9 @@ import org.apache.hadoop.fs.statistics.IOStatistics; 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.S3ATestUtils.enablePrefetching; + +import static org.apache.hadoop.fs.s3a.Constants.*; 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; @@ -71,11 +72,14 @@ public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest { private static final int INTERVAL_MILLIS = 500; private static final int BLOCK_SIZE = S_1K * 10; + @Override public Configuration createConfiguration() { Configuration conf = enablePrefetching(super.createConfiguration()); S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_BLOCK_SIZE_KEY); conf.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); + // When both Prefetching and Analytics Accelerator enabled Analytics Accelerator is used + conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, false); 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 5cc948a044dbb..413a1adc9aee1 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 @@ -42,9 +42,9 @@ import org.apache.hadoop.fs.statistics.IOStatistics; 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_MAX_BLOCKS_COUNT; import static org.apache.hadoop.fs.s3a.S3ATestUtils.enablePrefetching; + +import static org.apache.hadoop.fs.s3a.Constants.*; 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; @@ -91,6 +91,8 @@ public Configuration createConfiguration() { PREFETCH_BLOCK_SIZE_KEY); conf.setInt(PREFETCH_MAX_BLOCKS_COUNT, Integer.parseInt(maxBlocks)); conf.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); + // When both Prefetching and Analytics Accelerator enabled Analytics Accelerator is used + conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, false); return conf; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java new file mode 100644 index 0000000000000..c6ecee9505101 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java @@ -0,0 +1,130 @@ +/* + * 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.IOException; + +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX; +import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_ENABLED_KEY; +import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_CRT_ENABLED; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; + +import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamConfiguration; +import software.amazon.s3.analyticsaccelerator.common.ConnectorConfiguration; +import software.amazon.s3.analyticsaccelerator.util.PrefetchMode; + +public class ITestS3AS3SeekableStream extends AbstractS3ATestBase { + + private static final String PHYSICAL_IO_PREFIX = "physicalio"; + private static final String LOGICAL_IO_PREFIX = "logicalio"; + + public void testConnectorFrameWorkIntegration(boolean useCrtClient) throws IOException { + describe("Verify S3 connector framework integration"); + + Configuration conf = getConfiguration(); + removeBaseAndBucketOverrides(conf, ANALYTICS_ACCELERATOR_ENABLED_KEY); + conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, true); + conf.setBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, useCrtClient); + + String testFile = "s3a://noaa-cors-pds/raw/2023/017/ohfh/OHFH017d.23_.gz"; + S3AFileSystem s3AFileSystem = + (S3AFileSystem) FileSystem.newInstance(new Path(testFile).toUri(), conf); + byte[] buffer = new byte[500]; + + try (FSDataInputStream inputStream = s3AFileSystem.open(new Path(testFile))) { + inputStream.seek(5); + inputStream.read(buffer, 0, 500); + } + + } + + @Test + public void testConnectorFrameWorkIntegrationWithCrtClient() throws IOException { + testConnectorFrameWorkIntegration(true); + } + + @Test + public void testConnectorFrameWorkIntegrationWithoutCrtClient() throws IOException { + testConnectorFrameWorkIntegration(false); + } + + public void testConnectorFrameworkConfigurable(boolean useCrtClient) { + 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); + + conf.setBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, useCrtClient); + + ConnectorConfiguration connectorConfiguration = + new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); + + S3SeekableInputStreamConfiguration configuration = + S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration); + + assertSame("S3ASeekableStream configuration is not set to expected value", + PrefetchMode.ALL, configuration.getLogicalIOConfiguration().getPrefetchingMode()); + + assertEquals("S3ASeekableStream configuration is not set to expected value", + 1, configuration.getPhysicalIOConfiguration().getBlobStoreCapacity()); + } + + @Test + public void testConnectorFrameworkConfigurableWithoutCrtClient() throws IOException { + testConnectorFrameworkConfigurable(false); + } + + @Test + public void testConnectorFrameworkConfigurableWithCrtClient() throws IOException { + testConnectorFrameworkConfigurable(true); + } + + @Test + public void testInvalidConfigurationThrows() { + 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); + assertThrows("S3ASeekableStream illegal configuration does not throw", + IllegalArgumentException.class, () -> + S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration)); + } +} 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 a1b3821f7fa6b..571b4b1ad255d 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 @@ -576,6 +576,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.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, + ANALYTICS_ACCELERATOR_ENABLED_DEFAULT); + } + /** * Skip a test if the filesystem lacks a required capability. * @param fs filesystem 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..fac461371e62a 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; @@ -168,6 +169,10 @@ private void abortActiveStream() throws IOException { @Test public void testCostOfCreatingMagicFile() throws Throwable { describe("Files created under magic paths skip existence checks"); + + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3ASeekableInputStream does not support InputStreamStatistics"); + S3AFileSystem fs = getFileSystem(); Path destFile = methodSubPath("file.txt"); fs.delete(destFile.getParent(), true); @@ -245,6 +250,8 @@ public void testCostOfCreatingMagicFile() throws Throwable { public void testCostOfSavingLoadingPendingFile() throws Throwable { describe("Verify costs of saving .pending file under a magic path"); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3ASeekableInputStream does not support InputStreamStatistics"); 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/commit/ITestS3ACommitterFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java index 2561a69f60b59..a500bfb76a322 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java @@ -47,6 +47,7 @@ import org.apache.hadoop.security.UserGroupInformation; 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.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.COMMITTER_NAME_STAGING; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -182,6 +183,8 @@ public void setup() throws Exception { // destroy all filesystems from previous runs. FileSystem.closeAllForUGI(UserGroupInformation.getCurrentUser()); super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3ASeekableInputStream does not support InputStreamStatistics"); jobId = randomJobId(); attempt0 = "attempt_" + jobId + "_m_000000_0"; taskAttempt0 = TaskAttemptID.forName(attempt0); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java index cbfc23a2a29b6..b89740ae3120c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java @@ -46,6 +46,7 @@ import org.junit.runners.Parameterized; 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.S3AUtils.listAndFilter; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.impl.CommitUtilsWithMR.getMagicJobPath; @@ -77,6 +78,8 @@ protected String getCommitterName() { @Override public void setup() throws Exception { super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3ASeekableInputStream does not support InputStreamStatistics"); CommitUtils.verifyIsMagicCommitFS(getFileSystem()); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitProtocol.java index b19662c0117fd..cbd41fd1c93e6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitProtocol.java @@ -35,6 +35,7 @@ import org.apache.hadoop.mapreduce.JobStatus; import org.apache.hadoop.mapreduce.TaskAttemptContext; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.CONFLICT_MODE_APPEND; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_CONFLICT_MODE; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.S3A_COMMITTER_EXPERIMENTAL_COLLECT_IOSTATISTICS; @@ -42,6 +43,14 @@ /** ITest of the low level protocol methods. */ public class ITestDirectoryCommitProtocol extends ITestStagingCommitProtocol { + + @Override + public void setup() throws Exception { + super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Skipping because these tests will fail when using CRT client"); + } + @Override protected String suitename() { return "ITestDirectoryCommitProtocol"; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionedCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionedCommitProtocol.java index e3bc1500dab7c..b6438157e4e36 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionedCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionedCommitProtocol.java @@ -32,10 +32,19 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext; import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; /** ITest of the low level protocol methods. */ public class ITestPartitionedCommitProtocol extends ITestStagingCommitProtocol { + + @Override + public void setup() throws Exception { + super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Skipping because these tests will fail when using CRT client"); + } + @Override protected String suitename() { return "ITestPartitionedCommitProtocol"; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java index 81c3af812ab95..d2b4e1c52956b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java @@ -41,6 +41,7 @@ import org.apache.hadoop.mapreduce.JobStatus; import org.apache.hadoop.mapreduce.TaskAttemptContext; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; /** Test the staging committer's handling of the base protocol operations. */ @@ -65,6 +66,9 @@ protected Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Skipping because these tests will fail when using CRT client"); + // identify working dir for staging and delete Configuration conf = getConfiguration(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java index 08b6c21a863d5..5604aa7636968 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java @@ -32,9 +32,7 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_UPLOADS_ENABLED; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_NAME; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.S3A_COMMITTER_FACTORY_KEY; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -45,6 +43,14 @@ */ public class ITestStagingCommitProtocolFailure extends AbstractS3ATestBase { + + @Override + public void setup() throws Exception { + super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Skipping because these tests will fail when using CRT client"); + } + @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); 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..dc086f7c4237f 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,8 @@ public class ITestS3AFileContextStatistics extends FCStatisticsBaseTest { @Before public void setUp() throws Exception { conf = new Configuration(); + skipIfAnalyticsAcceleratorEnabled(conf, + "S3SeekableStream does not support File Context 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 aeb9629b3a6d1..58eb5c14dbc9e 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 @@ -60,6 +60,7 @@ 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.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC_PATH_PREFIX; import static org.apache.hadoop.fs.s3a.impl.ConfigurationHelper.setDurationAsMillis; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -145,6 +146,11 @@ public void teardown() throws Exception { @Test public void testGeneratePoolTimeouts() throws Throwable { skipIfClientSideEncryption(); + + // Assertions will fail when using CRTClient with Analytics Accelerator. + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Assertions will fail when using CRTClient with Analytics Accelerator"); + AWSClientConfig.setMinimumOperationDuration(Duration.ZERO); Configuration conf = timingOutConfiguration(); Path path = methodPath(); @@ -187,6 +193,11 @@ public void testGeneratePoolTimeouts() throws Throwable { @Test public void testObjectUploadTimeouts() throws Throwable { skipIfClientSideEncryption(); + + // Assertions will fail when using CRTClient with Analytics Accelerator. + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Assertions will fail when using CRTClient with Analytics Accelerator"); + AWSClientConfig.setMinimumOperationDuration(Duration.ZERO); final Path dir = methodPath(); Path file = new Path(dir, "file"); 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 37b1413ed0f5f..66f1b3c6a0359 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 @@ -52,11 +52,9 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile; import static org.apache.hadoop.fs.s3a.Constants.CHECKSUM_VALIDATION; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.assertStreamIsNotChecksummed; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getS3AInputStream; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.isPrefetchingEnabled; -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; @@ -109,6 +107,8 @@ public Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Assertions will fail as S3SeekableStream does not support Stream Statistics"); S3AFileSystem fs = getFileSystem(); testFile = methodPath(); @@ -387,7 +387,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)) { 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 caf723b95dedd..69e557f80eaf0 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 @@ -54,6 +54,7 @@ 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.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.s3a.impl.ConfigurationHelper.setDurationAsSeconds; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; @@ -117,6 +118,9 @@ public Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Skipping because getS3AInputStream will " + + "try to cast S3SeekableStream to S3AInputStream"); // now create a new FS with minimal http capacity and recovery // a separate one is used to avoid test teardown suffering 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..2b332b2b3ee24 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,11 @@ public List outputStreamStatisticKeys() { STREAM_WRITE_EXCEPTIONS); } + @Override + public void setup() throws Exception { + super.setup(); + skipIfAnalyticsAcceleratorEnabled(createConfiguration(), + "S3SeekableStream does not support Stream Statistics"); + } + } 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..5a3f7bb8fdbb2 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 { + // Assertions will fail as {@link S3ASeekableInputStream} + // do not support S3AFileSystemStatistics yet. + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3SeekableStream does not support File System Statistics"); S3AFileSystem fs = getFileSystem(); Path filePath = path(getMethodName()); byte[] oneKbBuf = new byte[ONE_KB]; From d45beae8e59678dd3929665de9b04aaf43e8a940 Mon Sep 17 00:00:00 2001 From: rajdchak Date: Tue, 28 Jan 2025 13:27:17 +0000 Subject: [PATCH 4/5] Integrate analytics-accelerator with factory (#7332) --- hadoop-tools/hadoop-aws/pom.xml | 5 - .../org/apache/hadoop/fs/s3a/Constants.java | 26 ----- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 57 ++--------- .../hadoop/fs/s3a/impl/S3AStoreImpl.java | 5 +- .../AbstractObjectInputStreamFactory.java | 4 +- .../streams/AnalyticsStream.java} | 47 ++++++--- .../impl/streams/AnalyticsStreamFactory.java | 96 +++++++++++++++++++ .../fs/s3a/impl/streams/InputStreamType.java | 7 +- .../streams/ObjectInputStreamFactory.java | 2 +- .../s3a/impl/streams/StreamIntegration.java | 3 +- .../fs/s3a/ITestS3APrefetchingCacheFiles.java | 2 - .../s3a/ITestS3APrefetchingInputStream.java | 6 +- .../s3a/ITestS3APrefetchingLruEviction.java | 2 - .../fs/s3a/ITestS3AS3SeekableStream.java | 55 ++++------- .../apache/hadoop/fs/s3a/S3ATestUtils.java | 5 +- .../s3a/commit/ITestCommitOperationCost.java | 7 +- .../s3a/commit/ITestS3ACommitterFactory.java | 2 +- .../magic/ITestMagicCommitProtocol.java | 2 +- .../ITestS3AFileSystemStatistic.java | 2 +- 19 files changed, 174 insertions(+), 161 deletions(-) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/{S3ASeekableStream.java => impl/streams/AnalyticsStream.java} (80%) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStreamFactory.java diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 28605c47f33b6..c754e75c94ff4 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -478,11 +478,6 @@ 0.0.2 compile - - software.amazon.awssdk.crt - aws-crt - 0.29.10 - 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 189448fcf1c45..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 @@ -1835,30 +1835,4 @@ private Constants() { public static final String ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX = "fs.s3a.analytics.accelerator"; - /** - * Config to enable Analytics Accelerator Library for Amazon S3. - * https://github.com/awslabs/analytics-accelerator-s3 - */ - public static final String ANALYTICS_ACCELERATOR_ENABLED_KEY = - ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + ".enabled"; - - /** - * Config to enable usage of crt client with Analytics Accelerator Library. - * It is by default true. - */ - public static final String ANALYTICS_ACCELERATOR_CRT_ENABLED = - "fs.s3a.analytics.accelerator.crt.client"; - - /** - * Default value for {@link #ANALYTICS_ACCELERATOR_ENABLED_KEY } - * Value {@value}. - */ - public static final boolean ANALYTICS_ACCELERATOR_ENABLED_DEFAULT = false; - - /** - * Default value for {@link #ANALYTICS_ACCELERATOR_CRT_ENABLED } - * Value {@value}. - */ - public static final boolean ANALYTICS_ACCELERATOR_CRT_ENABLED_DEFAULT = true; - } 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 1ba931ff728ce..d42fbe3f8035b 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,10 +51,9 @@ import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.Nullable; +import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType; import software.amazon.awssdk.core.exception.SdkException; -import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.s3.internal.crt.S3CrtAsyncClient; import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse; import software.amazon.awssdk.services.s3.model.GetBucketLocationRequest; @@ -85,11 +84,6 @@ import software.amazon.awssdk.transfer.s3.model.Copy; import software.amazon.awssdk.transfer.s3.model.CopyRequest; -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 org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -316,13 +310,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, */ private S3Client s3Client; - /** - * CRT-Based S3Client created of analytics accelerator library is enabled - * and managed by the S3AStoreImpl. Analytics accelerator library can be - * enabled with {@link Constants#ANALYTICS_ACCELERATOR_ENABLED_KEY} - */ - private S3AsyncClient s3AsyncClient; - // initial callback policy is fail-once; it's there just to assist // some mock tests and other codepaths trying to call the low level // APIs on an uninitialized filesystem. @@ -353,8 +340,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, // If true, S3SeekableInputStream from Analytics Accelerator for Amazon S3 will be used. private boolean analyticsAcceleratorEnabled; - private boolean analyticsAcceleratorCRTEnabled; - private int executorCapacity; private long multiPartThreshold; public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class); @@ -517,11 +502,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, */ private boolean s3AccessGrantsEnabled; - /** - * Factory to create S3SeekableInputStream if {@link this#analyticsAcceleratorEnabled} is true. - */ - private S3SeekableInputStreamFactory s3SeekableInputStreamFactory; - /** Add any deprecated keys. */ @SuppressWarnings("deprecation") private static void addDeprecatedKeys() { @@ -669,16 +649,12 @@ public void initialize(URI name, Configuration originalConf) s3ExpressStore); - this.analyticsAcceleratorEnabled = - conf.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, ANALYTICS_ACCELERATOR_ENABLED_DEFAULT); - this.analyticsAcceleratorCRTEnabled = - conf.getBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, - ANALYTICS_ACCELERATOR_CRT_ENABLED_DEFAULT); + this.analyticsAcceleratorEnabled = conf.getEnum(INPUT_STREAM_TYPE, InputStreamType.Classic) == InputStreamType.Analytics; this.isMultipartUploadEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED, DEFAULT_MULTIPART_UPLOAD_ENABLED); - if(this.analyticsAcceleratorEnabled && !analyticsAcceleratorCRTEnabled) { + if(this.analyticsAcceleratorEnabled) { // Temp change: Analytics Accelerator with S3AsyncClient do not support Multi-part upload. this.isMultipartUploadEnabled = false; } @@ -809,26 +785,6 @@ public void initialize(URI name, Configuration originalConf) // thread pool init requires store to be created initThreadPools(); - if (this.analyticsAcceleratorEnabled) { - LOG.info("Using S3SeekableInputStream"); - if(this.analyticsAcceleratorCRTEnabled) { - LOG.info("Using S3 CRT client for analytics accelerator S3"); - this.s3AsyncClient = S3CrtAsyncClient.builder().maxConcurrency(600).build(); - } else { - LOG.info("Using S3 async client for analytics accelerator S3"); - this.s3AsyncClient = store.getOrCreateAsyncClient(); - } - - ConnectorConfiguration configuration = new ConnectorConfiguration(conf, - ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); - S3SeekableInputStreamConfiguration seekableInputStreamConfiguration = - S3SeekableInputStreamConfiguration.fromConfiguration(configuration); - this.s3SeekableInputStreamFactory = - new S3SeekableInputStreamFactory( - new S3SdkObjectClient(this.s3AsyncClient), - seekableInputStreamConfiguration); - } - // The filesystem is now ready to perform operations against // S3 // This initiates a probe against S3 for the bucket existing. @@ -1922,6 +1878,7 @@ private FSDataInputStream executeOpen( .withContext(readContext.build()) .withObjectAttributes(createObjectAttributes(path, fileStatus)) .withStreamStatistics(inputStreamStats); + return new FSDataInputStream(getStore().readObject(parameters)); } @@ -4219,7 +4176,7 @@ PutObjectResponse executePut( throws IOException { String key = putObjectRequest.key(); ProgressableProgressListener listener = - new ProgressableProgressListener(store, key, progress); + new ProgressableProgressListener(getStore(), key, progress); UploadInfo info = putObject(putObjectRequest, file, listener); PutObjectResponse result = getStore().waitForUploadCompletion(key, info).response(); listener.uploadCompleted(info.getFileUpload()); @@ -4317,12 +4274,10 @@ protected synchronized void stopAllServices() { try { trackDuration(getDurationTrackerFactory(), FILESYSTEM_CLOSE.getSymbol(), () -> { - closeAutocloseables(LOG, getStore(), s3SeekableInputStreamFactory); + closeAutocloseables(LOG, getStore()); store = null; s3Client = null; - s3AsyncClient = null; - s3SeekableInputStreamFactory = null; // At this point the S3A client is shut down, // now the executor pools are closed 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 d284a098bcd3f..b86f316f1b0f3 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 @@ -91,8 +91,7 @@ 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.Constants.*; 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; @@ -948,7 +947,7 @@ public File createTemporaryFileForWriting(String pathStr, * All stream factory initialization required after {@code Service.init()}, * after all other services have themselves been initialized. */ - private void finishStreamFactoryInit() { + private void finishStreamFactoryInit() throws Exception { // must be on be invoked during service initialization Preconditions.checkState(isInState(STATE.INITED), "Store is in wrong state: %s", getServiceState()); 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 index c8f82d2abd6a3..f0b5c9614febd 100644 --- 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 @@ -18,6 +18,8 @@ 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; @@ -54,7 +56,7 @@ protected AbstractObjectInputStreamFactory(final String name) { * @param factoryBindingParameters parameters for the factory binding */ @Override - public void bind(final FactoryBindingParameters factoryBindingParameters) { + 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", diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStream.java similarity index 80% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStream.java index ef6a299081587..66a3bceaf415d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStream.java @@ -17,35 +17,34 @@ * under the License. */ -package org.apache.hadoop.fs.s3a; +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 org.apache.hadoop.fs.FSInputStream; - import software.amazon.s3.analyticsaccelerator.S3SeekableInputStream; -import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamFactory; import software.amazon.s3.analyticsaccelerator.util.S3URI; -public class S3ASeekableStream extends FSInputStream implements StreamCapabilities { +public class AnalyticsStream extends ObjectInputStream implements StreamCapabilities { private S3SeekableInputStream inputStream; private long lastReadCurrentPos = 0; - private final String key; private volatile boolean closed; - public static final Logger LOG = LoggerFactory.getLogger(S3ASeekableStream.class); + public static final Logger LOG = LoggerFactory.getLogger(AnalyticsStream.class); - public S3ASeekableStream(String bucket, String key, - S3SeekableInputStreamFactory s3SeekableInputStreamFactory) { - this.inputStream = s3SeekableInputStreamFactory.createStream(S3URI.of(bucket, key)); - this.key = key; + public AnalyticsStream(final ObjectReadParameters parameters, final S3SeekableInputStreamFactory s3SeekableInputStreamFactory) { + super(InputStreamType.Analytics, parameters); + S3ObjectAttributes s3Attributes = parameters.getObjectAttributes(); + this.inputStream = s3SeekableInputStreamFactory.createStream(S3URI.of(s3Attributes.getBucket(), s3Attributes.getKey())); } /** @@ -139,6 +138,24 @@ public int available() throws IOException { 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) { @@ -148,7 +165,7 @@ public synchronized void close() throws IOException { inputStream = null; super.close(); } catch (IOException ioe) { - LOG.debug("Failure closing stream {}: ", key); + LOG.debug("Failure closing stream {}: ", getKey()); throw ioe; } } @@ -165,11 +182,11 @@ private void onReadFailure(IOException ioe) throws IOException { if (LOG.isDebugEnabled()) { LOG.debug("Got exception while trying to read from stream {}, " + "not trying to recover:", - key, ioe); + getKey(), ioe); } else { LOG.info("Got exception while trying to read from stream {}, " + "not trying to recover:", - key, ioe); + getKey(), ioe); } this.close(); } @@ -177,7 +194,7 @@ private void onReadFailure(IOException ioe) throws IOException { protected void throwIfClosed() throws IOException { if (closed) { - throw new IOException(key + ": " + FSExceptionMessages.STREAM_IS_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..ed230ec75f933 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStreamFactory.java @@ -0,0 +1,96 @@ +/* + * 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; + +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, false, false, + vectorContext); + } + + +} 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 index 1775fa5f05c25..ff5112df4afc9 100644 --- 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 @@ -45,13 +45,11 @@ public enum InputStreamType { */ Prefetch(StreamIntegration.PREFETCH, 2, c -> new PrefetchingInputStreamFactory()), - /** * The analytics input stream. */ - Analytics(StreamIntegration.ANALYTICS, 3, c -> { - throw new IllegalArgumentException("not yet supported"); - }), + Analytics(StreamIntegration.ANALYTICS, 3, c -> + new AnalyticsStreamFactory()), /** * The a custom input stream. @@ -60,6 +58,7 @@ public enum InputStreamType { return loadCustomFactory(c); }); + /** * Name. */ 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 index d1c96c7411dc2..69637c89cd1dc 100644 --- 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 @@ -46,7 +46,7 @@ public interface ObjectInputStreamFactory * and {@code start()}. * @param factoryBindingParameters parameters for the factory binding */ - void bind(final FactoryBindingParameters factoryBindingParameters); + void bind(final FactoryBindingParameters factoryBindingParameters) throws IOException; /** * Create a new input stream. 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 index 83c498fd0b7d0..4fa5789dd8623 100644 --- 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 @@ -88,6 +88,8 @@ public final class StreamIntegration { LoggerFactory.getLogger( "org.apache.hadoop.conf.Configuration.deprecation"); + public static final Logger LOG = LoggerFactory.getLogger(StreamIntegration.class); + /** * Warn once on use of prefetch configuration option. */ @@ -114,7 +116,6 @@ private StreamIntegration() { * @throws RuntimeException any binding/loading/instantiation problem */ public static ObjectInputStreamFactory factoryFromConfig(final Configuration conf) { - // Construct the factory. return determineInputStreamType(conf) .factory() 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 837ecb067324c..343e5cb559e51 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 @@ -91,8 +91,6 @@ public Configuration createConfiguration() { final String bufferDirBase = configuration.get(BUFFER_DIR); bufferDir = bufferDirBase + "/" + UUID.randomUUID(); configuration.set(BUFFER_DIR, bufferDir); - // When both Prefetching and Analytics Accelerator enabled Analytics Accelerator is used - configuration.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, false); return configuration; } 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 5f9df4e4dbaa3..afc89454b2f2a 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 @@ -34,9 +34,10 @@ import org.apache.hadoop.fs.statistics.IOStatistics; 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.S3ATestUtils.enablePrefetching; -import static org.apache.hadoop.fs.s3a.Constants.*; 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; @@ -72,14 +73,11 @@ public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest { private static final int INTERVAL_MILLIS = 500; private static final int BLOCK_SIZE = S_1K * 10; - @Override public Configuration createConfiguration() { Configuration conf = enablePrefetching(super.createConfiguration()); S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_BLOCK_SIZE_KEY); conf.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); - // When both Prefetching and Analytics Accelerator enabled Analytics Accelerator is used - conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, false); 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 413a1adc9aee1..3e436b0e8a691 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 @@ -91,8 +91,6 @@ public Configuration createConfiguration() { PREFETCH_BLOCK_SIZE_KEY); conf.setInt(PREFETCH_MAX_BLOCKS_COUNT, Integer.parseInt(maxBlocks)); conf.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); - // When both Prefetching and Analytics Accelerator enabled Analytics Accelerator is used - conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, false); return conf; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java index c6ecee9505101..6ee6d6c6e1192 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java @@ -28,11 +28,11 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX; -import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_ENABLED_KEY; -import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_CRT_ENABLED; +import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import org.assertj.core.api.Assertions; + import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamConfiguration; import software.amazon.s3.analyticsaccelerator.common.ConnectorConfiguration; import software.amazon.s3.analyticsaccelerator.util.PrefetchMode; @@ -42,13 +42,13 @@ public class ITestS3AS3SeekableStream extends AbstractS3ATestBase { private static final String PHYSICAL_IO_PREFIX = "physicalio"; private static final String LOGICAL_IO_PREFIX = "logicalio"; - public void testConnectorFrameWorkIntegration(boolean useCrtClient) throws IOException { + @Test + public void testConnectorFrameWorkIntegration() throws IOException { describe("Verify S3 connector framework integration"); Configuration conf = getConfiguration(); - removeBaseAndBucketOverrides(conf, ANALYTICS_ACCELERATOR_ENABLED_KEY); - conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, true); - conf.setBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, useCrtClient); + removeBaseAndBucketOverrides(conf, INPUT_STREAM_TYPE); + conf.set(INPUT_STREAM_TYPE, "Analytics"); String testFile = "s3a://noaa-cors-pds/raw/2023/017/ohfh/OHFH017d.23_.gz"; S3AFileSystem s3AFileSystem = @@ -63,16 +63,7 @@ public void testConnectorFrameWorkIntegration(boolean useCrtClient) throws IOExc } @Test - public void testConnectorFrameWorkIntegrationWithCrtClient() throws IOException { - testConnectorFrameWorkIntegration(true); - } - - @Test - public void testConnectorFrameWorkIntegrationWithoutCrtClient() throws IOException { - testConnectorFrameWorkIntegration(false); - } - - public void testConnectorFrameworkConfigurable(boolean useCrtClient) { + public void testConnectorFrameworkConfigurable() { describe("Verify S3 connector framework reads configuration"); Configuration conf = getConfiguration(); @@ -86,33 +77,23 @@ public void testConnectorFrameworkConfigurable(boolean useCrtClient) { conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", 1); - conf.setBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, useCrtClient); - ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); S3SeekableInputStreamConfiguration configuration = S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration); - assertSame("S3ASeekableStream configuration is not set to expected value", - PrefetchMode.ALL, configuration.getLogicalIOConfiguration().getPrefetchingMode()); - - assertEquals("S3ASeekableStream configuration is not set to expected value", - 1, configuration.getPhysicalIOConfiguration().getBlobStoreCapacity()); - } + Assertions.assertThat(configuration.getLogicalIOConfiguration().getPrefetchingMode()) + .as("AnalyticsStream configuration is not set to expected value") + .isSameAs(PrefetchMode.ALL); - @Test - public void testConnectorFrameworkConfigurableWithoutCrtClient() throws IOException { - testConnectorFrameworkConfigurable(false); - } - - @Test - public void testConnectorFrameworkConfigurableWithCrtClient() throws IOException { - testConnectorFrameworkConfigurable(true); + Assertions.assertThat(configuration.getPhysicalIOConfiguration().getBlobStoreCapacity()) + .as("AnalyticsStream configuration is not set to expected value") + .isEqualTo(1); } @Test - public void testInvalidConfigurationThrows() { + public void testInvalidConfigurationThrows() throws Exception { describe("Verify S3 connector framework throws with invalid configuration"); Configuration conf = getConfiguration(); @@ -123,8 +104,8 @@ public void testInvalidConfigurationThrows() { ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); - assertThrows("S3ASeekableStream illegal configuration does not throw", - IllegalArgumentException.class, () -> - S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration)); + Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> + S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration)); } } 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 571b4b1ad255d..dd2dc3e91ee5a 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; @@ -587,8 +588,8 @@ public static void skipIfAnalyticsAcceleratorEnabled( } public static boolean isAnalyticsAcceleratorEnabled(final Configuration conf) { - return conf.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, - ANALYTICS_ACCELERATOR_ENABLED_DEFAULT); + return conf.getEnum(INPUT_STREAM_TYPE, + InputStreamType.Classic) == InputStreamType.Analytics; } /** 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 fac461371e62a..515313b009dc2 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 @@ -168,11 +168,10 @@ 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"); skipIfAnalyticsAcceleratorEnabled(getConfiguration(), - "S3ASeekableInputStream does not support InputStreamStatistics"); - + "AnalyticsStream does not support InputStreamStatistics"); S3AFileSystem fs = getFileSystem(); Path destFile = methodSubPath("file.txt"); fs.delete(destFile.getParent(), true); @@ -251,7 +250,7 @@ public void testCostOfSavingLoadingPendingFile() throws Throwable { describe("Verify costs of saving .pending file under a magic path"); skipIfAnalyticsAcceleratorEnabled(getConfiguration(), - "S3ASeekableInputStream does not support InputStreamStatistics"); + "AnalyticsStream does not support InputStreamStatistics"); 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/commit/ITestS3ACommitterFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java index a500bfb76a322..9057d1e366cc3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java @@ -184,7 +184,7 @@ public void setup() throws Exception { FileSystem.closeAllForUGI(UserGroupInformation.getCurrentUser()); super.setup(); skipIfAnalyticsAcceleratorEnabled(getConfiguration(), - "S3ASeekableInputStream does not support InputStreamStatistics"); + "AnalyticsStream does not support InputStreamStatistics"); jobId = randomJobId(); attempt0 = "attempt_" + jobId + "_m_000000_0"; taskAttempt0 = TaskAttemptID.forName(attempt0); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java index b89740ae3120c..542f6f2b7c772 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java @@ -79,7 +79,7 @@ protected String getCommitterName() { public void setup() throws Exception { super.setup(); skipIfAnalyticsAcceleratorEnabled(getConfiguration(), - "S3ASeekableInputStream does not support InputStreamStatistics"); + "AnalyticsStream does not support InputStreamStatistics"); CommitUtils.verifyIsMagicCommitFS(getFileSystem()); } 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 5a3f7bb8fdbb2..fb1cfb781e710 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 @@ -44,7 +44,7 @@ public class ITestS3AFileSystemStatistic extends AbstractS3ATestBase { */ @Test public void testBytesReadWithStream() throws IOException { - // Assertions will fail as {@link S3ASeekableInputStream} + // Assertions will fail as {@link AnalyticsStream} // do not support S3AFileSystemStatistics yet. skipIfAnalyticsAcceleratorEnabled(getConfiguration(), "S3SeekableStream does not support File System Statistics"); From dc2dc636a4fe4be18f6018ceb3f57183166cf1e5 Mon Sep 17 00:00:00 2001 From: Ahmar Suhail Date: Mon, 3 Feb 2025 13:26:00 +0000 Subject: [PATCH 5/5] enable MPU --- .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) 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 d42fbe3f8035b..3620d741c25a0 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 @@ -336,10 +336,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, */ private ExecutorServiceFuturePool futurePool; - - // If true, S3SeekableInputStream from Analytics Accelerator for Amazon S3 will be used. - private boolean analyticsAcceleratorEnabled; - private int executorCapacity; private long multiPartThreshold; public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class); @@ -648,17 +644,6 @@ public void initialize(URI name, Configuration originalConf) dirOperationsPurgeUploads = conf.getBoolean(DIRECTORY_OPERATIONS_PURGE_UPLOADS, s3ExpressStore); - - this.analyticsAcceleratorEnabled = conf.getEnum(INPUT_STREAM_TYPE, InputStreamType.Classic) == InputStreamType.Analytics; - - this.isMultipartUploadEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED, - DEFAULT_MULTIPART_UPLOAD_ENABLED); - - if(this.analyticsAcceleratorEnabled) { - // Temp change: Analytics Accelerator with S3AsyncClient do not support Multi-part upload. - this.isMultipartUploadEnabled = false; - } - // multipart copy and upload are the same; this just makes it explicit this.isMultipartCopyEnabled = isMultipartUploadEnabled; @@ -1836,7 +1821,6 @@ private FSDataInputStream executeOpen( final Path path, final OpenFileSupport.OpenFileInformation fileInformation) throws IOException { - // create the input stream statistics before opening // the file so that the time to prepare to open the file is included. S3AInputStreamStatistics inputStreamStats = @@ -1862,6 +1846,7 @@ private FSDataInputStream executeOpen( (requirements.vectorSupported() ? requirements.vectoredIOContext().getVectoredActiveRangeReads() : 0); + // create an executor which is a subset of the // bounded thread pool. final SemaphoredDelegatingExecutor pool = new SemaphoredDelegatingExecutor(