steveloughran commented on a change in pull request #3962: URL: https://github.com/apache/hadoop/pull/3962#discussion_r824017714
########## File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java ########## @@ -0,0 +1,99 @@ +/* + * 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.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 org.apache.hadoop.fs.statistics.IOStatisticAssertions; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; + +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Tests for Requester Pays feature. + */ +public class ITestS3ARequesterPays extends AbstractS3ATestBase { + + @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); + Path requesterPaysPath = getRequesterPaysPath(conf); + + 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(); + + // 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); + + // Check list calls work without error + fs.listFiles(requesterPaysPath.getParent(), false); + } + } + + @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); Review comment: It is very unlikely to surface here but a good thing to get into the habit of doing. Before tuning configuration options, call S3ATestUtils.removeBaseAndBucketOverrides and pass in all the options you plan to set. ``` removeBaseAndBucketOverrides(conf, ALLOW_REQUESTER_PAYS); ``` This is to avoid spurious test failures if some has set any per-bucket config option, which will then take priority over whatever you set in the test. like I say, not so likely here, but not uncommon with things like encryption settings, endpoints etc. ########## File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java ########## @@ -0,0 +1,99 @@ +/* + * 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.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 org.apache.hadoop.fs.statistics.IOStatisticAssertions; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; + +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Tests for Requester Pays feature. + */ +public class ITestS3ARequesterPays extends AbstractS3ATestBase { + + @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); Review comment: can you also set S3A_BUCKET_PROBE to 2? that will make sure we set that header on the bucketExists check, which could be the first failure point people might see -- 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]
