From 1af2a4015821acfe6d39394dd9f961bac25d5f21 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 6 Dec 2024 18:37:40 +0000 Subject: [PATCH 1/7] HADOOP-19354. S3AInputStream to be created by factory under S3AStore First iteration * Factory interface with a parameter object creation method * Base class AbstractS3AInputStream for all streams to create * S3AInputStream subclasses that and has a factory * Production and test code to use it Not done * Input stream callbacks pushed down to S3Store * S3Store to dynamically choose factory at startup, stop in close() * S3Store to implement the factory interface, completing final binding operations (callbacks, stats) Change-Id: I8d0f86ca1f3463d4987a43924f155ce0c0644180 --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 123 +++--- .../apache/hadoop/fs/s3a/S3AInputStream.java | 361 +++++------------- .../hadoop/fs/s3a/S3AReadOpContext.java | 2 +- .../org/apache/hadoop/fs/s3a/S3AStore.java | 31 +- .../hadoop/fs/s3a/impl/ClientManager.java | 4 - .../hadoop/fs/s3a/impl/S3AStoreImpl.java | 68 +++- .../s3a/prefetch/S3ACachingInputStream.java | 4 +- .../s3a/prefetch/S3AInMemoryInputStream.java | 4 +- .../prefetch/S3APrefetchingInputStream.java | 4 +- .../fs/s3a/prefetch/S3ARemoteInputStream.java | 6 +- .../fs/s3a/prefetch/S3ARemoteObject.java | 5 +- .../s3a/streams/AbstractS3AInputStream.java | 320 ++++++++++++++++ .../streams/ClassicInputStreamFactory.java | 41 ++ .../s3a/streams/FactoryStreamParameters.java | 148 +++++++ .../fs/s3a/streams/InputStreamFactory.java | 49 +++ .../fs/s3a/streams/StreamReadCallbacks.java | 64 ++++ .../hadoop/fs/s3a/streams/package-info.java | 28 ++ .../fs/contract/s3a/ITestS3AContractSeek.java | 2 +- .../hadoop/fs/s3a/ITestS3AConfiguration.java | 18 +- .../fs/s3a/TestS3AInputStreamRetry.java | 26 +- .../fs/s3a/prefetch/MockS3ARemoteObject.java | 6 +- .../fs/s3a/prefetch/S3APrefetchFakes.java | 12 +- .../prefetch/TestS3ARemoteInputStream.java | 4 +- .../fs/s3a/prefetch/TestS3ARemoteObject.java | 4 +- .../src/test/resources/log4j.properties | 4 + 25 files changed, 962 insertions(+), 376 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/AbstractS3AInputStream.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/ClassicInputStreamFactory.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/FactoryStreamParameters.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/InputStreamFactory.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/StreamReadCallbacks.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/package-info.java 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..53a785bb62d0c 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 @@ -149,6 +149,10 @@ 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.streams.ClassicInputStreamFactory; +import org.apache.hadoop.fs.s3a.streams.FactoryStreamParameters; +import org.apache.hadoop.fs.s3a.streams.InputStreamFactory; +import org.apache.hadoop.fs.s3a.streams.StreamReadCallbacks; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl; import org.apache.hadoop.fs.statistics.DurationTracker; @@ -301,9 +305,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private String username; - /** - * Store back end. - */ private S3AStore store; /** @@ -353,7 +354,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; /** @@ -788,12 +788,12 @@ 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(); // The filesystem is now ready to perform operations against // S3 // This initiates a probe against S3 for the bucket existing. @@ -836,7 +836,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 +845,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 +857,9 @@ protected S3AStore createS3AStore(final ClientManager clientManager, .withReadRateLimiter(unlimitedRate()) .withWriteRateLimiter(RateLimitingFactory.create(rateLimitCapacity)) .build(); + st.init(getConf()); + st.start(); + return st; } /** @@ -1329,6 +1332,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 +1546,7 @@ public S3Client getAmazonS3Client(String reason) { @Override public S3AStore getStore() { - return store; + return S3AFileSystem.this.getStore(); } /** @@ -1663,28 +1675,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); } /** @@ -1879,7 +1871,7 @@ private FSDataInputStream executeOpen( if (this.prefetchEnabled) { Configuration configuration = getConf(); - initLocalDirAllocatorIfNotInitialized(configuration); + return new FSDataInputStream( new S3APrefetchingInputStream( readContext.build(), @@ -1887,19 +1879,30 @@ private FSDataInputStream executeOpen( createInputStreamCallbacks(auditSpan), inputStreamStats, configuration, - directoryAllocator)); + getStore().getDirectoryAllocator())); } else { + + // create the factory. + // TODO: move into S3AStore and export the factory API through + // the store, which will add some of the features (callbacks, stats) + // before invoking the real factory + InputStreamFactory factory = new ClassicInputStreamFactory(); + factory.init(getConf()); + factory.start(); + FactoryStreamParameters parameters = new FactoryStreamParameters() + .withCallbacks(createInputStreamCallbacks(auditSpan)) + .withObjectAttributes(createObjectAttributes(path, fileStatus)) + .withContext(readContext.build()) + .withStreamStatistics(inputStreamStats) + .withBoundedThreadPool(new SemaphoredDelegatingExecutor( + boundedThreadPool, + vectoredActiveRangeReads, + true, + inputStreamStats)) + .build(); + return new FSDataInputStream( - new S3AInputStream( - readContext.build(), - createObjectAttributes(path, fileStatus), - createInputStreamCallbacks(auditSpan), - inputStreamStats, - new SemaphoredDelegatingExecutor( - boundedThreadPool, - vectoredActiveRangeReads, - true, - inputStreamStats))); + factory.create(parameters)); } } @@ -1907,7 +1910,7 @@ private FSDataInputStream executeOpen( * Override point: create the callbacks for S3AInputStream. * @return an implementation of the InputStreamCallbacks, */ - private S3AInputStream.InputStreamCallbacks createInputStreamCallbacks( + private StreamReadCallbacks createInputStreamCallbacks( final AuditSpan auditSpan) { return new InputStreamCallbacksImpl(auditSpan); } @@ -1916,7 +1919,7 @@ private S3AInputStream.InputStreamCallbacks createInputStreamCallbacks( * Operations needed by S3AInputStream to read data. */ private final class InputStreamCallbacksImpl implements - S3AInputStream.InputStreamCallbacks { + StreamReadCallbacks { /** * Audit span to activate before each call. @@ -1952,7 +1955,7 @@ public ResponseInputStream getObject(GetObjectRequest request IOException { // active the audit span used for the operation try (AuditSpan span = auditSpan.activate()) { - return fsHandler.getObject(store, request, getRequestFactory()); + return fsHandler.getObject(getStore(), request, getRequestFactory()); } } @@ -1982,7 +1985,7 @@ private final class WriteOperationHelperCallbacksImpl @Retries.OnceRaw public CompleteMultipartUploadResponse completeMultipartUpload( CompleteMultipartUploadRequest request) { - return store.completeMultipartUpload(request); + return getStore().completeMultipartUpload(request); } @Override @@ -1992,7 +1995,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); } } @@ -2756,7 +2759,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 +2990,7 @@ protected DurationTrackerFactory getDurationTrackerFactory() { */ protected DurationTrackerFactory nonNullDurationTrackerFactory( DurationTrackerFactory factory) { - return store.nonNullDurationTrackerFactory(factory); + return getStore().nonNullDurationTrackerFactory(factory); } /** @@ -3025,7 +3028,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 +3176,7 @@ public void incrementWriteOperations() { protected void deleteObject(String key) throws SdkException, IOException { incrementWriteOperations(); - store.deleteObject(getRequestFactory() + getStore().deleteObject(getRequestFactory() .newDeleteObjectRequestBuilder(key) .build()); } @@ -3227,7 +3230,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 +3273,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 +3372,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 +3383,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 +3394,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 +4262,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,7 +4357,7 @@ 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; @@ -4575,7 +4578,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()); @@ -5703,7 +5706,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..c61cc2bd358dc 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,6 +39,8 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.impl.LeakReporter; +import org.apache.hadoop.fs.s3a.streams.AbstractS3AInputStream; +import org.apache.hadoop.fs.s3a.streams.FactoryStreamParameters; import org.apache.hadoop.fs.statistics.StreamStatisticNames; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; @@ -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 AbstractS3AInputStream 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,31 @@ 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()); - } + public S3AInputStream(FactoryStreamParameters parameters) { - /** - * 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(); + super(parameters); + + + this.fileLength = Optional.of(getContentLength()); + S3AReadOpContext context = getContext(); + this.changeTracker = new ChangeTracker(getUri(), + context.getChangeDetectionPolicy(), + getStreamStatistics().getChangeTrackerStatistics(), + getObjectAttributes()); + setReadahead(context.getReadahead()); + this.asyncDrainThreshold = context.getAsyncDrainThreshold(); + this.vectoredIOContext = this.getContext().getVectoredIOContext(); } /** @@ -290,7 +201,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 +210,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 +222,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 +248,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 +289,7 @@ public synchronized void seek(long targetPos) throws IOException { + " " + targetPos); } - if (this.contentLength <= 0) { + if (this.getContentLength() <= 0) { return; } @@ -414,7 +307,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 +342,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 +357,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 +406,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 +425,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 +435,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 +447,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 +490,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 +531,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 +542,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 +578,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 +589,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 +610,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 +655,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 +671,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 +696,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 +718,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 +758,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 +799,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 +850,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 +871,26 @@ 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 +901,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 +909,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 +920,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 +1008,7 @@ private void drainUnnecessaryData( remaining -= readCount; } } finally { - streamStatistics.readVectoredBytesDiscarded(drainBytes); + getS3AStreamStatistics().readVectoredBytesDiscarded(drainBytes); LOG.debug("{} bytes drained from stream ", drainBytes); } } @@ -1140,7 +1019,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 +1034,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 +1153,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 +1191,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 +1276,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); } @@ -1423,12 +1290,9 @@ public boolean hasCapability(String 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 +1305,6 @@ public boolean isObjectStreamOpen() { return wrappedStream != null; } - @Override - public IOStatistics getIOStatistics() { - return ioStatistics; - } - /** * Get the wrapped stream. * This is for testing only. @@ -1457,38 +1316,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..67fc38d2c1ef8 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 { 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..c1cdc172600d6 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,6 +45,8 @@ 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.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.impl.ChangeTracker; import org.apache.hadoop.fs.s3a.impl.ClientManager; @@ -54,6 +56,7 @@ 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 +66,14 @@ * 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 Service, IOStatisticsSource, ClientManager { /** * Acquire write capacity for operations. @@ -302,4 +309,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..d29fd14cbdf46 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 @@ -76,8 +76,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/S3AStoreImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java index db07881345500..f0f8235934570 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,10 @@ 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.s3a.Invoker; import org.apache.hadoop.fs.s3a.ProgressableProgressListener; import org.apache.hadoop.fs.s3a.Retries; @@ -74,11 +77,14 @@ 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.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; @@ -109,7 +115,8 @@ * This is where lower level storage operations are intended * to move. */ -public class S3AStoreImpl implements S3AStore { +public class S3AStoreImpl extends CompositeService + implements S3AStore { private static final Logger LOG = LoggerFactory.getLogger(S3AStoreImpl.class); @@ -165,6 +172,11 @@ public class S3AStoreImpl implements S3AStore { */ private final FileSystem.Statistics fsStatistics; + /** + * Allocator of local FS storage. + */ + private LocalDirAllocator directoryAllocator; + /** Constructor to create S3A store. */ S3AStoreImpl(StoreContextFactory storeContextFactory, ClientManager clientManager, @@ -176,6 +188,7 @@ public class S3AStoreImpl implements S3AStore { RateLimiting writeRateLimiter, AuditSpanSource auditSpanSource, @Nullable FileSystem.Statistics fsStatistics) { + super("S3AStore"); this.storeContextFactory = requireNonNull(storeContextFactory); this.clientManager = requireNonNull(clientManager); this.durationTrackerFactory = requireNonNull(durationTrackerFactory); @@ -193,10 +206,26 @@ public class S3AStoreImpl implements S3AStore { } @Override - public void close() { + protected void serviceStart() throws Exception { + super.serviceStart(); + initLocalDirAllocator(); + } + + @Override + protected void serviceStop() throws Exception { + super.serviceStop(); clientManager.close(); } + /** + * 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}. */ @Override public Duration acquireWriteCapacity(final int capacity) { @@ -808,4 +837,39 @@ 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); + } + } 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..3a0fd6d247fa8 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.streams.StreamReadCallbacks; 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; @@ -75,7 +75,7 @@ public class S3ACachingInputStream extends S3ARemoteInputStream { public S3ACachingInputStream( S3AReadOpContext context, S3ObjectAttributes s3Attributes, - S3AInputStream.InputStreamCallbacks client, + StreamReadCallbacks client, S3AInputStreamStatistics streamStatistics, Configuration conf, LocalDirAllocator localDirAllocator) { 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..a64f48dfaa2ec 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.streams.StreamReadCallbacks; /** * Provides an {@code InputStream} that allows reading from an S3 file. @@ -61,7 +61,7 @@ public class S3AInMemoryInputStream extends S3ARemoteInputStream { public S3AInMemoryInputStream( S3AReadOpContext context, S3ObjectAttributes s3Attributes, - S3AInputStream.InputStreamCallbacks client, + StreamReadCallbacks client, S3AInputStreamStatistics streamStatistics) { super(context, s3Attributes, client, streamStatistics); int fileSize = (int) s3Attributes.getLen(); 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..f59b5b912ac2f 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 @@ -34,10 +34,10 @@ 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.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.streams.StreamReadCallbacks; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; @@ -90,7 +90,7 @@ public class S3APrefetchingInputStream public S3APrefetchingInputStream( S3AReadOpContext context, S3ObjectAttributes s3Attributes, - S3AInputStream.InputStreamCallbacks client, + StreamReadCallbacks client, S3AInputStreamStatistics streamStatistics, Configuration conf, LocalDirAllocator localDirAllocator) { 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..3f609b1b9532b 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.streams.StreamReadCallbacks; 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 StreamReadCallbacks client; /** Used for reporting input stream access statistics. */ private final S3AInputStreamStatistics streamStatistics; @@ -124,7 +124,7 @@ public abstract class S3ARemoteInputStream public S3ARemoteInputStream( S3AReadOpContext context, S3ObjectAttributes s3Attributes, - S3AInputStream.InputStreamCallbacks client, + StreamReadCallbacks client, S3AInputStreamStatistics streamStatistics) { this.context = requireNonNull(context); 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..446db4f88ac64 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.streams.StreamReadCallbacks; 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 StreamReadCallbacks client; /** * Used for reporting input stream access statistics. @@ -100,7 +101,7 @@ public class S3ARemoteObject { public S3ARemoteObject( S3AReadOpContext context, S3ObjectAttributes s3Attributes, - S3AInputStream.InputStreamCallbacks client, + StreamReadCallbacks client, S3AInputStreamStatistics streamStatistics, ChangeTracker changeTracker) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/AbstractS3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/AbstractS3AInputStream.java new file mode 100644 index 0000000000000..645df6a113ef4 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/AbstractS3AInputStream.java @@ -0,0 +1,320 @@ +/* + * 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.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 org.apache.hadoop.util.Preconditions; + +import static java.util.Objects.requireNonNull; +import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.util.StringUtils.toLowerCase; + +/** + * Base class for input streams returned by the factory, and therefore + * used within S3A code. + */ +public abstract class AbstractS3AInputStream extends FSInputStream + implements StreamCapabilities, IOStatisticsSource { + + private static final Logger LOG = + LoggerFactory.getLogger(AbstractS3AInputStream.class); + + /** + * IOStatistics report. + */ + protected final IOStatistics ioStatistics; + + /** + * Read-specific operation context structure. + */ + private final S3AReadOpContext context; + + /** + * Callbacks for reading input stream data from the S3 Store. + */ + private final StreamReadCallbacks 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; + + 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; + + /** + * Requested input policy. + */ + private S3AInputPolicy inputPolicy; + + /** + * Constructor. + * @param parameters extensible parameter list. + */ + protected AbstractS3AInputStream( + FactoryStreamParameters parameters) { + + objectAttributes = parameters.getObjectAttributes(); + Preconditions.checkArgument(isNotEmpty(objectAttributes.getBucket()), + "No Bucket"); + Preconditions.checkArgument(isNotEmpty(objectAttributes.getKey()), "No Key"); + long l = objectAttributes.getLen(); + Preconditions.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, + () -> 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; + } + + @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; + } + } + + + protected S3AReadOpContext getContext() { + return context; + } + + protected StreamReadCallbacks getCallbacks() { + return callbacks; + } + + protected ExecutorService getBoundedThreadPool() { + return boundedThreadPool; + } + + protected String getUri() { + return uri; + } + + protected String getBucket() { + return bucket; + } + + protected String getKey() { + return key; + } + + protected String getPathStr() { + return pathStr; + } + + protected long getContentLength() { + return contentLength; + } + + protected S3AInputStreamStatistics getStreamStatistics() { + return streamStatistics; + } + + protected IOStatisticsAggregator getThreadIOStatistics() { + return threadIOStatistics; + } + + protected S3ObjectAttributes getObjectAttributes() { + return objectAttributes; + } +} + + diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/ClassicInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/ClassicInputStreamFactory.java new file mode 100644 index 0000000000000..a2720b0d05058 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/ClassicInputStreamFactory.java @@ -0,0 +1,41 @@ +/* + * 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.streams; + +import java.io.IOException; + +import org.apache.hadoop.fs.s3a.S3AInputStream; +import org.apache.hadoop.service.AbstractService; + +/** + * Factory of classic {@link S3AInputStream} instances. + */ +public class ClassicInputStreamFactory extends AbstractService + implements InputStreamFactory { + + public ClassicInputStreamFactory() { + super("ClassicInputStreamFactory"); + } + + @Override + public AbstractS3AInputStream create(final FactoryStreamParameters parameters) + throws IOException { + return new S3AInputStream(parameters); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/FactoryStreamParameters.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/FactoryStreamParameters.java new file mode 100644 index 0000000000000..c9f2208c505d9 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/FactoryStreamParameters.java @@ -0,0 +1,148 @@ +/* + * 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.streams; + +import java.util.concurrent.ExecutorService; + +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 input streams created through + * {@link InputStreamFactory}. + * It is designed to be extensible; the {@link #build()} + * operation does not freeze the parameters -instead it simply + * verifies that all required values are set. + */ +public final class FactoryStreamParameters { + + private S3AReadOpContext context; + + private S3ObjectAttributes objectAttributes; + + private StreamReadCallbacks callbacks; + + private S3AInputStreamStatistics streamStatistics; + + private ExecutorService boundedThreadPool; + + /** + * Read operation context. + */ + public S3AReadOpContext getContext() { + return context; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public FactoryStreamParameters withContext(S3AReadOpContext value) { + context = value; + return this; + } + + /** + * Attributes of the object. + */ + public S3ObjectAttributes getObjectAttributes() { + return objectAttributes; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public FactoryStreamParameters withObjectAttributes(S3ObjectAttributes value) { + objectAttributes = value; + return this; + } + + /** + * Callbacks to the store. + */ + public StreamReadCallbacks getCallbacks() { + return callbacks; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public FactoryStreamParameters withCallbacks(StreamReadCallbacks value) { + callbacks = value; + return this; + } + + /** + * Stream statistics. + */ + public S3AInputStreamStatistics getStreamStatistics() { + return streamStatistics; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public FactoryStreamParameters withStreamStatistics(S3AInputStreamStatistics value) { + streamStatistics = value; + return this; + } + + /** + * Bounded thread pool for submitting asynchronous + * work. + */ + public ExecutorService getBoundedThreadPool() { + return boundedThreadPool; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public FactoryStreamParameters withBoundedThreadPool(ExecutorService value) { + boundedThreadPool = value; + return this; + } + + /** + * Validate that all attributes are as expected. + * Mock tests can skip this if required. + * @return the object. + */ + public FactoryStreamParameters build() { + requireNonNull(boundedThreadPool, "boundedThreadPool"); + requireNonNull(callbacks, "callbacks"); + requireNonNull(context, "context"); + requireNonNull(objectAttributes, "objectAttributes"); + requireNonNull(streamStatistics, "streamStatistics"); + requireNonNull(boundedThreadPool, "boundedThreadPool"); + return this; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/InputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/InputStreamFactory.java new file mode 100644 index 0000000000000..56b158e50f691 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/InputStreamFactory.java @@ -0,0 +1,49 @@ +/* + * 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.streams; + +import java.io.IOException; + +import org.apache.hadoop.service.Service; + +/** + * A Factory for input 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 FactoryStreamParameters} class, rather than change the + * interface signature. + */ +public interface InputStreamFactory extends Service { + + /** + * Create a new input stream. + * @param parameters parameters. + * @return the input stream + * @throws problem creating the stream. + */ + AbstractS3AInputStream create(FactoryStreamParameters parameters) + throws IOException; + +} + diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/StreamReadCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/StreamReadCallbacks.java new file mode 100644 index 0000000000000..c4441af07a2e2 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/StreamReadCallbacks.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.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 input stream data from the S3 Store. + */ +public interface StreamReadCallbacks 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/streams/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/package-info.java new file mode 100644 index 0000000000000..bb79fb129ef2b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/package-info.java @@ -0,0 +1,28 @@ +/* + * 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. + */ + +/** + * Stream factory and support. + */ + +@InterfaceAudience.LimitedPrivate("Extension Libraries") +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.streams; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file 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/TestS3AInputStreamRetry.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java index 6eccdc23dd5d5..a0a12eb1b3899 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.streams.FactoryStreamParameters; +import org.apache.hadoop.fs.s3a.streams.StreamReadCallbacks; 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) { + StreamReadCallbacks streamCallback) { Path path = new Path("test-path"); String eTag = "test-etag"; String versionId = "test-version-id"; @@ -187,12 +189,14 @@ private S3AInputStream getMockedS3AInputStream( s3AFileStatus, NoopSpan.INSTANCE); - return new S3AInputStream( - s3AReadOpContext, - s3ObjectAttributes, - streamCallback, - s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics(), - null); + FactoryStreamParameters parameters = new FactoryStreamParameters() + .withCallbacks(streamCallback) + .withObjectAttributes(s3ObjectAttributes) + .withContext(s3AReadOpContext) + .withStreamStatistics( s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics()) + .withBoundedThreadPool(null); + + return new S3AInputStream(parameters); } /** @@ -203,7 +207,7 @@ private S3AInputStream getMockedS3AInputStream( * @param ex exception to raise on failure * @return mocked object. */ - private S3AInputStream.InputStreamCallbacks failingInputStreamCallbacks( + private StreamReadCallbacks failingInputStreamCallbacks( final RuntimeException ex) { GetObjectResponse objectResponse = GetObjectResponse.builder() @@ -238,7 +242,7 @@ private S3AInputStream.InputStreamCallbacks failingInputStreamCallbacks( * @param ex exception to raise on failure * @return mocked object. */ - private S3AInputStream.InputStreamCallbacks maybeFailInGetCallback( + private StreamReadCallbacks maybeFailInGetCallback( final RuntimeException ex, final Function failurePredicate) { GetObjectResponse objectResponse = GetObjectResponse.builder() @@ -259,13 +263,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 StreamReadCallbacks mockInputStreamCallback( final RuntimeException ex, final Function failurePredicate, final Function> streamFactory) { - return new S3AInputStream.InputStreamCallbacks() { + return new StreamReadCallbacks() { private int attempt = 0; @Override 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..4a572441c56b7 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.streams.StreamReadCallbacks; import org.apache.hadoop.util.functional.CallableRaisingIOE; /** @@ -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 StreamReadCallbacks createClient(String bucketName) { + return new StreamReadCallbacks() { @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..eb7ed4fcaabfe 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.streams.StreamReadCallbacks; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryPolicy; @@ -187,7 +187,7 @@ public static ResponseInputStream createS3ObjectInputStream( AbortableInputStream.create(new ByteArrayInputStream(buffer), () -> {})); } - public static S3AInputStream.InputStreamCallbacks createInputStreamCallbacks( + public static StreamReadCallbacks createInputStreamCallbacks( String bucket) { GetObjectResponse objectResponse = GetObjectResponse.builder() @@ -197,7 +197,7 @@ public static S3AInputStream.InputStreamCallbacks createInputStreamCallbacks( ResponseInputStream responseInputStream = createS3ObjectInputStream(objectResponse, new byte[8]); - return new S3AInputStream.InputStreamCallbacks() { + return new StreamReadCallbacks() { @Override public ResponseInputStream getObject(GetObjectRequest request) { return responseInputStream; @@ -238,7 +238,7 @@ public static S3ARemoteInputStream createInputStream( prefetchBlockSize, prefetchBlockCount); - S3AInputStream.InputStreamCallbacks callbacks = + StreamReadCallbacks callbacks = createInputStreamCallbacks(bucket); S3AInputStreamStatistics stats = s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics(); @@ -289,7 +289,7 @@ public static class FakeS3AInMemoryInputStream public FakeS3AInMemoryInputStream( S3AReadOpContext context, S3ObjectAttributes s3Attributes, - S3AInputStream.InputStreamCallbacks client, + StreamReadCallbacks client, S3AInputStreamStatistics streamStatistics) { super(context, s3Attributes, client, streamStatistics); } @@ -391,7 +391,7 @@ public static class FakeS3ACachingInputStream extends S3ACachingInputStream { public FakeS3ACachingInputStream( S3AReadOpContext context, S3ObjectAttributes s3Attributes, - S3AInputStream.InputStreamCallbacks client, + StreamReadCallbacks client, S3AInputStreamStatistics streamStatistics) { super(context, s3Attributes, client, streamStatistics, CONF, new LocalDirAllocator( 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..35a08cc792a98 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.streams.StreamReadCallbacks; import org.apache.hadoop.test.AbstractHadoopTestBase; import static org.assertj.core.api.Assertions.assertThat; @@ -53,7 +53,7 @@ public class TestS3ARemoteInputStream extends AbstractHadoopTestBase { private final ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(threadPool); - private final S3AInputStream.InputStreamCallbacks client = + private final StreamReadCallbacks client = MockS3ARemoteObject.createClient("bucket"); @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..6cee43bb6c6b5 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.streams.StreamReadCallbacks; import org.apache.hadoop.test.AbstractHadoopTestBase; public class TestS3ARemoteObject extends AbstractHadoopTestBase { @@ -40,7 +40,7 @@ public class TestS3ARemoteObject extends AbstractHadoopTestBase { private final ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(threadPool); - private final S3AInputStream.InputStreamCallbacks client = + private final StreamReadCallbacks client = MockS3ARemoteObject.createClient("bucket"); @Test diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties index 7442a357f9777..f548e7c98f9eb 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 f7ea58411962088c4c7d0e7f692ba2ec65ced13d Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Sun, 8 Dec 2024 18:07:14 +0000 Subject: [PATCH 2/7] HADOOP-19354. S3AInputStream to be created by factory under S3AStore Revision API: Make clear this is part of the fundamental store Model: * abstract stream class is now ObjectInputStream * interface is ObjectInputStreamFactory * move to package org.apache.hadoop.fs.s3a.impl.model Implementation: Prefetching stream is created this way too; adds one extra parameter. Maybe we should pass conf down too Change-Id: I5bbb5dfe585528b047a649b6c82a9d0318c7e91e --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 81 +++++++++---------- .../apache/hadoop/fs/s3a/S3AInputStream.java | 10 ++- .../ClassicObjectInputStreamFactory.java} | 15 ++-- .../hadoop/fs/s3a/impl/S3AStoreImpl.java | 1 - .../model/ObjectInputStream.java} | 19 ++--- .../model/ObjectInputStreamCallbacks.java} | 6 +- .../model/ObjectInputStreamFactory.java} | 14 ++-- .../model/ObjectReadParameters.java} | 51 ++++++++---- .../{streams => impl/model}/package-info.java | 14 ++-- .../PrefetchingInputStreamFactory.java | 42 ++++++++++ .../s3a/prefetch/S3ACachingInputStream.java | 4 +- .../s3a/prefetch/S3AInMemoryInputStream.java | 4 +- .../prefetch/S3APrefetchingInputStream.java | 47 ++++++----- .../fs/s3a/prefetch/S3ARemoteInputStream.java | 6 +- .../fs/s3a/prefetch/S3ARemoteObject.java | 6 +- .../fs/s3a/TestS3AInputStreamRetry.java | 16 ++-- .../fs/s3a/prefetch/MockS3ARemoteObject.java | 6 +- .../fs/s3a/prefetch/S3APrefetchFakes.java | 12 +-- .../prefetch/TestS3ARemoteInputStream.java | 4 +- .../fs/s3a/prefetch/TestS3ARemoteObject.java | 4 +- 20 files changed, 222 insertions(+), 140 deletions(-) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/{streams/ClassicInputStreamFactory.java => impl/ClassicObjectInputStreamFactory.java} (68%) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/{streams/AbstractS3AInputStream.java => impl/model/ObjectInputStream.java} (94%) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/{streams/StreamReadCallbacks.java => impl/model/ObjectInputStreamCallbacks.java} (92%) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/{streams/InputStreamFactory.java => impl/model/ObjectInputStreamFactory.java} (75%) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/{streams/FactoryStreamParameters.java => impl/model/ObjectReadParameters.java} (69%) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/{streams => impl/model}/package-info.java (69%) 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/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 53a785bb62d0c..e45a72cd80f51 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,11 +148,11 @@ import org.apache.hadoop.fs.s3a.impl.StoreContextFactory; import org.apache.hadoop.fs.s3a.impl.UploadContentProviders; import org.apache.hadoop.fs.s3a.impl.CSEUtils; -import org.apache.hadoop.fs.s3a.prefetch.S3APrefetchingInputStream; -import org.apache.hadoop.fs.s3a.streams.ClassicInputStreamFactory; -import org.apache.hadoop.fs.s3a.streams.FactoryStreamParameters; -import org.apache.hadoop.fs.s3a.streams.InputStreamFactory; -import org.apache.hadoop.fs.s3a.streams.StreamReadCallbacks; +import org.apache.hadoop.fs.s3a.prefetch.PrefetchingInputStreamFactory; +import org.apache.hadoop.fs.s3a.impl.ClassicObjectInputStreamFactory; +import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamFactory; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamCallbacks; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl; import org.apache.hadoop.fs.statistics.DurationTracker; @@ -168,6 +168,7 @@ import org.apache.hadoop.fs.store.audit.ActiveThreadSpanSource; import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.fs.store.audit.AuditSpanSource; +import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.token.DelegationTokenIssuer; @@ -1868,58 +1869,54 @@ private FSDataInputStream executeOpen( auditSpan); fileInformation.applyOptions(readContext); LOG.debug("Opening '{}'", readContext); + final SemaphoredDelegatingExecutor pool = new SemaphoredDelegatingExecutor( + boundedThreadPool, + vectoredActiveRangeReads, + true, + inputStreamStats); + ObjectReadParameters parameters = new ObjectReadParameters() + .withBoundedThreadPool(pool) + .withCallbacks(createInputStreamCallbacks(auditSpan)) + .withContext(readContext.build()) + .withDirectoryAllocator(getStore().getDirectoryAllocator()) + .withObjectAttributes(createObjectAttributes(path, fileStatus)) + .withStreamStatistics(inputStreamStats) + .build(); - if (this.prefetchEnabled) { - Configuration configuration = getConf(); - - return new FSDataInputStream( - new S3APrefetchingInputStream( - readContext.build(), - createObjectAttributes(path, fileStatus), - createInputStreamCallbacks(auditSpan), - inputStreamStats, - configuration, - getStore().getDirectoryAllocator())); - } else { - - // create the factory. - // TODO: move into S3AStore and export the factory API through - // the store, which will add some of the features (callbacks, stats) - // before invoking the real factory - InputStreamFactory factory = new ClassicInputStreamFactory(); + // TODO: move into S3AStore and export the factory API through + // the store, which will add some of the features (callbacks, stats) + // before invoking the real factory + ObjectInputStreamFactory factory = null; + try { + // Choose factory. + if (prefetchEnabled) { + factory = new PrefetchingInputStreamFactory(); + } else { + factory = new ClassicObjectInputStreamFactory(); + } factory.init(getConf()); factory.start(); - FactoryStreamParameters parameters = new FactoryStreamParameters() - .withCallbacks(createInputStreamCallbacks(auditSpan)) - .withObjectAttributes(createObjectAttributes(path, fileStatus)) - .withContext(readContext.build()) - .withStreamStatistics(inputStreamStats) - .withBoundedThreadPool(new SemaphoredDelegatingExecutor( - boundedThreadPool, - vectoredActiveRangeReads, - true, - inputStreamStats)) - .build(); - - return new FSDataInputStream( - factory.create(parameters)); + return new FSDataInputStream(factory.readObject(parameters)); + } finally { + IOUtils.cleanupWithLogger(LOG, factory); } + } /** - * Override point: create the callbacks for S3AInputStream. - * @return an implementation of the InputStreamCallbacks, + * Override point: create the callbacks for ObjectInputStream. + * @return an implementation of callbacks, */ - private StreamReadCallbacks createInputStreamCallbacks( + private ObjectInputStreamCallbacks createInputStreamCallbacks( final AuditSpan auditSpan) { return new InputStreamCallbacksImpl(auditSpan); } /** - * Operations needed by S3AInputStream to read data. + * Operations needed by ObjectInputStreams to read data. */ private final class InputStreamCallbacksImpl implements - StreamReadCallbacks { + ObjectInputStreamCallbacks { /** * Audit span to activate before each call. 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 c61cc2bd358dc..9e9b3ca96dbd4 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 @@ -39,8 +39,8 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.impl.LeakReporter; -import org.apache.hadoop.fs.s3a.streams.AbstractS3AInputStream; -import org.apache.hadoop.fs.s3a.streams.FactoryStreamParameters; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStream; +import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; import org.apache.hadoop.fs.statistics.StreamStatisticNames; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; @@ -87,7 +87,7 @@ */ @InterfaceAudience.Private @InterfaceStability.Evolving -public class S3AInputStream extends AbstractS3AInputStream implements CanSetReadahead, +public class S3AInputStream extends ObjectInputStream implements CanSetReadahead, CanUnbuffer, StreamCapabilities, IOStatisticsSource { public static final String E_NEGATIVE_READAHEAD_VALUE @@ -179,8 +179,10 @@ public class S3AInputStream extends AbstractS3AInputStream implements CanSetRead * Create the stream. * This does not attempt to open it; that is only done on the first * actual read() operation. + * + * @param parameters creation parameters. */ - public S3AInputStream(FactoryStreamParameters parameters) { + public S3AInputStream(ObjectReadParameters parameters) { super(parameters); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/ClassicInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClassicObjectInputStreamFactory.java similarity index 68% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/ClassicInputStreamFactory.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClassicObjectInputStreamFactory.java index a2720b0d05058..76b91821e691c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/ClassicInputStreamFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClassicObjectInputStreamFactory.java @@ -16,25 +16,28 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.streams; +package org.apache.hadoop.fs.s3a.impl; import java.io.IOException; import org.apache.hadoop.fs.s3a.S3AInputStream; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStream; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamFactory; +import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; import org.apache.hadoop.service.AbstractService; /** * Factory of classic {@link S3AInputStream} instances. */ -public class ClassicInputStreamFactory extends AbstractService - implements InputStreamFactory { +public class ClassicObjectInputStreamFactory extends AbstractService + implements ObjectInputStreamFactory { - public ClassicInputStreamFactory() { - super("ClassicInputStreamFactory"); + public ClassicObjectInputStreamFactory() { + super("ClassicObjectInputStreamFactory"); } @Override - public AbstractS3AInputStream create(final FactoryStreamParameters parameters) + public ObjectInputStream readObject(final ObjectReadParameters parameters) throws IOException { return new S3AInputStream(parameters); } 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 f0f8235934570..8ca3bead0f161 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 @@ -837,7 +837,6 @@ public CompleteMultipartUploadResponse completeMultipartUpload( return getS3Client().completeMultipartUpload(request); } - /** * Get the directory allocator. * @return the directory allocator diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/AbstractS3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStream.java similarity index 94% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/AbstractS3AInputStream.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStream.java index 645df6a113ef4..2a9bde9bf3422 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/AbstractS3AInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStream.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.streams; +package org.apache.hadoop.fs.s3a.impl.model; import java.io.IOException; import java.util.concurrent.ExecutorService; @@ -45,14 +45,15 @@ import static org.apache.hadoop.util.StringUtils.toLowerCase; /** - * Base class for input streams returned by the factory, and therefore - * used within S3A code. + * A stream of data from an S3 object. + * The blase class includes common methods, stores + * common data and incorporates leak tracking. */ -public abstract class AbstractS3AInputStream extends FSInputStream +public abstract class ObjectInputStream extends FSInputStream implements StreamCapabilities, IOStatisticsSource { private static final Logger LOG = - LoggerFactory.getLogger(AbstractS3AInputStream.class); + LoggerFactory.getLogger(ObjectInputStream.class); /** * IOStatistics report. @@ -67,7 +68,7 @@ public abstract class AbstractS3AInputStream extends FSInputStream /** * Callbacks for reading input stream data from the S3 Store. */ - private final StreamReadCallbacks callbacks; + private final ObjectInputStreamCallbacks callbacks; /** * Thread pool used for vectored IO operation. @@ -124,8 +125,8 @@ public abstract class AbstractS3AInputStream extends FSInputStream * Constructor. * @param parameters extensible parameter list. */ - protected AbstractS3AInputStream( - FactoryStreamParameters parameters) { + protected ObjectInputStream( + ObjectReadParameters parameters) { objectAttributes = parameters.getObjectAttributes(); Preconditions.checkArgument(isNotEmpty(objectAttributes.getBucket()), @@ -276,7 +277,7 @@ protected S3AReadOpContext getContext() { return context; } - protected StreamReadCallbacks getCallbacks() { + protected ObjectInputStreamCallbacks getCallbacks() { return callbacks; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/StreamReadCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStreamCallbacks.java similarity index 92% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/StreamReadCallbacks.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStreamCallbacks.java index c4441af07a2e2..44fb0f068ee02 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/StreamReadCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStreamCallbacks.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.streams; +package org.apache.hadoop.fs.s3a.impl.model; import java.io.Closeable; import java.io.IOException; @@ -30,9 +30,9 @@ import org.apache.hadoop.util.functional.CallableRaisingIOE; /** - * Callbacks for reading input stream data from the S3 Store. + * Callbacks for reading objectd data from the S3 Store. */ -public interface StreamReadCallbacks extends Closeable { +public interface ObjectInputStreamCallbacks extends Closeable { /** * Create a GET request builder. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/InputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStreamFactory.java similarity index 75% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/InputStreamFactory.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStreamFactory.java index 56b158e50f691..a817291606fb7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/InputStreamFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStreamFactory.java @@ -16,14 +16,14 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.streams; +package org.apache.hadoop.fs.s3a.impl.model; import java.io.IOException; import org.apache.hadoop.service.Service; /** - * A Factory for input streams. + * A Factory for {@link ObjectInputStream} streams. *

* This class is instantiated during initialization of * {@code S3AStore}, it then follows the same service @@ -31,18 +31,20 @@ *

* Note for maintainers: do try and keep this mostly stable. * If new parameters need to be added, expand the - * {@link FactoryStreamParameters} class, rather than change the + * {@link ObjectReadParameters} class, rather than change the * interface signature. */ -public interface InputStreamFactory extends Service { +public interface ObjectInputStreamFactory extends Service { /** * 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 problem creating the stream. + * @throws IOException problem creating the stream. */ - AbstractS3AInputStream create(FactoryStreamParameters parameters) + ObjectInputStream readObject(ObjectReadParameters parameters) throws IOException; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/FactoryStreamParameters.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectReadParameters.java similarity index 69% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/FactoryStreamParameters.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectReadParameters.java index c9f2208c505d9..e8114d7e5e7dd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/FactoryStreamParameters.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectReadParameters.java @@ -16,10 +16,11 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.streams; +package org.apache.hadoop.fs.s3a.impl.model; 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; @@ -27,24 +28,31 @@ import static java.util.Objects.requireNonNull; /** - * Parameters for input streams created through - * {@link InputStreamFactory}. - * It is designed to be extensible; the {@link #build()} + * Parameters for object input streams created through + * {@link ObjectInputStreamFactory}. + * It is designed to support extra parameters added + * in future. + *

Note that the {@link #build()} * operation does not freeze the parameters -instead it simply * verifies that all required values are set. */ -public final class FactoryStreamParameters { +public final class ObjectReadParameters { private S3AReadOpContext context; private S3ObjectAttributes objectAttributes; - private StreamReadCallbacks callbacks; + private ObjectInputStreamCallbacks callbacks; private S3AInputStreamStatistics streamStatistics; private ExecutorService boundedThreadPool; + /** + * Allocator of local FS storage. + */ + private LocalDirAllocator directoryAllocator; + /** * Read operation context. */ @@ -57,7 +65,7 @@ public S3AReadOpContext getContext() { * @param value new value * @return the builder */ - public FactoryStreamParameters withContext(S3AReadOpContext value) { + public ObjectReadParameters withContext(S3AReadOpContext value) { context = value; return this; } @@ -74,7 +82,7 @@ public S3ObjectAttributes getObjectAttributes() { * @param value new value * @return the builder */ - public FactoryStreamParameters withObjectAttributes(S3ObjectAttributes value) { + public ObjectReadParameters withObjectAttributes(S3ObjectAttributes value) { objectAttributes = value; return this; } @@ -82,7 +90,7 @@ public FactoryStreamParameters withObjectAttributes(S3ObjectAttributes value) { /** * Callbacks to the store. */ - public StreamReadCallbacks getCallbacks() { + public ObjectInputStreamCallbacks getCallbacks() { return callbacks; } @@ -91,7 +99,7 @@ public StreamReadCallbacks getCallbacks() { * @param value new value * @return the builder */ - public FactoryStreamParameters withCallbacks(StreamReadCallbacks value) { + public ObjectReadParameters withCallbacks(ObjectInputStreamCallbacks value) { callbacks = value; return this; } @@ -108,7 +116,7 @@ public S3AInputStreamStatistics getStreamStatistics() { * @param value new value * @return the builder */ - public FactoryStreamParameters withStreamStatistics(S3AInputStreamStatistics value) { + public ObjectReadParameters withStreamStatistics(S3AInputStreamStatistics value) { streamStatistics = value; return this; } @@ -126,23 +134,38 @@ public ExecutorService getBoundedThreadPool() { * @param value new value * @return the builder */ - public FactoryStreamParameters withBoundedThreadPool(ExecutorService value) { + 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 FactoryStreamParameters build() { + public ObjectReadParameters build() { + // please keep in alphabetical order. requireNonNull(boundedThreadPool, "boundedThreadPool"); requireNonNull(callbacks, "callbacks"); requireNonNull(context, "context"); + requireNonNull(directoryAllocator, "directoryAllocator"); requireNonNull(objectAttributes, "objectAttributes"); requireNonNull(streamStatistics, "streamStatistics"); - requireNonNull(boundedThreadPool, "boundedThreadPool"); return this; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/package-info.java similarity index 69% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/package-info.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/package-info.java index bb79fb129ef2b..82bb61c2313c4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/streams/package-info.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/package-info.java @@ -17,12 +17,14 @@ */ /** - * Stream factory and support. + * This describes the model of S3 for layers above to interact with, and + * possibly extend. + *

+ * This intended for internal use, as a way of separating the layers + * above from how the store is actually interacted with. */ -@InterfaceAudience.LimitedPrivate("Extension Libraries") -@InterfaceStability.Unstable -package org.apache.hadoop.fs.s3a.streams; +@InterfaceAudience.Private +package org.apache.hadoop.fs.s3a.impl.model; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file +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/PrefetchingInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchingInputStreamFactory.java new file mode 100644 index 0000000000000..d2a0685c64dfa --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchingInputStreamFactory.java @@ -0,0 +1,42 @@ +/* + * 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.fs.s3a.impl.model.ObjectInputStream; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamFactory; +import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; +import org.apache.hadoop.service.AbstractService; + +/** + * Factory for prefetching streams. + */ +public class PrefetchingInputStreamFactory extends AbstractService + implements ObjectInputStreamFactory { + + public PrefetchingInputStreamFactory() { + super("PrefetchingInputStreamFactory"); + } + + @Override + public ObjectInputStream readObject(final ObjectReadParameters parameters) throws IOException { + return new S3APrefetchingInputStream(parameters, getConfig()); + } +} 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 3a0fd6d247fa8..544e3af849959 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 @@ -35,7 +35,7 @@ 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.streams.StreamReadCallbacks; +import org.apache.hadoop.fs.s3a.impl.model.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; @@ -75,7 +75,7 @@ public class S3ACachingInputStream extends S3ARemoteInputStream { public S3ACachingInputStream( S3AReadOpContext context, S3ObjectAttributes s3Attributes, - StreamReadCallbacks client, + ObjectInputStreamCallbacks client, S3AInputStreamStatistics streamStatistics, Configuration conf, LocalDirAllocator localDirAllocator) { 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 a64f48dfaa2ec..ede6a493324c3 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 @@ -30,7 +30,7 @@ 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.streams.StreamReadCallbacks; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamCallbacks; /** * Provides an {@code InputStream} that allows reading from an S3 file. @@ -61,7 +61,7 @@ public class S3AInMemoryInputStream extends S3ARemoteInputStream { public S3AInMemoryInputStream( S3AReadOpContext context, S3ObjectAttributes s3Attributes, - StreamReadCallbacks client, + ObjectInputStreamCallbacks client, S3AInputStreamStatistics streamStatistics) { super(context, s3Attributes, client, streamStatistics); int fileSize = (int) s3Attributes.getLen(); 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 f59b5b912ac2f..0bd8d0ed5bdc4 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,14 +30,15 @@ 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.S3AReadOpContext; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStream; +import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; -import org.apache.hadoop.fs.s3a.streams.StreamReadCallbacks; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamCallbacks; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; @@ -48,7 +49,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 +75,21 @@ 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. + * @throws IllegalArgumentException required parameterss are null */ - public S3APrefetchingInputStream( - S3AReadOpContext context, - S3ObjectAttributes s3Attributes, - StreamReadCallbacks client, - S3AInputStreamStatistics streamStatistics, - Configuration conf, - LocalDirAllocator localDirAllocator) { + public S3APrefetchingInputStream(final ObjectReadParameters parameters, Configuration conf) { + super(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"); @@ -198,6 +194,21 @@ public synchronized void close() throws IOException { } } + + @Override + protected boolean isStreamOpen() { + return !isClosed(); + } + + @Override + protected void abortInFinalizer() { + try { + close(); + } catch (IOException ignored) { + + } + } + /** * Updates internal data such that the next read will take place at the given {@code pos}. * 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 3f609b1b9532b..63762d063228c 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 @@ -40,7 +40,7 @@ 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.streams.StreamReadCallbacks; +import org.apache.hadoop.fs.s3a.impl.model.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 StreamReadCallbacks client; + private ObjectInputStreamCallbacks client; /** Used for reporting input stream access statistics. */ private final S3AInputStreamStatistics streamStatistics; @@ -124,7 +124,7 @@ public abstract class S3ARemoteInputStream public S3ARemoteInputStream( S3AReadOpContext context, S3ObjectAttributes s3Attributes, - StreamReadCallbacks client, + ObjectInputStreamCallbacks client, S3AInputStreamStatistics streamStatistics) { this.context = requireNonNull(context); 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 446db4f88ac64..a35291033da47 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,7 +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.streams.StreamReadCallbacks; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamCallbacks; import org.apache.hadoop.fs.statistics.DurationTracker; @@ -61,7 +61,7 @@ public class S3ARemoteObject { /** * Callbacks used for interacting with the underlying S3 client. */ - private final StreamReadCallbacks client; + private final ObjectInputStreamCallbacks client; /** * Used for reporting input stream access statistics. @@ -101,7 +101,7 @@ public class S3ARemoteObject { public S3ARemoteObject( S3AReadOpContext context, S3ObjectAttributes s3Attributes, - StreamReadCallbacks client, + ObjectInputStreamCallbacks client, S3AInputStreamStatistics streamStatistics, ChangeTracker changeTracker) { 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 a0a12eb1b3899..b86206b0b2cca 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,8 +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.streams.FactoryStreamParameters; -import org.apache.hadoop.fs.s3a.streams.StreamReadCallbacks; +import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamCallbacks; import org.apache.hadoop.util.functional.CallableRaisingIOE; import org.apache.http.NoHttpResponseException; @@ -166,7 +166,7 @@ private static void assertReadValueMatchesOffset( * @return a stream. */ private S3AInputStream getMockedS3AInputStream( - StreamReadCallbacks streamCallback) { + ObjectInputStreamCallbacks streamCallback) { Path path = new Path("test-path"); String eTag = "test-etag"; String versionId = "test-version-id"; @@ -189,7 +189,7 @@ private S3AInputStream getMockedS3AInputStream( s3AFileStatus, NoopSpan.INSTANCE); - FactoryStreamParameters parameters = new FactoryStreamParameters() + ObjectReadParameters parameters = new ObjectReadParameters() .withCallbacks(streamCallback) .withObjectAttributes(s3ObjectAttributes) .withContext(s3AReadOpContext) @@ -207,7 +207,7 @@ private S3AInputStream getMockedS3AInputStream( * @param ex exception to raise on failure * @return mocked object. */ - private StreamReadCallbacks failingInputStreamCallbacks( + private ObjectInputStreamCallbacks failingInputStreamCallbacks( final RuntimeException ex) { GetObjectResponse objectResponse = GetObjectResponse.builder() @@ -242,7 +242,7 @@ private StreamReadCallbacks failingInputStreamCallbacks( * @param ex exception to raise on failure * @return mocked object. */ - private StreamReadCallbacks maybeFailInGetCallback( + private ObjectInputStreamCallbacks maybeFailInGetCallback( final RuntimeException ex, final Function failurePredicate) { GetObjectResponse objectResponse = GetObjectResponse.builder() @@ -263,13 +263,13 @@ private StreamReadCallbacks maybeFailInGetCallback( * @param streamFactory factory for the stream to return on the given attempt. * @return mocked object. */ - private StreamReadCallbacks mockInputStreamCallback( + private ObjectInputStreamCallbacks mockInputStreamCallback( final RuntimeException ex, final Function failurePredicate, final Function> streamFactory) { - return new StreamReadCallbacks() { + return new ObjectInputStreamCallbacks() { private int attempt = 0; @Override 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 4a572441c56b7..3ac9863ab2e85 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 @@ -30,7 +30,7 @@ import org.apache.hadoop.fs.impl.prefetch.Validate; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; -import org.apache.hadoop.fs.s3a.streams.StreamReadCallbacks; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamCallbacks; import org.apache.hadoop.util.functional.CallableRaisingIOE; /** @@ -95,8 +95,8 @@ public static byte byteAtOffset(int offset) { return (byte) (offset % 128); } - public static StreamReadCallbacks createClient(String bucketName) { - return new StreamReadCallbacks() { + 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 eb7ed4fcaabfe..6f03068129585 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 @@ -61,7 +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.streams.StreamReadCallbacks; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamCallbacks; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryPolicy; @@ -187,7 +187,7 @@ public static ResponseInputStream createS3ObjectInputStream( AbortableInputStream.create(new ByteArrayInputStream(buffer), () -> {})); } - public static StreamReadCallbacks createInputStreamCallbacks( + public static ObjectInputStreamCallbacks createInputStreamCallbacks( String bucket) { GetObjectResponse objectResponse = GetObjectResponse.builder() @@ -197,7 +197,7 @@ public static StreamReadCallbacks createInputStreamCallbacks( ResponseInputStream responseInputStream = createS3ObjectInputStream(objectResponse, new byte[8]); - return new StreamReadCallbacks() { + return new ObjectInputStreamCallbacks() { @Override public ResponseInputStream getObject(GetObjectRequest request) { return responseInputStream; @@ -238,7 +238,7 @@ public static S3ARemoteInputStream createInputStream( prefetchBlockSize, prefetchBlockCount); - StreamReadCallbacks callbacks = + ObjectInputStreamCallbacks callbacks = createInputStreamCallbacks(bucket); S3AInputStreamStatistics stats = s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics(); @@ -289,7 +289,7 @@ public static class FakeS3AInMemoryInputStream public FakeS3AInMemoryInputStream( S3AReadOpContext context, S3ObjectAttributes s3Attributes, - StreamReadCallbacks client, + ObjectInputStreamCallbacks client, S3AInputStreamStatistics streamStatistics) { super(context, s3Attributes, client, streamStatistics); } @@ -391,7 +391,7 @@ public static class FakeS3ACachingInputStream extends S3ACachingInputStream { public FakeS3ACachingInputStream( S3AReadOpContext context, S3ObjectAttributes s3Attributes, - StreamReadCallbacks client, + ObjectInputStreamCallbacks client, S3AInputStreamStatistics streamStatistics) { super(context, s3Attributes, client, streamStatistics, CONF, new LocalDirAllocator( 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 35a08cc792a98..f779840b09029 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 @@ -35,7 +35,7 @@ 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.streams.StreamReadCallbacks; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamCallbacks; import org.apache.hadoop.test.AbstractHadoopTestBase; import static org.assertj.core.api.Assertions.assertThat; @@ -53,7 +53,7 @@ public class TestS3ARemoteInputStream extends AbstractHadoopTestBase { private final ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(threadPool); - private final StreamReadCallbacks client = + private final ObjectInputStreamCallbacks client = MockS3ARemoteObject.createClient("bucket"); @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 6cee43bb6c6b5..4818df284f661 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 @@ -30,7 +30,7 @@ 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.streams.StreamReadCallbacks; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamCallbacks; import org.apache.hadoop.test.AbstractHadoopTestBase; public class TestS3ARemoteObject extends AbstractHadoopTestBase { @@ -40,7 +40,7 @@ public class TestS3ARemoteObject extends AbstractHadoopTestBase { private final ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(threadPool); - private final StreamReadCallbacks client = + private final ObjectInputStreamCallbacks client = MockS3ARemoteObject.createClient("bucket"); @Test From e82d33f8c98ec60e25cbec46122b5c475fe90dbd Mon Sep 17 00:00:00 2001 From: Ahmar Suhail Date: Fri, 20 Dec 2024 07:18:07 +0000 Subject: [PATCH 3/7] initial commit Change-Id: If42bdd0b227c4da07c62a410a998e6d8c35581f6 --- .../org/apache/hadoop/fs/s3a/Constants.java | 5 ++ .../apache/hadoop/fs/s3a/InputStreamType.java | 29 +++++++ .../apache/hadoop/fs/s3a/S3AFileSystem.java | 79 +---------------- .../org/apache/hadoop/fs/s3a/S3AStore.java | 5 +- .../fs/s3a/impl/InputStreamCallbacksImpl.java | 87 +++++++++++++++++++ .../hadoop/fs/s3a/impl/S3AStoreImpl.java | 40 ++++++++- 6 files changed, 168 insertions(+), 77 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InputStreamType.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InputStreamCallbacksImpl.java 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..ceecd78e13859 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 @@ -1580,6 +1580,11 @@ private Constants() { */ public static final String AWS_AUTH_CLASS_PREFIX = "com.amazonaws.auth"; + + public static final String INPUT_STREAM_TYPE = "fs.s3a.input.stream.type"; + + public static final String INPUT_STREAM_TYPE_DEFAULT = "classic"; + /** * Controls whether the prefetching input stream is enabled. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InputStreamType.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InputStreamType.java new file mode 100644 index 0000000000000..a6fdde8a05e8d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InputStreamType.java @@ -0,0 +1,29 @@ +package org.apache.hadoop.fs.s3a; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public enum InputStreamType { + CLASSIC("classic"), + PREFETCH("prefetch"), + ANALYTICS("analytics"); + + private final String name; + + private static final Logger LOG = LoggerFactory.getLogger(InputStreamType.class); + + InputStreamType(String name) { + this.name = name; + } + + public static InputStreamType fromString(String inputStreamType) { + for (InputStreamType value : values()) { + if (value.name.equalsIgnoreCase(inputStreamType)) { + return value; + } + } + LOG.warn("Unknown input stream type {}, using default classic stream.", inputStreamType); + + return CLASSIC; + } +} 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 e45a72cd80f51..860807c059f6c 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 @@ -125,6 +125,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; @@ -1869,6 +1870,7 @@ private FSDataInputStream executeOpen( auditSpan); fileInformation.applyOptions(readContext); LOG.debug("Opening '{}'", readContext); + // QUESTION: why are we creating a new executor on each open? final SemaphoredDelegatingExecutor pool = new SemaphoredDelegatingExecutor( boundedThreadPool, vectoredActiveRangeReads, @@ -1886,21 +1888,7 @@ private FSDataInputStream executeOpen( // TODO: move into S3AStore and export the factory API through // the store, which will add some of the features (callbacks, stats) // before invoking the real factory - ObjectInputStreamFactory factory = null; - try { - // Choose factory. - if (prefetchEnabled) { - factory = new PrefetchingInputStreamFactory(); - } else { - factory = new ClassicObjectInputStreamFactory(); - } - factory.init(getConf()); - factory.start(); - return new FSDataInputStream(factory.readObject(parameters)); - } finally { - IOUtils.cleanupWithLogger(LOG, factory); - } - + return new FSDataInputStream(getStore().readObject(parameters)); } /** @@ -1909,68 +1897,9 @@ private FSDataInputStream executeOpen( */ private ObjectInputStreamCallbacks createInputStreamCallbacks( final AuditSpan auditSpan) { - return new InputStreamCallbacksImpl(auditSpan); + return new InputStreamCallbacksImpl(auditSpan, getStore(), fsHandler, unboundedThreadPool); } - /** - * Operations needed by ObjectInputStreams to read data. - */ - private final class InputStreamCallbacksImpl implements - ObjectInputStreamCallbacks { - - /** - * 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(getStore(), 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. 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 c1cdc172600d6..44f3bff604319 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 @@ -46,6 +46,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.impl.ChangeTracker; @@ -53,6 +54,8 @@ 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.model.ObjectInputStreamFactory; +import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.fs.statistics.IOStatisticsSource; @@ -73,7 +76,7 @@ */ @InterfaceAudience.LimitedPrivate("Extensions") @InterfaceStability.Unstable -public interface S3AStore extends Service, IOStatisticsSource, ClientManager { +public interface S3AStore extends Service, IOStatisticsSource, ClientManager, ObjectInputStreamFactory { /** * Acquire write capacity for operations. 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..c8c0c720b7075 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InputStreamCallbacksImpl.java @@ -0,0 +1,87 @@ +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.model.ObjectInputStreamCallbacks; +import org.apache.hadoop.fs.store.audit.AuditSpan; +import org.apache.hadoop.util.LambdaUtils; +import org.apache.hadoop.util.functional.CallableRaisingIOE; + +import static java.util.Objects.requireNonNull; + +public class InputStreamCallbacksImpl implements ObjectInputStreamCallbacks { + + /** + * Audit span to activate before each call. + */ + private final AuditSpan auditSpan; + + private final S3AStore store; + + private final S3AFileSystemOperations fsHandler; + + private static final Logger LOG = LoggerFactory.getLogger(InputStreamCallbacksImpl.class); + + private final ThreadPoolExecutor unboundedThreadPool; + + /** + * Create. + * @param auditSpan Audit span to activate before each call. + */ + public InputStreamCallbacksImpl(final AuditSpan auditSpan, final S3AStore store, + S3AFileSystemOperations fsHandler, ThreadPoolExecutor unboundedThreadPool) { + this.auditSpan = requireNonNull(auditSpan); + this.store = requireNonNull(store); + this.fsHandler = requireNonNull(fsHandler); + this.unboundedThreadPool = requireNonNull(unboundedThreadPool); + } + + /** + * 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 fsHandler.getObject(store, request, store.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; + } +} 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 8ca3bead0f161..f32b97c154abd 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 @@ -58,9 +58,11 @@ import software.amazon.awssdk.transfer.s3.model.UploadFileRequest; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.InputStreamType; import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.ProgressableProgressListener; import org.apache.hadoop.fs.s3a.Retries; @@ -72,6 +74,10 @@ 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.model.ObjectInputStream; +import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamFactory; +import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; +import org.apache.hadoop.fs.s3a.prefetch.PrefetchingInputStreamFactory; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.statistics.DurationTracker; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; @@ -85,6 +91,8 @@ 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.INPUT_STREAM_TYPE; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_DEFAULT; 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; @@ -108,6 +116,7 @@ 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.io.IOUtils.cleanupWithLogger; import static org.apache.hadoop.util.Preconditions.checkArgument; /** @@ -116,7 +125,7 @@ * to move. */ public class S3AStoreImpl extends CompositeService - implements S3AStore { + implements S3AStore, ObjectInputStreamFactory { private static final Logger LOG = LoggerFactory.getLogger(S3AStoreImpl.class); @@ -177,6 +186,8 @@ public class S3AStoreImpl extends CompositeService */ private LocalDirAllocator directoryAllocator; + private ObjectInputStreamFactory objectInputStreamFactory; + /** Constructor to create S3A store. */ S3AStoreImpl(StoreContextFactory storeContextFactory, ClientManager clientManager, @@ -209,12 +220,14 @@ public class S3AStoreImpl extends CompositeService protected void serviceStart() throws Exception { super.serviceStart(); initLocalDirAllocator(); + initInputStreamFactory(); } @Override protected void serviceStop() throws Exception { super.serviceStop(); clientManager.close(); + cleanupWithLogger(LOG, objectInputStreamFactory); } /** @@ -226,6 +239,20 @@ private void initLocalDirAllocator() { directoryAllocator = new LocalDirAllocator(bufferDir); } + private void initInputStreamFactory() { + InputStreamType inputStreamType = InputStreamType.fromString(getConfig().get(INPUT_STREAM_TYPE, INPUT_STREAM_TYPE_DEFAULT)); + switch (inputStreamType) { + case PREFETCH: + this.objectInputStreamFactory = new PrefetchingInputStreamFactory(); + break; + default: + this.objectInputStreamFactory = new ClassicObjectInputStreamFactory(); + } + + this.objectInputStreamFactory.init(getConfig()); + this.objectInputStreamFactory.start(); + } + /** Acquire write capacity for rate limiting {@inheritDoc}. */ @Override public Duration acquireWriteCapacity(final int capacity) { @@ -871,4 +898,15 @@ public File createTemporaryFileForWriting(String pathStr, return File.createTempFile(prefix, null, dir); } + @Override + public ObjectInputStream readObject(ObjectReadParameters parameters) + throws IOException { + if (objectInputStreamFactory != null) { + return objectInputStreamFactory.readObject(parameters); + } else { + // TODO: Find the right exception to throw if factory has not yet been initialised, or closed. + throw new IOException("Factory not initialized!"); + } + } + } From 6cf7e9e27b394292b426db5eab6772b0fe2639f6 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 30 Dec 2024 18:55:32 +0000 Subject: [PATCH 4/7] HADOOP-19354. stream factories Moves all prefetching stream related options into the prefetching stream factory; the standard ReadOpContext removes them, so a new PrefetchingOptions is passed around. Stream factories can now declare how many extra shared threads they want and whether or not to create a future pool around the bounded pool. This is used in S3AFileSystem when creating its thread pools -this class no longer reads in any of the prefetching options. All tests which enable/disable prefetching, or probe for its state, now use S3ATestUtils methods for this. This avoids them having to now explicitly unset two properties, set the new input stream type, and any more complications in test setup in future. Everything under S3AStore is a service, so service lifecycle matches everywhere -and store just adds to the list of managed services for start/stop/close integration. + adjust assertions in ITestS3AInputStreamLeakage for prefetching + update the prefetching.md doc for factory changs + javadocs + add string values of type names to Constants Once the analytics stream is in, a full doc on "stream performance" will be needed. package for this stuff is now impl.streams Change-Id: Id6356d2ded2c477ba16cbb9027ac0cfbece2a542 --- .../dev-support/findbugs-exclude.xml | 2 +- .../org/apache/hadoop/fs/s3a/Constants.java | 24 +++++- .../apache/hadoop/fs/s3a/InputStreamType.java | 29 ------- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 60 +++++--------- .../apache/hadoop/fs/s3a/S3AInputStream.java | 17 ++-- .../hadoop/fs/s3a/S3AReadOpContext.java | 42 ++-------- .../org/apache/hadoop/fs/s3a/S3AStore.java | 10 ++- .../impl/ClassicObjectInputStreamFactory.java | 6 +- .../hadoop/fs/s3a/impl/ClientManager.java | 5 +- .../hadoop/fs/s3a/impl/ClientManagerImpl.java | 56 +++++-------- .../fs/s3a/impl/InputStreamCallbacksImpl.java | 62 +++++++++++--- .../hadoop/fs/s3a/impl/S3AStoreImpl.java | 81 +++++++++---------- .../fs/s3a/impl/streams/InputStreamType.java | 64 +++++++++++++++ .../{model => streams}/ObjectInputStream.java | 53 ++++++------ .../ObjectInputStreamCallbacks.java | 2 +- .../ObjectInputStreamFactory.java | 44 +++++++++- .../ObjectReadParameters.java | 29 +++++-- .../s3a/impl/streams/StreamIntegration.java | 71 ++++++++++++++++ .../impl/{model => streams}/package-info.java | 9 +-- .../fs/s3a/prefetch/PrefetchOptions.java | 67 +++++++++++++++ .../PrefetchingInputStreamFactory.java | 57 ++++++++++++- .../s3a/prefetch/S3ACachingInputStream.java | 14 ++-- .../s3a/prefetch/S3AInMemoryInputStream.java | 5 +- .../prefetch/S3APrefetchingInputStream.java | 37 +++++---- .../fs/s3a/prefetch/S3ARemoteInputStream.java | 10 +-- .../fs/s3a/prefetch/S3ARemoteObject.java | 2 +- .../markdown/tools/hadoop-aws/prefetching.md | 14 +++- .../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 | 7 +- .../fs/s3a/impl/ITestConnectionTimeouts.java | 7 +- .../fs/s3a/performance/ITestS3AOpenCost.java | 7 +- .../performance/ITestUnbufferDraining.java | 5 +- .../fs/s3a/prefetch/MockS3ARemoteObject.java | 4 +- .../fs/s3a/prefetch/S3APrefetchFakes.java | 33 ++++---- .../prefetch/TestS3ARemoteInputStream.java | 17 ++-- .../fs/s3a/prefetch/TestS3ARemoteObject.java | 4 +- .../scale/ITestS3AInputStreamPerformance.java | 6 +- 42 files changed, 698 insertions(+), 368 deletions(-) delete mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InputStreamType.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/InputStreamType.java rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/{model => streams}/ObjectInputStream.java (86%) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/{model => streams}/ObjectInputStreamCallbacks.java (97%) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/{model => streams}/ObjectInputStreamFactory.java (58%) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/{model => streams}/ObjectReadParameters.java (92%) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamIntegration.java rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/{model => streams}/package-info.java (77%) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchOptions.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 ceecd78e13859..eb5e76d690a8e 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 @@ -1581,9 +1581,31 @@ 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"; - public static final String INPUT_STREAM_TYPE_DEFAULT = "classic"; + /** + * The classic input stream: {@value}. + */ + public static final String INPUT_STREAM_TYPE_CLASSIC = "classic"; + + /** + * The prefetching input stream: {@value}. + */ + public static final String INPUT_STREAM_TYPE_PREFETCH = "prefetch"; + + /** + * The analytics input stream: {@value}. + */ + public static final String INPUT_STREAM_TYPE_ANALYTICS = "analytics"; + + /** + * The default input stream. + * Currently {@link #INPUT_STREAM_TYPE_CLASSIC} + */ + public static final String INPUT_STREAM_TYPE_DEFAULT = INPUT_STREAM_TYPE_CLASSIC; /** * Controls whether the prefetching input stream is enabled. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InputStreamType.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InputStreamType.java deleted file mode 100644 index a6fdde8a05e8d..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InputStreamType.java +++ /dev/null @@ -1,29 +0,0 @@ -package org.apache.hadoop.fs.s3a; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public enum InputStreamType { - CLASSIC("classic"), - PREFETCH("prefetch"), - ANALYTICS("analytics"); - - private final String name; - - private static final Logger LOG = LoggerFactory.getLogger(InputStreamType.class); - - InputStreamType(String name) { - this.name = name; - } - - public static InputStreamType fromString(String inputStreamType) { - for (InputStreamType value : values()) { - if (value.name.equalsIgnoreCase(inputStreamType)) { - return value; - } - } - LOG.warn("Unknown input stream type {}, using default classic stream.", inputStreamType); - - return CLASSIC; - } -} 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 860807c059f6c..68059bb62b441 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; @@ -149,11 +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.PrefetchingInputStreamFactory; -import org.apache.hadoop.fs.s3a.impl.ClassicObjectInputStreamFactory; -import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; -import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamFactory; -import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamCallbacks; +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.ObjectInputStreamCallbacks; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl; import org.apache.hadoop.fs.statistics.DurationTracker; @@ -169,7 +164,6 @@ import org.apache.hadoop.fs.store.audit.ActiveThreadSpanSource; import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.fs.store.audit.AuditSpanSource; -import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.token.DelegationTokenIssuer; @@ -337,18 +331,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); @@ -663,22 +650,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 " + @@ -796,6 +772,10 @@ public void initialize(URI name, Configuration originalConf) // directly through the client manager. // this is to aid mocking. 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. @@ -944,12 +924,15 @@ public Statistics getInstanceStatistics() { } /** - * Initialize the thread pool. + * Initialize the thread pools. * This must be re-invoked after replacing the S3Client during test * runs. * @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) { @@ -965,7 +948,9 @@ private void initThreadPools(Configuration conf) { TimeUnit.SECONDS, Duration.ZERO).getSeconds(); - int numPrefetchThreads = this.prefetchEnabled ? this.prefetchBlockCount : 0; + final ObjectInputStreamFactory.ThreadOptions requirements = + getStore().prefetchThreadRequirements(); + int numPrefetchThreads = requirements.sharedThreads(); int activeTasksForBoundedThreadPool = maxThreads; int waitingTasksForBoundedThreadPool = maxThreads + totalTasks + numPrefetchThreads; @@ -983,7 +968,7 @@ private void initThreadPools(Configuration conf) { unboundedThreadPool.allowCoreThreadTimeOut(true); executorCapacity = intOption(conf, EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1); - if (prefetchEnabled) { + if (requirements.createFuturePool()) { final S3AInputStreamStatistics s3AInputStreamStatistics = statisticsContext.newInputStreamStatistics(); futurePool = new ExecutorServiceFuturePool( @@ -1945,9 +1930,8 @@ protected S3AReadOpContext createReadContext( fileStatus, vectoredIOContext, IOStatisticsContext.getCurrentIOStatisticsContext().getAggregator(), - futurePool, - prefetchBlockSize, - prefetchBlockCount) + futurePool + ) .withAuditSpan(auditSpan); openFileHelper.applyDefaultOptions(roc); return roc.build(); @@ -5365,7 +5349,7 @@ public boolean hasPathCapability(final Path path, final String capability) // stream leak detection. case StreamStatisticNames.STREAM_LEAKS: - return !prefetchEnabled; + return true; default: // is it a performance flag? 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 9e9b3ca96dbd4..859eb8dfa0a4a 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 @@ -39,9 +39,8 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.impl.LeakReporter; -import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStream; -import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; -import org.apache.hadoop.fs.statistics.StreamStatisticNames; +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; @@ -191,7 +190,7 @@ public S3AInputStream(ObjectReadParameters parameters) { S3AReadOpContext context = getContext(); this.changeTracker = new ChangeTracker(getUri(), context.getChangeDetectionPolicy(), - getStreamStatistics().getChangeTrackerStatistics(), + getS3AStreamStatistics().getChangeTrackerStatistics(), getObjectAttributes()); setReadahead(context.getReadahead()); this.asyncDrainThreshold = context.getAsyncDrainThreshold(); @@ -873,7 +872,8 @@ public synchronized void readVectored(List ranges, if (isOrderedDisjoint(sortedRanges, 1, minSeekForVectorReads())) { LOG.debug("Not merging the ranges as they are disjoint"); - getS3AStreamStatistics().readVectoredOperationStarted(sortedRanges.size(), sortedRanges.size()); + getS3AStreamStatistics().readVectoredOperationStarted(sortedRanges.size(), + sortedRanges.size()); for (FileRange range: sortedRanges) { ByteBuffer buffer = allocate.apply(range.getLength()); getBoundedThreadPool().submit(() -> readSingleRange(range, buffer)); @@ -883,7 +883,8 @@ public synchronized void readVectored(List ranges, List combinedFileRanges = mergeSortedRanges(sortedRanges, 1, minSeekForVectorReads(), maxReadSizeForVectorReads()); - getS3AStreamStatistics().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) { @@ -1289,9 +1290,9 @@ 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: return true; default: return super.hasCapability(capability); 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 67fc38d2c1ef8..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 @@ -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 44f3bff604319..1dccd9d950869 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 @@ -46,7 +46,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.impl.ChangeTracker; @@ -54,8 +53,7 @@ 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.model.ObjectInputStreamFactory; -import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; +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; @@ -76,7 +74,11 @@ */ @InterfaceAudience.LimitedPrivate("Extensions") @InterfaceStability.Unstable -public interface S3AStore extends Service, IOStatisticsSource, ClientManager, ObjectInputStreamFactory { +public interface S3AStore extends + ClientManager, + IOStatisticsSource, + ObjectInputStreamFactory, + Service { /** * Acquire write capacity for operations. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClassicObjectInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClassicObjectInputStreamFactory.java index 76b91821e691c..7d401b11cb2a5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClassicObjectInputStreamFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClassicObjectInputStreamFactory.java @@ -21,9 +21,9 @@ import java.io.IOException; import org.apache.hadoop.fs.s3a.S3AInputStream; -import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStream; -import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamFactory; -import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; +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.service.AbstractService; /** 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 d29fd14cbdf46..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. 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 index c8c0c720b7075..fde21f7a9b920 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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; @@ -11,38 +29,56 @@ import software.amazon.awssdk.services.s3.model.GetObjectResponse; import org.apache.hadoop.fs.s3a.S3AStore; -import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamCallbacks; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; import org.apache.hadoop.fs.store.audit.AuditSpan; -import org.apache.hadoop.util.LambdaUtils; 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; - private final S3AFileSystemOperations fsHandler; - - private static final Logger LOG = LoggerFactory.getLogger(InputStreamCallbacksImpl.class); + /** + * crypto FS operations. + */ + private final S3AFileSystemOperations fsOperations; - private final ThreadPoolExecutor unboundedThreadPool; + /** + * 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, - S3AFileSystemOperations fsHandler, ThreadPoolExecutor unboundedThreadPool) { + public InputStreamCallbacksImpl( + final AuditSpan auditSpan, + final S3AStore store, + final S3AFileSystemOperations fsOperations, + final ThreadPoolExecutor threadPool) { this.auditSpan = requireNonNull(auditSpan); this.store = requireNonNull(store); - this.fsHandler = requireNonNull(fsHandler); - this.unboundedThreadPool = requireNonNull(unboundedThreadPool); + this.fsOperations = requireNonNull(fsOperations); + this.threadPool = requireNonNull(threadPool); } /** @@ -66,15 +102,15 @@ public ResponseInputStream getObject(GetObjectRequest request IOException { // active the audit span used for the operation try (AuditSpan span = auditSpan.activate()) { - return fsHandler.getObject(store, request, store.getRequestFactory()); + return fsOperations.getObject(store, request, store.getRequestFactory()); } } @Override public CompletableFuture submit(final CallableRaisingIOE operation) { CompletableFuture result = new CompletableFuture<>(); - unboundedThreadPool.submit(() -> - LambdaUtils.eval(result, () -> { + threadPool.submit(() -> + eval(result, () -> { LOG.debug("Starting submitted operation in {}", auditSpan.getSpanId()); try (AuditSpan span = auditSpan.activate()) { return operation.apply(); 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 f32b97c154abd..ac84ae74859a9 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 @@ -58,11 +58,9 @@ import software.amazon.awssdk.transfer.s3.model.UploadFileRequest; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.InputStreamType; import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.ProgressableProgressListener; import org.apache.hadoop.fs.s3a.Retries; @@ -74,10 +72,9 @@ 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.model.ObjectInputStream; -import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamFactory; -import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; -import org.apache.hadoop.fs.s3a.prefetch.PrefetchingInputStreamFactory; +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.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.statistics.DurationTracker; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; @@ -91,8 +88,6 @@ 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.INPUT_STREAM_TYPE; -import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_DEFAULT; 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; @@ -113,10 +108,10 @@ 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.io.IOUtils.cleanupWithLogger; import static org.apache.hadoop.util.Preconditions.checkArgument; /** @@ -124,7 +119,8 @@ * This is where lower level storage operations are intended * to move. */ -public class S3AStoreImpl extends CompositeService +public class S3AStoreImpl + extends CompositeService implements S3AStore, ObjectInputStreamFactory { private static final Logger LOG = LoggerFactory.getLogger(S3AStoreImpl.class); @@ -186,9 +182,15 @@ public class S3AStoreImpl extends CompositeService */ private LocalDirAllocator directoryAllocator; + /** + * Factory for input streams. + */ private ObjectInputStreamFactory objectInputStreamFactory; - /** Constructor to create S3A store. */ + /** + * Constructor to create S3A store. + * Package private, as {@link S3AStoreBuilder} creates them. + * */ S3AStoreImpl(StoreContextFactory storeContextFactory, ClientManager clientManager, DurationTrackerFactory durationTrackerFactory, @@ -200,34 +202,42 @@ public class S3AStoreImpl extends CompositeService AuditSpanSource auditSpanSource, @Nullable FileSystem.Statistics fsStatistics) { super("S3AStore"); - this.storeContextFactory = requireNonNull(storeContextFactory); + 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 serviceStart() throws Exception { - super.serviceStart(); - initLocalDirAllocator(); - initInputStreamFactory(); + protected void serviceInit(final Configuration conf) throws Exception { + + objectInputStreamFactory = createStreamFactory(conf); + addService(objectInputStreamFactory); + + // init all child services + super.serviceInit(conf); } @Override - protected void serviceStop() throws Exception { - super.serviceStop(); - clientManager.close(); - cleanupWithLogger(LOG, objectInputStreamFactory); + protected void serviceStart() throws Exception { + super.serviceStart(); + initLocalDirAllocator(); } /** @@ -239,20 +249,6 @@ private void initLocalDirAllocator() { directoryAllocator = new LocalDirAllocator(bufferDir); } - private void initInputStreamFactory() { - InputStreamType inputStreamType = InputStreamType.fromString(getConfig().get(INPUT_STREAM_TYPE, INPUT_STREAM_TYPE_DEFAULT)); - switch (inputStreamType) { - case PREFETCH: - this.objectInputStreamFactory = new PrefetchingInputStreamFactory(); - break; - default: - this.objectInputStreamFactory = new ClassicObjectInputStreamFactory(); - } - - this.objectInputStreamFactory.init(getConfig()); - this.objectInputStreamFactory.start(); - } - /** Acquire write capacity for rate limiting {@inheritDoc}. */ @Override public Duration acquireWriteCapacity(final int capacity) { @@ -898,15 +894,14 @@ public File createTemporaryFileForWriting(String pathStr, return File.createTempFile(prefix, null, dir); } - @Override + @Override /* ObjectInputStreamFactory */ public ObjectInputStream readObject(ObjectReadParameters parameters) throws IOException { - if (objectInputStreamFactory != null) { - return objectInputStreamFactory.readObject(parameters); - } else { - // TODO: Find the right exception to throw if factory has not yet been initialised, or closed. - throw new IOException("Factory not initialized!"); - } + return objectInputStreamFactory.readObject(parameters); } + @Override /* ObjectInputStreamFactory */ + public ThreadOptions prefetchThreadRequirements() { + return objectInputStreamFactory.prefetchThreadRequirements(); + } } 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..f2c6b4a920fe2 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/InputStreamType.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 static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_ANALYTICS; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_CLASSIC; +import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_PREFETCH; + +/** + * Enum of input stream types. + */ +public enum InputStreamType { + /** + * The classic input stream. + */ + Classic(INPUT_STREAM_TYPE_CLASSIC), + /** + * The prefetching input stream. + */ + Prefetch(INPUT_STREAM_TYPE_PREFETCH), + /** + * The analytics input stream. + */ + Analytics(INPUT_STREAM_TYPE_ANALYTICS); + + /** + * Name. + */ + private final String name; + + /** + * String name. + * @return the name + */ + public String getName() { + return name; + } + + InputStreamType(String name) { + this.name = name; + } + + /** + * 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/model/ObjectInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStream.java similarity index 86% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStream.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStream.java index 2a9bde9bf3422..e3e6a86cb5771 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStream.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.impl.model; +package org.apache.hadoop.fs.s3a.impl.streams; import java.io.IOException; import java.util.concurrent.ExecutorService; @@ -38,10 +38,10 @@ import org.apache.hadoop.fs.statistics.IOStatisticsAggregator; import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.fs.statistics.StreamStatisticNames; -import org.apache.hadoop.util.Preconditions; 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; /** @@ -58,7 +58,7 @@ public abstract class ObjectInputStream extends FSInputStream /** * IOStatistics report. */ - protected final IOStatistics ioStatistics; + private final IOStatistics ioStatistics; /** * Read-specific operation context structure. @@ -129,11 +129,11 @@ protected ObjectInputStream( ObjectReadParameters parameters) { objectAttributes = parameters.getObjectAttributes(); - Preconditions.checkArgument(isNotEmpty(objectAttributes.getBucket()), + checkArgument(isNotEmpty(objectAttributes.getBucket()), "No Bucket"); - Preconditions.checkArgument(isNotEmpty(objectAttributes.getKey()), "No Key"); + checkArgument(isNotEmpty(objectAttributes.getKey()), "No Key"); long l = objectAttributes.getLen(); - Preconditions.checkArgument(l >= 0, "Negative content length"); + checkArgument(l >= 0, "Negative content length"); this.context = parameters.getContext(); this.contentLength = l; @@ -152,7 +152,7 @@ protected ObjectInputStream( this.leakReporter = new LeakReporter( "Stream not closed while reading " + uri, this::isStreamOpen, - () -> abortInFinalizer()); + this::abortInFinalizer); } /** @@ -257,63 +257,64 @@ 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 StreamCapabilities.IOSTATISTICS_CONTEXT: case StreamStatisticNames.STREAM_LEAKS: - case StreamCapabilities.READAHEAD: - case StreamCapabilities.UNBUFFER: - case StreamCapabilities.VECTOREDIO: return true; default: return false; } } - - protected S3AReadOpContext getContext() { + protected final S3AReadOpContext getContext() { return context; } - protected ObjectInputStreamCallbacks getCallbacks() { + protected final ObjectInputStreamCallbacks getCallbacks() { return callbacks; } - protected ExecutorService getBoundedThreadPool() { + protected final ExecutorService getBoundedThreadPool() { return boundedThreadPool; } - protected String getUri() { + protected final String getUri() { return uri; } - protected String getBucket() { + protected final String getBucket() { return bucket; } - protected String getKey() { + protected final String getKey() { return key; } - protected String getPathStr() { + protected final String getPathStr() { return pathStr; } - protected long getContentLength() { + protected final long getContentLength() { return contentLength; } - protected S3AInputStreamStatistics getStreamStatistics() { - return streamStatistics; - } - - protected IOStatisticsAggregator getThreadIOStatistics() { + protected final IOStatisticsAggregator getThreadIOStatistics() { return threadIOStatistics; } - protected S3ObjectAttributes getObjectAttributes() { + protected final S3ObjectAttributes getObjectAttributes() { return objectAttributes; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStreamCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamCallbacks.java similarity index 97% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStreamCallbacks.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamCallbacks.java index 44fb0f068ee02..211b3476cd1cc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStreamCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamCallbacks.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.impl.model; +package org.apache.hadoop.fs.s3a.impl.streams; import java.io.Closeable; import java.io.IOException; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamFactory.java similarity index 58% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStreamFactory.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamFactory.java index a817291606fb7..b9bc86471fce1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectInputStreamFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamFactory.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.impl.model; +package org.apache.hadoop.fs.s3a.impl.streams; import java.io.IOException; @@ -47,5 +47,47 @@ public interface ObjectInputStreamFactory extends Service { ObjectInputStream readObject(ObjectReadParameters parameters) throws IOException; + /** + * Get the number of prefetch threads required for this factory. + * @return the count of prefetch threads. + */ + default ThreadOptions prefetchThreadRequirements() { + return new ThreadOptions(0, false); + } + + /** + * Will streams created through this factory have the requested capability? + * @param capability capability to probe for. + * @return true if a capability is known to be supported. + */ + default boolean hasStreamCapability(String capability) { + return false; + } + + /** + * Options for threading. + */ + class ThreadOptions { + /** number of shared threads to included in the bounded pool. */ + private final int sharedThreads; + + /** + * flag to enable creation of a future pool around the bounded thread pool. + */ + private final boolean createFuturePool; + + public ThreadOptions(final int sharedThreads, final boolean createFuturePool) { + this.sharedThreads = sharedThreads; + this.createFuturePool = createFuturePool; + } + + public int sharedThreads() { + return sharedThreads; + } + + public boolean createFuturePool() { + return createFuturePool; + } + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectReadParameters.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectReadParameters.java similarity index 92% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectReadParameters.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectReadParameters.java index e8114d7e5e7dd..8ee16b1050a29 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/ObjectReadParameters.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectReadParameters.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.impl.model; +package org.apache.hadoop.fs.s3a.impl.streams; import java.util.concurrent.ExecutorService; @@ -38,14 +38,30 @@ */ 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; /** @@ -54,7 +70,7 @@ public final class ObjectReadParameters { private LocalDirAllocator directoryAllocator; /** - * Read operation context. + * @return Read operation context. */ public S3AReadOpContext getContext() { return context; @@ -71,7 +87,7 @@ public ObjectReadParameters withContext(S3AReadOpContext value) { } /** - * Attributes of the object. + * @return Attributes of the object. */ public S3ObjectAttributes getObjectAttributes() { return objectAttributes; @@ -88,7 +104,7 @@ public ObjectReadParameters withObjectAttributes(S3ObjectAttributes value) { } /** - * Callbacks to the store. + * @return callbacks to the store. */ public ObjectInputStreamCallbacks getCallbacks() { return callbacks; @@ -105,7 +121,7 @@ public ObjectReadParameters withCallbacks(ObjectInputStreamCallbacks value) { } /** - * Stream statistics. + * @return Stream statistics. */ public S3AInputStreamStatistics getStreamStatistics() { return streamStatistics; @@ -122,8 +138,7 @@ public ObjectReadParameters withStreamStatistics(S3AInputStreamStatistics value) } /** - * Bounded thread pool for submitting asynchronous - * work. + * @return Bounded thread pool for submitting asynchronous work. */ public ExecutorService getBoundedThreadPool() { return boundedThreadPool; 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..2b74ccc7aad4d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamIntegration.java @@ -0,0 +1,71 @@ +/* + * 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.s3a.impl.ClassicObjectInputStreamFactory; +import org.apache.hadoop.fs.s3a.prefetch.PrefetchingInputStreamFactory; +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 S3AStoreImpl. + */ +public final class StreamIntegration { + + private static final Logger LOG_DEPRECATION = + LoggerFactory.getLogger( + "org.apache.hadoop.conf.Configuration.deprecation"); + + /** + * Warn once on use of prefetch boolean flag rather than enum. + */ + private static final LogExactlyOnce WARN_PREFETCH_KEY = new LogExactlyOnce(LOG_DEPRECATION); + + /** + * Create the input stream factory the configuration asks for. + * @param conf configuration + * @return a stream factory. + */ + public static ObjectInputStreamFactory createStreamFactory(final Configuration conf) { + ObjectInputStreamFactory objectInputStreamFactory; + InputStreamType defaultStream = InputStreamType.Classic; + if (conf.getBoolean(PREFETCH_ENABLED_KEY, false)) { + WARN_PREFETCH_KEY.info("Using {} is deprecated: choose the appropriate stream in {}", + PREFETCH_ENABLED_KEY, INPUT_STREAM_TYPE); + defaultStream = InputStreamType.Prefetch; + } + InputStreamType inputStreamType = conf.getEnum(INPUT_STREAM_TYPE, + defaultStream); + switch (inputStreamType) { + case Prefetch: + objectInputStreamFactory = new PrefetchingInputStreamFactory(); + break; + case Classic: + default: + objectInputStreamFactory = new ClassicObjectInputStreamFactory(); + } + return objectInputStreamFactory; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/package-info.java similarity index 77% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/package-info.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/package-info.java index 82bb61c2313c4..514e8dbd70376 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/model/package-info.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/package-info.java @@ -17,14 +17,13 @@ */ /** - * This describes the model of S3 for layers above to interact with, and - * possibly extend. + * Input and Output stream support. *

- * This intended for internal use, as a way of separating the layers - * above from how the store is actually interacted with. + * 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.model; +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 index d2a0685c64dfa..82e88810268db 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 @@ -20,23 +20,72 @@ import java.io.IOException; -import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStream; -import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamFactory; -import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; +import org.apache.hadoop.conf.Configuration; +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.service.AbstractService; +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 AbstractService implements ObjectInputStreamFactory { + /** 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 prefatch 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()); + return new S3APrefetchingInputStream(parameters, + getConfig(), + prefetchOptions); + } + + /** + * The thread count is calculated from the configuration. + * @return a positive thread count. + */ + @Override + public ThreadOptions prefetchThreadRequirements() { + return new ThreadOptions(prefetchBlockCount, true); } } 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 544e3af849959..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 @@ -35,7 +35,7 @@ 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.model.ObjectInputStreamCallbacks; +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, 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 ede6a493324c3..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 @@ -30,7 +30,7 @@ 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.model.ObjectInputStreamCallbacks; +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, 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 0bd8d0ed5bdc4..49a30646bbf59 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 @@ -35,13 +35,15 @@ import org.apache.hadoop.fs.impl.prefetch.Validate; import org.apache.hadoop.fs.s3a.S3AReadOpContext; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; -import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStream; -import org.apache.hadoop.fs.s3a.impl.model.ObjectReadParameters; +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.model.ObjectInputStreamCallbacks; +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. * @@ -78,16 +80,19 @@ public class S3APrefetchingInputStream /** * Initializes a new instance of the {@code S3APrefetchingInputStream} class. - * * @param parameters creation parameters. * @param conf the configuration. - * @throws IllegalArgumentException required parameterss are null + * @param prefetchOptions prefetch stream specific options + * @throws NullPointerException if a required parameter is null. */ - public S3APrefetchingInputStream(final ObjectReadParameters parameters, Configuration conf) { + public S3APrefetchingInputStream( + final ObjectReadParameters parameters, + final Configuration conf, + final PrefetchOptions prefetchOptions) { super(parameters); - S3ObjectAttributes s3Attributes = parameters.getObjectAttributes(); - ObjectInputStreamCallbacks client = parameters.getCallbacks(); - S3AInputStreamStatistics streamStatistics = parameters.getStreamStatistics(); + S3ObjectAttributes s3Attributes = parameters.getObjectAttributes(); + ObjectInputStreamCallbacks client = parameters.getCallbacks(); + S3AInputStreamStatistics streamStatistics = parameters.getStreamStatistics(); final S3AReadOpContext context = parameters.getContext(); LocalDirAllocator localDirAllocator = parameters.getDirectoryAllocator(); @@ -102,10 +107,11 @@ public S3APrefetchingInputStream(final ObjectReadParameters parameters, Configur 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); @@ -113,6 +119,7 @@ public S3APrefetchingInputStream(final ObjectReadParameters parameters, Configur LOG.debug("Creating in caching input stream for {}", context.getPath()); this.inputStream = new S3ACachingInputStream( context, + prefetchOptions, s3Attributes, client, streamStatistics, @@ -202,6 +209,7 @@ protected boolean isStreamOpen() { @Override protected void abortInFinalizer() { + getS3AStreamStatistics().streamLeaked(); try { close(); } catch (IOException ignored) { @@ -241,11 +249,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 63762d063228c..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 @@ -40,7 +40,7 @@ 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.model.ObjectInputStreamCallbacks; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; @@ -113,16 +113,16 @@ 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, ObjectInputStreamCallbacks client, S3AInputStreamStatistics streamStatistics) { @@ -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 a35291033da47..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,7 +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.model.ObjectInputStreamCallbacks; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; import org.apache.hadoop.fs.statistics.DurationTracker; 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/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 b86206b0b2cca..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,8 +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.model.ObjectReadParameters; -import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamCallbacks; +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; @@ -193,7 +193,8 @@ private S3AInputStream getMockedS3AInputStream( .withCallbacks(streamCallback) .withObjectAttributes(s3ObjectAttributes) .withContext(s3AReadOpContext) - .withStreamStatistics( s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics()) + .withStreamStatistics( + s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics()) .withBoundedThreadPool(null); return new S3AInputStream(parameters); 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 3ac9863ab2e85..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 @@ -30,7 +30,7 @@ import org.apache.hadoop.fs.impl.prefetch.Validate; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; -import org.apache.hadoop.fs.s3a.impl.model.ObjectInputStreamCallbacks; +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, 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 6f03068129585..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 @@ -61,7 +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.model.ObjectInputStreamCallbacks; +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)) @@ -234,21 +231,25 @@ public static S3ARemoteInputStream createInputStream( S3AReadOpContext s3AReadOpContext = createReadContext( futurePool, key, - fileSize, - prefetchBlockSize, - prefetchBlockCount); + fileSize + ); 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, 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, 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 f779840b09029..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 @@ -35,7 +35,7 @@ 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.model.ObjectInputStreamCallbacks; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; import org.apache.hadoop.test.AbstractHadoopTestBase; import static org.assertj.core.api.Assertions.assertThat; @@ -59,7 +59,8 @@ public class TestS3ARemoteInputStream extends AbstractHadoopTestBase { @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 4818df284f661..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 @@ -30,7 +30,7 @@ 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.model.ObjectInputStreamCallbacks; +import org.apache.hadoop.fs.s3a.impl.streams.ObjectInputStreamCallbacks; import org.apache.hadoop.test.AbstractHadoopTestBase; public class TestS3ARemoteObject extends AbstractHadoopTestBase { @@ -46,7 +46,7 @@ public class TestS3ARemoteObject extends AbstractHadoopTestBase { @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..bb46d5ca1e292 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; @@ -99,15 +100,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; } From 03bc2cdfc2c4c86ee977bad4b013ca54e9677871 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 7 Jan 2025 14:36:23 +0000 Subject: [PATCH 5/7] HADOOP-19354. Input Stream Factory Push factory construction into the enum itself Store implements stream capabilities, which are then relayed to the active factory. This avoids the FS having to know what capabilities are available in the stream. Abstract base class for stream factories. Change-Id: Ib757e6696f29cc7e0e8edd1119e738c6adc6f98f --- .../org/apache/hadoop/fs/s3a/Constants.java | 16 ++-- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 62 +++++++++------ .../org/apache/hadoop/fs/s3a/S3AStore.java | 2 + .../hadoop/fs/s3a/impl/S3AStoreImpl.java | 46 ++++++++++- .../AbstractObjectInputStreamFactory.java | 48 ++++++++++++ .../ClassicObjectInputStreamFactory.java | 36 +++++++-- .../fs/s3a/impl/streams/InputStreamType.java | 35 +++++++-- .../streams/ObjectInputStreamFactory.java | 46 ++--------- .../impl/streams/ObjectReadParameters.java | 4 +- .../s3a/impl/streams/StreamIntegration.java | 23 ++---- .../s3a/impl/streams/StreamThreadOptions.java | 76 +++++++++++++++++++ .../PrefetchingInputStreamFactory.java | 11 ++- 12 files changed, 293 insertions(+), 112 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AbstractObjectInputStreamFactory.java rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/{ => streams}/ClassicObjectInputStreamFactory.java (58%) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamThreadOptions.java 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 eb5e76d690a8e..7cc7a012f4ef0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -21,6 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.Options; +import org.apache.hadoop.fs.s3a.impl.streams.InputStreamType; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import java.time.Duration; @@ -1580,7 +1581,6 @@ private Constants() { */ public static final String AWS_AUTH_CLASS_PREFIX = "com.amazonaws.auth"; - /** * Input stream type: {@value}. */ @@ -1589,23 +1589,25 @@ private Constants() { /** * The classic input stream: {@value}. */ - public static final String INPUT_STREAM_TYPE_CLASSIC = "classic"; + public static final String INPUT_STREAM_TYPE_CLASSIC = + InputStreamType.Classic.getName(); /** - * The prefetching input stream: {@value}. + * The prefetching input stream: "prefetch". */ - public static final String INPUT_STREAM_TYPE_PREFETCH = "prefetch"; + public static final String INPUT_STREAM_TYPE_PREFETCH = InputStreamType.Prefetch.getName(); /** - * The analytics input stream: {@value}. + * The analytics input stream: "analytics". */ - public static final String INPUT_STREAM_TYPE_ANALYTICS = "analytics"; + public static final String INPUT_STREAM_TYPE_ANALYTICS = + InputStreamType.Analytics.getName(); /** * The default input stream. * Currently {@link #INPUT_STREAM_TYPE_CLASSIC} */ - public static final String INPUT_STREAM_TYPE_DEFAULT = 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 68059bb62b441..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 @@ -146,9 +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.impl.streams.ObjectInputStreamFactory; 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; @@ -925,8 +924,11 @@ public Statistics getInstanceStatistics() { /** * 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() { @@ -948,9 +950,9 @@ private void initThreadPools() { TimeUnit.SECONDS, Duration.ZERO).getSeconds(); - final ObjectInputStreamFactory.ThreadOptions requirements = - getStore().prefetchThreadRequirements(); - int numPrefetchThreads = requirements.sharedThreads(); + final StreamThreadOptions threadRequirements = + getStore().threadRequirements(); + int numPrefetchThreads = threadRequirements.sharedThreads(); int activeTasksForBoundedThreadPool = maxThreads; int waitingTasksForBoundedThreadPool = maxThreads + totalTasks + numPrefetchThreads; @@ -968,7 +970,8 @@ private void initThreadPools() { unboundedThreadPool.allowCoreThreadTimeOut(true); executorCapacity = intOption(conf, EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1); - if (requirements.createFuturePool()) { + if (threadRequirements.createFuturePool()) { + // create a future pool. final S3AInputStreamStatistics s3AInputStreamStatistics = statisticsContext.newInputStreamStatistics(); futurePool = new ExecutorServiceFuturePool( @@ -1855,24 +1858,32 @@ private FSDataInputStream executeOpen( auditSpan); fileInformation.applyOptions(readContext); LOG.debug("Opening '{}'", readContext); - // QUESTION: why are we creating a new executor on each open? + + // 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, - vectoredActiveRangeReads, + 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()) - .withDirectoryAllocator(getStore().getDirectoryAllocator()) .withObjectAttributes(createObjectAttributes(path, fileStatus)) - .withStreamStatistics(inputStreamStats) - .build(); - - // TODO: move into S3AStore and export the factory API through - // the store, which will add some of the features (callbacks, stats) - // before invoking the real factory + .withStreamStatistics(inputStreamStats); return new FSDataInputStream(getStore().readObject(parameters)); } @@ -4273,16 +4284,19 @@ protected synchronized void stopAllServices() { // 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), @@ -5347,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 true; - 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 } 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 1dccd9d950869..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 @@ -47,6 +47,7 @@ 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; @@ -78,6 +79,7 @@ public interface S3AStore extends ClientManager, IOStatisticsSource, ObjectInputStreamFactory, + PathCapabilities, Service { /** 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 ac84ae74859a9..e2fbb07f2440c 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 @@ -61,6 +61,7 @@ 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; @@ -75,6 +76,7 @@ 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; @@ -113,6 +115,7 @@ 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. @@ -240,12 +243,46 @@ protected void serviceStart() throws Exception { initLocalDirAllocator(); } + + /** + * Return the store 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 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; + ? BUFFER_DIR + : HADOOP_TMP_DIR; directoryAllocator = new LocalDirAllocator(bufferDir); } @@ -897,11 +934,12 @@ public File createTemporaryFileForWriting(String pathStr, @Override /* ObjectInputStreamFactory */ public ObjectInputStream readObject(ObjectReadParameters parameters) throws IOException { - return objectInputStreamFactory.readObject(parameters); + parameters.withDirectoryAllocator(getDirectoryAllocator()); + return objectInputStreamFactory.readObject(parameters.validate()); } @Override /* ObjectInputStreamFactory */ - public ThreadOptions prefetchThreadRequirements() { - return objectInputStreamFactory.prefetchThreadRequirements(); + public StreamThreadOptions threadRequirements() { + return objectInputStreamFactory.threadRequirements(); } } 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..d3b6c67113997 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AbstractObjectInputStreamFactory.java @@ -0,0 +1,48 @@ +/* + * 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 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); + } + + @Override + public boolean hasCapability(final String capability) { + switch (toLowerCase(capability)) { + case StreamCapabilities.IOSTATISTICS: + case StreamStatisticNames.STREAM_LEAKS: + return true; + default: + return false; + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClassicObjectInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ClassicObjectInputStreamFactory.java similarity index 58% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClassicObjectInputStreamFactory.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ClassicObjectInputStreamFactory.java index 7d401b11cb2a5..030e9bad01a5e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ClassicObjectInputStreamFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ClassicObjectInputStreamFactory.java @@ -16,21 +16,19 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.impl; +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 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.service.AbstractService; + +import static org.apache.hadoop.util.StringUtils.toLowerCase; /** * Factory of classic {@link S3AInputStream} instances. */ -public class ClassicObjectInputStreamFactory extends AbstractService - implements ObjectInputStreamFactory { +public class ClassicObjectInputStreamFactory extends AbstractObjectInputStreamFactory { public ClassicObjectInputStreamFactory() { super("ClassicObjectInputStreamFactory"); @@ -41,4 +39,28 @@ 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 index f2c6b4a920fe2..4ca9a6305a237 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 @@ -18,32 +18,42 @@ package org.apache.hadoop.fs.s3a.impl.streams; -import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_ANALYTICS; -import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_CLASSIC; -import static org.apache.hadoop.fs.s3a.Constants.INPUT_STREAM_TYPE_PREFETCH; +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(INPUT_STREAM_TYPE_CLASSIC), + Classic("classic", c -> + new ClassicObjectInputStreamFactory()), + /** * The prefetching input stream. */ - Prefetch(INPUT_STREAM_TYPE_PREFETCH), + Prefetch("prefetch", c -> + new PrefetchingInputStreamFactory()), + /** * The analytics input stream. */ - Analytics(INPUT_STREAM_TYPE_ANALYTICS); + Analytics("analytics", c -> { + throw new IllegalArgumentException("not yet supported"); + }); /** * Name. */ private final String name; + private final Function factory; /** * String name. * @return the name @@ -52,8 +62,17 @@ public String getName() { return name; } - InputStreamType(String name) { + InputStreamType(String name, final Function factory) { this.name = name; + this.factory = factory; + } + + /** + * Factory constructor. + * @return the factory associated with this stream type. + */ + public Function factory() { + return factory; } /** @@ -61,4 +80,6 @@ public String getName() { */ 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/ObjectInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamFactory.java index b9bc86471fce1..1293bd4f92a06 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 @@ -20,6 +20,7 @@ import java.io.IOException; +import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.service.Service; /** @@ -34,7 +35,8 @@ * {@link ObjectReadParameters} class, rather than change the * interface signature. */ -public interface ObjectInputStreamFactory extends Service { +public interface ObjectInputStreamFactory + extends Service, StreamCapabilities { /** * Create a new input stream. @@ -48,46 +50,10 @@ ObjectInputStream readObject(ObjectReadParameters parameters) throws IOException; /** - * Get the number of prefetch threads required for this factory. - * @return the count of prefetch threads. + * Get the number of background threads required for this factory. + * @return the count of background threads. */ - default ThreadOptions prefetchThreadRequirements() { - return new ThreadOptions(0, false); - } + StreamThreadOptions threadRequirements(); - /** - * Will streams created through this factory have the requested capability? - * @param capability capability to probe for. - * @return true if a capability is known to be supported. - */ - default boolean hasStreamCapability(String capability) { - return false; - } - - /** - * Options for threading. - */ - class ThreadOptions { - /** number of shared threads to included in the bounded pool. */ - private final int sharedThreads; - - /** - * flag to enable creation of a future pool around the bounded thread pool. - */ - private final boolean createFuturePool; - - public ThreadOptions(final int sharedThreads, final boolean createFuturePool) { - this.sharedThreads = sharedThreads; - this.createFuturePool = createFuturePool; - } - - public int sharedThreads() { - return sharedThreads; - } - - public boolean createFuturePool() { - return createFuturePool; - } - } } 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 index 8ee16b1050a29..32a2487d3f7e7 100644 --- 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 @@ -32,7 +32,7 @@ * {@link ObjectInputStreamFactory}. * It is designed to support extra parameters added * in future. - *

Note that the {@link #build()} + *

Note that the {@link #validate()} * operation does not freeze the parameters -instead it simply * verifies that all required values are set. */ @@ -173,7 +173,7 @@ public ObjectReadParameters withDirectoryAllocator(final LocalDirAllocator value * Mock tests can skip this if required. * @return the object. */ - public ObjectReadParameters build() { + public ObjectReadParameters validate() { // please keep in alphabetical order. requireNonNull(boundedThreadPool, "boundedThreadPool"); requireNonNull(callbacks, "callbacks"); 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 2b74ccc7aad4d..85ca41beaf914 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 @@ -22,15 +22,13 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.s3a.impl.ClassicObjectInputStreamFactory; -import org.apache.hadoop.fs.s3a.prefetch.PrefetchingInputStreamFactory; 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 S3AStoreImpl. + * Stream integration, including factory construction. */ public final class StreamIntegration { @@ -45,27 +43,20 @@ public final class StreamIntegration { /** * Create the input stream factory the configuration asks for. + * This does not initialize the factory. * @param conf configuration * @return a stream factory. */ public static ObjectInputStreamFactory createStreamFactory(final Configuration conf) { - ObjectInputStreamFactory objectInputStreamFactory; - InputStreamType defaultStream = InputStreamType.Classic; + // choose the default input stream type + InputStreamType defaultStream = InputStreamType.DEFAULT_STREAM_TYPE; if (conf.getBoolean(PREFETCH_ENABLED_KEY, false)) { WARN_PREFETCH_KEY.info("Using {} is deprecated: choose the appropriate stream in {}", PREFETCH_ENABLED_KEY, INPUT_STREAM_TYPE); defaultStream = InputStreamType.Prefetch; } - InputStreamType inputStreamType = conf.getEnum(INPUT_STREAM_TYPE, - defaultStream); - switch (inputStreamType) { - case Prefetch: - objectInputStreamFactory = new PrefetchingInputStreamFactory(); - break; - case Classic: - default: - objectInputStreamFactory = new ClassicObjectInputStreamFactory(); - } - return objectInputStreamFactory; + 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..d4d629b94e430 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamThreadOptions.java @@ -0,0 +1,76 @@ +/* + * 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/prefetch/PrefetchingInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchingInputStreamFactory.java index 82e88810268db..b490294b7696d 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,10 @@ 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.ObjectInputStreamFactory; import org.apache.hadoop.fs.s3a.impl.streams.ObjectReadParameters; -import org.apache.hadoop.service.AbstractService; +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; @@ -39,8 +39,7 @@ *

* Reads and validates prefetch configuration options during service init. */ -public class PrefetchingInputStreamFactory extends AbstractService - implements ObjectInputStreamFactory { +public class PrefetchingInputStreamFactory extends AbstractObjectInputStreamFactory { /** Size in bytes of a single prefetch block. */ private int prefetchBlockSize; @@ -85,7 +84,7 @@ public ObjectInputStream readObject(final ObjectReadParameters parameters) throw * @return a positive thread count. */ @Override - public ThreadOptions prefetchThreadRequirements() { - return new ThreadOptions(prefetchBlockCount, true); + public StreamThreadOptions threadRequirements() { + return new StreamThreadOptions(prefetchBlockCount, 0, true, false); } } From 7923913dd3b1a1b3e6b8e7bfc2b1561fcf02dbd7 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 8 Jan 2025 18:19:18 +0000 Subject: [PATCH 6/7] HADOOP-19354. review comments Change-Id: Id79f8aa019095c1601bb0b2a282c51bdb0b7b817 --- .../hadoop/fs/s3a/impl/streams/ObjectInputStreamCallbacks.java | 2 +- hadoop-tools/hadoop-aws/src/test/resources/log4j.properties | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 index 211b3476cd1cc..f4ed384ed977a 100644 --- 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 @@ -30,7 +30,7 @@ import org.apache.hadoop.util.functional.CallableRaisingIOE; /** - * Callbacks for reading objectd data from the S3 Store. + * Callbacks for reading object data from the S3 Store. */ public interface ObjectInputStreamCallbacks extends Closeable { diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties index f548e7c98f9eb..7b8dd3c11fcdc 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties @@ -100,5 +100,5 @@ log4j.logger.org.apache.hadoop.fs.s3a.S3AStorageStatistics=INFO # debug service lifecycle of components such as S3AStore and # services it launches itself. -log4.logger.org.apache.hadoop.service=DEBUG +# log4.logger.org.apache.hadoop.service=DEBUG From 88ee1d2898269899c3e6056e6c936003e880a6b1 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 22 Jan 2025 16:51:31 +0000 Subject: [PATCH 7/7] HADOOP-19354. Support stream factory callbacks - Add callbacks from stream factories to creator. - Initial operation is to ask for an async client. - Callbacks and wiring up done in S3AStoreImpl. Change-Id: I544f05da15e3b57e9a538d337b972e4e07dc8877 --- .../hadoop/fs/s3a/impl/S3AStoreImpl.java | 61 ++++++++++++++++++- .../AbstractObjectInputStreamFactory.java | 35 +++++++++++ .../streams/ObjectInputStreamFactory.java | 23 +++++++ .../s3a/impl/streams/StreamIntegration.java | 8 +++ .../PrefetchingInputStreamFactory.java | 3 +- 5 files changed, 126 insertions(+), 4 deletions(-) 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 e2fbb07f2440c..0fabf4f2c64ad 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 @@ -84,6 +84,7 @@ 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; @@ -230,22 +231,28 @@ public class S3AStoreImpl @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 + // 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 capabilities. + * 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. @@ -262,6 +269,7 @@ public boolean hasPathCapability(final Path path, final String capability) { } } + /** * Return the capabilities of input streams created * through the store. @@ -931,6 +939,25 @@ public File createTemporaryFileForWriting(String pathStr, 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 { @@ -942,4 +969,32 @@ public ObjectInputStream readObject(ObjectReadParameters parameters) 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 index d3b6c67113997..7c20f7d66f61b 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 @@ -21,6 +21,7 @@ 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; @@ -34,6 +35,33 @@ 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)) { @@ -45,4 +73,11 @@ public boolean hasCapability(final String capability) { } } + /** + * 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/ObjectInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/ObjectInputStreamFactory.java index 1293bd4f92a06..d8fe87f9cf7fd 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 @@ -20,6 +20,8 @@ import java.io.IOException; +import software.amazon.awssdk.services.s3.S3AsyncClient; + import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.service.Service; @@ -38,6 +40,14 @@ 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 @@ -55,5 +65,18 @@ ObjectInputStream readObject(ObjectReadParameters parameters) */ 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/StreamIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/StreamIntegration.java index 85ca41beaf914..dfe2efbb97c4f 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 @@ -49,12 +49,20 @@ public final class StreamIntegration { */ 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/prefetch/PrefetchingInputStreamFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/PrefetchingInputStreamFactory.java index b490294b7696d..4109580c4ce5d 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 @@ -62,7 +62,7 @@ protected void serviceInit(final Configuration conf) throws Exception { long prefetchBlockSizeLong = longBytesOption(conf, PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE, 1); checkState(prefetchBlockSizeLong < Integer.MAX_VALUE, - "S3A prefatch block size exceeds int limit"); + "S3A prefetch block size exceeds int limit"); prefetchBlockSize = (int) prefetchBlockSizeLong; prefetchBlockCount = intOption(conf, PREFETCH_BLOCK_COUNT_KEY, PREFETCH_BLOCK_DEFAULT_COUNT, 1); @@ -87,4 +87,5 @@ public ObjectInputStream readObject(final ObjectReadParameters parameters) throw public StreamThreadOptions threadRequirements() { return new StreamThreadOptions(prefetchBlockCount, 0, true, false); } + }