[
https://issues.apache.org/jira/browse/HADOOP-15583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16551488#comment-16551488
]
Steve Loughran commented on HADOOP-15583:
-----------------------------------------
If you have S3Guard enabled but you don't have s3:getBucketLocation, the first
S3 interaction to call will be the getBucketLocation operation used to find the
region for the S3Guard table; this error message can be improved
{code}
[ERROR]
testAssumeRoleRestrictedPolicyFS(org.apache.hadoop.fs.s3a.auth.ITestAssumeRole)
Time elapsed: 6.396 s <<< ERROR!
java.nio.file.AccessDeniedException: hwdev-steve-ireland-new:
getBucketLocation() on hwdev-steve-ireland-new:
com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service:
Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID:
A8B11AD20AE20391; S3 Extended Request ID:
+SSWoeRU5/mwTCs7WnWScs52x8aSJbVfhQ/lVM3fXb5G/0EU0T4BdRRMgPJImufrd/jDDl8/uWk=),
S3 Extended Request ID:
+SSWoeRU5/mwTCs7WnWScs52x8aSJbVfhQ/lVM3fXb5G/0EU0T4BdRRMgPJImufrd/jDDl8/uWk=:AccessDenied
at
org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:229)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111)
at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:260)
at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:317)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:256)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:231)
at
org.apache.hadoop.fs.s3a.S3AFileSystem.getBucketLocation(S3AFileSystem.java:528)
at
org.apache.hadoop.fs.s3a.S3AFileSystem.getBucketLocation(S3AFileSystem.java:516)
at
org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.initialize(DynamoDBMetadataStore.java:294)
at
org.apache.hadoop.fs.s3a.s3guard.S3Guard.getMetadataStore(S3Guard.java:99)
at
org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:339)
at
org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474)
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
at
org.apache.hadoop.fs.s3a.auth.ITestAssumeRole.testAssumeRoleRestrictedPolicyFS(ITestAssumeRole.java:311)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
at
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
at
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
at
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
at
org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
at
org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
at
org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied
(Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID:
A8B11AD20AE20391; S3 Extended Request ID:
+SSWoeRU5/mwTCs7WnWScs52x8aSJbVfhQ/lVM3fXb5G/0EU0T4BdRRMgPJImufrd/jDDl8/uWk=),
S3 Extended Request ID:
+SSWoeRU5/mwTCs7WnWScs52x8aSJbVfhQ/lVM3fXb5G/0EU0T4BdRRMgPJImufrd/jDDl8/uWk=
at
com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1639)
at
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1304)
at
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1056)
at
com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
at
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
at
com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at
com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at
com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at
com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4325)
at
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4272)
at
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4266)
at
com.amazonaws.services.s3.AmazonS3Client.getBucketLocation(AmazonS3Client.java:949)
at
com.amazonaws.services.s3.AmazonS3Client.getBucketLocation(AmazonS3Client.java:955)
at
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$getBucketLocation$3(S3AFileSystem.java:529)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:109)
... 25 more
{code}
> Stabilize S3A Assumed Role support
> ----------------------------------
>
> Key: HADOOP-15583
> URL: https://issues.apache.org/jira/browse/HADOOP-15583
> Project: Hadoop Common
> Issue Type: Sub-task
> Components: fs/s3
> Affects Versions: 3.1.0
> Reporter: Steve Loughran
> Assignee: Steve Loughran
> Priority: Blocker
> Attachments: HADOOP-15583-001.patch, HADOOP-15583-002.patch
>
>
> started off just on sharing credentials across S3A and S3Guard, but in the
> process it has grown to becoming one of stabilising the assumed role support
> so it can be used for more than just testing.
> Was: "S3Guard to get AWS Credential chain from S3AFS; credentials closed() on
> shutdown"
> h3. Issue: lack of auth chain sharing causes ddb and s3 to get out of sync
> S3Guard builds its DDB auth chain itself, which stops it having to worry
> about being created standalone vs part of an S3AFS, but it means its
> authenticators are in a separate chain.
> When you are using short-lived assumed roles or other session credentials
> updated in the S3A FS authentication chain, you need that same set of
> credentials picked up by DDB. Otherwise, at best you are doubling load, at
> worse: the DDB connector may not get refreshed credentials.
> Proposed: {{DynamoDBClientFactory.createDynamoDBClient()}} to take an
> optional ref to aws credentials. If set: don't create a new set.
> There's one little complication here: our {{AWSCredentialProviderList}} list
> is autocloseable; it's close() will go through all children and close them.
> Apparently the AWS S3 client (And hopefully the DDB client) will close this
> when they are closed themselves. If DDB has the same set of credentials as
> the FS, then there could be trouble if they are closed in one place when the
> other still wants to use them.
> Solution; have a use count the uses of the credentials list, starting at one:
> every close() call decrements, and when this hits zero the cleanup is kicked
> off
> h3. Issue: {{AssumedRoleCredentialProvider}} connector to STS not picking up
> the s3a connection settings, including proxy.
> h3. issue: we're not using getPassword() to get user/password for proxy
> binding for STS. Fix: use that and pass down the bucket ref for per-bucket
> secrets in a JCEKS file.
> h3. Issue; hard to debug what's going wrong :)
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]