[ https://issues.apache.org/jira/browse/HADOOP-13761?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16367954#comment-16367954 ]
Aaron Fabbri commented on HADOOP-13761: --------------------------------------- {quote}Mark up the seek/read calls with @RetryPolicy, so it's easier to see what happens transitively {quote} Humm.. these codepaths are sort of a new case. They retry sometimes, depending on s3guard state. Should we add some new annotations S3GuardRetry[Mixed,Raw,Translated]? Or just call them Retry with a clarifying comment–since they "are covered WRT retries that we actually want". I'm thinking the latter. {quote}only have the s3guardInvoker field set to S3GuardExistsRetryPolicy if S3Guard is enabled, otherwise just set s3guardInvoker=invoker. and maybe just call it readInvoker then you can go context.readInvoker on those calls without much logic in the input stream. {quote} I thought about this. I'm concerned about misuse of the new retry policy breaking fail-fast behavior where it is needed, so I opted to make each codepath very explicit where the change in behavior happens and why. Let me take a look at it again. {quote}the invoked operations themselves can be stuck inline {quote} This statement is true. ;) Thanks for taking the time to think through this stuff–I wanted extra eyes on it. I was a bit on the fence about the read failpoints, not knowing if they really occur in the wild: getObject() seems to actually open an HTTP stream and read usually consumes that, so not sure you can really get FNFE there. Wondering if we could get throttling there too–but it depends on unknown S3 service behavior (i.e. does throttling only affect new read sessions and metadata requests, or can it affect an open HTTP/TCP stream. I suppose we could have a reconnect that hits another shard and gets FNFE?). > 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-006.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