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

Aaron Fabbri commented on HADOOP-13761:
---------------------------------------

Thanks for the feedback on the scale test patch [~ste...@apache.org].  On the 
retries, I've been looking at open(), delete(), and rename() in terms of what 
happens when MetadataStore has metadata but we hit eventual consistency when we 
start operating on S3.

 
 * open(): This is the only code that creates a S3AInputStream constructor, and 
it does a getFileStatus() first, so failures should be happening later in 
read() paths.  There is already a single handler for read failures: 
{{onReadFailure()}}.  I suggest we add invoker/retry policy to this function. 
Thoughts? As is, it appears to only retry once without any backoff timer.
 * delete(): this uses deleteObject(), which already has retries. I think we 
are good here?
 * rename(): this is more complicated, and we probably need to do more in-depth 
rename() rewrite to include HADOOP-15183.  One thing I noticed is, 
{{copyFile()}} is annotated {{RetryMixed}} but only uses {{once()}} internally, 
so I might fix that.

I'd like to address Steve's feedback on the scale test and make the above 
changes if it sounds good. I think rename() would benefit from its own 
patch/JIRA, (possibly even add a new experimental "v2" rename codepath that can 
be switched in via config while we improve and test with it, dunno).

> S3Guard: implement retries for DDB failures and throttling; translate 
> exceptions
> --------------------------------------------------------------------------------
>
>                 Key: HADOOP-13761
>                 URL: https://issues.apache.org/jira/browse/HADOOP-13761
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/s3
>    Affects Versions: 3.0.0-beta1
>            Reporter: Aaron Fabbri
>            Assignee: Aaron Fabbri
>            Priority: Blocker
>         Attachments: HADOOP-13761.001.patch, HADOOP-13761.002.patch
>
>
> Following the S3AFileSystem integration patch in HADOOP-13651, we need to add 
> retry logic.
> In HADOOP-13651, I added TODO comments in most of the places retry loops are 
> needed, including:
> - open(path).  If MetadataStore reflects recent create/move of file path, but 
> we fail to read it from S3, retry.
> - delete(path).  If deleteObject() on S3 fails, but MetadataStore shows the 
> file exists, retry.
> - rename(src,dest).  If source path is not visible in S3 yet, retry.
> - listFiles(). Skip for now. Not currently implemented in S3Guard. I will 
> create a separate JIRA for this as it will likely require interface changes 
> (i.e. prefix or subtree scan).
> We may miss some cases initially and we should do failure injection testing 
> to make sure we're covered.  Failure injection tests can be a separate JIRA 
> to make this easier to review.
> We also need basic configuration parameters around retry policy.  There 
> should be a way to specify maximum retry duration, as some applications would 
> prefer to receive an error eventually, than waiting indefinitely.  We should 
> also be keeping statistics when inconsistency is detected and we enter a 
> retry loop.



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

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to