[
https://issues.apache.org/jira/browse/HADOOP-13761?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16365040#comment-16365040
]
Aaron Fabbri commented on HADOOP-13761:
---------------------------------------
Just posted v3 patch.
{noformat}
Feedback from previous patch
- info -> debug
- switch to provisionTableBlocking() in DDBMS#updateParameters()
- use getClass().getSimpleName() for test folder
- Add warning to testing doc: -Dscale + -Ds3guard + -Ddynamo requires you use
a private table that you can afford to mess up.
New retry policy when failing read() with S3Guard.
- ITestS3AInconsistency#testOpenFailOnRead() shows the new retry behavior maybe
read that first.
- Add new failure injection around S3Object API.
- New class InconsistentS3Object subclasses the AWS S3Object,
and produces the new InconsistentS3InputStream.
- InconsistentS3Input stream just wraps S3ObjectInputStream, adding failpoints
to read() and read(byte[], int, int).
- Factor some S3AInputStream constructor args into a new struct, S3AOpContext.
(I expect we will want to subclass this and add more per-operation state
when we rewrite rename(). Any per-op state, such as S3Guard details on
expected metadata, can go here.)
- Add retry to S3AInputStream#onReadFailure(), only when metadata store is in
use. Currently we infer that isS3GuardEnabled ==> S3guard had metadata for
the file, which is how the code is currently structured, but we could make
this more explicit.
- Move failure injection config / policy into separate class,
FailureInjectionPolicy
- Refactor S3ARetryPolicy to make the new S3GuardExistsRetryPolicy possible.
{noformat}
> 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,
> HADOOP-13761.003.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: [email protected]
For additional commands, e-mail: [email protected]