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

Steve Loughran commented on HADOOP-13786:
-----------------------------------------

Patch 032

Main change here is implementing failure resilience on all the commit 
algorithms. This is done with some major (and long outstanding) enhancments to 
S3A's ability to recognise different AWS exceptions, including throttling, and 
implement retries depending on what's happened

Specifically

# new Exceptions for specific problems 
([throttling|https://github.com/steveloughran/hadoop/blob/s3guard/HADOOP-13786-committer/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSServiceThrottledException.java],
 [bad 
request|https://github.com/steveloughran/hadoop/blob/s3guard/HADOOP-13786-committer/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSBadRequestException.java])
#  S3AUtils.translateException [to map errors to 
them|https://github.com/steveloughran/hadoop/blob/s3guard/HADOOP-13786-committer/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java#L164]
# 
[S3ARetryPolicy|https://github.com/steveloughran/hadoop/blob/s3guard/HADOOP-13786-committer/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java]
 to choose actions base on exception type. Failfast for unrecoverables 
(network, auth), fail with exponential backoff for throttling, fail with 
limited retries for everything else.
# extended S3a [Lambda 
integration|https://github.com/steveloughran/hadoop/blob/s3guard/HADOOP-13786-committer/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ALambda.java#L156]
 to wrap any l-exp with the retry policy.
# WriteOperationsHelper to wrap all operations with retry invocation 
[example|https://github.com/steveloughran/hadoop/blob/s3guard/HADOOP-13786-committer/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java#L219]

Essentially, it's now trivial to take any S3 interaction and wrap it with retry 
logic. This works for all the existing FS ops too

{code}
  protected FileStatus getFileStatus(Path path) throws IOException {
    return lambda.retry("getFileStatus", path.toString(), true,
        () -> fs.getFileStatus(path));
  }
{code}

I've only fitted this to the writeOperationsHelper new methods & the commit 
logic; left the retry logic commiting an{{S3ABlockOutputStream}} alone, and 
doing nothing to the S3AFS methods instead. But we should be doing the latter 
*for all operations which don't go through AWS xfer manager*

I'd like to pull this feature into HADOOP-13345 once I've had a bit more 
experience of failure tuning (example: does making BadRequest retriable help?). 
It can be the basis for all the DDB retry logic too. Then we can look at 
wrapping all other FS operations which are at risk of failing. 



> Add S3Guard committer for zero-rename commits to 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, HADOOP-13786-HADOOP-13345-022.patch, 
> HADOOP-13786-HADOOP-13345-023.patch, HADOOP-13786-HADOOP-13345-024.patch, 
> HADOOP-13786-HADOOP-13345-025.patch, HADOOP-13786-HADOOP-13345-026.patch, 
> HADOOP-13786-HADOOP-13345-027.patch, HADOOP-13786-HADOOP-13345-028.patch, 
> HADOOP-13786-HADOOP-13345-028.patch, HADOOP-13786-HADOOP-13345-029.patch, 
> HADOOP-13786-HADOOP-13345-030.patch, HADOOP-13786-HADOOP-13345-031.patch, 
> HADOOP-13786-HADOOP-13345-032.patch, objectstore.pdf, 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.4.14#64029)

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

Reply via email to