From 4abf17d729a179b9fd787378de493daff05a3707 Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Fri, 4 Feb 2022 09:11:12 +0000 Subject: [PATCH 01/17] HADOOP-14661. Add S3 requester pays bucket support to S3A --- .../org/apache/hadoop/fs/s3a/Constants.java | 4 + .../hadoop/fs/s3a/DefaultS3ClientFactory.java | 6 ++ .../apache/hadoop/fs/s3a/S3AFileSystem.java | 1 + .../apache/hadoop/fs/s3a/S3ClientFactory.java | 4 +- .../site/markdown/tools/hadoop-aws/index.md | 15 +++ .../tools/hadoop-aws/troubleshooting_s3a.md | 14 +++ .../hadoop/fs/s3a/ITestS3ARequesterPays.java | 100 ++++++++++++++++++ .../hadoop/fs/s3a/S3ATestConstants.java | 11 ++ 8 files changed, 153 insertions(+), 2 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index dd7e425880962..2dc9f62a80d98 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 @@ -157,6 +157,10 @@ private Constants() { "fs.s3a.connection.ssl.enabled"; public static final boolean DEFAULT_SECURE_CONNECTIONS = true; + // allow access to requester pay buckets + public static final String ALLOW_REQUESTER_PAYS = "fs.s3a.requester-pays.enabled"; + public static final boolean DEFAULT_ALLOW_REQUESTER_PAYS = false; + // use OpenSSL or JSEE for secure connections public static final String SSL_CHANNEL_MODE = "fs.s3a.ssl.channel.mode"; public static final DelegatingSSLSocketFactory.SSLChannelMode diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index c14558adf54b3..e3737c05f0818 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java @@ -32,6 +32,7 @@ import com.amazonaws.services.s3.AmazonS3ClientBuilder; import com.amazonaws.services.s3.AmazonS3EncryptionClientV2Builder; import com.amazonaws.services.s3.AmazonS3EncryptionV2; +import com.amazonaws.services.s3.Headers; import com.amazonaws.services.s3.S3ClientOptions; import com.amazonaws.services.s3.internal.ServiceUtils; import com.amazonaws.services.s3.model.CryptoConfigurationV2; @@ -118,6 +119,11 @@ public AmazonS3 createS3Client( parameters.getHeaders().forEach((h, v) -> awsConf.addHeader(h, v)); + if (parameters.isRequesterPays()) { + // All calls must acknowledge requester will pay via header. + awsConf.addHeader(Headers.REQUESTER_PAYS_HEADER, "requester"); + } + // When EXPERIMENTAL_AWS_INTERNAL_THROTTLING is false // throttling is explicitly disabled on the S3 client so that // all failures are collected in S3A instrumentation, and its 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 c8a73d956d844..3ccdecde832f7 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 @@ -837,6 +837,7 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException { .withMetrics(statisticsContext.newStatisticsFromAwsSdk()) .withPathStyleAccess(conf.getBoolean(PATH_STYLE_ACCESS, false)) .withUserAgentSuffix(uaSuffix) + .withRequesterPays(conf.getBoolean(ALLOW_REQUESTER_PAYS, DEFAULT_ALLOW_REQUESTER_PAYS)) .withRequestHandlers(auditManager.createRequestHandlers()); s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java index 5ef99ed6f5c3c..34674c788901f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java @@ -101,7 +101,7 @@ final class S3ClientCreationParameters { private boolean pathStyleAccess; /** - * This is in the settings awaiting wiring up and testing. + * Permit requests to requester pays buckets. */ private boolean requesterPays; @@ -168,7 +168,7 @@ public S3ClientCreationParameters withMetrics( } /** - * Requester pays option. Not yet wired up. + * Set requester pays option. * @param value new value * @return the builder */ diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index f390f1d5f8d57..df8436e03ec67 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -1630,6 +1630,21 @@ Before using Access Points make sure you're not impacted by the following: considering endpoints, if you have any custom signers that use the host endpoint property make sure to update them if needed; +## Requester Pays buckets + +S3A supports buckets with [Requester Pays](https://docs.aws.amazon.com/AmazonS3/latest/userguide/RequesterPaysBuckets.html) enabled. When a bucket is configured with requester pays, the requester must cover the per-request cost. + +For requests to be successful, the S3 client must acknowledge that they will pay for these requests by setting a request flag, usually a header, on each request. + +To enable this feature within S3A, configure the `fs.s3a.requester-pays.enabled` property. + +```xml + + fs.s3a.requester-pays.enabled + true + +``` + ## How S3A writes data to S3 The original S3A client implemented file writes by diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md index 3019b8525dbfb..c6f437cef9497 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md @@ -547,6 +547,20 @@ When trying to write or read SEE-KMS-encrypted data, the client gets a The caller does not have the permissions to access the key with which the data was encrypted. +### `AccessDeniedException` when using a "Requester Pays" enabled bucket + +When making cross-account requests to a requester pays enabled bucket, all calls must acknowledge via a header that the requester will be billed. + +If you don't enable this acknowledgement within S3A, then you will see a message similar to this: + +``` +java.nio.file.AccessDeniedException: s3a://my-bucket/my-object: getFileStatus on s3a://my-bucket/my-object: +com.amazonaws.services.s3.model.AmazonS3Exception: Forbidden (Service: Amazon S3; Status Code: 403; +Error Code: 403 Forbidden; Request ID: myshortreqid; S3 Extended Request ID: mylongreqid):403 Forbidden +``` + +To enable requester pays, set `fs.s3a.requester-pays.enabled` property to `true`. + ### "Unable to find a region via the region provider chain." when using session credentials. Region must be provided when requesting session credentials, or an exception will be thrown with the message: diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java new file mode 100644 index 0000000000000..cceb273dbd94e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -0,0 +1,100 @@ +/* + * 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 org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.contract.s3a.S3AContract; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; +import org.junit.Test; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; + +import java.nio.file.AccessDeniedException; + +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Tests for Requester Pays feature. + */ +public class ITestS3ARequesterPays extends AbstractS3ATestBase { + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + S3ATestUtils.disableFilesystemCaching(conf); + return conf; + } + + @Test + public void testRequesterPaysOptionSuccess() throws Throwable { + describe("Test requester pays enabled case by reading last then first byte"); + + Configuration conf = this.createConfiguration(); + conf.setBoolean(Constants.ALLOW_REQUESTER_PAYS, true); + S3AContract contract = (S3AContract) createContract(conf); + contract.init(); + + Path requesterPaysPath = getRequesterPaysPath(conf); + FileSystem fs = contract.getFileSystem(requesterPaysPath.toUri()); + + long fileLength = fs.getFileStatus(requesterPaysPath).getLen(); + FSDataInputStream inputStream = fs.open(requesterPaysPath); + + inputStream.seek(fileLength - 1); + inputStream.readByte(); + + // Jump back to the start, triggering a new GetObject request. + inputStream.seek(0); + inputStream.readByte(); + + assertEquals( + "Expected two GetObject requests", // We want to test header is applied to all requests made + 2, + inputStream.getIOStatistics().counters().get(StreamStatisticNames.STREAM_READ_OPENED).intValue() + ); + } + + @Test + public void testRequesterPaysDisabledFails() throws Throwable { + describe("Verify expected negative behavior for requester pays buckets when client has it disabled"); + + Configuration conf = this.createConfiguration(); + conf.setBoolean(Constants.ALLOW_REQUESTER_PAYS, false); + S3AContract contract = (S3AContract) createContract(conf); + contract.init(); + + Path requesterPaysPath = getRequesterPaysPath(conf); + FileSystem fs = contract.getFileSystem(requesterPaysPath.toUri()); + + intercept( + AccessDeniedException.class, + "403 Forbidden", + "Expected requester pays bucket to fail without header set", + () -> fs.open(requesterPaysPath) + ); + } + + private Path getRequesterPaysPath(Configuration conf) { + String requesterPaysFile = conf.getTrimmed(KEY_REQUESTER_PAYS_FILE, DEFAULT_REQUESTER_PAYS_FILE); + S3ATestUtils.assume("Empty test property: " + KEY_REQUESTER_PAYS_FILE, !requesterPaysFile.isEmpty()); + return new Path(requesterPaysFile); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java index aca622a9e20b3..74c778c0ca89d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java @@ -97,6 +97,17 @@ public interface S3ATestConstants { */ String DEFAULT_CSVTEST_FILE = LANDSAT_BUCKET + "scene_list.gz"; + /** + * Configuration key for an existing object in a requester pays bucket: {@value}. + * If not set, defaults to {@value DEFAULT_REQUESTER_PAYS_FILE}. + */ + String KEY_REQUESTER_PAYS_FILE = TEST_FS_S3A + "requester-pays.file"; + + /** + * Default path for an S3 object inside a requester pays enabled bucket: {@value}. + */ + String DEFAULT_REQUESTER_PAYS_FILE = "s3a://usgs-landsat/collection02/catalog.json"; + /** * Name of the property to define the timeout for scale tests: {@value}. * Measured in seconds. From a401c3632a4b9e61092c08ed617befaf54bf7359 Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Mon, 7 Feb 2022 17:11:10 +0000 Subject: [PATCH 02/17] HADOOP-14661. Fix checkstyle --- .../hadoop/fs/s3a/ITestS3ARequesterPays.java | 127 +++++++++--------- 1 file changed, 66 insertions(+), 61 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index cceb273dbd94e..59c0c1e8c911c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -35,66 +35,71 @@ */ public class ITestS3ARequesterPays extends AbstractS3ATestBase { - @Override - protected Configuration createConfiguration() { - Configuration conf = super.createConfiguration(); - S3ATestUtils.disableFilesystemCaching(conf); - return conf; - } - - @Test - public void testRequesterPaysOptionSuccess() throws Throwable { - describe("Test requester pays enabled case by reading last then first byte"); - - Configuration conf = this.createConfiguration(); - conf.setBoolean(Constants.ALLOW_REQUESTER_PAYS, true); - S3AContract contract = (S3AContract) createContract(conf); - contract.init(); - - Path requesterPaysPath = getRequesterPaysPath(conf); - FileSystem fs = contract.getFileSystem(requesterPaysPath.toUri()); - - long fileLength = fs.getFileStatus(requesterPaysPath).getLen(); - FSDataInputStream inputStream = fs.open(requesterPaysPath); - - inputStream.seek(fileLength - 1); - inputStream.readByte(); - - // Jump back to the start, triggering a new GetObject request. - inputStream.seek(0); - inputStream.readByte(); - - assertEquals( - "Expected two GetObject requests", // We want to test header is applied to all requests made - 2, - inputStream.getIOStatistics().counters().get(StreamStatisticNames.STREAM_READ_OPENED).intValue() - ); - } - - @Test - public void testRequesterPaysDisabledFails() throws Throwable { - describe("Verify expected negative behavior for requester pays buckets when client has it disabled"); - - Configuration conf = this.createConfiguration(); - conf.setBoolean(Constants.ALLOW_REQUESTER_PAYS, false); - S3AContract contract = (S3AContract) createContract(conf); - contract.init(); - - Path requesterPaysPath = getRequesterPaysPath(conf); - FileSystem fs = contract.getFileSystem(requesterPaysPath.toUri()); - - intercept( - AccessDeniedException.class, - "403 Forbidden", - "Expected requester pays bucket to fail without header set", - () -> fs.open(requesterPaysPath) - ); - } - - private Path getRequesterPaysPath(Configuration conf) { - String requesterPaysFile = conf.getTrimmed(KEY_REQUESTER_PAYS_FILE, DEFAULT_REQUESTER_PAYS_FILE); - S3ATestUtils.assume("Empty test property: " + KEY_REQUESTER_PAYS_FILE, !requesterPaysFile.isEmpty()); - return new Path(requesterPaysFile); - } + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + S3ATestUtils.disableFilesystemCaching(conf); + return conf; + } + + @Test + public void testRequesterPaysOptionSuccess() throws Throwable { + describe("Test requester pays enabled case by reading last then first byte"); + + Configuration conf = this.createConfiguration(); + conf.setBoolean(Constants.ALLOW_REQUESTER_PAYS, true); + S3AContract contract = (S3AContract) createContract(conf); + contract.init(); + + Path requesterPaysPath = getRequesterPaysPath(conf); + FileSystem fs = contract.getFileSystem(requesterPaysPath.toUri()); + + long fileLength = fs.getFileStatus(requesterPaysPath).getLen(); + FSDataInputStream inputStream = fs.open(requesterPaysPath); + + inputStream.seek(fileLength - 1); + inputStream.readByte(); + + // Jump back to the start, triggering a new GetObject request. + inputStream.seek(0); + inputStream.readByte(); + + assertEquals( + "Expected two GetObject requests", // We want to test header is applied to all requests made + 2, + inputStream.getIOStatistics().counters() + .get(StreamStatisticNames.STREAM_READ_OPENED).intValue() + ); + } + + @Test + public void testRequesterPaysDisabledFails() throws Throwable { + describe("Verify expected failure for requester pays buckets when client has it disabled"); + + Configuration conf = this.createConfiguration(); + conf.setBoolean(Constants.ALLOW_REQUESTER_PAYS, false); + S3AContract contract = (S3AContract) createContract(conf); + contract.init(); + + Path requesterPaysPath = getRequesterPaysPath(conf); + FileSystem fs = contract.getFileSystem(requesterPaysPath.toUri()); + + intercept( + AccessDeniedException.class, + "403 Forbidden", + "Expected requester pays bucket to fail without header set", + () -> fs.open(requesterPaysPath) + ); + } + + private Path getRequesterPaysPath(Configuration conf) { + String requesterPaysFile = + conf.getTrimmed(KEY_REQUESTER_PAYS_FILE, DEFAULT_REQUESTER_PAYS_FILE); + S3ATestUtils.assume( + "Empty test property: " + KEY_REQUESTER_PAYS_FILE, + !requesterPaysFile.isEmpty() + ); + return new Path(requesterPaysFile); + } } From 8d205b5814be39c26ef9d96efcf33ed63f8228fb Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Mon, 28 Feb 2022 14:02:57 +0000 Subject: [PATCH 03/17] Add close statements for FS and InputStream in Requester Pays S3A integ test --- .../hadoop/fs/s3a/ITestS3ARequesterPays.java | 59 +++++++++---------- 1 file changed, 28 insertions(+), 31 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index 59c0c1e8c911c..eb7f867b409cd 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -20,7 +20,6 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.contract.s3a.S3AContract; import org.apache.hadoop.fs.statistics.StreamStatisticNames; import org.junit.Test; import org.apache.hadoop.conf.Configuration; @@ -48,28 +47,28 @@ public void testRequesterPaysOptionSuccess() throws Throwable { Configuration conf = this.createConfiguration(); conf.setBoolean(Constants.ALLOW_REQUESTER_PAYS, true); - S3AContract contract = (S3AContract) createContract(conf); - contract.init(); - Path requesterPaysPath = getRequesterPaysPath(conf); - FileSystem fs = contract.getFileSystem(requesterPaysPath.toUri()); - - long fileLength = fs.getFileStatus(requesterPaysPath).getLen(); - FSDataInputStream inputStream = fs.open(requesterPaysPath); - - inputStream.seek(fileLength - 1); - inputStream.readByte(); - - // Jump back to the start, triggering a new GetObject request. - inputStream.seek(0); - inputStream.readByte(); - assertEquals( - "Expected two GetObject requests", // We want to test header is applied to all requests made - 2, - inputStream.getIOStatistics().counters() - .get(StreamStatisticNames.STREAM_READ_OPENED).intValue() - ); + try ( + FileSystem fs = requesterPaysPath.getFileSystem(conf); + FSDataInputStream inputStream = fs.open(requesterPaysPath); + ) { + long fileLength = fs.getFileStatus(requesterPaysPath).getLen(); + + inputStream.seek(fileLength - 1); + inputStream.readByte(); + + // Jump back to the start, triggering a new GetObject request. + inputStream.seek(0); + inputStream.readByte(); + + assertEquals( + "Expected two GetObject requests", + 2, + inputStream.getIOStatistics().counters() + .get(StreamStatisticNames.STREAM_READ_OPENED).intValue() + ); + } } @Test @@ -78,18 +77,16 @@ public void testRequesterPaysDisabledFails() throws Throwable { Configuration conf = this.createConfiguration(); conf.setBoolean(Constants.ALLOW_REQUESTER_PAYS, false); - S3AContract contract = (S3AContract) createContract(conf); - contract.init(); - Path requesterPaysPath = getRequesterPaysPath(conf); - FileSystem fs = contract.getFileSystem(requesterPaysPath.toUri()); - intercept( - AccessDeniedException.class, - "403 Forbidden", - "Expected requester pays bucket to fail without header set", - () -> fs.open(requesterPaysPath) - ); + try (FileSystem fs = requesterPaysPath.getFileSystem(conf)) { + intercept( + AccessDeniedException.class, + "403 Forbidden", + "Expected requester pays bucket to fail without header set", + () -> fs.open(requesterPaysPath).close() + ); + } } private Path getRequesterPaysPath(Configuration conf) { From 257fe5b96b971c32a3a060fb7419a6b1acf0dc0b Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Mon, 28 Feb 2022 14:11:43 +0000 Subject: [PATCH 04/17] Move requester pays header value to constant --- .../org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index e3737c05f0818..c374ef7397c97 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java @@ -32,7 +32,6 @@ import com.amazonaws.services.s3.AmazonS3ClientBuilder; import com.amazonaws.services.s3.AmazonS3EncryptionClientV2Builder; import com.amazonaws.services.s3.AmazonS3EncryptionV2; -import com.amazonaws.services.s3.Headers; import com.amazonaws.services.s3.S3ClientOptions; import com.amazonaws.services.s3.internal.ServiceUtils; import com.amazonaws.services.s3.model.CryptoConfigurationV2; @@ -55,6 +54,7 @@ import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector; import org.apache.hadoop.fs.store.LogExactlyOnce; +import static com.amazonaws.services.s3.Headers.REQUESTER_PAYS_HEADER; import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION; import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_CENTRAL_REGION; import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING; @@ -76,6 +76,8 @@ public class DefaultS3ClientFactory extends Configured private static final String S3_SERVICE_NAME = "s3"; + private static final String REQUESTER_PAYS_HEADER_VALUE = "requester"; + /** * Subclasses refer to this. */ @@ -121,7 +123,7 @@ public AmazonS3 createS3Client( if (parameters.isRequesterPays()) { // All calls must acknowledge requester will pay via header. - awsConf.addHeader(Headers.REQUESTER_PAYS_HEADER, "requester"); + awsConf.addHeader(REQUESTER_PAYS_HEADER, REQUESTER_PAYS_HEADER_VALUE); } // When EXPERIMENTAL_AWS_INTERNAL_THROTTLING is false From 4ed59598ee2032c87b1f96401d3a4e6e269c7eb7 Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Mon, 28 Feb 2022 14:22:29 +0000 Subject: [PATCH 05/17] Deprecate unused requester pays flag in S3A's RequestFactory --- .../fs/s3a/impl/RequestFactoryImpl.java | 22 ------------------- 1 file changed, 22 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java index 04cff49be3d80..8e8d9b6052af7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java @@ -106,13 +106,6 @@ public class RequestFactoryImpl implements RequestFactory { */ private final long multipartPartCountLimit; - /** - * Requester Pays. - * This is to be wired up in a PR with its - * own tests and docs. - */ - private final boolean requesterPays; - /** * Callback to prepare requests. */ @@ -133,7 +126,6 @@ protected RequestFactoryImpl( this.cannedACL = builder.cannedACL; this.encryptionSecrets = builder.encryptionSecrets; this.multipartPartCountLimit = builder.multipartPartCountLimit; - this.requesterPays = builder.requesterPays; this.requestPreparer = builder.requestPreparer; this.contentEncoding = builder.contentEncoding; } @@ -616,9 +608,6 @@ public static final class RequestFactoryBuilder { */ private CannedAccessControlList cannedACL = null; - /** Requester Pays flag. */ - private boolean requesterPays = false; - /** Content Encoding. */ private String contentEncoding; @@ -685,17 +674,6 @@ public RequestFactoryBuilder withCannedACL( return this; } - /** - * Requester Pays flag. - * @param value new value - * @return the builder - */ - public RequestFactoryBuilder withRequesterPays( - final boolean value) { - requesterPays = value; - return this; - } - /** * Multipart limit. * @param value new value From 6fcea06733838fce7ef4ac4c6b3a525d684cc1e7 Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Wed, 9 Mar 2022 13:55:29 +0000 Subject: [PATCH 06/17] Add JavaDoc to ALLOW_REQUESTER_PAYS and related variables --- .../src/main/java/org/apache/hadoop/fs/s3a/Constants.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) 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 2dc9f62a80d98..e8a3e40574e90 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 @@ -157,8 +157,13 @@ private Constants() { "fs.s3a.connection.ssl.enabled"; public static final boolean DEFAULT_SECURE_CONNECTIONS = true; - // allow access to requester pay buckets + /** + * Configuration option for S3 Requester Pays feature: {@value}. + */ public static final String ALLOW_REQUESTER_PAYS = "fs.s3a.requester-pays.enabled"; + /** + * Default configuration for {@value ALLOW_REQUESTER_PAYS}: {@value}. + */ public static final boolean DEFAULT_ALLOW_REQUESTER_PAYS = false; // use OpenSSL or JSEE for secure connections From 41e801663f8dd96a21165e2db8d5854c4c511c1f Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Wed, 9 Mar 2022 13:55:49 +0000 Subject: [PATCH 07/17] Replace . with - in requester pays config --- .../src/main/java/org/apache/hadoop/fs/s3a/Constants.java | 2 +- .../hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md | 4 ++-- .../src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md | 2 +- .../test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java | 2 +- 4 files changed, 5 insertions(+), 5 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 e8a3e40574e90..cb3d72e566854 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 @@ -160,7 +160,7 @@ private Constants() { /** * Configuration option for S3 Requester Pays feature: {@value}. */ - public static final String ALLOW_REQUESTER_PAYS = "fs.s3a.requester-pays.enabled"; + public static final String ALLOW_REQUESTER_PAYS = "fs.s3a.requester.pays.enabled"; /** * Default configuration for {@value ALLOW_REQUESTER_PAYS}: {@value}. */ diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index 1fa23269c2d6b..82872287a6fbd 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -1639,11 +1639,11 @@ S3A supports buckets with [Requester Pays](https://docs.aws.amazon.com/AmazonS3/ For requests to be successful, the S3 client must acknowledge that they will pay for these requests by setting a request flag, usually a header, on each request. -To enable this feature within S3A, configure the `fs.s3a.requester-pays.enabled` property. +To enable this feature within S3A, configure the `fs.s3a.requester.pays.enabled` property. ```xml - fs.s3a.requester-pays.enabled + fs.s3a.requester.pays.enabled true ``` diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md index c6f437cef9497..96e6e287deaaf 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md @@ -559,7 +559,7 @@ com.amazonaws.services.s3.model.AmazonS3Exception: Forbidden (Service: Amazon S3 Error Code: 403 Forbidden; Request ID: myshortreqid; S3 Extended Request ID: mylongreqid):403 Forbidden ``` -To enable requester pays, set `fs.s3a.requester-pays.enabled` property to `true`. +To enable requester pays, set `fs.s3a.requester.pays.enabled` property to `true`. ### "Unable to find a region via the region provider chain." when using session credentials. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java index 74c778c0ca89d..742c22ac5a51e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java @@ -101,7 +101,7 @@ public interface S3ATestConstants { * Configuration key for an existing object in a requester pays bucket: {@value}. * If not set, defaults to {@value DEFAULT_REQUESTER_PAYS_FILE}. */ - String KEY_REQUESTER_PAYS_FILE = TEST_FS_S3A + "requester-pays.file"; + String KEY_REQUESTER_PAYS_FILE = TEST_FS_S3A + "requester.pays.file"; /** * Default path for an S3 object inside a requester pays enabled bucket: {@value}. From 15ed59d4d429c1bfd1dc79773ae28448bc33371c Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Wed, 9 Mar 2022 14:27:49 +0000 Subject: [PATCH 08/17] Fix imports on ITestS3ARequesterPays --- .../apache/hadoop/fs/s3a/ITestS3ARequesterPays.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index eb7f867b409cd..a2c30c9ae9cdb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -18,14 +18,15 @@ package org.apache.hadoop.fs.s3a; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.statistics.StreamStatisticNames; +import java.nio.file.AccessDeniedException; + 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 java.nio.file.AccessDeniedException; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; import static org.apache.hadoop.test.LambdaTestUtils.intercept; From a3768a5ca262faa9be3f1bb85e382db7684c8f96 Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Wed, 9 Mar 2022 14:30:07 +0000 Subject: [PATCH 09/17] Remove unnecessary configuration disabling FS caching in ITestS3ARequesterPays --- .../org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index a2c30c9ae9cdb..67f6861579f38 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -35,13 +35,6 @@ */ public class ITestS3ARequesterPays extends AbstractS3ATestBase { - @Override - protected Configuration createConfiguration() { - Configuration conf = super.createConfiguration(); - S3ATestUtils.disableFilesystemCaching(conf); - return conf; - } - @Test public void testRequesterPaysOptionSuccess() throws Throwable { describe("Test requester pays enabled case by reading last then first byte"); From 059b8d66bdb09b14f8f8daf1850078cadcf01aae Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Wed, 9 Mar 2022 14:33:32 +0000 Subject: [PATCH 10/17] Use IOStatisticAssertions --- .../apache/hadoop/fs/s3a/ITestS3ARequesterPays.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index 67f6861579f38..c14edc252865a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatisticAssertions; import org.apache.hadoop.fs.statistics.StreamStatisticNames; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -56,12 +57,11 @@ public void testRequesterPaysOptionSuccess() throws Throwable { inputStream.seek(0); inputStream.readByte(); - assertEquals( - "Expected two GetObject requests", - 2, - inputStream.getIOStatistics().counters() - .get(StreamStatisticNames.STREAM_READ_OPENED).intValue() - ); + // Verify > 1 call was made, so we're sure it is correctly configured for each request + IOStatisticAssertions + .assertThatStatisticCounter(inputStream.getIOStatistics(), + StreamStatisticNames.STREAM_READ_OPENED) + .isGreaterThan(1); } } From 152c567c18b65f2935810ba9e4d5267fe8ef3358 Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Wed, 9 Mar 2022 14:41:30 +0000 Subject: [PATCH 11/17] Update documentation to split over multiple lines --- .../src/site/markdown/tools/hadoop-aws/index.md | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index 82872287a6fbd..0eb65d558ac98 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -1635,9 +1635,13 @@ sure to update them if needed; ## Requester Pays buckets -S3A supports buckets with [Requester Pays](https://docs.aws.amazon.com/AmazonS3/latest/userguide/RequesterPaysBuckets.html) enabled. When a bucket is configured with requester pays, the requester must cover the per-request cost. +S3A supports buckets with +[Requester Pays](https://docs.aws.amazon.com/AmazonS3/latest/userguide/RequesterPaysBuckets.html) +enabled. When a bucket is configured with requester pays, the requester must cover +the per-request cost. -For requests to be successful, the S3 client must acknowledge that they will pay for these requests by setting a request flag, usually a header, on each request. +For requests to be successful, the S3 client must acknowledge that they will pay +for these requests by setting a request flag, usually a header, on each request. To enable this feature within S3A, configure the `fs.s3a.requester.pays.enabled` property. From 876103f64d3263676d840d18d81f74834453c24e Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Thu, 10 Mar 2022 13:30:55 +0000 Subject: [PATCH 12/17] Add listFiles call to S3A requester pays integ test --- .../java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index c14edc252865a..818e517c1fa6f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -62,6 +62,9 @@ public void testRequesterPaysOptionSuccess() throws Throwable { .assertThatStatisticCounter(inputStream.getIOStatistics(), StreamStatisticNames.STREAM_READ_OPENED) .isGreaterThan(1); + + // Check list calls work without error + fs.listFiles(requesterPaysPath.getParent(), false); } } From bbccc7238b583507867b693aa6ee0e876b99271d Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Thu, 10 Mar 2022 18:39:09 +0000 Subject: [PATCH 13/17] Add removal of base/bucket overrides for requester pays tests --- .../org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index 818e517c1fa6f..c1398fe685ca0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -36,6 +36,13 @@ */ public class ITestS3ARequesterPays extends AbstractS3ATestBase { + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + S3ATestUtils.removeBaseAndBucketOverrides(conf, Constants.ALLOW_REQUESTER_PAYS); + return conf; + } + @Test public void testRequesterPaysOptionSuccess() throws Throwable { describe("Test requester pays enabled case by reading last then first byte"); From 9dd0bae79d22a453c0d4a8dbd39c755f354bf9f1 Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Thu, 10 Mar 2022 18:39:52 +0000 Subject: [PATCH 14/17] Enable full bucket exists check in requester pays tests --- .../org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index c1398fe685ca0..77aaa379d1c18 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -39,7 +39,9 @@ public class ITestS3ARequesterPays extends AbstractS3ATestBase { @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); - S3ATestUtils.removeBaseAndBucketOverrides(conf, Constants.ALLOW_REQUESTER_PAYS); + S3ATestUtils.removeBaseAndBucketOverrides(conf, + Constants.ALLOW_REQUESTER_PAYS, + Constants.S3A_BUCKET_PROBE); return conf; } @@ -49,6 +51,9 @@ public void testRequesterPaysOptionSuccess() throws Throwable { Configuration conf = this.createConfiguration(); conf.setBoolean(Constants.ALLOW_REQUESTER_PAYS, true); + // Enable bucket exists check, the first failure point people may encounter + conf.setInt(Constants.S3A_BUCKET_PROBE, 2); + Path requesterPaysPath = getRequesterPaysPath(conf); try ( From d58b0f97d9805c2cdfe8b640592b71f284e2b6a5 Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Thu, 10 Mar 2022 18:53:36 +0000 Subject: [PATCH 15/17] Add note in testing.md on how to configure requester pays tests --- .../site/markdown/tools/hadoop-aws/testing.md | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md index 559687a3fdb3f..22a0d3af1e2e2 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md @@ -593,6 +593,31 @@ your `core-site.xml` file, so that trying to use S3 select fails fast with a meaningful error ("S3 Select not supported") rather than a generic Bad Request exception. +### Testing Requester Pays + +By default, the requester pays tests will look for a bucket that exists on Amazon S3 +in us-east-1. + +If the endpoint does support requester pays, you can specify an alternative object. +The test only requires an object of at least a few bytes in order +to check that lists and basic reads work. + +```xml + + test.fs.s3a.requester.pays.file + s3a://my-req-pays-enabled-bucket/on-another-endpoint.json + +``` + +If the endpoint does not support requester pays, you can also disable the tests by configuring +the test URI as a single space. + +```xml + + test.fs.s3a.requester.pays.file + + +``` ### Testing Session Credentials From 1fbbc487adcf267ad51e88462a08a63af1729b46 Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Thu, 10 Mar 2022 18:56:50 +0000 Subject: [PATCH 16/17] Update ITestS3ARequesterPays to use static imports for constants --- .../apache/hadoop/fs/s3a/ITestS3ARequesterPays.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index 77aaa379d1c18..c2e7684cad6da 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -29,6 +29,8 @@ import org.apache.hadoop.fs.statistics.IOStatisticAssertions; import org.apache.hadoop.fs.statistics.StreamStatisticNames; +import static org.apache.hadoop.fs.s3a.Constants.ALLOW_REQUESTER_PAYS; +import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** @@ -40,8 +42,8 @@ public class ITestS3ARequesterPays extends AbstractS3ATestBase { protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); S3ATestUtils.removeBaseAndBucketOverrides(conf, - Constants.ALLOW_REQUESTER_PAYS, - Constants.S3A_BUCKET_PROBE); + ALLOW_REQUESTER_PAYS, + S3A_BUCKET_PROBE); return conf; } @@ -50,9 +52,9 @@ public void testRequesterPaysOptionSuccess() throws Throwable { describe("Test requester pays enabled case by reading last then first byte"); Configuration conf = this.createConfiguration(); - conf.setBoolean(Constants.ALLOW_REQUESTER_PAYS, true); + conf.setBoolean(ALLOW_REQUESTER_PAYS, true); // Enable bucket exists check, the first failure point people may encounter - conf.setInt(Constants.S3A_BUCKET_PROBE, 2); + conf.setInt(S3A_BUCKET_PROBE, 2); Path requesterPaysPath = getRequesterPaysPath(conf); @@ -85,7 +87,7 @@ public void testRequesterPaysDisabledFails() throws Throwable { describe("Verify expected failure for requester pays buckets when client has it disabled"); Configuration conf = this.createConfiguration(); - conf.setBoolean(Constants.ALLOW_REQUESTER_PAYS, false); + conf.setBoolean(ALLOW_REQUESTER_PAYS, false); Path requesterPaysPath = getRequesterPaysPath(conf); try (FileSystem fs = requesterPaysPath.getFileSystem(conf)) { From ae8e812e11148bcf007d51fbf94356229df4fae1 Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Fri, 11 Mar 2022 07:21:17 +0000 Subject: [PATCH 17/17] Fix blanks in documentation --- .../src/site/markdown/tools/hadoop-aws/index.md | 8 ++++---- .../src/site/markdown/tools/hadoop-aws/testing.md | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index 0eb65d558ac98..df08a969e9551 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -1635,12 +1635,12 @@ sure to update them if needed; ## Requester Pays buckets -S3A supports buckets with -[Requester Pays](https://docs.aws.amazon.com/AmazonS3/latest/userguide/RequesterPaysBuckets.html) -enabled. When a bucket is configured with requester pays, the requester must cover +S3A supports buckets with +[Requester Pays](https://docs.aws.amazon.com/AmazonS3/latest/userguide/RequesterPaysBuckets.html) +enabled. When a bucket is configured with requester pays, the requester must cover the per-request cost. -For requests to be successful, the S3 client must acknowledge that they will pay +For requests to be successful, the S3 client must acknowledge that they will pay for these requests by setting a request flag, usually a header, on each request. To enable this feature within S3A, configure the `fs.s3a.requester.pays.enabled` property. diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md index 22a0d3af1e2e2..2641b870d2ea5 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md @@ -595,11 +595,11 @@ exception. ### Testing Requester Pays -By default, the requester pays tests will look for a bucket that exists on Amazon S3 +By default, the requester pays tests will look for a bucket that exists on Amazon S3 in us-east-1. If the endpoint does support requester pays, you can specify an alternative object. -The test only requires an object of at least a few bytes in order +The test only requires an object of at least a few bytes in order to check that lists and basic reads work. ```xml @@ -609,7 +609,7 @@ to check that lists and basic reads work. ``` -If the endpoint does not support requester pays, you can also disable the tests by configuring +If the endpoint does not support requester pays, you can also disable the tests by configuring the test URI as a single space. ```xml