hadoop-common-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Aaron Fabbri (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HADOOP-13761) S3Guard: implement retries for DDB failures and throttling; translate exceptions
Date Fri, 16 Feb 2018 06:32:00 GMT

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

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

About to post v6 patch.  Changes from v5:

- Run testOpenFailOnRead() in a loop with 0.5 failure probability. (My idea about making failpoint
probing more deterministic wasn't worth the trouble.)  It runs 20 times, but I tested it with
500 iterations.  I no longer assert non-s3guard fails since it is now RNG.
- Add status=404 failpoint to InconsistentAmazonS3Client#getObject(): that is the first GET
AFAICT
- Add FNFE failpoint to to skip() (which may use read() underneath to advance stream, I believe).
- Add s3guard-specific retry() around lazySeek().  Now both callers of reopen() are covered
in a retry, only when S3Guard is enabled.

I'll add the v6 patch as well as a v5-to-v6 diff so you can skim the changes..(diff-diff is
not exact: it contains a change or two from your v5 tweaks).


> 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-004-to-005.patch, HADOOP-13761-005-to-006-approx.diff.txt,
HADOOP-13761-005.patch, HADOOP-13761.001.patch, HADOOP-13761.002.patch, HADOOP-13761.003.patch,
HADOOP-13761.004.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-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


Mime
View raw message