Closing this discussion thread and the PIP. Apart from the discussion present in this thread, I presented the detailed requirements in a dev meet on 23rd November and the conclusion was that we will actually go ahead and implement the requirements in pulsar itself. There was a pre-requisite of refactoring rate limiter codebase which is already covered by Lari in PIP-322.
I will be creating a new parent PIP soon about the high level requirements. Thank you everyone who participated in the thread and the discussion on 23rd dev meeting. Regards On Thu, Nov 23, 2023 at 8:26 PM Girish Sharma <scrapmachi...@gmail.com> wrote: > I've captured our requirements in detail in this document - > https://docs.google.com/document/d/1-y5nBaC9QuAUHKUGMVVe4By-SmMZIL4w09U1byJBbMc/edit > Added it to agenda document as well. Will join the meeting and discuss. > > Regards > > On Wed, Nov 22, 2023 at 10:49 PM Lari Hotari <lhot...@apache.org> wrote: > >> I have written a long blog post that contains the context, the summary >> of my view point about PIP-310 and the proposal for proceeding: >> >> https://codingthestreams.com/pulsar/2023/11/22/pulsar-slos-and-rate-limiting.html >> >> Let's discuss this tomorrow in the Pulsar community meeting [1]. Let's >> coordinate on Pulsar Slack's #dev channel if the are issues in joining >> the meeting. >> See you tomorrow! >> >> -Lari >> >> 1 - https://github.com/apache/pulsar/wiki/Community-Meetings >> >> On Mon, 20 Nov 2023 at 20:48, Lari Hotari <lhot...@apache.org> wrote: >> > >> > Hi Girish, >> > >> > replies inline and after that there are some updates about my >> > preparation for the community meeting on Thursday. (there's >> > https://github.com/lhotari/async-tokenbucket with a PoC for a >> > low-level high performance token bucket implementation) >> > >> > On Sat, 11 Nov 2023 at 17:25, Girish Sharma <scrapmachi...@gmail.com> >> wrote: >> > > Actually, the capacity is meant to simulate that particular rate >> limit. if >> > > we have 2 buckets anyways, the one managing the fixed rate limit part >> > > shouldn't generally have a capacity more than the fixed rate, right? >> > >> > There are multiple ways to model and understand a dual token bucket >> > implementation. >> > I view the 2 buckets in a dual token bucket implementation as separate >> > buckets. They are like an AND rule, so if either bucket is empty, >> > there will be a need to pause to wait for new tokens. >> > Since we aren't working with code yet, these comments could be out of >> context. >> > >> > > I think it can be done, especially with that one thing you mentioned >> about >> > > holding off filling the second bucket for 10 minutes.. but it does >> become >> > > quite complicated in terms of managing the flow of the tokens.. >> because >> > > while we only fill the second bucket once every 10 minutes, after the >> 10th >> > > minute, it needs to be filled continuously for a while (the duration >> we >> > > want to support the bursting for).. and the capacity of this second >> bucket >> > > also is governed by and exactly matches the burst value. >> > >> > There might not be a need for this complexity of the "filling bucket" >> > in the first place. It was more of a demonstration that it's possible >> > to implement the desired behavior of limited bursting by tweaking the >> > basic token bucket algorithm slightly. >> > I'd rather avoid this additional complexity. >> > >> > > Agreed that it is much higher than a single topics' max throughput.. >> but >> > > the context of my example had multiple topics lying on the same >> > > broker/bookie ensemble bursting together at the same time because >> they had >> > > been saving up on tokens in the bucket. >> > >> > Yes, that makes sense. >> > >> > > always be a need to overprovision resources. You usually don't want to >> > > > go beyond 60% or 70% utilization on disk, cpu or network resources >> so >> > > > that queues in the system don't start to increase and impacting >> > > > latencies. In Pulsar/Bookkeeper, the storage solution has a very >> > > > effective load balancing, especially for writing. In Bookkeeper each >> > > > ledger (the segment) of a topic selects the "ensemble" and the >> "write >> > > > quorum", the set of bookies to write to, when the ledger is opened. >> > > > The bookkeeper client could also change the ensemble in the middle >> of >> > > > a ledger due to some event like a bookie becoming read-only or >> > > > >> > > >> > > While it does do that on complete failure of bookie or a bookie disk, >> or >> > > broker going down, degradations aren't handled this well. So if all >> topics >> > > in a bookie are bursting due to the fact that they had accumulated >> tokens, >> > > then all it will lead to is breach of write latency SLA because at one >> > > point, the disks/cpu/network etc will start choking. (even after >> > > considering the 70% utilization i.e. 30% buffer) >> > >> > Yes. >> > >> > > That's only in the case of the default rate limiter where the >> tryAcquire >> > > isn't even implemented.. since the default rate limiter checks for >> breach >> > > only at a fixed rate rather than before every produce call. But in >> case of >> > > precise rate limiter, the response of `tryAcquire` is respected. >> > >> > This is one of many reasons why I think it's better to improve the >> > maintainability of the current solution and remove the unnecessary >> > options between "precise" and the default one. >> > >> > > True, and actually, due to the fact that pulsar auto distributes >> topics >> > > based on load shedding parameters, we can actually focus on a single >> > > broker's or a single bookie ensemble and assume that it works as we >> scale >> > > it. Of course this means that putting a reasonable limit in terms of >> > > cpu/network/partition/throughput limits at each broker level and >> pulsar >> > > provides ways to do that automatically. >> > >> > We do have plans to improve the Pulsar so that things would simply >> > work properly under heavy load. Optimally, things would work without >> > the need to tune and tweak the system rigorously. These improvements >> > go beyond rate limiting. >> > >> > > While I have shared the core requirements over these threads (fixed >> rate + >> > > burst multiplier for upto X duration every Y minutes).. We are >> finalizing >> > > the details requirements internally to present. As I replied in my >> previous >> > > mail, one outcome of detailed internal discussion was the discovery of >> > > throughput contention. >> > >> > Sounds good. Sharing your experiences is extremely valuable. >> > >> > > We do use resource groups for certain namespace level quotas, but >> even in >> > > our use case, rate limiter and resource groups are two separate >> tangents. >> > > At least for foreseeable future. >> > >> > At this point they are separate, but there is a need to improve. Jack >> > Vanlightly's blog post series "The Architecture Of Serverless Data >> > Systems" [1] explains the competition in this area very well. >> > Multi-tenant capacity management and SLAs are at the core of >> > "serverless" data systems. >> > >> > 1 - >> https://jack-vanlightly.com/blog/2023/11/14/the-architecture-of-serverless-data-systems >> > >> > > Will close this before the 23rd. >> > >> > Yes. :) >> > >> > > This is actually not a resource group limitation, but a general broker >> > > level example. Even if resource groups weren't in picture, this issue >> > > remains. The fact remains that since we need to have reasonable broker >> > > level limits (as a result of our NFR testing etc), there will be >> clash of >> > > topics where some topics trying to burst are taking up broker's >> capacity of >> > > serving the fixed rate for other topics. This needs special handling >> even >> > > after dual token approach. I will have detailed requirement with >> examples >> > > by 23rd. >> > >> > Sounds good. >> > >> > > > That is the one of the reasons for putting a strict check on the >> amount, >> > > > > duration and frequency of bursting. >> > > > >> > > > Please explain more about this. It would be helpful for me in trying >> > > > to understand your requirements. >> > > > One question about the solution of "strict check on amount, duration >> > > > and frequency of bursting". >> > > > Do you already have a PoC where you have validated that the solution >> > > > actually solves the problem? >> > > > >> > > >> > > No, this is still theoretical based on our understanding of rate >> limiter, >> > > dual tokens, current pulsar code, etc. If topics are allowed to burst >> > > without a real duration based limitation, then the chances of more >> and more >> > > topics contending for broker's actual capacity is high and thus it >> > > hinders/contents with (a) a new topics trying to burst and make use >> of the >> > > bursting feature with its SLA and (b) another topics , not bursting, >> way >> > > within its fixed rate, still being rate limited due to lack of >> capacity >> > > (which is taken up by bursting topics) at a broker level. >> > >> > There are more factors involved in the bursting behavior and the >> > applying back pressure in Pulsar. >> > If there's proper end-to-end flow control and backpressure, a possible >> > "over commitment" situation could be handled by simply handling the >> > load as fast as possible. The bursting rates aren't necessarily part >> > of strict SLAs so it's possible that even SLAs are met while operating >> > as-fast-as possible. When the Pulsar load balancing is improved, the >> > Pulsar load manager could move load off an overloaded broker and that >> > would help resolve the situation quicker. >> > Later on, the SLAs should become part of the rate limiting and >> > capacity/load management decisions within Pulsar. For example, not all >> > topics need very low latency. When there's information about the >> > expected latency information, it would be possible to priority and >> > optimize the work load in different ways. >> > When things are properly configured, the Pulsar system shouldn't get >> > into a state where the system collapses when it gets overloaded. If >> > brokers crash because of out-of-memory errors, the whole system will >> > quickly collapse, since recovering from a broker crash will increase >> > the overall load. >> > I have doubts that focusing solely on a custom rate limiting >> > implementation wouldn't be impactful. My current assumption is that >> > adding support for bursting to rate limiting will be sufficient for >> > making rate limiting good enough for most Pulsar use cases, and we >> > could continue to improve rate limiting when there's a specific need. >> > I'll be happy to be proven to be wrong and change my opinion when >> > there's sufficient evidence that custom rate limiters are needed in >> > Pulsar. >> > >> > > > Could you simply write a PoC by forking apache/pulsar and changing >> the >> > > > code directly without having it pluggable initially? >> > > > >> > > > >> > > This would be a last resort :) . At most if things don't go as >> desired, >> > > then we would end up doing this and adding plugging logic in a pulsar >> fork. >> > > From a PoC perspective, we may try it out soon. >> > >> > I think it's useful to work with a PoC since it will show the >> > practical limits of using rate limiters for handling SLA and capacity >> > management issues. >> > >> > To start improving the current rate limiter implementation in Pulsar >> > core, I have made a PoC of an asynchronous token bucket implementation >> > class, which is optimized for performance without causing contention >> > in a multi-threaded setup. >> > The repository https://github.com/lhotari/async-tokenbucket contains >> > the PoC code and performance test that measures the token bucket >> > calculation logic overhead. >> > On my laptop, it's possible to achieve about 165M ops/s with 100 >> > threads with a single token bucket. That's to prove that the >> > calculation logic isn't blocking and has very low contention. There's >> > no need for a scheduler to maintain the calculations. A scheduler >> > would be needed in an actual rate limiter implementation to unblock a >> > paused Netty channel by setting auto read to true (in the case where >> > backpressure is applied by setting auto read to false). The >> > AsyncTokenBucket class is intended to be a pure function without side >> > effects. The implementation idea is to use the composition pattern to >> > build higher level rate limiter constructs. >> > >> > Let's meet on Thursday at the Pulsar community meeting (schedule at >> > https://github.com/apache/pulsar/wiki/Community-Meetings) to present >> > our summaries and decide on the next steps together. >> > >> > >> > -Lari >> > > > -- > Girish Sharma > -- Girish Sharma