-
Notifications
You must be signed in to change notification settings - Fork 8.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
HADOOP-17764. S3AInputStream read does not re-open the input stream on the second read retry attempt #3109
Conversation
163bfd4
to
7ba901e
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for this.
Failure handling here is tricky here because it is relatively rare that it does happen, so are real world coverage maybe in adequate. One problem we have encountered in the past, which relates to this very section of code, is the challenge of making sure that retrys don't try and recycle the same HTTP connection which has just failed for a network error. There is something all the reviews will need to satisfy themselves with.
I say all the reviewers, as as I am on holiday this week. I nominate: @mukund-thakur @mehakmeet and @bogthe ... if any of you can comment, please do so
Testing
-
I can see you are deep into the S3A code base. This is good. It does mean though that you get to declare which S3 region you ran the full
mvn verify
test suites against. Thanks. -
My opinions on mocking are fairly well known, as in "they are an utter nightmare from both future maintenance and back porting perspectives". This is that particular problem for me as I still have to backport stuff to mockito 1.x; and inevitably mocking-based tests will fail, leaving me to work out whether this is a real regression or just over-brittle tests. They are also incredibly brittle to any minor change in how different layers work. eg: any change in the s3 integration, a suite of mock tests suddenly fail and the developer is left to sort out if this is a regression or false alarm. If they cause too many problems I will just delete them.
I understand and we do use them occasionally for fault injection, but like to try and write code which can be tested without going near them; the callback interface design pattern has that as one of its goals.
S3AInputStreams now take an implementation of S3AInputStream.InputStreamCallbacks
to open files. You shouldn't need to bind an S3AFS to a mock S3 client (with all the maintenance costs that implies) when a custom implementation of S3AInputStream.InputStreamCallbacks
can be written for the test.
Accordingly: I don't think that you need to mock classes when 1+ callback implementation can trigger the failures in a way we can both backport and maintain in the future. This also means we don't need to make any more private methods VisibleForTesting.
- no idea why yetus is complaining about TestS3AAWSCredentialsProvider.java. Unrelated, obviously.
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java
Outdated
Show resolved
Hide resolved
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java
Outdated
Show resolved
Hide resolved
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
Outdated
Show resolved
Hide resolved
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java
Outdated
Show resolved
Hide resolved
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java
Outdated
Show resolved
Hide resolved
…n the second read retry attempt
Thanks for the review! |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for this PR too!
To start off, I'll reiterate what Steve said, once you touch S3a code you would have to run the testing suite and comment on this PR what region you ran them and if they were successful or not.
For the test suite that was added TestS3AInputStreamRetry
:
- There's still a bit too much mocking going on. Out of everything the most important part is
.getObjectContent()
as this is what theread
function uses, everything else that's not needed is just added complexity to the test; - It would be good to have some test cases which cover failure scenarios, not just the failures that are expected on "the happy path";
} catch (EOFException e) { | ||
return -1; | ||
} catch (IOException e) { | ||
onReadFailure(e, length, e instanceof SocketTimeoutException); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I see you're calling onReadFailure
with length
instead of 1
. Any reasoning for this?
That is used to calculate the range for a GetObjectRequest
when the stream is being reopened. If it's intended then I would be curious of the impact it has on larger objects, have you done any testing around it?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This method is used on both read()
and read(b, off, len)
we use length = 1 for read()
and variable length for read(b, off, len)
. It's intended to keep the current behaviour
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@bogthe see calculateRequestLimit()
; its only part of the information used to calculate the range. on random IO the range will be that of fs.s3a.readahead (unless the file is shorter). It's more important in read(buffer[])
as then the full buffer length is requested; if something is asking for a 2MB buffer then that's what it gets.
}; | ||
} | ||
|
||
// Get mocked S3ObjectInputStream where we can trigger IOException to sumulate the read failure |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit*: sumulate
-> simulate
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Updated
@VisibleForTesting | ||
protected S3AReadOpContext createReadContext( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm not really convinced that this is needed. Check the main comment for details.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since the read context attach the retry policy and the retry logic that being used by the input stream, this is seems to be the entry where we can get for the test without exposing many internal implementations.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Afraid we currently do. However, if we move to a builder API for that ReadOpContext then the test could construct something very minimal (would only need the Invoker ref). I'd support that change here as it would help future tests.
Hi @bogthe thanks for the feedback. And for the testing part:
|
Really a corner case scenario. Nice catch. Though I was wondering, can the same be achieved by always re-opening the stream and throwing exception such that invoker does the retry. |
} catch (EOFException e) { | ||
return -1; | ||
} catch (IOException e) { | ||
onReadFailure(e, length, e instanceof SocketTimeoutException); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@bogthe see calculateRequestLimit()
; its only part of the information used to calculate the range. on random IO the range will be that of fs.s3a.readahead (unless the file is shorter). It's more important in read(buffer[])
as then the full buffer length is requested; if something is asking for a 2MB buffer then that's what it gets.
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java
Outdated
Show resolved
Hide resolved
@VisibleForTesting | ||
protected S3AReadOpContext createReadContext( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Afraid we currently do. However, if we move to a builder API for that ReadOpContext then the test could construct something very minimal (would only need the Invoker ref). I'd support that change here as it would help future tests.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You've done some good work here identifying a problem & coming up with a fix.
Afraid Yetus is unappy about style issues, including indentation and line with. You do you have to fix these. Line length it's a long-standing source of contention, but until the policy changes we have to live with it except in cases where it would actually make readability much worse.
I really like what you've done with the new test; it's set an example for new ones.
Regarding the production code: do we actually need to have that second attempt explicitly performed, now that it is identical to the first.
instead it can invoked once, and Invoker.retry() handling the failure there (SocketTimeoutException) is an idempotent retry entry.
The sole disadvantage with cutting it would be that the first retry triggers a sleep() of a few hundred millis. That's all though, and any costs related to re-negotiating a new HTTPS connection are already there. So let's just do that: remove the second attempt entirely, but make sure the connection is closed on a socket timeout so we don't get the same connection again.
Now, big question: Which S3 endpooint are you running the mvn verify
tests against, and what parameters did you pass the maven?
We need something like "aws ireland" and -Dparallel-tests -DtestsThreadCount=7 -Dmarkers=keep -Dscale
. We need the endpoint to make sure you've done that you diligence and the parameters so we can see what's the coverage is and what different configurations should any other people running the tests try.
(Ignore that Yetus complaint about deprecation. I understand what the issue is -created HADOOP-17768).
Agreed on adding slight delay on second retry won't do any harm, I have updated the main code to only just throwing the exception without having any manual try. I'm running the test on |
I got an error that requires me to provide DynamoDB table to run the S3A test, is this expected? |
thanks for running the tests.
|
Here are the failing tests:
|
Thanks for the detials.
probably a variant on (https://issues.apache.org/jira/browse/HADOOP-17628)[https://issues.apache.org/jira/browse/HADOOP-17628]: we need to make the test directory tree smaller. it'd make the test faster for all too. Patches welcome :)
you aren't alone here; its read() returning an undeful buffer. We can't switch to readFully() as the test really wants to call read(). Ignore it. Happens when I use many threads in parallel runs.
same transient; ignore
Looks like you are seeing https://issues.apache.org/jira/browse/HADOOP-17457
This is new. Can you file a JIRA with the stack trace, just so we have a history of it.
This is very new, which makes it interesting. If you are seeing this, it means it may surface in the wild. I suspect it's because you've got an IAM permission set up blocking access to this (public) dataset. Can you file a JIRA with this too? I'll probably give you some tasks to find out more about the cause, but at least there'll be an indexed reference to the issue. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK. production code all good; suggested one more test case for the tests. Other than that it's down to the minor style nits, including yetus complaints in https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3109/9/artifact/out/results-checkstyle-hadoop-tools_hadoop-aws.txt
+1 pending those changes; as usual, you'll need to do a test run in the hadoop-aws module. Don't worry about tests which want dynamoDB -but do tell me which of these tests there are. They should all be skipping/downgrading
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java
Outdated
Show resolved
Hide resolved
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java
Outdated
Show resolved
Hide resolved
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java
Outdated
Show resolved
Hide resolved
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java
Outdated
Show resolved
Hide resolved
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java
Outdated
Show resolved
Hide resolved
Final changes LGTM; if the test run is happy then it's ready to commit |
I have re-run all the
For the second test that you mentioned: https://issues.apache.org/jira/browse/HADOOP-17457, |
ok. just fix those line length checkstyles and we are good to merge. As these are just formatting, no need to rerun the tests. regarding the second failure -I've updated the JIRA to "lets just cut it"; it's part of the fault injection of inconsistencies we needed to test S3Guard. Now s3 is consistent, just a needless failure |
There seems to be no complaint from Yetus now :) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yep, yetus is happy.
FWIW, sometimes yetus is wrong, but we try to minimise all complaints to the final few
…n the second read retry attempt (#3109) Contributed by Zamil Majdy. Change-Id: I680d9c425c920ff1a7cd4764d62e10e6ac78bee4
thanks. Merged to trunk then (locally) cherrypicked that to branch-3.3, ran the new test (and only that test!) and pushed up. @majdyz thanks! your contribution is appreciated |
Note: we could also backport to 3.2.x if you want to cherrypick the 3.3 changes and retest...then provide a new PR. |
…n the second read retry attempt (apache#3109) Contributed by Zamil Majdy.
…put stream on the second read retry attempt (apache#3109) Contributed by Zamil Majdy. Change-Id: I680d9c425c920ff1a7cd4764d62e10e6ac78bee4
NOTICE
Please create an issue in ASF JIRA before opening a pull request,
and you need to set the title of the pull request which starts with
the corresponding JIRA issue number. (e.g. HADOOP-XXXXX. Fix a typo in YYY.)
For more details, please see https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute