[
https://issues.apache.org/jira/browse/HADOOP-13786?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15945207#comment-15945207
]
Steve Loughran commented on HADOOP-13786:
-----------------------------------------
h2. IT Test {{testMapFileOutputCommitter}} failures with unique IDs enabled
This is failing as the unique ID goes onto the path such as
{{dest/part-m-0000/index-job-attempt_01}}, {{part-m-000/data-job-attempt_01}}.
As a result, the files index and data
aren't found in the test or the reducer. The naming needs to go into the
{{part-m-0000-$UUID/index}} and {{part-m-0000-$UUID/index}}.
I've disabled unique filenames by default; adapted files to match
for the partitioned committer, it doesn't work with mappers anyway, because of
how things are handled in the commit (as far as I can tell, though I could be
confused here.). Disabled the relevant IT test.
h2. TestStatingMRJob Failures
Reasons for failure in Yetus unclear. Locally, *appears* to work, but in the AM
log there's a stack warning that s3a isn't bonding
{code}
2017-03-24 16:56:37,468 WARN [CommitterEvent Processor #1]
org.apache.hadoop.fs.s3a.S3AUtils: Failed to delete
s3a://bucket-name/output/path/_temporary
java.nio.file.AccessDeniedException: s3a://bucket-name/output/path/_temporary:
getFileStatus on s3a://bucket-name/output/path/_temporary:
com.amazonaws.services.s3.model.AmazonS3Exception: Forbidden (Service: Amazon
S3; Status Code: 403; Error Code: 403 Forbidden; Request ID: 5F7696A22A66FAAF),
S3 Extended Request ID:
cVVOEgwybZL0YQbQ0fwFbPMv7GO/TOBAVJed//EQsQUX9dGNkgzCrLy1U6ZbMHmb9qQyKzZSOcw=
at
org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:159)
at
org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:102)
at
org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:1832)
at
org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:1802)
at
org.apache.hadoop.fs.s3a.S3AFileSystem.delete(S3AFileSystem.java:1416)
at
org.apache.hadoop.fs.s3a.S3AUtils.deleteWithWarning(S3AUtils.java:756)
at
org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.deleteDestinationPaths(StagingS3GuardCommitter.java:817)
at
org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.cleanup(StagingS3GuardCommitter.java:780)
at
org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.commitJobInternal(StagingS3GuardCommitter.java:685)
at
org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.commitJob(StagingS3GuardCommitter.java:625)
at
org.apache.hadoop.fs.s3a.commit.staging.MockedStagingCommitter.commitJob(MockedStagingCommitter.java:84)
at
org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventHandler$EventProcessor.handleJobCommit(CommitterEventHandler.java:286)
at
org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventHandler$EventProcessor.run(CommitterEventHandler.java:238)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Forbidden
(Service: Amazon S3; Status Code: 403; Error Code: 403 Forbidden; Request ID:
5F7696A22A66FAAF), S3 Extended Request ID:
cVVOEgwybZL0YQbQ0fwFbPMv7GO/TOBAVJed//EQsQUX9dGNkgzCrLy1U6ZbMHmb9qQyKzZSOcw=
at
com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1586)
at
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1254)
at
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1035)
at
com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:747)
at
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:721)
at
com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:704)
at
com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:672)
at
com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:654)
at
com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:518)
at
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4185)
at
com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4132)
at
com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1245)
at
org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1081)
at
org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:1817)
... 13 more
2017-03-24 16:56:37,470 INFO [CommitterEvent Processor #1]
org.apache.hadoop.fs.s3a.commit.
{code}
What's happening? resumably, the mock S3 client isn't getting as far as the
MockedStagingCommitter in the AM: a new process is starting and so the special
committer pushed into the FileSystem.get() cache isn't being found.
Fix? Presumably to revert to the trick as the original mock tests did: patch in
a new fs.s3a.impl class into the job submission and use it.
Not we do not want to do that for the real IT tests.
> Add S3Guard committer for zero-rename commits to consistent S3 endpoints
> ------------------------------------------------------------------------
>
> Key: HADOOP-13786
> URL: https://issues.apache.org/jira/browse/HADOOP-13786
> Project: Hadoop Common
> Issue Type: New Feature
> Components: fs/s3
> Affects Versions: HADOOP-13345
> Reporter: Steve Loughran
> Assignee: Steve Loughran
> Attachments: HADOOP-13786-HADOOP-13345-001.patch,
> HADOOP-13786-HADOOP-13345-002.patch, HADOOP-13786-HADOOP-13345-003.patch,
> HADOOP-13786-HADOOP-13345-004.patch, HADOOP-13786-HADOOP-13345-005.patch,
> HADOOP-13786-HADOOP-13345-006.patch, HADOOP-13786-HADOOP-13345-006.patch,
> HADOOP-13786-HADOOP-13345-007.patch, HADOOP-13786-HADOOP-13345-009.patch,
> HADOOP-13786-HADOOP-13345-010.patch, HADOOP-13786-HADOOP-13345-011.patch,
> HADOOP-13786-HADOOP-13345-012.patch, HADOOP-13786-HADOOP-13345-013.patch,
> HADOOP-13786-HADOOP-13345-015.patch, HADOOP-13786-HADOOP-13345-016.patch,
> HADOOP-13786-HADOOP-13345-017.patch, HADOOP-13786-HADOOP-13345-018.patch,
> HADOOP-13786-HADOOP-13345-019.patch, HADOOP-13786-HADOOP-13345-020.patch,
> HADOOP-13786-HADOOP-13345-021.patch, s3committer-master.zip
>
>
> A goal of this code is "support O(1) commits to S3 repositories in the
> presence of failures". Implement it, including whatever is needed to
> demonstrate the correctness of the algorithm. (that is, assuming that s3guard
> provides a consistent view of the presence/absence of blobs, show that we can
> commit directly).
> I consider ourselves free to expose the blobstore-ness of the s3 output
> streams (ie. not visible until the close()), if we need to use that to allow
> us to abort commit operations.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]