From 2d23c5b9e316fbc790b4025dfd9df19983ed720f Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Tue, 26 Nov 2024 20:38:23 +0000 Subject: [PATCH 01/20] Integrate Analytics Accelerator Library for Amazon S3 This commits is the initial integration of Analytics Accelerator Library for Amazon S3 to S3A. It performs integration by introducing a new S3ASeekableStream and modifying S3AFileSystem. Use of the Analytics Accelerator Library is controlled by a configration and it is off by default. --- hadoop-tools/hadoop-aws/pom.xml | 11 +++ .../org/apache/hadoop/fs/s3a/Constants.java | 18 +++++ .../apache/hadoop/fs/s3a/S3AFileSystem.java | 48 +++++++++++- .../hadoop/fs/s3a/S3ASeekableStream.java | 68 ++++++++++++++++ .../fs/s3a/ITestS3AS3SeekableStream.java | 78 +++++++++++++++++++ 5 files changed, 222 insertions(+), 1 deletion(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index d28704b7c334e..0fa9cf9b5a166 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -525,6 +525,17 @@ amazon-s3-encryption-client-java provided + + software.amazon.s3.analyticsaccelerator + analyticsaccelerator-s3 + 0.0.1 + compile + + + software.amazon.awssdk.crt + aws-crt + 0.29.10 + org.assertj assertj-core diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index b03c41c7bb1d2..b70dd9cd2aa48 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 @@ -1760,4 +1760,22 @@ private Constants() { * Value: {@value}. */ public static final String S3A_IO_RATE_LIMIT = "fs.s3a.io.rate.limit"; + + /** + * Config to enable Analytics Accelerator Library for Amazon S3 + * https://github.com/awslabs/analytics-accelerator-s3 + */ + public static final String ANALYTICS_ACCELERATOR_ENABLED_KEY = "fs.s3a.analytics.accelerator.enabled"; + + /** + * Default value for {@link #ANALYTICS_ACCELERATOR_ENABLED_KEY } + * Value {@value}. + */ + public static final boolean ANALYTICS_ACCELERATOR_ENABLED_DEFAULT = false; + + /** + * Prefix to configure Analytics Accelerator Library + */ + public static final String ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX = "fs.s3a.analytics.accelerator"; + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index c0e530cb5ce40..69859a95d5c49 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 @@ -54,7 +54,9 @@ import software.amazon.awssdk.core.ResponseInputStream; import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.internal.crt.S3CrtAsyncClient; import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse; import software.amazon.awssdk.services.s3.model.GetBucketLocationRequest; @@ -87,6 +89,11 @@ import software.amazon.awssdk.transfer.s3.model.Copy; import software.amazon.awssdk.transfer.s3.model.CopyRequest; +import software.amazon.s3.analyticsaccelerator.S3SdkObjectClient; +import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamConfiguration; +import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamFactory; +import software.amazon.s3.analyticsaccelerator.common.ConnectorConfiguration; + import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -317,6 +324,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, */ private S3Client s3Client; + /** + * CRT-Based S3Client created of analytics accelerator library is enabled + * and managed by the ClientManager. Analytics accelerator library can be + * enabled with {@link Constants#ANALYTICS_ACCELERATOR_ENABLED_KEY} + */ + private S3AsyncClient crtClient; + // initial callback policy is fail-once; it's there just to assist // some mock tests and other codepaths trying to call the low level // APIs on an uninitialized filesystem. @@ -344,6 +358,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, // If true, the prefetching input stream is used for reads. private boolean prefetchEnabled; + // If true, S3SeekableInputStream from Analytics Accelerator for Amazon S3 will be used. + private boolean analyticsAcceleratorEnabled; + // Size in bytes of a single prefetch block. private int prefetchBlockSize; @@ -525,6 +542,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, */ private boolean s3AccessGrantsEnabled; + /** + * Factory to create S3SeekableInputStream if {@link this#analyticsAcceleratorEnabled} is true + */ + private S3SeekableInputStreamFactory s3SeekableInputStreamFactory; + /** Add any deprecated keys. */ @SuppressWarnings("deprecation") private static void addDeprecatedKeys() { @@ -672,6 +694,7 @@ public void initialize(URI name, Configuration originalConf) s3ExpressStore); this.prefetchEnabled = conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT); + this.analyticsAcceleratorEnabled = conf.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, ANALYTICS_ACCELERATOR_ENABLED_DEFAULT); long prefetchBlockSizeLong = longBytesOption(conf, PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE, 1); if (prefetchBlockSizeLong > (long) Integer.MAX_VALUE) { @@ -718,6 +741,17 @@ public void initialize(URI name, Configuration originalConf) // the encryption algorithms) ClientManager clientManager = createClientManager(name, delegationTokensEnabled); + if (this.analyticsAcceleratorEnabled) { + LOG.info("Using S3SeekableInputStream"); + this.crtClient = S3CrtAsyncClient.builder().maxConcurrency(600).build(); + ConnectorConfiguration configuration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); + S3SeekableInputStreamConfiguration seekableInputStreamConfiguration = + S3SeekableInputStreamConfiguration.fromConfiguration(configuration); + this.s3SeekableInputStreamFactory = + new S3SeekableInputStreamFactory( + new S3SdkObjectClient(this.crtClient), seekableInputStreamConfiguration); + } + inputPolicy = S3AInputPolicy.getPolicy( conf.getTrimmed(INPUT_FADVISE, Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT), @@ -1876,6 +1910,8 @@ private FSDataInputStream executeOpen( final Path path, final OpenFileSupport.OpenFileInformation fileInformation) throws IOException { + + // create the input stream statistics before opening // the file so that the time to prepare to open the file is included. S3AInputStreamStatistics inputStreamStats = @@ -1892,6 +1928,14 @@ private FSDataInputStream executeOpen( fileInformation.applyOptions(readContext); LOG.debug("Opening '{}'", readContext); + if (this.analyticsAcceleratorEnabled) { + return new FSDataInputStream( + new S3ASeekableStream( + this.bucket, + pathToKey(path), + s3SeekableInputStreamFactory)); + } + if (this.prefetchEnabled) { Configuration configuration = getConf(); initLocalDirAllocatorIfNotInitialized(configuration); @@ -4421,9 +4465,11 @@ public void close() throws IOException { protected synchronized void stopAllServices() { try { trackDuration(getDurationTrackerFactory(), FILESYSTEM_CLOSE.getSymbol(), () -> { - closeAutocloseables(LOG, store); + closeAutocloseables(LOG, store, s3SeekableInputStreamFactory); store = null; s3Client = null; + crtClient = null; + s3SeekableInputStreamFactory = null; // At this point the S3A client is shut down, // now the executor pools are closed diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java new file mode 100644 index 0000000000000..23b4a603e640c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java @@ -0,0 +1,68 @@ +package org.apache.hadoop.fs.s3a; + +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSInputStream; + +import software.amazon.s3.analyticsaccelerator.S3SeekableInputStream; +import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamFactory; +import software.amazon.s3.analyticsaccelerator.util.S3URI; + +public class S3ASeekableStream extends FSInputStream { + + private S3SeekableInputStream inputStream; + private final String key; + + public static final Logger LOG = LoggerFactory.getLogger(S3ASeekableStream.class); + + + public S3ASeekableStream(String bucket, String key, S3SeekableInputStreamFactory s3SeekableInputStreamFactory) + throws IOException { + this.inputStream = s3SeekableInputStreamFactory.createStream(S3URI.of(bucket, key)); + this.key = key; + } + + @Override + public int read() throws IOException { + return inputStream.read(); + } + + @Override + public void seek(long pos) throws IOException { + inputStream.seek(pos); + } + + @Override + public long getPos() throws IOException { + return inputStream.getPos(); + } + + @Override + public void close() throws IOException { + if (inputStream != null) { + inputStream.close(); + inputStream = null; + super.close(); + } + } + + + public void readTail(byte[] buf, int off, int n) throws IOException { + inputStream.readTail(buf, off, n); + } + + @Override + public int read(byte[] buf, int off, int len) throws IOException { + return inputStream.read(buf, off, len); + } + + + @Override + public boolean seekToNewSource(long l) throws IOException { + return false; + } + +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java new file mode 100644 index 0000000000000..aa2de18dc231a --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java @@ -0,0 +1,78 @@ +package org.apache.hadoop.fs.s3a; + +import java.io.IOException; + +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX; +import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_ENABLED_KEY; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; + +import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamConfiguration; +import software.amazon.s3.analyticsaccelerator.common.ConnectorConfiguration; +import software.amazon.s3.analyticsaccelerator.util.PrefetchMode; + +public class ITestS3AS3SeekableStream extends AbstractS3ATestBase { + + final String PHYSICAL_IO_PREFIX = "physicalio"; + final String LOGICAL_IO_PREFIX = "logicalio"; + + @Test + public void testConnectorFrameWorkIntegration() throws IOException { + describe("Verify S3 connector framework integration"); + + Configuration conf = getConfiguration(); + removeBaseAndBucketOverrides(conf, ANALYTICS_ACCELERATOR_ENABLED_KEY); + conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, true); + + String testFile = "s3a://noaa-cors-pds/raw/2023/017/ohfh/OHFH017d.23_.gz"; + S3AFileSystem s3AFileSystem = (S3AFileSystem) FileSystem.newInstance(new Path(testFile).toUri(), conf); + byte[] buffer = new byte[500]; + + try (FSDataInputStream inputStream = s3AFileSystem.open(new Path(testFile))) { + inputStream.seek(5); + inputStream.read(buffer, 0, 500); + } + + } + + + @Test + public void testConnectorFrameworkConfigurable() { + describe("Verify S3 connector framework reads configuration"); + + Configuration conf = getConfiguration(); + removeBaseAndBucketOverrides(conf); + + //Disable Predictive Prefetching + conf.set(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + LOGICAL_IO_PREFIX + ".prefetching.mode", "all"); + + //Set Blobstore Capacity + conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", 1); + + ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); + + S3SeekableInputStreamConfiguration configuration = S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration); + assertEquals(configuration.getLogicalIOConfiguration().getPrefetchingMode(), PrefetchMode.ALL); + assert configuration.getPhysicalIOConfiguration().getBlobStoreCapacity() == 1; + } + + @Test + public void testInvalidConfigurationThrows() { + describe("Verify S3 connector framework throws with invalid configuration"); + + Configuration conf = getConfiguration(); + removeBaseAndBucketOverrides(conf); + //Disable Sequential Prefetching + conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", -1); + + ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); + assertThrows(IllegalArgumentException.class, () -> + S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration)); + } +} From 693fe9ecce82296fee1b2935b007743c11a6ae53 Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Wed, 27 Nov 2024 13:35:59 +0000 Subject: [PATCH 02/20] Update S3ASeekableStream Implementation to throw when stream is closed --- .../hadoop/fs/s3a/S3ASeekableStream.java | 41 +++++++++++++------ 1 file changed, 29 insertions(+), 12 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java index 23b4a603e640c..d3b7c65deb76a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java @@ -2,6 +2,7 @@ import java.io.IOException; +import org.apache.hadoop.fs.FSExceptionMessages; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -14,48 +15,46 @@ public class S3ASeekableStream extends FSInputStream { private S3SeekableInputStream inputStream; + private long lastReadCurrentPos = 0; private final String key; public static final Logger LOG = LoggerFactory.getLogger(S3ASeekableStream.class); - - public S3ASeekableStream(String bucket, String key, S3SeekableInputStreamFactory s3SeekableInputStreamFactory) - throws IOException { + public S3ASeekableStream(String bucket, String key, S3SeekableInputStreamFactory s3SeekableInputStreamFactory) { this.inputStream = s3SeekableInputStreamFactory.createStream(S3URI.of(bucket, key)); this.key = key; } @Override public int read() throws IOException { + throwIfClosed(); return inputStream.read(); } @Override public void seek(long pos) throws IOException { + throwIfClosed(); inputStream.seek(pos); } - @Override - public long getPos() throws IOException { - return inputStream.getPos(); - } @Override - public void close() throws IOException { - if (inputStream != null) { - inputStream.close(); - inputStream = null; - super.close(); + public synchronized long getPos() { + if (!isClosed()) { + lastReadCurrentPos = inputStream.getPos(); } + return lastReadCurrentPos; } public void readTail(byte[] buf, int off, int n) throws IOException { + throwIfClosed(); inputStream.readTail(buf, off, n); } @Override public int read(byte[] buf, int off, int len) throws IOException { + throwIfClosed(); return inputStream.read(buf, off, len); } @@ -65,4 +64,22 @@ public boolean seekToNewSource(long l) throws IOException { return false; } + @Override + public void close() throws IOException { + if (inputStream != null) { + inputStream.close(); + inputStream = null; + super.close(); + } + } + + protected void throwIfClosed() throws IOException { + if (isClosed()) { + throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + } + } + + protected boolean isClosed() { + return inputStream == null; + } } \ No newline at end of file From f78df7397af07dea39a9ff1bb29d7b2b186f3487 Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Wed, 27 Nov 2024 16:55:04 +0000 Subject: [PATCH 03/20] Add license text to new files --- .../hadoop/fs/s3a/S3ASeekableStream.java | 28 +++++++++++++++++++ .../fs/s3a/ITestS3AS3SeekableStream.java | 19 +++++++++++++ 2 files changed, 47 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java index d3b7c65deb76a..6045e673ea915 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.hadoop.fs.s3a; import java.io.IOException; @@ -47,6 +66,15 @@ public synchronized long getPos() { } + /** + * Reads the last n bytes from the stream into a byte buffer. Blocks until end of stream is + * reached. Leaves the position of the stream unaltered. + * + * @param buf buffer to read data into + * @param off start position in buffer at which data is written + * @param n the number of bytes to read; the n-th byte should be the last byte of the stream. + * @return the total number of bytes read into the buffer + */ public void readTail(byte[] buf, int off, int n) throws IOException { throwIfClosed(); inputStream.readTail(buf, off, n); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java index aa2de18dc231a..b5334fc3925fa 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.hadoop.fs.s3a; import java.io.IOException; From 6a1a6602b819548f3345a0aa9e887dc8d0159de9 Mon Sep 17 00:00:00 2001 From: rajdchak Date: Thu, 12 Dec 2024 10:37:54 +0000 Subject: [PATCH 04/20] Make crt client configurable --- .../org/apache/hadoop/fs/s3a/Constants.java | 11 +++++ .../apache/hadoop/fs/s3a/S3AFileSystem.java | 46 ++++++++++++------- .../fs/s3a/ITestS3AS3SeekableStream.java | 27 +++++++++-- 3 files changed, 65 insertions(+), 19 deletions(-) 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 b70dd9cd2aa48..daf244b3a5ca6 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 @@ -1767,12 +1767,23 @@ private Constants() { */ public static final String ANALYTICS_ACCELERATOR_ENABLED_KEY = "fs.s3a.analytics.accelerator.enabled"; + /** + * Config to specify usage of crt client with Analytics Accelerator Library for Amazon S3 and it is by default true + */ + public static final String USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR = "fs.s3a.analytics.accelerator.crt.client"; + /** * Default value for {@link #ANALYTICS_ACCELERATOR_ENABLED_KEY } * Value {@value}. */ public static final boolean ANALYTICS_ACCELERATOR_ENABLED_DEFAULT = false; + /** + * Default value for {@link #USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR } + * Value {@value}. + */ + public static final boolean USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR_DEFAULT = true; + /** * Prefix to configure Analytics Accelerator Library */ 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 69859a95d5c49..35e22fb179014 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 @@ -329,7 +329,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, * and managed by the ClientManager. Analytics accelerator library can be * enabled with {@link Constants#ANALYTICS_ACCELERATOR_ENABLED_KEY} */ - private S3AsyncClient crtClient; + private S3AsyncClient s3AsyncClient; // initial callback policy is fail-once; it's there just to assist // some mock tests and other codepaths trying to call the low level @@ -694,7 +694,6 @@ public void initialize(URI name, Configuration originalConf) s3ExpressStore); this.prefetchEnabled = conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT); - this.analyticsAcceleratorEnabled = conf.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, ANALYTICS_ACCELERATOR_ENABLED_DEFAULT); long prefetchBlockSizeLong = longBytesOption(conf, PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE, 1); if (prefetchBlockSizeLong > (long) Integer.MAX_VALUE) { @@ -703,8 +702,15 @@ public void initialize(URI name, Configuration originalConf) 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); + + this.analyticsAcceleratorEnabled = conf.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, ANALYTICS_ACCELERATOR_ENABLED_DEFAULT); + + if(!analyticsAcceleratorEnabled) { + this.isMultipartUploadEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED, + DEFAULT_MULTIPART_UPLOAD_ENABLED); + } else { + this.isMultipartUploadEnabled = false; + } // multipart copy and upload are the same; this just makes it explicit this.isMultipartCopyEnabled = isMultipartUploadEnabled; @@ -741,17 +747,6 @@ public void initialize(URI name, Configuration originalConf) // the encryption algorithms) ClientManager clientManager = createClientManager(name, delegationTokensEnabled); - if (this.analyticsAcceleratorEnabled) { - LOG.info("Using S3SeekableInputStream"); - this.crtClient = S3CrtAsyncClient.builder().maxConcurrency(600).build(); - ConnectorConfiguration configuration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); - S3SeekableInputStreamConfiguration seekableInputStreamConfiguration = - S3SeekableInputStreamConfiguration.fromConfiguration(configuration); - this.s3SeekableInputStreamFactory = - new S3SeekableInputStreamFactory( - new S3SdkObjectClient(this.crtClient), seekableInputStreamConfiguration); - } - inputPolicy = S3AInputPolicy.getPolicy( conf.getTrimmed(INPUT_FADVISE, Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT), @@ -843,6 +838,25 @@ public void initialize(URI name, Configuration originalConf) // directly through the client manager. // this is to aid mocking. s3Client = store.getOrCreateS3Client(); + + if (this.analyticsAcceleratorEnabled) { + LOG.info("Using S3SeekableInputStream"); + if(conf.getBoolean(USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR, USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR_DEFAULT)) { + LOG.info("Using S3CrtClient"); + this.s3AsyncClient = S3CrtAsyncClient.builder().maxConcurrency(600).build(); + } else { + LOG.info("Using S3Client"); + this.s3AsyncClient = store.getOrCreateAsyncClient(); + } + + ConnectorConfiguration configuration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); + S3SeekableInputStreamConfiguration seekableInputStreamConfiguration = + S3SeekableInputStreamConfiguration.fromConfiguration(configuration); + this.s3SeekableInputStreamFactory = + new S3SeekableInputStreamFactory( + new S3SdkObjectClient(this.s3AsyncClient), seekableInputStreamConfiguration); + } + // The filesystem is now ready to perform operations against // S3 // This initiates a probe against S3 for the bucket existing. @@ -4468,7 +4482,7 @@ protected synchronized void stopAllServices() { closeAutocloseables(LOG, store, s3SeekableInputStreamFactory); store = null; s3Client = null; - crtClient = null; + s3AsyncClient = null; s3SeekableInputStreamFactory = null; // At this point the S3A client is shut down, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java index b5334fc3925fa..343015c63e14e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java @@ -30,6 +30,7 @@ import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX; import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_ENABLED_KEY; +import static org.apache.hadoop.fs.s3a.Constants.USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamConfiguration; @@ -41,13 +42,13 @@ public class ITestS3AS3SeekableStream extends AbstractS3ATestBase { final String PHYSICAL_IO_PREFIX = "physicalio"; final String LOGICAL_IO_PREFIX = "logicalio"; - @Test - public void testConnectorFrameWorkIntegration() throws IOException { + public void testConnectorFrameWorkIntegration(boolean useCrtClient) throws IOException { describe("Verify S3 connector framework integration"); Configuration conf = getConfiguration(); removeBaseAndBucketOverrides(conf, ANALYTICS_ACCELERATOR_ENABLED_KEY); conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, true); + conf.setBoolean(USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR, useCrtClient); String testFile = "s3a://noaa-cors-pds/raw/2023/017/ohfh/OHFH017d.23_.gz"; S3AFileSystem s3AFileSystem = (S3AFileSystem) FileSystem.newInstance(new Path(testFile).toUri(), conf); @@ -60,9 +61,17 @@ public void testConnectorFrameWorkIntegration() throws IOException { } + @Test + public void testConnectorFrameWorkIntegrationWithCrtClient() throws IOException { + testConnectorFrameWorkIntegration(true); + } @Test - public void testConnectorFrameworkConfigurable() { + public void testConnectorFrameWorkIntegrationWithoutCrtClient() throws IOException { + testConnectorFrameWorkIntegration(false); + } + + public void testConnectorFrameworkConfigurable(boolean useCrtClient) { describe("Verify S3 connector framework reads configuration"); Configuration conf = getConfiguration(); @@ -74,6 +83,8 @@ public void testConnectorFrameworkConfigurable() { //Set Blobstore Capacity conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", 1); + conf.setBoolean(USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR, useCrtClient); + ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); S3SeekableInputStreamConfiguration configuration = S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration); @@ -81,6 +92,16 @@ public void testConnectorFrameworkConfigurable() { assert configuration.getPhysicalIOConfiguration().getBlobStoreCapacity() == 1; } + @Test + public void testConnectorFrameworkConfigurableWithoutCrtClient() throws IOException { + testConnectorFrameworkConfigurable(false); + } + + @Test + public void testConnectorFrameworkConfigurableWithCrtClient() throws IOException { + testConnectorFrameworkConfigurable(true); + } + @Test public void testInvalidConfigurationThrows() { describe("Verify S3 connector framework throws with invalid configuration"); From 87fa86da36829738e13688c60ced1bb4989c718b Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Thu, 12 Dec 2024 13:42:43 +0000 Subject: [PATCH 05/20] Skip Tests --- .../hadoop/fs/s3a/S3ASeekableStream.java | 25 ++++++++++++++++++- .../contract/s3a/ITestS3AContractCreate.java | 10 ++++++-- .../contract/s3a/ITestS3AContractDistCp.java | 7 ++++++ .../contract/s3a/ITestS3AContractRename.java | 7 ++++++ .../s3a/ITestS3AContractVectoredRead.java | 11 ++++++++ .../hadoop/fs/s3a/ITestS3AEncryptionSSEC.java | 5 ++-- .../fs/s3a/ITestS3AIOStatisticsContext.java | 3 +++ .../s3a/ITestS3APrefetchingLruEviction.java | 7 ++++++ .../apache/hadoop/fs/s3a/S3ATestUtils.java | 14 +++++++++++ .../magic/ITestMagicCommitProtocol.java | 2 ++ .../ITestS3AContractStreamIOStatistics.java | 7 ++++++ 11 files changed, 92 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java index 6045e673ea915..c833449a0dd16 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java @@ -19,9 +19,11 @@ package org.apache.hadoop.fs.s3a; +import java.io.EOFException; import java.io.IOException; import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.StreamCapabilities; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,7 +33,7 @@ import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamFactory; import software.amazon.s3.analyticsaccelerator.util.S3URI; -public class S3ASeekableStream extends FSInputStream { +public class S3ASeekableStream extends FSInputStream implements StreamCapabilities { private S3SeekableInputStream inputStream; private long lastReadCurrentPos = 0; @@ -44,6 +46,17 @@ public S3ASeekableStream(String bucket, String key, S3SeekableInputStreamFactory this.key = key; } + /** + * Indicates whether the given {@code capability} is supported by this stream. + * + * @param capability the capability to check. + * @return true if the given {@code capability} is supported by this stream, false otherwise. + */ + @Override + public boolean hasCapability(String capability) { + return false; + } + @Override public int read() throws IOException { throwIfClosed(); @@ -53,6 +66,10 @@ public int read() throws IOException { @Override public void seek(long pos) throws IOException { throwIfClosed(); + if (pos < 0) { + throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK + + " " + pos); + } inputStream.seek(pos); } @@ -92,6 +109,12 @@ public boolean seekToNewSource(long l) throws IOException { return false; } + @Override + public int available() throws IOException { + throwIfClosed(); + return super.available(); + } + @Override public void close() throws IOException { if (inputStream != null) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java index 000caf328837e..7fa663f461b14 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java @@ -30,8 +30,7 @@ import org.apache.hadoop.fs.s3a.S3ATestUtils; import static org.apache.hadoop.fs.s3a.Constants.CONNECTION_EXPECT_CONTINUE; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.setPerformanceFlags; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; /** * S3A contract tests creating files. @@ -88,6 +87,13 @@ protected Configuration createConfiguration() { return conf; } + @Override + public void testOverwriteExistingFile() throws Throwable { + super.testOverwriteExistingFile(); + skipIfAnalyticsAcceleratorEnabled(this.createConfiguration()); + + } + @Override public void testOverwriteNonEmptyDirectory() throws Throwable { try { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java index e761e0d14bf83..806a6cd80027d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestConstants.SCALE_TEST_TIMEOUT_MILLIS; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.StorageStatistics; @@ -51,6 +52,12 @@ protected Configuration createConfiguration() { return newConf; } + @Override + public void setup() throws Exception { + super.setup(); + skipIfAnalyticsAcceleratorEnabled(createConfiguration()); + } + @Override protected boolean shouldUseDirectWrite() { return true; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java index d3ba7373cc944..c67163c29013f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java @@ -36,6 +36,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents; import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; import static org.apache.hadoop.fs.s3a.S3ATestConstants.S3A_TEST_TIMEOUT; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; /** * S3A contract tests covering rename. @@ -45,6 +46,12 @@ public class ITestS3AContractRename extends AbstractContractRenameTest { public static final Logger LOG = LoggerFactory.getLogger( ITestS3AContractRename.class); + @Override + public void setup() throws Exception { + super.setup(); + skipIfAnalyticsAcceleratorEnabled(getContract().getConf()); + + } @Override protected int getTestTimeoutMillis() { return S3A_TEST_TIMEOUT; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java index 8096f55bcd54c..c0c88802a8443 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java @@ -61,6 +61,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.range; import static org.apache.hadoop.fs.contract.ContractTestUtils.returnBuffersToPoolPostRead; import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture; @@ -84,6 +85,16 @@ protected AbstractFSContract createContract(Configuration conf) { return new S3AContract(conf); } + /** + * Analytics Accelerator Library for Amazon S3 does not support Vectored Reads + * @throws Exception + */ + @Override + public void setup() throws Exception { + skipIfAnalyticsAcceleratorEnabled(createConfiguration()); + super.setup(); + } + /** * Verify response to a vector read request which is beyond the * real length of the file. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java index d22de3b06d81b..d4ce466f8444f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java @@ -38,9 +38,7 @@ import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_KEY; import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM; import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_KEY; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeSkipRootTests; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** @@ -93,6 +91,7 @@ protected Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration()); assumeEnabled(); // although not a root dir test, this confuses paths enough it shouldn't be run in // parallel with other jobs diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AIOStatisticsContext.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AIOStatisticsContext.java index 70dc5ee476c47..d94140f19e314 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AIOStatisticsContext.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AIOStatisticsContext.java @@ -43,6 +43,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; import static org.apache.hadoop.fs.s3a.S3ATestUtils.disablePrefetching; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES; @@ -77,6 +78,8 @@ protected Configuration createConfiguration() { public void setup() throws Exception { super.setup(); executor = HadoopExecutors.newFixedThreadPool(SMALL_THREADS); + skipIfAnalyticsAcceleratorEnabled(getContract().getConf()); + } @Override 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 0fa08f37cf909..8417b05eb6b75 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 @@ -45,6 +45,7 @@ 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.skipIfAnalyticsAcceleratorEnabled; 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; @@ -68,6 +69,12 @@ public static Collection params() { }); } + @Override + public void setup() throws Exception { + super.setup(); + skipIfAnalyticsAcceleratorEnabled(createConfiguration()); + } + public ITestS3APrefetchingLruEviction(final String maxBlocks) { super(true); this.maxBlocks = maxBlocks; 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 308838c2927fa..089cac79c12e4 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 @@ -574,6 +574,20 @@ public static boolean isS3ExpressTestBucket(final Configuration conf) { return S3ExpressStorage.isS3ExpressStore(getTestBucketName(conf), ""); } + /** + * Skip a test if the Analytics Accelerator Library for Amazon S3 is enabled. + * @param configuration configuration to probe + */ + public static void skipIfAnalyticsAcceleratorEnabled( + Configuration configuration) { + assume("Skipping test as Analytics Accelerator Library does not support this functionality", + !isAnalyticsAcceleratorEnabled(configuration)); + } + + public static boolean isAnalyticsAcceleratorEnabled(final Configuration conf) { + return conf.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, ANALYTICS_ACCELERATOR_ENABLED_DEFAULT); + } + /** * Skip a test if the filesystem lacks a required capability. * @param fs filesystem diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java index cbfc23a2a29b6..8f04d46a47901 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java @@ -46,6 +46,7 @@ import org.junit.runners.Parameterized; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.s3a.S3AUtils.listAndFilter; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.impl.CommitUtilsWithMR.getMagicJobPath; @@ -77,6 +78,7 @@ protected String getCommitterName() { @Override public void setup() throws Exception { super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration()); CommitUtils.verifyIsMagicCommitFS(getFileSystem()); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java index 0f6b69cd54d89..0e73e2d74ae5b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.contract.s3a.S3AContract; import org.apache.hadoop.fs.statistics.StreamStatisticNames; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.statistics.StreamStatisticNames.*; /** @@ -78,4 +79,10 @@ public List outputStreamStatisticKeys() { STREAM_WRITE_EXCEPTIONS); } + @Override + public void setup() throws Exception { + super.setup(); + skipIfAnalyticsAcceleratorEnabled(getContract().getConf()); + } + } From 122dd590b42c682567ae376bd6de03636d8c6b5f Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Thu, 12 Dec 2024 16:14:35 +0000 Subject: [PATCH 06/20] Disable PrefetchingInputStream tests and write-only CommitProtocol tests --- .../hadoop/fs/s3a/ITestS3APrefetchingInputStream.java | 7 +++++++ .../integration/ITestPartitionedCommitProtocol.java | 8 ++++++++ 2 files changed, 15 insertions(+) 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 4998cbc946e12..b9aed3d82c41c 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 @@ -36,6 +36,7 @@ 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.skipIfAnalyticsAcceleratorEnabled; 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; @@ -75,6 +76,12 @@ public ITestS3APrefetchingInputStream() { private static final int INTERVAL_MILLIS = 500; private static final int BLOCK_SIZE = S_1K * 10; + @Override + public void setup() throws Exception { + super.setup(); + skipIfAnalyticsAcceleratorEnabled(this.createConfiguration()); + } + @Override public Configuration createConfiguration() { Configuration conf = super.createConfiguration(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionedCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionedCommitProtocol.java index e3bc1500dab7c..9d7bb6c41f551 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionedCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionedCommitProtocol.java @@ -32,10 +32,18 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext; import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; /** ITest of the low level protocol methods. */ public class ITestPartitionedCommitProtocol extends ITestStagingCommitProtocol { + + @Override + public void setup() throws Exception { + super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + } + @Override protected String suitename() { return "ITestPartitionedCommitProtocol"; From 22d4e8d4bea900163c2cfb36197d37d5bc1df6b4 Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Thu, 12 Dec 2024 16:43:37 +0000 Subject: [PATCH 07/20] skip remaining commitProtocol tests --- .../integration/ITestDirectoryCommitProtocol.java | 8 ++++++++ .../integration/ITestStagingCommitProtocol.java | 3 +++ .../ITestStagingCommitProtocolFailure.java | 11 ++++++++--- 3 files changed, 19 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitProtocol.java index b19662c0117fd..6e5ae77aa3761 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitProtocol.java @@ -35,6 +35,7 @@ import org.apache.hadoop.mapreduce.JobStatus; import org.apache.hadoop.mapreduce.TaskAttemptContext; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.CONFLICT_MODE_APPEND; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_CONFLICT_MODE; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.S3A_COMMITTER_EXPERIMENTAL_COLLECT_IOSTATISTICS; @@ -42,6 +43,13 @@ /** ITest of the low level protocol methods. */ public class ITestDirectoryCommitProtocol extends ITestStagingCommitProtocol { + + @Override + public void setup() throws Exception { + super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + } + @Override protected String suitename() { return "ITestDirectoryCommitProtocol"; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java index 81c3af812ab95..513df74bcbdff 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java @@ -41,6 +41,7 @@ import org.apache.hadoop.mapreduce.JobStatus; import org.apache.hadoop.mapreduce.TaskAttemptContext; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; /** Test the staging committer's handling of the base protocol operations. */ @@ -65,6 +66,8 @@ protected Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + // identify working dir for staging and delete Configuration conf = getConfiguration(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java index 08b6c21a863d5..fcbf9f1d8902b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java @@ -32,9 +32,7 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_UPLOADS_ENABLED; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_NAME; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.S3A_COMMITTER_FACTORY_KEY; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -45,6 +43,13 @@ */ public class ITestStagingCommitProtocolFailure extends AbstractS3ATestBase { + + @Override + public void setup() throws Exception { + super.setup(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + } + @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); From 9b82163257aca7136c4b1b3ad3e37d8a1d9715e1 Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Fri, 13 Dec 2024 14:34:16 +0000 Subject: [PATCH 08/20] Address review comments --- .../org/apache/hadoop/fs/s3a/Constants.java | 19 +++++++------------ .../hadoop/fs/s3a/S3ASeekableStream.java | 2 +- .../fs/s3a/ITestS3APrefetchingCacheFiles.java | 3 ++- 3 files changed, 10 insertions(+), 14 deletions(-) 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 daf244b3a5ca6..f73b0d7a41e2e 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 @@ -1761,11 +1761,17 @@ private Constants() { */ public static final String S3A_IO_RATE_LIMIT = "fs.s3a.io.rate.limit"; + + /** + * Prefix to configure Analytics Accelerator Library + */ + public static final String ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX = "fs.s3a.analytics.accelerator"; + /** * Config to enable Analytics Accelerator Library for Amazon S3 * https://github.com/awslabs/analytics-accelerator-s3 */ - public static final String ANALYTICS_ACCELERATOR_ENABLED_KEY = "fs.s3a.analytics.accelerator.enabled"; + public static final String ANALYTICS_ACCELERATOR_ENABLED_KEY = ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + ".enabled"; /** * Config to specify usage of crt client with Analytics Accelerator Library for Amazon S3 and it is by default true @@ -1778,15 +1784,4 @@ private Constants() { */ public static final boolean ANALYTICS_ACCELERATOR_ENABLED_DEFAULT = false; - /** - * Default value for {@link #USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR } - * Value {@value}. - */ - public static final boolean USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR_DEFAULT = true; - - /** - * Prefix to configure Analytics Accelerator Library - */ - public static final String ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX = "fs.s3a.analytics.accelerator"; - } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java index c833449a0dd16..9362e9f322c59 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java @@ -126,7 +126,7 @@ public void close() throws IOException { protected void throwIfClosed() throws IOException { if (isClosed()) { - throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + throw new IOException(key + ": " + FSExceptionMessages.STREAM_IS_CLOSED); } } 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 5e6731ed520ad..9ecf2203c37f2 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 @@ -41,6 +41,7 @@ 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.skipIfAnalyticsAcceleratorEnabled; 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; @@ -77,7 +78,7 @@ public void setUp() throws Exception { super.setup(); // Sets BUFFER_DIR by calling S3ATestUtils#prepareTestConfiguration conf = createConfiguration(); - + skipIfAnalyticsAcceleratorEnabled(conf); testFile = getExternalData(conf); prefetchBlockSize = conf.getInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); fs = FileSystem.get(testFile.toUri(), conf); From e38b482a3f465288fb8d9990bb67a91767a9db9a Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Fri, 13 Dec 2024 14:43:55 +0000 Subject: [PATCH 09/20] Fix build failure --- .../src/main/java/org/apache/hadoop/fs/s3a/Constants.java | 6 ++++++ .../hadoop/fs/contract/s3a/ITestS3AContractCreate.java | 3 +-- 2 files changed, 7 insertions(+), 2 deletions(-) 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 f73b0d7a41e2e..29557506419bd 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 @@ -1784,4 +1784,10 @@ private Constants() { */ public static final boolean ANALYTICS_ACCELERATOR_ENABLED_DEFAULT = false; + /** + * Default value for {@link #USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR } + * Value {@value}. + */ + public static final boolean USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR_DEFAULT = true; + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java index 7fa663f461b14..e7a6bb6e4c3f5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java @@ -89,9 +89,8 @@ protected Configuration createConfiguration() { @Override public void testOverwriteExistingFile() throws Throwable { - super.testOverwriteExistingFile(); skipIfAnalyticsAcceleratorEnabled(this.createConfiguration()); - + super.testOverwriteExistingFile(); } @Override From e56882ec06750656c584fe92b1715dcdea175eea Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Fri, 13 Dec 2024 16:16:35 +0000 Subject: [PATCH 10/20] Add exception handling. Fix Multi-part uploads --- .../org/apache/hadoop/fs/s3a/Constants.java | 6 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 15 +++-- .../hadoop/fs/s3a/S3ASeekableStream.java | 62 ++++++++++++++++--- .../fs/s3a/ITestS3AS3SeekableStream.java | 6 +- .../s3a/commit/ITestS3ACommitterFactory.java | 2 + 5 files changed, 72 insertions(+), 19 deletions(-) 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 29557506419bd..850551f8dbbb4 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 @@ -1776,7 +1776,7 @@ private Constants() { /** * Config to specify usage of crt client with Analytics Accelerator Library for Amazon S3 and it is by default true */ - public static final String USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR = "fs.s3a.analytics.accelerator.crt.client"; + public static final String ANALYTICS_ACCELERATOR_CRT_ENABLED = "fs.s3a.analytics.accelerator.crt.client"; /** * Default value for {@link #ANALYTICS_ACCELERATOR_ENABLED_KEY } @@ -1785,9 +1785,9 @@ private Constants() { public static final boolean ANALYTICS_ACCELERATOR_ENABLED_DEFAULT = false; /** - * Default value for {@link #USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR } + * Default value for {@link #ANALYTICS_ACCELERATOR_CRT_ENABLED } * Value {@value}. */ - public static final boolean USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR_DEFAULT = true; + public static final boolean ANALYTICS_ACCELERATOR_CRT_ENABLED_DEFAULT = true; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 35e22fb179014..f9ca6d5d8ddc6 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 @@ -361,6 +361,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, // If true, S3SeekableInputStream from Analytics Accelerator for Amazon S3 will be used. private boolean analyticsAcceleratorEnabled; + private boolean analyticsAcceleratorCRTEnabled; + // Size in bytes of a single prefetch block. private int prefetchBlockSize; @@ -704,13 +706,16 @@ public void initialize(URI name, Configuration originalConf) intOption(conf, PREFETCH_BLOCK_COUNT_KEY, PREFETCH_BLOCK_DEFAULT_COUNT, 1); this.analyticsAcceleratorEnabled = conf.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, ANALYTICS_ACCELERATOR_ENABLED_DEFAULT); + this.analyticsAcceleratorCRTEnabled = conf.getBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, ANALYTICS_ACCELERATOR_CRT_ENABLED_DEFAULT); - if(!analyticsAcceleratorEnabled) { - this.isMultipartUploadEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED, - DEFAULT_MULTIPART_UPLOAD_ENABLED); - } else { + this.isMultipartUploadEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED, + DEFAULT_MULTIPART_UPLOAD_ENABLED); + + if(this.analyticsAcceleratorEnabled && !analyticsAcceleratorCRTEnabled) { + // Temp change: Analytics Accelerator with S3AsyncClient do not support Multi-part upload. this.isMultipartUploadEnabled = false; } + // multipart copy and upload are the same; this just makes it explicit this.isMultipartCopyEnabled = isMultipartUploadEnabled; @@ -841,7 +846,7 @@ public void initialize(URI name, Configuration originalConf) if (this.analyticsAcceleratorEnabled) { LOG.info("Using S3SeekableInputStream"); - if(conf.getBoolean(USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR, USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR_DEFAULT)) { + if(this.analyticsAcceleratorCRTEnabled) { LOG.info("Using S3CrtClient"); this.s3AsyncClient = S3CrtAsyncClient.builder().maxConcurrency(600).build(); } else { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java index 9362e9f322c59..7c6494d1bcf3e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java @@ -60,7 +60,14 @@ public boolean hasCapability(String capability) { @Override public int read() throws IOException { throwIfClosed(); - return inputStream.read(); + int bytesRead; + try { + bytesRead = inputStream.read(); + } catch (IOException ioe){ + onReadFailure(ioe); + throw ioe; + } + return bytesRead; } @Override @@ -89,18 +96,32 @@ public synchronized long getPos() { * * @param buf buffer to read data into * @param off start position in buffer at which data is written - * @param n the number of bytes to read; the n-th byte should be the last byte of the stream. + * @param len the number of bytes to read; the n-th byte should be the last byte of the stream. * @return the total number of bytes read into the buffer */ - public void readTail(byte[] buf, int off, int n) throws IOException { + public int readTail(byte[] buf, int off, int len) throws IOException { throwIfClosed(); - inputStream.readTail(buf, off, n); + int bytesRead; + try { + bytesRead = inputStream.readTail(buf, off, len); + } catch (IOException ioe) { + onReadFailure(ioe); + throw ioe; + } + return bytesRead; } @Override public int read(byte[] buf, int off, int len) throws IOException { throwIfClosed(); - return inputStream.read(buf, off, len); + int bytesRead; + try { + bytesRead = inputStream.read(buf, off, len); + } catch (IOException ioe) { + onReadFailure(ioe); + throw ioe; + } + return bytesRead; } @@ -118,12 +139,37 @@ public int available() throws IOException { @Override public void close() throws IOException { if (inputStream != null) { - inputStream.close(); - inputStream = null; - super.close(); + try { + inputStream.close(); + inputStream = null; + super.close(); + } catch (IOException ioe) { + LOG.debug("Failure closing stream {}: ", key); + throw ioe; + } } } + /** + * Close the stream on read failure. + * No attempt to recover from failure + * @param ioe exception caught. + */ + @Retries.OnceTranslated + private void onReadFailure(IOException ioe) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("Got exception while trying to read from stream {}, " + + "not trying to recover:", + key, ioe); + } else { + LOG.info("Got exception while trying to read from stream {}, " + + "not trying to recover:", + key, ioe); + } + this.close(); + } + + protected void throwIfClosed() throws IOException { if (isClosed()) { throw new IOException(key + ": " + FSExceptionMessages.STREAM_IS_CLOSED); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java index 343015c63e14e..080efc3c296a6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java @@ -30,7 +30,7 @@ import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX; import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_ENABLED_KEY; -import static org.apache.hadoop.fs.s3a.Constants.USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR; +import static org.apache.hadoop.fs.s3a.Constants.ANALYTICS_ACCELERATOR_CRT_ENABLED; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamConfiguration; @@ -48,7 +48,7 @@ public void testConnectorFrameWorkIntegration(boolean useCrtClient) throws IOExc Configuration conf = getConfiguration(); removeBaseAndBucketOverrides(conf, ANALYTICS_ACCELERATOR_ENABLED_KEY); conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, true); - conf.setBoolean(USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR, useCrtClient); + conf.setBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, useCrtClient); String testFile = "s3a://noaa-cors-pds/raw/2023/017/ohfh/OHFH017d.23_.gz"; S3AFileSystem s3AFileSystem = (S3AFileSystem) FileSystem.newInstance(new Path(testFile).toUri(), conf); @@ -83,7 +83,7 @@ public void testConnectorFrameworkConfigurable(boolean useCrtClient) { //Set Blobstore Capacity conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", 1); - conf.setBoolean(USE_CRT_CLIENT_WITH_S3A_ANALYTICS_ACCELERATOR, useCrtClient); + conf.setBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, useCrtClient); ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java index 2561a69f60b59..a224798eb5d99 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java @@ -47,6 +47,7 @@ import org.apache.hadoop.security.UserGroupInformation; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.COMMITTER_NAME_STAGING; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -182,6 +183,7 @@ public void setup() throws Exception { // destroy all filesystems from previous runs. FileSystem.closeAllForUGI(UserGroupInformation.getCurrentUser()); super.setup(); + skipIfAnalyticsAcceleratorEnabled(createConfiguration()); jobId = randomJobId(); attempt0 = "attempt_" + jobId + "_m_000000_0"; taskAttempt0 = TaskAttemptID.forName(attempt0); From 5e988730951b8547066c36afee6b15926dcfe2aa Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Fri, 13 Dec 2024 16:57:17 +0000 Subject: [PATCH 11/20] Skip stats and S3AInputStream cast tests --- .../apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java | 7 ++----- .../hadoop/fs/s3a/performance/ITestUnbufferDraining.java | 2 ++ 2 files changed, 4 insertions(+), 5 deletions(-) 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 8e61225e17ef5..72e26540d38ec 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 @@ -54,10 +54,7 @@ import static org.apache.hadoop.fs.s3a.Constants.CHECKSUM_VALIDATION; import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_DEFAULT; import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.assertStreamIsNotChecksummed; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getS3AInputStream; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES_READ_CLOSE; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_OPENED; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_SEEK_BYTES_SKIPPED; @@ -114,6 +111,7 @@ public Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); + skipIfAnalyticsAcceleratorEnabled(createConfiguration()); S3AFileSystem fs = getFileSystem(); testFile = methodPath(); @@ -392,7 +390,6 @@ public void testPositionedReadableReadPastEOF() throws Throwable { describe("PositionedReadable.read() past the end of the file"); assumeNoPrefetching(); - verifyMetrics(() -> { try (FSDataInputStream in = openFile(longLen, FS_OPTION_OPENFILE_READ_POLICY_RANDOM)) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestUnbufferDraining.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestUnbufferDraining.java index 00bae1519f5eb..cd62e23f14db7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestUnbufferDraining.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestUnbufferDraining.java @@ -54,6 +54,7 @@ import static org.apache.hadoop.fs.s3a.Constants.RETRY_LIMIT; import static org.apache.hadoop.fs.s3a.Constants.SOCKET_TIMEOUT; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.s3a.impl.ConfigurationHelper.setDurationAsSeconds; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; @@ -125,6 +126,7 @@ public Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); + skipIfAnalyticsAcceleratorEnabled(createConfiguration()); // now create a new FS with minimal http capacity and recovery // a separate one is used to avoid test teardown suffering From 667317ccfee239fd7b3b46bf9b7eb891a0972a3d Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Mon, 16 Dec 2024 14:55:45 +0000 Subject: [PATCH 12/20] skip overwrite tests --- .../hadoop/fs/s3a/ITestS3AFSMainOperations.java | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFSMainOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFSMainOperations.java index 0281c57f5cbce..a7e30c5949529 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFSMainOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFSMainOperations.java @@ -29,9 +29,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.s3a.S3AContract; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.createTestPath; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.isCreatePerformanceEnabled; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.setPerformanceFlags; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; /** * S3A Test suite for the FSMainOperationsBaseTest tests. @@ -79,6 +77,18 @@ public void testCopyToLocalWithUseRawLocalFileSystemOption() } @Override + public void testWriteReadAndDeleteOneAndAHalfBlocks() throws Exception { + //Skipping this test for AnalyticsAccelerator as it is acting as an overwrite test + skipIfAnalyticsAcceleratorEnabled(this.contract.getConf()); + } + + @Override + public void testWriteReadAndDeleteTwoBlocks() throws Exception { + //Skipping this test for AnalyticsAccelerator as it is acting as an overwrite test + skipIfAnalyticsAcceleratorEnabled(this.contract.getConf()); + } + + @Override public void testOverwrite() throws IOException { boolean createPerformance = isCreatePerformanceEnabled(fSys); try { From 9fdea6845810df021bd00cb3e4143ac8a9732e6b Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Mon, 16 Dec 2024 16:11:19 +0000 Subject: [PATCH 13/20] Skip Statistics tests --- hadoop-tools/hadoop-aws/pom.xml | 2 +- .../hadoop/fs/s3a/commit/ITestCommitOperationCost.java | 6 ++++++ .../s3a/fileContext/ITestS3AFileContextStatistics.java | 3 +++ .../hadoop/fs/s3a/impl/ITestConnectionTimeouts.java | 9 +++++++++ .../fs/s3a/statistics/ITestS3AFileSystemStatistic.java | 5 +++++ 5 files changed, 24 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 0fa9cf9b5a166..b00ca14332e7f 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -528,7 +528,7 @@ software.amazon.s3.analyticsaccelerator analyticsaccelerator-s3 - 0.0.1 + SNAPSHOT compile diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java index fbe1a0a3120bc..53a8871a73580 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java @@ -40,6 +40,7 @@ import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; import org.apache.hadoop.fs.statistics.IOStatisticsLogging; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.s3a.Statistic.ACTION_HTTP_GET_REQUEST; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MAGIC_FILES_CREATED; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MAGIC_MARKER_PUT; @@ -203,6 +204,9 @@ private void abortActiveStream() throws IOException { @Test public void testCostOfCreatingMagicFile() throws Throwable { describe("Files created under magic paths skip existence checks and marker deletes"); + + // Assertions will fail as {@link S3ASeekableInputStream} do not support InputStreamStatistics yes + skipIfAnalyticsAcceleratorEnabled(getConfiguration()); S3AFileSystem fs = getFileSystem(); Path destFile = methodSubPath("file.txt"); fs.delete(destFile.getParent(), true); @@ -282,6 +286,8 @@ public void testCostOfCreatingMagicFile() throws Throwable { public void testCostOfSavingLoadingPendingFile() throws Throwable { describe("Verify costs of saving .pending file under a magic path"); + // Assertions will fail as {@link S3ASeekableInputStream} do not support InputStreamStatistics yes + skipIfAnalyticsAcceleratorEnabled(getConfiguration()); S3AFileSystem fs = getFileSystem(); Path partDir = methodSubPath("file.pending"); Path destFile = new Path(partDir, "file.pending"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java index 1724006a83198..3fda6f24386c5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java @@ -30,6 +30,8 @@ import org.junit.Assert; import org.junit.Before; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; + /** * S3a implementation of FCStatisticsBaseTest. */ @@ -44,6 +46,7 @@ public class ITestS3AFileContextStatistics extends FCStatisticsBaseTest { @Before public void setUp() throws Exception { conf = new Configuration(); + skipIfAnalyticsAcceleratorEnabled(conf); fc = S3ATestUtils.createTestFileContext(conf); testRootPath = fileContextTestHelper.getTestRootPath(fc, "test"); fc.mkdir(testRootPath, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestConnectionTimeouts.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestConnectionTimeouts.java index a4cc5cadc5da0..d9071ef16c2d3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestConnectionTimeouts.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestConnectionTimeouts.java @@ -60,6 +60,7 @@ import static org.apache.hadoop.fs.s3a.Constants.RETRY_LIMIT; import static org.apache.hadoop.fs.s3a.Constants.SOCKET_TIMEOUT; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC_PATH_PREFIX; import static org.apache.hadoop.fs.s3a.impl.ConfigurationHelper.setDurationAsMillis; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -146,6 +147,10 @@ public void teardown() throws Exception { @Test public void testGeneratePoolTimeouts() throws Throwable { skipIfClientSideEncryption(); + + // Assertions will fail when using CRTClient with SeekableStream. + skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + AWSClientConfig.setMinimumOperationDuration(Duration.ZERO); Configuration conf = timingOutConfiguration(); Path path = methodPath(); @@ -188,6 +193,10 @@ public void testGeneratePoolTimeouts() throws Throwable { @Test public void testObjectUploadTimeouts() throws Throwable { skipIfClientSideEncryption(); + + // Assertions will fail when using CRTClient with SeekableStream. + skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + AWSClientConfig.setMinimumOperationDuration(Duration.ZERO); final Path dir = methodPath(); Path file = new Path(dir, "file"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java index 0d5d2a789a02a..227d511522990 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java @@ -31,6 +31,8 @@ import org.apache.hadoop.fs.statistics.IOStatisticAssertions; import org.apache.hadoop.fs.statistics.StreamStatisticNames; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; + public class ITestS3AFileSystemStatistic extends AbstractS3ATestBase { private static final int ONE_KB = 1024; @@ -42,6 +44,9 @@ public class ITestS3AFileSystemStatistic extends AbstractS3ATestBase { */ @Test public void testBytesReadWithStream() throws IOException { + // Assertions will fail as {@link S3ASeekableInputStream} do not support InputStreamStatistics yes + skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + S3AFileSystem fs = getFileSystem(); Path filePath = path(getMethodName()); byte[] oneKbBuf = new byte[ONE_KB]; From 5bc0c19b59ccca8ecc969cdbf330b3b693c3cef2 Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Mon, 16 Dec 2024 16:22:43 +0000 Subject: [PATCH 14/20] Skip Statistics tests --- .../test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java index 3bfe69c2bca91..420f49bd097ba 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.io.InputStream; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; /** @@ -51,6 +52,8 @@ public void testMetricsRegister() @Test public void testStreamStatistics() throws IOException { + skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + S3AFileSystem fs = getFileSystem(); Path file = path("testStreamStatistics"); byte[] data = "abcdefghijklmnopqrstuvwxyz".getBytes(); From 92f775c21644d6f76fd70a5893fdd059a00b82bd Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Mon, 16 Dec 2024 16:39:45 +0000 Subject: [PATCH 15/20] update contract tests and stream leak test --- .../apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java | 9 +++++++-- .../apache/hadoop/fs/s3a/ITestS3AInputStreamLeakage.java | 5 ++++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java index 4808145765822..f4b52f5057f34 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java @@ -34,8 +34,7 @@ import org.apache.hadoop.fs.Path; import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.isCreatePerformanceEnabled; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.setPerformanceFlags; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.junit.Assume.*; import static org.junit.Assert.*; @@ -160,4 +159,10 @@ public void testOverwrite() throws IOException { } } } + + @Override + public void testOverWriteAndRead() throws Exception { + skipIfAnalyticsAcceleratorEnabled(fs.getConf()); + super.testOverWriteAndRead(); + } } 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..d944bdb1b683e 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 @@ -34,6 +34,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_LEAKS; import static org.apache.hadoop.test.GenericTestUtils.LogCapturer.captureLogs; @@ -58,7 +59,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)); } @@ -89,6 +90,8 @@ public void setup() throws Exception { @Test public void testFinalizer() throws Throwable { Path path = methodPath(); + skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + final S3AFileSystem fs = getFileSystem(); ContractTestUtils.createFile(fs, path, true, DATASET); From 814f3218cf61420ea44b2a94fb8b733c82cd81cc Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Mon, 16 Dec 2024 20:59:31 +0000 Subject: [PATCH 16/20] revert pom change --- hadoop-tools/hadoop-aws/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index b00ca14332e7f..0fa9cf9b5a166 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -528,7 +528,7 @@ software.amazon.s3.analyticsaccelerator analyticsaccelerator-s3 - SNAPSHOT + 0.0.1 compile From 28714ac91104637fe9a321c01f6a9a85e8ef0076 Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Mon, 16 Dec 2024 21:32:31 +0000 Subject: [PATCH 17/20] fix checkstyle, address review comments --- .../org/apache/hadoop/fs/s3a/Constants.java | 16 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 10 +- .../hadoop/fs/s3a/S3ASeekableStream.java | 265 +++++++++--------- .../fs/s3a/ITestS3AS3SeekableStream.java | 116 ++++---- 4 files changed, 209 insertions(+), 198 deletions(-) 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 850551f8dbbb4..d563181bb6155 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 @@ -1763,20 +1763,24 @@ private Constants() { /** - * Prefix to configure Analytics Accelerator Library + * Prefix to configure Analytics Accelerator Library. */ - public static final String ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX = "fs.s3a.analytics.accelerator"; + public static final String ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX = + "fs.s3a.analytics.accelerator"; /** - * Config to enable Analytics Accelerator Library for Amazon S3 + * Config to enable Analytics Accelerator Library for Amazon S3. * https://github.com/awslabs/analytics-accelerator-s3 */ - public static final String ANALYTICS_ACCELERATOR_ENABLED_KEY = ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + ".enabled"; + public static final String ANALYTICS_ACCELERATOR_ENABLED_KEY = + ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + ".enabled"; /** - * Config to specify usage of crt client with Analytics Accelerator Library for Amazon S3 and it is by default true + * Config to enable usage of crt client with Analytics Accelerator Library. + * It is by default true. */ - public static final String ANALYTICS_ACCELERATOR_CRT_ENABLED = "fs.s3a.analytics.accelerator.crt.client"; + public static final String ANALYTICS_ACCELERATOR_CRT_ENABLED = + "fs.s3a.analytics.accelerator.crt.client"; /** * Default value for {@link #ANALYTICS_ACCELERATOR_ENABLED_KEY } 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 f9ca6d5d8ddc6..01b06e894b5ae 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 @@ -854,12 +854,14 @@ public void initialize(URI name, Configuration originalConf) this.s3AsyncClient = store.getOrCreateAsyncClient(); } - ConnectorConfiguration configuration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); + ConnectorConfiguration configuration = new ConnectorConfiguration(conf, + ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); S3SeekableInputStreamConfiguration seekableInputStreamConfiguration = - S3SeekableInputStreamConfiguration.fromConfiguration(configuration); + S3SeekableInputStreamConfiguration.fromConfiguration(configuration); this.s3SeekableInputStreamFactory = - new S3SeekableInputStreamFactory( - new S3SdkObjectClient(this.s3AsyncClient), seekableInputStreamConfiguration); + new S3SeekableInputStreamFactory( + new S3SdkObjectClient(this.s3AsyncClient), + seekableInputStreamConfiguration); } // The filesystem is now ready to perform operations against diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java index 7c6494d1bcf3e..067306ff3c4a1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java @@ -35,148 +35,151 @@ public class S3ASeekableStream extends FSInputStream implements StreamCapabilities { - private S3SeekableInputStream inputStream; - private long lastReadCurrentPos = 0; - private final String key; - - public static final Logger LOG = LoggerFactory.getLogger(S3ASeekableStream.class); - - public S3ASeekableStream(String bucket, String key, S3SeekableInputStreamFactory s3SeekableInputStreamFactory) { - this.inputStream = s3SeekableInputStreamFactory.createStream(S3URI.of(bucket, key)); - this.key = key; + private S3SeekableInputStream inputStream; + private long lastReadCurrentPos = 0; + private final String key; + + public static final Logger LOG = LoggerFactory.getLogger(S3ASeekableStream.class); + + public S3ASeekableStream(String bucket, String key, + S3SeekableInputStreamFactory s3SeekableInputStreamFactory) { + this.inputStream = s3SeekableInputStreamFactory.createStream(S3URI.of(bucket, key)); + this.key = key; + } + + /** + * Indicates whether the given {@code capability} is supported by this stream. + * + * @param capability the capability to check. + * @return true if the given {@code capability} is supported by this stream, false otherwise. + */ + @Override + public boolean hasCapability(String capability) { + return false; + } + + @Override + public int read() throws IOException { + throwIfClosed(); + int bytesRead; + try { + bytesRead = inputStream.read(); + } catch (IOException ioe) { + onReadFailure(ioe); + throw ioe; } - - /** - * Indicates whether the given {@code capability} is supported by this stream. - * - * @param capability the capability to check. - * @return true if the given {@code capability} is supported by this stream, false otherwise. - */ - @Override - public boolean hasCapability(String capability) { - return false; - } - - @Override - public int read() throws IOException { - throwIfClosed(); - int bytesRead; - try { - bytesRead = inputStream.read(); - } catch (IOException ioe){ - onReadFailure(ioe); - throw ioe; - } - return bytesRead; - } - - @Override - public void seek(long pos) throws IOException { - throwIfClosed(); - if (pos < 0) { - throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK - + " " + pos); - } - inputStream.seek(pos); - } - - - @Override - public synchronized long getPos() { - if (!isClosed()) { - lastReadCurrentPos = inputStream.getPos(); - } - return lastReadCurrentPos; + return bytesRead; + } + + @Override + public void seek(long pos) throws IOException { + throwIfClosed(); + if (pos < 0) { + throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK + + " " + pos); } + inputStream.seek(pos); + } - /** - * Reads the last n bytes from the stream into a byte buffer. Blocks until end of stream is - * reached. Leaves the position of the stream unaltered. - * - * @param buf buffer to read data into - * @param off start position in buffer at which data is written - * @param len the number of bytes to read; the n-th byte should be the last byte of the stream. - * @return the total number of bytes read into the buffer - */ - public int readTail(byte[] buf, int off, int len) throws IOException { - throwIfClosed(); - int bytesRead; - try { - bytesRead = inputStream.readTail(buf, off, len); - } catch (IOException ioe) { - onReadFailure(ioe); - throw ioe; - } - return bytesRead; + @Override + public synchronized long getPos() { + if (!isClosed()) { + lastReadCurrentPos = inputStream.getPos(); } - - @Override - public int read(byte[] buf, int off, int len) throws IOException { - throwIfClosed(); - int bytesRead; - try { - bytesRead = inputStream.read(buf, off, len); - } catch (IOException ioe) { - onReadFailure(ioe); - throw ioe; - } - return bytesRead; + return lastReadCurrentPos; + } + + + /** + * Reads the last n bytes from the stream into a byte buffer. Blocks until end of stream is + * reached. Leaves the position of the stream unaltered. + * + * @param buf buffer to read data into + * @param off start position in buffer at which data is written + * @param len the number of bytes to read; the n-th byte should be the last byte of the stream. + * @return the total number of bytes read into the buffer + * @throws IOException if an I/O error occurs + */ + public int readTail(byte[] buf, int off, int len) throws IOException { + throwIfClosed(); + int bytesRead; + try { + bytesRead = inputStream.readTail(buf, off, len); + } catch (IOException ioe) { + onReadFailure(ioe); + throw ioe; } - - - @Override - public boolean seekToNewSource(long l) throws IOException { - return false; + return bytesRead; + } + + @Override + public int read(byte[] buf, int off, int len) throws IOException { + throwIfClosed(); + int bytesRead; + try { + bytesRead = inputStream.read(buf, off, len); + } catch (IOException ioe) { + onReadFailure(ioe); + throw ioe; } - - @Override - public int available() throws IOException { - throwIfClosed(); - return super.available(); + return bytesRead; + } + + + @Override + public boolean seekToNewSource(long l) throws IOException { + return false; + } + + @Override + public int available() throws IOException { + throwIfClosed(); + return super.available(); + } + + @Override + public void close() throws IOException { + if (inputStream != null) { + try { + inputStream.close(); + inputStream = null; + super.close(); + } catch (IOException ioe) { + LOG.debug("Failure closing stream {}: ", key); + throw ioe; + } } - - @Override - public void close() throws IOException { - if (inputStream != null) { - try { - inputStream.close(); - inputStream = null; - super.close(); - } catch (IOException ioe) { - LOG.debug("Failure closing stream {}: ", key); - throw ioe; - } - } - } - - /** - * Close the stream on read failure. - * No attempt to recover from failure - * @param ioe exception caught. - */ - @Retries.OnceTranslated - private void onReadFailure(IOException ioe) throws IOException { - if (LOG.isDebugEnabled()) { - LOG.debug("Got exception while trying to read from stream {}, " + - "not trying to recover:", - key, ioe); - } else { - LOG.info("Got exception while trying to read from stream {}, " + - "not trying to recover:", - key, ioe); - } - this.close(); + } + + /** + * Close the stream on read failure. + * No attempt to recover from failure + * + * @param ioe exception caught. + */ + @Retries.OnceTranslated + private void onReadFailure(IOException ioe) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("Got exception while trying to read from stream {}, " + + "not trying to recover:", + key, ioe); + } else { + LOG.info("Got exception while trying to read from stream {}, " + + "not trying to recover:", + key, ioe); } + this.close(); + } - protected void throwIfClosed() throws IOException { - if (isClosed()) { - throw new IOException(key + ": " + FSExceptionMessages.STREAM_IS_CLOSED); - } + protected void throwIfClosed() throws IOException { + if (isClosed()) { + throw new IOException(key + ": " + FSExceptionMessages.STREAM_IS_CLOSED); } + } - protected boolean isClosed() { - return inputStream == null; - } + protected boolean isClosed() { + return inputStream == null; + } } \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java index 080efc3c296a6..b7352617a8060 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java @@ -39,80 +39,82 @@ public class ITestS3AS3SeekableStream extends AbstractS3ATestBase { - final String PHYSICAL_IO_PREFIX = "physicalio"; - final String LOGICAL_IO_PREFIX = "logicalio"; + final String PHYSICAL_IO_PREFIX = "physicalio"; + final String LOGICAL_IO_PREFIX = "logicalio"; - public void testConnectorFrameWorkIntegration(boolean useCrtClient) throws IOException { - describe("Verify S3 connector framework integration"); + public void testConnectorFrameWorkIntegration(boolean useCrtClient) throws IOException { + describe("Verify S3 connector framework integration"); - Configuration conf = getConfiguration(); - removeBaseAndBucketOverrides(conf, ANALYTICS_ACCELERATOR_ENABLED_KEY); - conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, true); - conf.setBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, useCrtClient); + Configuration conf = getConfiguration(); + removeBaseAndBucketOverrides(conf, ANALYTICS_ACCELERATOR_ENABLED_KEY); + conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, true); + conf.setBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, useCrtClient); - String testFile = "s3a://noaa-cors-pds/raw/2023/017/ohfh/OHFH017d.23_.gz"; - S3AFileSystem s3AFileSystem = (S3AFileSystem) FileSystem.newInstance(new Path(testFile).toUri(), conf); - byte[] buffer = new byte[500]; - - try (FSDataInputStream inputStream = s3AFileSystem.open(new Path(testFile))) { - inputStream.seek(5); - inputStream.read(buffer, 0, 500); - } + String testFile = "s3a://noaa-cors-pds/raw/2023/017/ohfh/OHFH017d.23_.gz"; + S3AFileSystem s3AFileSystem = (S3AFileSystem) FileSystem.newInstance(new Path(testFile).toUri(), conf); + byte[] buffer = new byte[500]; + try (FSDataInputStream inputStream = s3AFileSystem.open(new Path(testFile))) { + inputStream.seek(5); + inputStream.read(buffer, 0, 500); } - @Test - public void testConnectorFrameWorkIntegrationWithCrtClient() throws IOException { - testConnectorFrameWorkIntegration(true); - } + } - @Test - public void testConnectorFrameWorkIntegrationWithoutCrtClient() throws IOException { - testConnectorFrameWorkIntegration(false); - } + @Test + public void testConnectorFrameWorkIntegrationWithCrtClient() throws IOException { + testConnectorFrameWorkIntegration(true); + } - public void testConnectorFrameworkConfigurable(boolean useCrtClient) { - describe("Verify S3 connector framework reads configuration"); + @Test + public void testConnectorFrameWorkIntegrationWithoutCrtClient() throws IOException { + testConnectorFrameWorkIntegration(false); + } - Configuration conf = getConfiguration(); - removeBaseAndBucketOverrides(conf); + public void testConnectorFrameworkConfigurable(boolean useCrtClient) { + describe("Verify S3 connector framework reads configuration"); - //Disable Predictive Prefetching - conf.set(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + LOGICAL_IO_PREFIX + ".prefetching.mode", "all"); + Configuration conf = getConfiguration(); + removeBaseAndBucketOverrides(conf); - //Set Blobstore Capacity - conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", 1); + //Disable Predictive Prefetching + conf.set(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + LOGICAL_IO_PREFIX + ".prefetching.mode", "all"); - conf.setBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, useCrtClient); + //Set Blobstore Capacity + conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", 1); - ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); + conf.setBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, useCrtClient); - S3SeekableInputStreamConfiguration configuration = S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration); - assertEquals(configuration.getLogicalIOConfiguration().getPrefetchingMode(), PrefetchMode.ALL); - assert configuration.getPhysicalIOConfiguration().getBlobStoreCapacity() == 1; - } + ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); - @Test - public void testConnectorFrameworkConfigurableWithoutCrtClient() throws IOException { - testConnectorFrameworkConfigurable(false); - } + S3SeekableInputStreamConfiguration configuration = S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration); - @Test - public void testConnectorFrameworkConfigurableWithCrtClient() throws IOException { - testConnectorFrameworkConfigurable(true); - } + assertSame("S3ASeekableStream configuration is not set to expected value", PrefetchMode.ALL, configuration.getLogicalIOConfiguration().getPrefetchingMode()); - @Test - public void testInvalidConfigurationThrows() { - describe("Verify S3 connector framework throws with invalid configuration"); + assertEquals("S3ASeekableStream configuration is not set to expected value", 1, configuration.getPhysicalIOConfiguration().getBlobStoreCapacity()); + } - Configuration conf = getConfiguration(); - removeBaseAndBucketOverrides(conf); - //Disable Sequential Prefetching - conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", -1); + @Test + public void testConnectorFrameworkConfigurableWithoutCrtClient() throws IOException { + testConnectorFrameworkConfigurable(false); + } - ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); - assertThrows(IllegalArgumentException.class, () -> - S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration)); - } + @Test + public void testConnectorFrameworkConfigurableWithCrtClient() throws IOException { + testConnectorFrameworkConfigurable(true); + } + + @Test + public void testInvalidConfigurationThrows() { + describe("Verify S3 connector framework throws with invalid configuration"); + + Configuration conf = getConfiguration(); + removeBaseAndBucketOverrides(conf); + //Disable Sequential Prefetching + conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", -1); + + ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); + assertThrows("S3ASeekableStream illegal configuration does not throw", IllegalArgumentException.class, () -> + S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration)); + } } From db015244559698b62c3625465cfbe29f5b60aba8 Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Tue, 17 Dec 2024 17:21:23 +0000 Subject: [PATCH 18/20] Address review feedback v2 --- hadoop-tools/hadoop-aws/pom.xml | 2 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 14 ++++--- .../hadoop/fs/s3a/S3ASeekableStream.java | 14 +++---- .../contract/s3a/ITestS3AContractCreate.java | 4 +- .../contract/s3a/ITestS3AContractDistCp.java | 14 ++++--- .../contract/s3a/ITestS3AContractRename.java | 3 +- .../s3a/ITestS3AContractVectoredRead.java | 5 ++- .../hadoop/fs/s3a/ITestS3ADelayedFNF.java | 3 ++ .../hadoop/fs/s3a/ITestS3AEncryptionSSEC.java | 3 +- .../fs/s3a/ITestS3AFSMainOperations.java | 12 +++--- .../fs/s3a/ITestS3AFileSystemContract.java | 4 +- .../fs/s3a/ITestS3AIOStatisticsContext.java | 4 +- .../fs/s3a/ITestS3AInputStreamLeakage.java | 4 +- .../apache/hadoop/fs/s3a/ITestS3AMetrics.java | 4 +- .../fs/s3a/ITestS3APrefetchingCacheFiles.java | 9 ++--- .../s3a/ITestS3APrefetchingInputStream.java | 11 ++---- .../s3a/ITestS3APrefetchingLruEviction.java | 13 ++----- .../fs/s3a/ITestS3AS3SeekableStream.java | 38 ++++++++++++------- .../apache/hadoop/fs/s3a/S3ATestUtils.java | 7 ++-- .../s3a/commit/ITestCommitOperationCost.java | 8 ++-- .../s3a/commit/ITestS3ACommitterFactory.java | 3 +- .../magic/ITestMagicCommitProtocol.java | 3 +- .../ITestDirectoryCommitProtocol.java | 3 +- .../ITestPartitionedCommitProtocol.java | 3 +- .../ITestStagingCommitProtocol.java | 3 +- .../ITestStagingCommitProtocolFailure.java | 3 +- .../ITestS3AFileContextStatistics.java | 3 +- .../fs/s3a/impl/ITestConnectionTimeouts.java | 10 +++-- .../fs/s3a/performance/ITestS3AOpenCost.java | 3 +- .../performance/ITestUnbufferDraining.java | 4 +- .../ITestS3AContractStreamIOStatistics.java | 3 +- .../ITestS3AFileSystemStatistic.java | 6 +-- 32 files changed, 126 insertions(+), 97 deletions(-) diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 0fa9cf9b5a166..f047cde9cfbfc 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -528,7 +528,7 @@ software.amazon.s3.analyticsaccelerator analyticsaccelerator-s3 - 0.0.1 + 0.0.2 compile 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 01b06e894b5ae..3e030b9c60b97 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 @@ -326,7 +326,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, /** * CRT-Based S3Client created of analytics accelerator library is enabled - * and managed by the ClientManager. Analytics accelerator library can be + * and managed by the S3AStoreImpl. Analytics accelerator library can be * enabled with {@link Constants#ANALYTICS_ACCELERATOR_ENABLED_KEY} */ private S3AsyncClient s3AsyncClient; @@ -545,7 +545,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private boolean s3AccessGrantsEnabled; /** - * Factory to create S3SeekableInputStream if {@link this#analyticsAcceleratorEnabled} is true + * Factory to create S3SeekableInputStream if {@link this#analyticsAcceleratorEnabled} is true. */ private S3SeekableInputStreamFactory s3SeekableInputStreamFactory; @@ -705,8 +705,10 @@ public void initialize(URI name, Configuration originalConf) this.prefetchBlockCount = intOption(conf, PREFETCH_BLOCK_COUNT_KEY, PREFETCH_BLOCK_DEFAULT_COUNT, 1); - this.analyticsAcceleratorEnabled = conf.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, ANALYTICS_ACCELERATOR_ENABLED_DEFAULT); - this.analyticsAcceleratorCRTEnabled = conf.getBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, ANALYTICS_ACCELERATOR_CRT_ENABLED_DEFAULT); + this.analyticsAcceleratorEnabled = + conf.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, ANALYTICS_ACCELERATOR_ENABLED_DEFAULT); + this.analyticsAcceleratorCRTEnabled = + conf.getBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, ANALYTICS_ACCELERATOR_CRT_ENABLED_DEFAULT); this.isMultipartUploadEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED, DEFAULT_MULTIPART_UPLOAD_ENABLED); @@ -847,10 +849,10 @@ public void initialize(URI name, Configuration originalConf) if (this.analyticsAcceleratorEnabled) { LOG.info("Using S3SeekableInputStream"); if(this.analyticsAcceleratorCRTEnabled) { - LOG.info("Using S3CrtClient"); + LOG.info("Using S3 CRT client for analytics accelerator S3"); this.s3AsyncClient = S3CrtAsyncClient.builder().maxConcurrency(600).build(); } else { - LOG.info("Using S3Client"); + LOG.info("Using S3 async client for analytics accelerator S3"); this.s3AsyncClient = store.getOrCreateAsyncClient(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java index 067306ff3c4a1..ef6a299081587 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ASeekableStream.java @@ -38,6 +38,7 @@ public class S3ASeekableStream extends FSInputStream implements StreamCapabiliti private S3SeekableInputStream inputStream; private long lastReadCurrentPos = 0; private final String key; + private volatile boolean closed; public static final Logger LOG = LoggerFactory.getLogger(S3ASeekableStream.class); @@ -84,7 +85,7 @@ public void seek(long pos) throws IOException { @Override public synchronized long getPos() { - if (!isClosed()) { + if (!closed) { lastReadCurrentPos = inputStream.getPos(); } return lastReadCurrentPos; @@ -139,8 +140,9 @@ public int available() throws IOException { } @Override - public void close() throws IOException { - if (inputStream != null) { + public synchronized void close() throws IOException { + if(!closed) { + closed = true; try { inputStream.close(); inputStream = null; @@ -174,12 +176,8 @@ private void onReadFailure(IOException ioe) throws IOException { protected void throwIfClosed() throws IOException { - if (isClosed()) { + if (closed) { throw new IOException(key + ": " + FSExceptionMessages.STREAM_IS_CLOSED); } } - - protected boolean isClosed() { - return inputStream == null; - } } \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java index e7a6bb6e4c3f5..fd569422b429e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractCreate.java @@ -89,7 +89,9 @@ protected Configuration createConfiguration() { @Override public void testOverwriteExistingFile() throws Throwable { - skipIfAnalyticsAcceleratorEnabled(this.createConfiguration()); + // Will remove this when Analytics Accelerator supports overwrites + skipIfAnalyticsAcceleratorEnabled(this.createConfiguration(), + "Analytics Accelerator does not support overwrites yet"); super.testOverwriteExistingFile(); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java index 806a6cd80027d..f6127700b5f1f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java @@ -52,12 +52,6 @@ protected Configuration createConfiguration() { return newConf; } - @Override - public void setup() throws Exception { - super.setup(); - skipIfAnalyticsAcceleratorEnabled(createConfiguration()); - } - @Override protected boolean shouldUseDirectWrite() { return true; @@ -85,6 +79,14 @@ public void testNonDirectWrite() throws Exception { getRenameOperationCount() - renames); } + @Override + public void testDistCpUpdateCheckFileSkip() throws Exception { + //Will remove this when Analytics Accelerator supports overwrites + skipIfAnalyticsAcceleratorEnabled(createConfiguration(), + "Analytics Accelerator Library does not support update to existing files"); + super.testDistCpUpdateCheckFileSkip(); + } + private long getRenameOperationCount() { return getFileSystem().getStorageStatistics() .getLong(StorageStatistics.CommonStatisticNames.OP_RENAME); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java index c67163c29013f..9f5246d9ddec5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java @@ -49,7 +49,8 @@ public class ITestS3AContractRename extends AbstractContractRenameTest { @Override public void setup() throws Exception { super.setup(); - skipIfAnalyticsAcceleratorEnabled(getContract().getConf()); + skipIfAnalyticsAcceleratorEnabled(createConfiguration(), + "Analytics Accelerator does not support rename"); } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java index c0c88802a8443..41aa90434cdd3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java @@ -86,12 +86,13 @@ protected AbstractFSContract createContract(Configuration conf) { } /** - * Analytics Accelerator Library for Amazon S3 does not support Vectored Reads + * Analytics Accelerator Library for Amazon S3 does not support Vectored Reads. * @throws Exception */ @Override public void setup() throws Exception { - skipIfAnalyticsAcceleratorEnabled(createConfiguration()); + skipIfAnalyticsAcceleratorEnabled(createConfiguration(), + "Analytics Accelerator does not support vectored reads"); super.setup(); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java index ca9d185c3e9e1..4793092b717dd 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADelayedFNF.java @@ -36,6 +36,7 @@ import static org.apache.hadoop.fs.s3a.Constants.RETRY_INTERVAL; import static org.apache.hadoop.fs.s3a.Constants.RETRY_LIMIT; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; /** * Tests behavior of a FileNotFound error that happens after open(), i.e. on @@ -65,6 +66,8 @@ protected Configuration createConfiguration() { */ @Test public void testNotFoundFirstRead() throws Exception { + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Temporarily disabling to fix Exception handling on Analytics Accelerator"); S3AFileSystem fs = getFileSystem(); ChangeDetectionPolicy changeDetectionPolicy = fs.getChangeDetectionPolicy(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java index d4ce466f8444f..12e5ef3841a64 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java @@ -91,7 +91,8 @@ protected Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); - skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Analytics Accelerator does not support SSEC"); assumeEnabled(); // although not a root dir test, this confuses paths enough it shouldn't be run in // parallel with other jobs diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFSMainOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFSMainOperations.java index a7e30c5949529..1d3806e75e329 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFSMainOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFSMainOperations.java @@ -78,17 +78,19 @@ public void testCopyToLocalWithUseRawLocalFileSystemOption() @Override public void testWriteReadAndDeleteOneAndAHalfBlocks() throws Exception { - //Skipping this test for AnalyticsAccelerator as it is acting as an overwrite test - skipIfAnalyticsAcceleratorEnabled(this.contract.getConf()); + // Skipping this test for AnalyticsAccelerator as it is acting as an overwrite test + skipIfAnalyticsAcceleratorEnabled(this.contract.getConf(), + "Analytics Accelerator does not support overwrites"); } @Override public void testWriteReadAndDeleteTwoBlocks() throws Exception { - //Skipping this test for AnalyticsAccelerator as it is acting as an overwrite test - skipIfAnalyticsAcceleratorEnabled(this.contract.getConf()); + // Skipping this test for AnalyticsAccelerator as it is acting as an overwrite test + skipIfAnalyticsAcceleratorEnabled(this.contract.getConf(), + "Analytics Accelerator does not support overwrites"); } - @Override + @Override public void testOverwrite() throws IOException { boolean createPerformance = isCreatePerformanceEnabled(fSys); try { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java index f4b52f5057f34..32d9a511a4159 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileSystemContract.java @@ -162,7 +162,9 @@ public void testOverwrite() throws IOException { @Override public void testOverWriteAndRead() throws Exception { - skipIfAnalyticsAcceleratorEnabled(fs.getConf()); + //Will remove this when Analytics Accelerator supports overwrites + skipIfAnalyticsAcceleratorEnabled(fs.getConf(), + "Analytics Accelerator does not support overwrites"); super.testOverWriteAndRead(); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AIOStatisticsContext.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AIOStatisticsContext.java index d94140f19e314..005b2fbf91d41 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AIOStatisticsContext.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AIOStatisticsContext.java @@ -78,7 +78,9 @@ protected Configuration createConfiguration() { public void setup() throws Exception { super.setup(); executor = HadoopExecutors.newFixedThreadPool(SMALL_THREADS); - skipIfAnalyticsAcceleratorEnabled(getContract().getConf()); + // TODO: Add IOStatistics Support to S3SeekableStream + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3SeekableStream does not support IOStatisticsContext"); } 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 d944bdb1b683e..4d8956d38e7be 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 @@ -90,7 +90,9 @@ public void setup() throws Exception { @Test public void testFinalizer() throws Throwable { Path path = methodPath(); - skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + // TODO: Add Leak Detection to S3SeekableStream + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3SeekableStream does not support leak detection"); final S3AFileSystem fs = getFileSystem(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java index 420f49bd097ba..61d159996357f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java @@ -52,7 +52,9 @@ public void testMetricsRegister() @Test public void testStreamStatistics() throws IOException { - skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + // TODO: Add StreamStatistics support to S3SeekableStream + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3SeekableStream does not support stream statistics"); S3AFileSystem fs = getFileSystem(); Path file = path("testStreamStatistics"); 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 9ecf2203c37f2..0b366aff146ed 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java @@ -37,11 +37,7 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; -import static org.apache.hadoop.fs.s3a.Constants.BUFFER_DIR; -import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT; -import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY; -import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; +import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.getExternalData; import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.isUsingDefaultExternalDataFile; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; @@ -78,7 +74,6 @@ public void setUp() throws Exception { super.setup(); // Sets BUFFER_DIR by calling S3ATestUtils#prepareTestConfiguration conf = createConfiguration(); - skipIfAnalyticsAcceleratorEnabled(conf); testFile = getExternalData(conf); prefetchBlockSize = conf.getInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); fs = FileSystem.get(testFile.toUri(), conf); @@ -99,6 +94,8 @@ public Configuration createConfiguration() { final String bufferDirBase = configuration.get(BUFFER_DIR); bufferDir = bufferDirBase + "/" + UUID.randomUUID(); configuration.set(BUFFER_DIR, bufferDir); + // When both Prefetching and Analytics Accelerator enabled Analytics Accelerator is used + conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, false); return configuration; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java index b9aed3d82c41c..b97be9d18e914 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java @@ -34,9 +34,7 @@ import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.test.LambdaTestUtils; -import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY; -import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; +import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMaximum; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue; @@ -76,11 +74,6 @@ public ITestS3APrefetchingInputStream() { private static final int INTERVAL_MILLIS = 500; private static final int BLOCK_SIZE = S_1K * 10; - @Override - public void setup() throws Exception { - super.setup(); - skipIfAnalyticsAcceleratorEnabled(this.createConfiguration()); - } @Override public Configuration createConfiguration() { @@ -89,6 +82,8 @@ public Configuration createConfiguration() { S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_BLOCK_SIZE_KEY); conf.setBoolean(PREFETCH_ENABLED_KEY, true); conf.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); + // When both Prefetching and Analytics Accelerator enabled Analytics Accelerator is used + conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, false); return conf; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java index 8417b05eb6b75..8aefcc8535af6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java @@ -42,10 +42,7 @@ import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.test.LambdaTestUtils; -import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY; -import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; -import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_MAX_BLOCKS_COUNT; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled; +import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValues; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue; @@ -69,12 +66,6 @@ public static Collection params() { }); } - @Override - public void setup() throws Exception { - super.setup(); - skipIfAnalyticsAcceleratorEnabled(createConfiguration()); - } - public ITestS3APrefetchingLruEviction(final String maxBlocks) { super(true); this.maxBlocks = maxBlocks; @@ -100,6 +91,8 @@ public Configuration createConfiguration() { conf.setBoolean(PREFETCH_ENABLED_KEY, true); conf.setInt(PREFETCH_MAX_BLOCKS_COUNT, Integer.parseInt(maxBlocks)); conf.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); + // When both Prefetching and Analytics Accelerator enabled Analytics Accelerator is used + conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, false); return conf; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java index b7352617a8060..c6ecee9505101 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AS3SeekableStream.java @@ -39,8 +39,8 @@ public class ITestS3AS3SeekableStream extends AbstractS3ATestBase { - final String PHYSICAL_IO_PREFIX = "physicalio"; - final String LOGICAL_IO_PREFIX = "logicalio"; + private static final String PHYSICAL_IO_PREFIX = "physicalio"; + private static final String LOGICAL_IO_PREFIX = "logicalio"; public void testConnectorFrameWorkIntegration(boolean useCrtClient) throws IOException { describe("Verify S3 connector framework integration"); @@ -51,7 +51,8 @@ public void testConnectorFrameWorkIntegration(boolean useCrtClient) throws IOExc conf.setBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, useCrtClient); String testFile = "s3a://noaa-cors-pds/raw/2023/017/ohfh/OHFH017d.23_.gz"; - S3AFileSystem s3AFileSystem = (S3AFileSystem) FileSystem.newInstance(new Path(testFile).toUri(), conf); + S3AFileSystem s3AFileSystem = + (S3AFileSystem) FileSystem.newInstance(new Path(testFile).toUri(), conf); byte[] buffer = new byte[500]; try (FSDataInputStream inputStream = s3AFileSystem.open(new Path(testFile))) { @@ -78,20 +79,26 @@ public void testConnectorFrameworkConfigurable(boolean useCrtClient) { removeBaseAndBucketOverrides(conf); //Disable Predictive Prefetching - conf.set(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + LOGICAL_IO_PREFIX + ".prefetching.mode", "all"); + conf.set(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + + "." + LOGICAL_IO_PREFIX + ".prefetching.mode", "all"); //Set Blobstore Capacity - conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", 1); + conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", 1); conf.setBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, useCrtClient); - ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); + ConnectorConfiguration connectorConfiguration = + new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); - S3SeekableInputStreamConfiguration configuration = S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration); + S3SeekableInputStreamConfiguration configuration = + S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration); - assertSame("S3ASeekableStream configuration is not set to expected value", PrefetchMode.ALL, configuration.getLogicalIOConfiguration().getPrefetchingMode()); + assertSame("S3ASeekableStream configuration is not set to expected value", + PrefetchMode.ALL, configuration.getLogicalIOConfiguration().getPrefetchingMode()); - assertEquals("S3ASeekableStream configuration is not set to expected value", 1, configuration.getPhysicalIOConfiguration().getBlobStoreCapacity()); + assertEquals("S3ASeekableStream configuration is not set to expected value", + 1, configuration.getPhysicalIOConfiguration().getBlobStoreCapacity()); } @Test @@ -111,10 +118,13 @@ public void testInvalidConfigurationThrows() { Configuration conf = getConfiguration(); removeBaseAndBucketOverrides(conf); //Disable Sequential Prefetching - conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", -1); - - ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); - assertThrows("S3ASeekableStream illegal configuration does not throw", IllegalArgumentException.class, () -> - S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration)); + conf.setInt(ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX + + "." + PHYSICAL_IO_PREFIX + ".blobstore.capacity", -1); + + ConnectorConfiguration connectorConfiguration = + new ConnectorConfiguration(conf, ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX); + assertThrows("S3ASeekableStream illegal configuration does not throw", + IllegalArgumentException.class, () -> + S3SeekableInputStreamConfiguration.fromConfiguration(connectorConfiguration)); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index 089cac79c12e4..e9ebd98169fa4 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 @@ -579,13 +579,14 @@ public static boolean isS3ExpressTestBucket(final Configuration conf) { * @param configuration configuration to probe */ public static void skipIfAnalyticsAcceleratorEnabled( - Configuration configuration) { - assume("Skipping test as Analytics Accelerator Library does not support this functionality", + Configuration configuration, String message) { + assume(message, !isAnalyticsAcceleratorEnabled(configuration)); } public static boolean isAnalyticsAcceleratorEnabled(final Configuration conf) { - return conf.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, ANALYTICS_ACCELERATOR_ENABLED_DEFAULT); + return conf.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, + ANALYTICS_ACCELERATOR_ENABLED_DEFAULT); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java index 53a8871a73580..9041a20aeeb6c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java @@ -205,8 +205,8 @@ private void abortActiveStream() throws IOException { public void testCostOfCreatingMagicFile() throws Throwable { describe("Files created under magic paths skip existence checks and marker deletes"); - // Assertions will fail as {@link S3ASeekableInputStream} do not support InputStreamStatistics yes - skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3ASeekableInputStream does not support InputStreamStatistics"); S3AFileSystem fs = getFileSystem(); Path destFile = methodSubPath("file.txt"); fs.delete(destFile.getParent(), true); @@ -286,8 +286,8 @@ public void testCostOfCreatingMagicFile() throws Throwable { public void testCostOfSavingLoadingPendingFile() throws Throwable { describe("Verify costs of saving .pending file under a magic path"); - // Assertions will fail as {@link S3ASeekableInputStream} do not support InputStreamStatistics yes - skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3ASeekableInputStream does not support InputStreamStatistics"); S3AFileSystem fs = getFileSystem(); Path partDir = methodSubPath("file.pending"); Path destFile = new Path(partDir, "file.pending"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java index a224798eb5d99..a500bfb76a322 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java @@ -183,7 +183,8 @@ public void setup() throws Exception { // destroy all filesystems from previous runs. FileSystem.closeAllForUGI(UserGroupInformation.getCurrentUser()); super.setup(); - skipIfAnalyticsAcceleratorEnabled(createConfiguration()); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3ASeekableInputStream does not support InputStreamStatistics"); jobId = randomJobId(); attempt0 = "attempt_" + jobId + "_m_000000_0"; taskAttempt0 = TaskAttemptID.forName(attempt0); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java index 8f04d46a47901..b89740ae3120c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java @@ -78,7 +78,8 @@ protected String getCommitterName() { @Override public void setup() throws Exception { super.setup(); - skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3ASeekableInputStream does not support InputStreamStatistics"); CommitUtils.verifyIsMagicCommitFS(getFileSystem()); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitProtocol.java index 6e5ae77aa3761..cbd41fd1c93e6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestDirectoryCommitProtocol.java @@ -47,7 +47,8 @@ public class ITestDirectoryCommitProtocol extends ITestStagingCommitProtocol { @Override public void setup() throws Exception { super.setup(); - skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Skipping because these tests will fail when using CRT client"); } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionedCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionedCommitProtocol.java index 9d7bb6c41f551..b6438157e4e36 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionedCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestPartitionedCommitProtocol.java @@ -41,7 +41,8 @@ public class ITestPartitionedCommitProtocol extends ITestStagingCommitProtocol { @Override public void setup() throws Exception { super.setup(); - skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Skipping because these tests will fail when using CRT client"); } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java index 513df74bcbdff..d2b4e1c52956b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java @@ -66,7 +66,8 @@ protected Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); - skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Skipping because these tests will fail when using CRT client"); // identify working dir for staging and delete diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java index fcbf9f1d8902b..5604aa7636968 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java @@ -47,7 +47,8 @@ public class ITestStagingCommitProtocolFailure extends AbstractS3ATestBase { @Override public void setup() throws Exception { super.setup(); - skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Skipping because these tests will fail when using CRT client"); } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java index 3fda6f24386c5..dc086f7c4237f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java @@ -46,7 +46,8 @@ public class ITestS3AFileContextStatistics extends FCStatisticsBaseTest { @Before public void setUp() throws Exception { conf = new Configuration(); - skipIfAnalyticsAcceleratorEnabled(conf); + skipIfAnalyticsAcceleratorEnabled(conf, + "S3SeekableStream does not support File Context Statistics"); fc = S3ATestUtils.createTestFileContext(conf); testRootPath = fileContextTestHelper.getTestRootPath(fc, "test"); fc.mkdir(testRootPath, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestConnectionTimeouts.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestConnectionTimeouts.java index d9071ef16c2d3..2a6605b93421f 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 @@ -148,8 +148,9 @@ public void teardown() throws Exception { public void testGeneratePoolTimeouts() throws Throwable { skipIfClientSideEncryption(); - // Assertions will fail when using CRTClient with SeekableStream. - skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + // Assertions will fail when using CRTClient with Analytics Accelerator. + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Assertions will fail when using CRTClient with Analytics Accelerator"); AWSClientConfig.setMinimumOperationDuration(Duration.ZERO); Configuration conf = timingOutConfiguration(); @@ -194,8 +195,9 @@ public void testGeneratePoolTimeouts() throws Throwable { public void testObjectUploadTimeouts() throws Throwable { skipIfClientSideEncryption(); - // Assertions will fail when using CRTClient with SeekableStream. - skipIfAnalyticsAcceleratorEnabled(getConfiguration()); + // Assertions will fail when using CRTClient with Analytics Accelerator. + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Assertions will fail when using CRTClient with Analytics Accelerator"); AWSClientConfig.setMinimumOperationDuration(Duration.ZERO); final Path dir = methodPath(); 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 72e26540d38ec..c07c5dd5086c0 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 @@ -111,7 +111,8 @@ public Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); - skipIfAnalyticsAcceleratorEnabled(createConfiguration()); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Assertions will fail as S3SeekableStream does not support Stream Statistics"); S3AFileSystem fs = getFileSystem(); testFile = methodPath(); 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 cd62e23f14db7..a0dec8d476bda 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 @@ -126,7 +126,9 @@ public Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); - skipIfAnalyticsAcceleratorEnabled(createConfiguration()); + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "Skipping because getS3AInputStream will " + + "try to cast S3SeekableStream to S3AInputStream"); // now create a new FS with minimal http capacity and recovery // a separate one is used to avoid test teardown suffering diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java index 0e73e2d74ae5b..2b332b2b3ee24 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AContractStreamIOStatistics.java @@ -82,7 +82,8 @@ public List outputStreamStatisticKeys() { @Override public void setup() throws Exception { super.setup(); - skipIfAnalyticsAcceleratorEnabled(getContract().getConf()); + skipIfAnalyticsAcceleratorEnabled(createConfiguration(), + "S3SeekableStream does not support Stream Statistics"); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java index 227d511522990..68821001bff83 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java @@ -44,9 +44,9 @@ public class ITestS3AFileSystemStatistic extends AbstractS3ATestBase { */ @Test public void testBytesReadWithStream() throws IOException { - // Assertions will fail as {@link S3ASeekableInputStream} do not support InputStreamStatistics yes - skipIfAnalyticsAcceleratorEnabled(getConfiguration()); - + // Assertions will fail as {@link S3ASeekableInputStream} do not support S3AFileSystemStatistics yes + skipIfAnalyticsAcceleratorEnabled(getConfiguration(), + "S3SeekableStream does not support File System Statistics"); S3AFileSystem fs = getFileSystem(); Path filePath = path(getMethodName()); byte[] oneKbBuf = new byte[ONE_KB]; From 848ea407d8982a7c816ab18f4174984611cf17e3 Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Tue, 17 Dec 2024 17:47:39 +0000 Subject: [PATCH 19/20] fix typo --- .../org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 0b366aff146ed..5e37e68d8bea7 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 @@ -95,7 +95,7 @@ public Configuration createConfiguration() { bufferDir = bufferDirBase + "/" + UUID.randomUUID(); configuration.set(BUFFER_DIR, bufferDir); // When both Prefetching and Analytics Accelerator enabled Analytics Accelerator is used - conf.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, false); + configuration.setBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, false); return configuration; } From b040612bd120bd88ba21fe46969db51c940301e3 Mon Sep 17 00:00:00 2001 From: Fuat Basik Date: Wed, 18 Dec 2024 08:48:08 +0000 Subject: [PATCH 20/20] fix checkstyle --- .../src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java | 3 ++- .../hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 3e030b9c60b97..e85cae3942b84 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 @@ -708,7 +708,8 @@ public void initialize(URI name, Configuration originalConf) this.analyticsAcceleratorEnabled = conf.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, ANALYTICS_ACCELERATOR_ENABLED_DEFAULT); this.analyticsAcceleratorCRTEnabled = - conf.getBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, ANALYTICS_ACCELERATOR_CRT_ENABLED_DEFAULT); + conf.getBoolean(ANALYTICS_ACCELERATOR_CRT_ENABLED, + ANALYTICS_ACCELERATOR_CRT_ENABLED_DEFAULT); this.isMultipartUploadEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED, DEFAULT_MULTIPART_UPLOAD_ENABLED); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java index 68821001bff83..5a3f7bb8fdbb2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestS3AFileSystemStatistic.java @@ -44,7 +44,8 @@ public class ITestS3AFileSystemStatistic extends AbstractS3ATestBase { */ @Test public void testBytesReadWithStream() throws IOException { - // Assertions will fail as {@link S3ASeekableInputStream} do not support S3AFileSystemStatistics yes + // Assertions will fail as {@link S3ASeekableInputStream} + // do not support S3AFileSystemStatistics yet. skipIfAnalyticsAcceleratorEnabled(getConfiguration(), "S3SeekableStream does not support File System Statistics"); S3AFileSystem fs = getFileSystem();