[ 
https://issues.apache.org/jira/browse/HADOOP-15990?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16714073#comment-16714073
 ] 

Steve Loughran commented on HADOOP-15990:
-----------------------------------------

Patch 006. Patch 005, with
* rebase to trunk
* ability to revert back to v1 check. Covered in docs and the 
{{ITestS3AContractGetFileStatusV1List}} explicitly reverts to it, 
* confiig options != 1 or 2 get check as v2. Leaves place open for a v3 option 
if it ever arises. Manually tested.
* core-default changed
* docs
* Fix {{MockS3ClientFactory}} so that mocked v2 probe returns true. Without 
that mock tests were failing

Tested, S3 ireland w/ DDB + s3guard

Failures: 

# the known bouncy castle issue fixed in HADOOP-14556 broke the miniyarn test 
(ignoring that; its not new)
# ITestS3ATemporaryCredentials

I don't know if that second one was new, not tested trunk for a while; 
HADOOP-14556 changes things a lot there. And when I switch back to V1, things 
still fail

What's interesting here is that the error message has changed from a 400/bad 
request to 400/invalid token. This is actually good, if the error text is more 
meaningful. But: it should't be happening here at all.

{code}
[ERROR] testSTS(org.apache.hadoop.fs.s3a.ITestS3ATemporaryCredentials)  Time 
elapsed: 2.882 s  <<< ERROR!
org.apache.hadoop.fs.s3a.AWSBadRequestException: doesBucketExist on 
hwdev-steve-ireland-new: com.amazonaws.services.s3.model.AmazonS3Exception: Bad 
Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; 
Request ID: 683CC237B804A9E4; S3 Extended Request ID: 
hrEJrEbB+TmWrHmSJkGKI0mwOFuxvxIuuOvRmd/WbkkSqJpjCUT1gg1nykT7j96OltzrI4xhM/4=), 
S3 Extended Request ID: 
hrEJrEbB+TmWrHmSJkGKI0mwOFuxvxIuuOvRmd/WbkkSqJpjCUT1gg1nykT7j96OltzrI4xhM/4=:400
 Bad Request: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 
400 Bad Request; Request ID: 683CC237B804A9E4; S3 Extended Request ID: 
hrEJrEbB+TmWrHmSJkGKI0mwOFuxvxIuuOvRmd/WbkkSqJpjCUT1gg1nykT7j96OltzrI4xhM/4=)
        at 
org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:224)
        at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111)
        at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:265)
        at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:322)
        at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:261)
        at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:236)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:400)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:313)
        at 
org.apache.hadoop.fs.s3a.S3ATestUtils.createTestFileSystem(S3ATestUtils.java:143)
        at 
org.apache.hadoop.fs.s3a.S3ATestUtils.createTestFileSystem(S3ATestUtils.java:102)
        at 
org.apache.hadoop.fs.s3a.ITestS3ATemporaryCredentials.testSTS(ITestS3ATemporaryCredentials.java:117)
        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:50)
        at 
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
        at 
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
        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$CallableStatement.call(FailOnTimeout.java:298)
        at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.lang.Thread.run(Thread.java:745)
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Bad Request 
(Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: 
683CC237B804A9E4; S3 Extended Request ID: 
hrEJrEbB+TmWrHmSJkGKI0mwOFuxvxIuuOvRmd/WbkkSqJpjCUT1gg1nykT7j96OltzrI4xhM/4=), 
S3 Extended Request ID: 
hrEJrEbB+TmWrHmSJkGKI0mwOFuxvxIuuOvRmd/WbkkSqJpjCUT1gg1nykT7j96OltzrI4xhM/4=
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1640)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1304)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1058)
        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:4368)
        at 
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4315)
        at 
com.amazonaws.services.s3.AmazonS3Client.headBucket(AmazonS3Client.java:1344)
        at 
com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1284)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$verifyBucketExists$1(S3AFileSystem.java:402)
        at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:109)
        ... 24 more
{code}

And this is the v2 check

{code}
[ERROR] testSTS(org.apache.hadoop.fs.s3a.ITestS3ATemporaryCredentials)  Time 
elapsed: 3.345 s  <<< ERROR!
org.apache.hadoop.fs.s3a.AWSBadRequestException: doesBucketExist on 
hwdev-steve-ireland-new: com.amazonaws.services.s3.model.AmazonS3Exception: The 
provided token is malformed or otherwise invalid. (Service: Amazon S3; Status 
Code: 400; Error Code: InvalidToken; Request ID: 45CD4E5F45D54595; S3 Extended 
Request ID: 
qh5eoRVoawdHxE9EdncrrAE7FeQVurGkdxQ1sUpkPLf8GUaT63hSbKBH0dvjmJaebWXpB4WH8p4=), 
S3 Extended Request ID: 
qh5eoRVoawdHxE9EdncrrAE7FeQVurGkdxQ1sUpkPLf8GUaT63hSbKBH0dvjmJaebWXpB4WH8p4=:InvalidToken:
 The provided token is malformed or otherwise invalid. (Service: Amazon S3; 
Status Code: 400; Error Code: InvalidToken; Request ID: 45CD4E5F45D54595; S3 
Extended Request ID: 
qh5eoRVoawdHxE9EdncrrAE7FeQVurGkdxQ1sUpkPLf8GUaT63hSbKBH0dvjmJaebWXpB4WH8p4=)
        at 
org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:224)
        at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111)
        at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:265)
        at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:322)
        at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:261)
        at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:236)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:400)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:313)
        at 
org.apache.hadoop.fs.s3a.S3ATestUtils.createTestFileSystem(S3ATestUtils.java:143)
        at 
org.apache.hadoop.fs.s3a.S3ATestUtils.createTestFileSystem(S3ATestUtils.java:102)
        at 
org.apache.hadoop.fs.s3a.ITestS3ATemporaryCredentials.testSTS(ITestS3ATemporaryCredentials.java:118)
        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:50)
        at 
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
        at 
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
        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$CallableStatement.call(FailOnTimeout.java:298)
        at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.lang.Thread.run(Thread.java:745)
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: The provided 
token is malformed or otherwise invalid. (Service: Amazon S3; Status Code: 400; 
Error Code: InvalidToken; Request ID: 45CD4E5F45D54595; S3 Extended Request ID: 
qh5eoRVoawdHxE9EdncrrAE7FeQVurGkdxQ1sUpkPLf8GUaT63hSbKBH0dvjmJaebWXpB4WH8p4=), 
S3 Extended Request ID: 
qh5eoRVoawdHxE9EdncrrAE7FeQVurGkdxQ1sUpkPLf8GUaT63hSbKBH0dvjmJaebWXpB4WH8p4=
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1640)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1304)
        at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1058)
        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:4368)
        at 
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4315)
        at 
com.amazonaws.services.s3.AmazonS3Client.getAcl(AmazonS3Client.java:3512)
        at 
com.amazonaws.services.s3.AmazonS3Client.getBucketAcl(AmazonS3Client.java:1178)
        at 
com.amazonaws.services.s3.AmazonS3Client.getBucketAcl(AmazonS3Client.java:1168)
        at 
com.amazonaws.services.s3.AmazonS3Client.doesBucketExistV2(AmazonS3Client.java:1303)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$verifyBucketExists$1(S3AFileSystem.java:403)
        at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:109)
        ... 24 more
{code}

> S3AFileSystem.verifyBucketExists to move to s3.doesBucketExistV2
> ----------------------------------------------------------------
>
>                 Key: HADOOP-15990
>                 URL: https://issues.apache.org/jira/browse/HADOOP-15990
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/s3
>    Affects Versions: 3.2.0
>            Reporter: Steve Loughran
>            Assignee: lqjacklee
>            Priority: Major
>         Attachments: HADOOP-15409-005.patch
>
>
> in S3AFileSystem.initialize(), we check for the bucket existing with 
> verifyBucketExists(), which calls s3.doesBucketExist(). But that doesn't 
> check for auth issues. 
> s3. doesBucketExistV2() does at least validate credentials, and should be 
> switched to. This will help things fail faster 
> See SPARK-24000
> (this is a dupe of HADOOP-15409; moving off git PRs so we can get yetus to 
> test everything)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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

Reply via email to