-
Notifications
You must be signed in to change notification settings - Fork 3.3k
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
HBASE-28672 Ensure large batches are not indefinitely blocked by quotas #6003
Conversation
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
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 like this improvement and the behavior you demonstrate in the tests. It seems like a fair kind of latching mechanism for letting requests trickle through.
Previously, this limiting system acted as a gate keeper in terms of placing an effective upper bound on a user. This patch removes that upper bound. Should we consider also introducing explicitly a hard-cap/upper bound?
That's a good question. Despite the existing behavior, I think the intention was always for this to be more of a soft limit — the TimedQuota proto actually calls the field
That said, intention is one thing and reality is another. I could also see people missing the hard limits if that's how they're using quotas. I'd probably prefer to make the addition of a hard limit a separate effort, though, because my guess is that it will actually be quite a bit of work relative to its value. For example, coming up with good enough estimations of IO-heavy request load to earnestly guarantee any sort of "hard limit" is going to be tricky/impossible |
Ideally for a multi request it would be able to execute whatever actions fit within the available quota. This maintains the hard limit but also gives a user more likelihood to eventually finish their request through multiple rpc calls. Similar to the handle of MultiActionResultTooLarge. |
I agree with you, and considered the MARTL approach internally but didn't do a good job of documenting this publicly. My pushback is not that we shouldn't add support for some sort of It's definitely righteous work to also think about improving the server's respect of the limit via request chunking, but I think that's sort of its own task and, given the difference in changeset required, I don't think a lack of MABTL exceptions should block our fixing of this bug |
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.
Sounds good. I agree we shouldn't let this create an impossible to succeed request. If we wanted to do that, we should throw a DoNotRetryException instead of a throttling exception.
// the next request should be rejected | ||
assertThrows(RpcThrottlingException.class, () -> quota.checkBatchQuota(0, 1)); | ||
|
||
Thread.sleep(1000); |
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 think it's preferable not to sleep. Instead usually you can influence the timestamp by adding 1s to it or whatever.
The reason not to sleep is that you're more subject to random Jenkins slowdowns or other problems. We do it in a lot of places of course, but iirc the quota stuff already can do some timestamp manipulation?
Would we have the same problem with write size? |
Oh yeah, good point. I bet we would. I can look into a fix there too |
Actually, there's no realistic problem with write size. At least, no more than read size. Both make very naive lowball estimates based on the number of reads/writes: hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/DefaultOperationQuota.java Lines 191 to 203 in 1ffa127
It's theoretically possible for the estimate to exceed the max IO allowed though, so maybe it's worth putting limits in place for both the read and write size estimates here? What do you guys think? |
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
I pushed an addition which ensures we don't block requests indefinitely for read, write, or request size estimates |
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestDefaultOperationQuota.java
Show resolved
Hide resolved
🎊 +1 overall
This message was automatically generated. |
🎊 +1 overall
This message was automatically generated. |
…as (apache#6003) Co-authored-by: Ray Mattingly <rmattingly@hubspot.com> Signed-off-by: Bryan Beaudreault < bbeaudreault@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
…as (apache#6003) Co-authored-by: Ray Mattingly <rmattingly@hubspot.com> Signed-off-by: Bryan Beaudreault < bbeaudreault@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
…as (apache#6003) Co-authored-by: Ray Mattingly <rmattingly@hubspot.com> Signed-off-by: Bryan Beaudreault < bbeaudreault@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
…as (#6003) Co-authored-by: Ray Mattingly <rmattingly@hubspot.com> Signed-off-by: Bryan Beaudreault < bbeaudreault@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
…as (#6003) Co-authored-by: Ray Mattingly <rmattingly@hubspot.com> Signed-off-by: Bryan Beaudreault < bbeaudreault@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
…as (#6003) Co-authored-by: Ray Mattingly <rmattingly@hubspot.com> Signed-off-by: Bryan Beaudreault < bbeaudreault@apache.org> Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
https://issues.apache.org/jira/browse/HBASE-28672
At my day job we are trying to implement default quotas for a variety of access patterns. We began by introducing a default read IO limit per-user, per-machine — this has been very successful in reducing hotspots, even on clusters with thousands of distinct users.
While implementing a default writes/second throttle, I realized that doing so would put us in a precarious situation where large enough batches may never succeed. If your batch size is greater than your TimeLimiter's max throughput, then you will always fail in the quota estimation stage; this will stick the client in a backoff-and-retry loop until it times out. Meanwhile IO estimates are more optimistic, deliberately, which can let large requests do targeted oversubscription of an IO quota:
This is okay because the Limiter's availability will go negative and force a longer backoff on subsequent requests. I believe this is preferable UX compared to a doomed throttling loop.
In this PR I introduce batch estimation that takes the limiter's max throughput into account, and I've added unit tests to validate the intended behavior.
@ndimiduk @hgromer @bozzkar