-
Notifications
You must be signed in to change notification settings - Fork 9.2k
HDFS-16155: Allow configurable exponential backoff in DFSInputStream refetchLocations #3271
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
HDFS-16155: Allow configurable exponential backoff in DFSInputStream refetchLocations #3271
Conversation
|
💔 -1 overall
This message was automatically generated. |
36ae933 to
6c13196
Compare
|
🎊 +1 overall
This message was automatically generated. |
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.
Note: If i were adding this feature as brand new, I probably wouldn't include the * failures here -- the base and exponential are good enough IMO. But I needed this to maintain 100% parity with the existing backoff strategy
9fdd67a to
400b799
Compare
|
🎊 +1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
400b799 to
5ab105f
Compare
|
💔 -1 overall
This message was automatically generated. |
5ab105f to
d51a70f
Compare
|
🎊 +1 overall
This message was automatically generated. |
Hexiaoqiao
left a comment
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 @bbeaudreault for your works here. It makes sense to me almost. I prefer to keep the default action even if we used FetchBlockLocationsRetryer. After the first review, it seems the action is different here? Please correct me if something I missed. Thanks again.
|
|
||
| <property> | ||
| <name>dfs.client.retry.window.max</name> | ||
| <value>2147483647</value> |
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.
The default value for dfs.client.retry.window.max is too high here. In some corner case, it will sleep very long time?
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 so much for the review @Hexiaoqiao. The reason I chose this value was that I wanted the changes in this PR to be totally transparent to existing users -- so the backoff should work exactly as it does today for anyone who upgrades. I don't know how people have tuned their backoffs today, so adding a lower max might affect their configured backoffs. The default case will be well-bounded by the default retries of 3. That said, I agree that there's very little utility in waiting many minutes on a backoff. What if I put this to 30s?
Was this the only concern in terms of the default action? My test case testDefaultRetryPolicy proves that the default case remains unchanged from trunk. The default case was determined based on the comment in DFSInputStream, the old implementation details, and my own testing of the backoff policy prior to this change.
I also created this spreadsheet that helped me to determine how different multiplier values might affect the backoff: https://docs.google.com/spreadsheets/d/1I9ejqDtJ6-krSh-YBt0qHTf3JwZu5zRlrOhbzY0kJAg/edit?usp=sharing
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.
@Hexiaoqiao I've pushed a commit which lowers the window max to 30s. As mentioned above, this may cap some custom backoffs people have configured. But that may be beneficial. It should not affect the default case, given the default of 3 retries does not reach 30s. Let me know if you'd prefer a different default.
| } | ||
|
|
||
| @Test | ||
| public void testDefaultRetryPolicy() { |
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.
Per the comment in the original backoff policy:
// Introducing a random factor to the wait time before another retry. // The wait time is dependent on # of failures and a random factor. // At the first time of getting a BlockMissingException, the wait time // is a random number between 0..3000 ms. If the first retry // still fails, we will wait 3000 ms grace period before the 2nd retry. // Also at the second retry, the waiting window is expanded to 6000 ms // alleviating the request rate from the server. Similarly the 3rd retry // will wait 6000ms grace period before retry and the waiting window is // expanded to 9000ms.
- The first backoff should be between 0-3000ms.
- The second should be 3000 plus a random number between 0-6000ms. So the full range is 3000-9000.
- The third retry should be 6000 plus a random number between 0-9000ms. So the full range is 6000-15000ms.
This test proves that this original retry strategy continues to work with the new code. It's hard to test with randomness, so the random factor is disabled. We're left with only the worst case scenario (if rand() returned 1). See the assertions below to see that the results adhere to the original description above.
|
🎊 +1 overall
This message was automatically generated. |
|
Any other comments on this patch? As tests demonstrate, it should have no impact on existing use-cases, aside from the requested backoff ceiling. It will enable operators to unlock faster retries if desired, and is much easier to read and test code. |
|
@Hexiaoqiao can this be merged? |
|
Thanks @bbeaudreault for your great works here and sorry for the late response. It looks good to me in general. +1 from my side. I would like to wait if any other guys are interested for this improvement. |
|
Thanks for the approval @Hexiaoqiao. Is there a downside to jsut merging this? It's been open for over 6 months, so I doubt anyone else will be jumping in any time soon. |
|
We're closing this stale PR because it has been open for 100 days with no activity. This isn't a judgement on the merit of the PR in any way. It's just a way of keeping the PR queue manageable. |
Per https://issues.apache.org/jira/browse/HDFS-16155, we would like the ability to customize the backoff strategy when BlockMissingException occurs. This can happen when the balancer moves blocks, and in low latency clusters the existing backoff is too conservative. Drastically reducing the existing window base config would help but expose the namenode to a potential DDOS if many blocks became missing, because the current backoff would grow slowly.
Adding a configurable exponential component allows for aggressive early retries that back off quickly enough to mitigate stampeding herds. We make the backoff configurable by adding two new configs:
dfs.client.retry.window.multiplier: defaults to 1 to preserve existing behavior. Increasing this can result in a steeper backoff curve when desireddfs.client.retry.window.max: defaults to Int.MAX to preserve existing behavior. Decreasing this can help put a ceiling on exponential backoffs that could quickly grow to effectively unlimited levels.As described, the default behavior is maintained and I've added a test case to verify that. Someone looking for a more aggressive initial retry that backs off quickly in case of continuous failure could try setting
window.baseto 10,window.multiplierto 5, andwindow.maxto 10000. This would result in a quick initial retry of max 50ms, but quickly backoff to a few seconds within 3 retries.In order to improve the testability of this feature, I pulled out the existing refetchLocations retry configs into a FetchBlockLocationsRetryer class. I also improved the readability of the comment describing the backoff strategy, and fully tested the new retryer in TestFetchBlockLocationsRetryer.