steveloughran commented on a change in pull request #3962:
URL: https://github.com/apache/hadoop/pull/3962#discussion_r821208245



##########
File path: 
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";

Review comment:
       can we make this `fs.s3a.requester.pays.enabled`; avoids confusion about 
when to use use - over .?

##########
File path: 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;
 
+## <a name="requester_pays"></a>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.

Review comment:
       ideally split these down such as by sentence, 100 chars max. helps 
viewing diffs

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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;

Review comment:
       nit: junit (and other non org.apache.*) imports should go into their own 
block above the apache ones. Yes, all our existing code isn't always so good 
,no, we can't fix that without ruining the lives of anyone cherrypicking. But 
we try...

##########
File path: 
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

Review comment:
       nit: make a javadoc, with a reference to {@value} in it. Good for IDE 
popup hints

##########
File path: 
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.
 
+### <a name="access_denied_requester_pays"></a>`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:

Review comment:
       in the troubleshooting doc, add a link to this
   
https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md#access_denied

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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 {

Review comment:
       this test should be skipped when running against third party stores 
(e.g. private ones)
   
   subclass the setup() method, look for the path, and if it is empty use 
S3ATestUtils.assume() to downgrade to a skip. See ITestAssumeRole for an 
example.

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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() {

Review comment:
       no need for this, just create and then close new filesystems in the test 
cases.

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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(

Review comment:
       use the assertions in IOStatisticAssertions for this. it is what they 
are foe




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to