You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
I searched in the issues and found nothing similar.
Read release policy
I understand that unsupported versions don't get bug fixes. I will attempt to reproduce the issue on a supported version of Pulsar client and Pulsar broker.
Version
> 3.x
Minimal reproduce step
It sounds like your current token bucket rate-limiter suffers from issues due to eventual consistency, leading to a situation where a sudden traffic spike causes an excessive increase in pendingConsumedTokens, pushing the bucket into deep negative values. This, in turn, results in prolonged pauses in publish/dispatch operations.
Since tokens are refreshed at fixed intervals, a large negative value requires multiple refresh cycles to recover.
This leads to long pauses, causing the publish/dispatch system to get stuck.
3. Previous RateLimiter Had a Controlled Spike Handling:
Previous Rate Limiter was able to handle the spike and it doesn't; allow topic or dispatcher to stuck without dispatching messages.
Alternative: Bound Negative Token Values: Implement a threshold to prevent tokens from going excessively negative, limiting recovery time.
Let's step back and fix this issue fundamentally without adding any more patches, and we should at least revert to Dispatch Rate-Limiter until we have a matured solution in place because the current Rate-Limiter has certainly changed the behavior when the broker hits the spike in dispatch or publish.
This can be reproduced with below test
void testAsyncToken() throws Exception {
int rate = 2000;
int resolutionTimeNano = 8;
asyncTokenBucket = AsyncTokenBucket.builder().rate(rate).ratePeriodNanos(TimeUnit.SECONDS.toNanos(1)).clock(
new DefaultMonotonicSnapshotClock(TimeUnit.MILLISECONDS.toNanos(resolutionTimeNano), System::nanoTime))
.build();
for (int j = 0; j < (1000); j++) {
for (int i = 0; i < (1000); i++) {
long token = asyncTokenBucket.getTokens();
if (token < 0) {
// sleep to allow add more tokens
Thread.sleep(resolutionTimeNano * 5);
assertTrue(asyncTokenBucket.getTokens() > 0);
}
// calling consumeTokens iteratively to simulate calling this method multiple times from multiple
// threads
asyncTokenBucket.consumeTokens(100);
}
}
}
What did you expect to see?
Rate-Limiter should not be blocked for long time but like previous Rate-Limiter it should handle spike and allow configured rate at every second(or configured rate time)
What did you see instead?
Rate-Limiter should blocks publish/consume for long time
Anything else?
No response
Are you willing to submit a PR?
I'm willing to submit a PR!
The text was updated successfully, but these errors were encountered:
It sounds like your current token bucket rate-limiter suffers from issues due to eventual consistency, leading to a situation where a sudden traffic spike causes an excessive increase in pendingConsumedTokens, pushing the bucket into deep negative values. This, in turn, results in prolonged pauses in publish/dispatch operations.
This seems to already be fixed by #23930. @rdhabalia would you be able to test with #23930?
Search before asking
Read release policy
Version
> 3.x
Minimal reproduce step
It sounds like your current token bucket rate-limiter suffers from issues due to eventual consistency, leading to a situation where a sudden traffic spike causes an excessive increase in
pendingConsumedTokens
, pushing the bucket into deep negative values. This, in turn, results in prolonged pauses in publish/dispatch operations.Fundamental Issues Identified:
1. Eventual Consistency Across Multiple Threads:
Each thread maintains its own counter for
pendingConsumedTokens
, which gets aggregated periodically.During a spike, multiple producers concurrently increase pendingConsumedTokens, leading to an aggregated count that significantly exceeds the configured rate.
2. Deep Negative Token Count Problem:
Since tokens are refreshed at fixed intervals, a large negative value requires multiple refresh cycles to recover.
This leads to long pauses, causing the publish/dispatch system to get stuck.
3. Previous RateLimiter Had a Controlled Spike Handling:
Previous Rate Limiter was able to handle the spike and it doesn't; allow topic or dispatcher to stuck without dispatching messages.
Alternative:
Bound Negative Token Values: Implement a threshold to prevent tokens from going excessively negative, limiting recovery time.
Let's step back and fix this issue fundamentally without adding any more patches, and we should at least revert to Dispatch Rate-Limiter until we have a matured solution in place because the current Rate-Limiter has certainly changed the behavior when the broker hits the spike in dispatch or publish.
This can be reproduced with below test
What did you expect to see?
Rate-Limiter should not be blocked for long time but like previous Rate-Limiter it should handle spike and allow configured rate at every second(or configured rate time)
What did you see instead?
Rate-Limiter should blocks publish/consume for long time
Anything else?
No response
Are you willing to submit a PR?
The text was updated successfully, but these errors were encountered: