-
Notifications
You must be signed in to change notification settings - Fork 9.1k
HADOOP-14661. Add S3 requester pays bucket support to S3A #3962
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
18 commits
Select commit
Hold shift + click to select a range
4abf17d
HADOOP-14661. Add S3 requester pays bucket support to S3A
dannycjones a401c36
HADOOP-14661. Fix checkstyle
dannycjones 038956f
Merge branch 'apache:trunk' into HADOOP-14661
dannycjones 8d205b5
Add close statements for FS and InputStream in Requester Pays S3A int…
dannycjones 257fe5b
Move requester pays header value to constant
dannycjones 4ed5959
Deprecate unused requester pays flag in S3A's RequestFactory
dannycjones 6fcea06
Add JavaDoc to ALLOW_REQUESTER_PAYS and related variables
dannycjones 41e8016
Replace . with - in requester pays config
dannycjones 15ed59d
Fix imports on ITestS3ARequesterPays
dannycjones a3768a5
Remove unnecessary configuration disabling FS caching in ITestS3ARequ…
dannycjones 059b8d6
Use IOStatisticAssertions
dannycjones 152c567
Update documentation to split over multiple lines
dannycjones 876103f
Add listFiles call to S3A requester pays integ test
dannycjones bbccc72
Add removal of base/bucket overrides for requester pays tests
dannycjones 9dd0bae
Enable full bucket exists check in requester pays tests
dannycjones d58b0f9
Add note in testing.md on how to configure requester pays tests
dannycjones 1fbbc48
Update ITestS3ARequesterPays to use static imports for constants
dannycjones ae8e812
Fix blanks in documentation
dannycjones File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
113 changes: 113 additions & 0 deletions
113
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,113 @@ | ||
/* | ||
* 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; | ||
dannycjones marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
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.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; | ||
|
||
/** | ||
* Tests for Requester Pays feature. | ||
*/ | ||
public class ITestS3ARequesterPays extends AbstractS3ATestBase { | ||
dannycjones marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
@Override | ||
protected Configuration createConfiguration() { | ||
dannycjones marked this conversation as resolved.
Show resolved
Hide resolved
|
||
Configuration conf = super.createConfiguration(); | ||
S3ATestUtils.removeBaseAndBucketOverrides(conf, | ||
ALLOW_REQUESTER_PAYS, | ||
S3A_BUCKET_PROBE); | ||
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(ALLOW_REQUESTER_PAYS, true); | ||
// Enable bucket exists check, the first failure point people may encounter | ||
conf.setInt(S3A_BUCKET_PROBE, 2); | ||
|
||
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(ALLOW_REQUESTER_PAYS, false); | ||
Path requesterPaysPath = getRequesterPaysPath(conf); | ||
|
||
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) { | ||
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); | ||
} | ||
|
||
} |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.