[ https://issues.apache.org/jira/browse/HADOOP-13761?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16366138#comment-16366138 ]
Steve Loughran commented on HADOOP-13761: ----------------------------------------- I think you forgot to hit the submit button. I do that all too often. Here's my review. I've done it in the IDE and am attaching (a) the new patch and (b) the diff between yours and mine. That'll simplify your reviewing my changes. One thing I've not done is the change in S3AInputStream invoke on read, where I have three comments # I think the S3Guard invoker should be around/in lazySeek(nextReadPos, len) -that is where the GET is initiated. The read() is just trying to work with an existing stream. # what happens in the state that the file really isn't there? That is: how rapidly do we expect to timeout on retries for it? # context.dstFileStatus.getPath().toString() => move to a const to avoid recreating it on every read() call. The key issues is #1: is the retry in the right place? I don't think so. h2. changes in the patch h3. S3AFileSystem * L83 remove unused import * L704 reformatted new input stream constructor invocation for clarity; do like the new structure BTW. * L2106 reinstated the log debug statement, unless you really don't want it. h3. ITestS3AInconsistency * L120: this stream needs closing in a try-with-resources call * L131 the fail() call should nest the inner exception as the cause * move waitUntilDeleted to java 8 lambda h3. Other * InconsistentS3Object: IDE complains that the LOG is non serializable; mark as transient to get it to quieten down * AbstractCommitITest, ITestDynamoDBMetadataStoreScale, FailureInjectionPolicy, S3AOpContext, S3GuardExistsRetryPolicy : import ordering * InconsistentS3Object: import ordering & tell IDE to stop complaining about serialization issues on this subclass * S3ARetryPolicy - fixed up something wrong in the original code; duplicate AWSRedirectException in map > 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.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-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org