mukund-thakur commented on a change in pull request #3962:
URL: https://github.com/apache/hadoop/pull/3962#discussion_r815621192



##########
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:
       Never seen "-" used in any config name. 

##########
File path: 
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java
##########
@@ -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");

Review comment:
       create a local constant for "requester" string.

##########
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);

Review comment:
       Close inputStream. use try-with-resources. 

##########
File path: 
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
##########
@@ -97,6 +97,17 @@
    */
   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";

Review comment:
       I hope this doesn't gets removed in future. 

##########
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());

Review comment:
       We can use requesterPaysPath.getFileSystem(conf) as well.
   
   Also close the filesystem as it is created locally

##########
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:
       This seems like a generic AccessDeniedException. Is there any way caller 
will know that this is a requester pays bucket and I have to enable the flag? 
Otherwise debugging this will be a bit harder no?




-- 
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: common-issues-unsubscr...@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to