I've updated the proposal with suggestions from Lari about utilization based rate limit exceptions on clients, along with a minor change in the blocking section to ensure ordering is maintained. Please have a look again.
Regarding this comment: > Well, even if we have throttle producer protocol, if client app is keep > producing messages then client app will see high timeout and to fast fail > this issue, Pulsar Client has internal producer queue and client can always > tune that queue. once that queue is fail, client can configure to fast fail > or wait by blocking client thread but in both ways, client application will > be aware that publish is taking longer time and client app can always do > backoff if needed, So, this is very well known issue and it's already > solved in Pulsar. The core issue here is about communicating back to the client about throttling, which is missing today. Yes, clients can tune their send timeouts and pending queue size and rely solely on timeouts, but that wastes a lot of resources.. If clients were aware of throttling, i.e. the server is not reading any more messages anyway, then the client can make smart decisions to fail fast etc. For example, suppose a client has a contract with its upstream components about retries, then when the client is well aware of throttling, it can inform its upstream about the same as well and fail fast rather than holding on pending connections until the timeout. This is especially true when a REST bus system is using pulsar as a backend and the HTTP call does not exit until a send receipt from pulsar is received. Moreover, if you now combine this "rely on pending queue size to fail fast or block" approach with "separate client per topic or partition to segregate TCP connection" approach, it leads to more issues, specifically around memory usage. If an app has to produce to 100 partitions, it now has to divide the available memory it has by 100 while setting for each individual pulsar client. This may be very suboptimal. Or otherwise, the app will have to make some assumptions and oversubscribe the available memory between those 100 clients which can lead to OOM if many partitions are throttling. Hope this helps and gives more context around how the PIP is useful. Regards On Sat, Oct 5, 2024 at 12:53 PM Girish Sharma <scrapmachi...@gmail.com> wrote: > Hi Rajan, > Thanks for taking the time and going through the PIP. > > >>> Well, even if we have throttle producer protocol, if client app is keep > >>> producing messages then client app will see high timeout and to fast > fail > >>> this issue, Pulsar Client has internal producer queue and client can > always > >>> tune that queue. once that queue is fail, client can configure to fast > fail > >>> or wait by blocking client thread but in both ways, client application > will > >>> be aware that publish is taking longer time and client app can always > do > >>> backoff if needed, So, this is very well known issue and it's already > >>> solved in Pulsar. > Your github comments are missing this point about the client timeout, > producer queue etc. Could you please paste it there itself so that we can > keep the discussion contained at one place? > > Regards > > On Sat, Oct 5, 2024 at 4:58 AM Rajan Dhabalia <rdhaba...@apache.org> > wrote: > >> Hi Girish, >> >> I have gone through the proposal and you mentioned few problems as a >> motivation of this improvements >> >> >> Noisy neighbors - Even if one topic is exceeding the quota, since the >> entire channel read is paused, all topics sharing the same connect (for >> example - using the same java client object) get rate limited. >> >> I don't think it's a noisy neighbor issue. There are many ways: clients >> can >> use a separate connection for different topics by increasing the number of >> connections and more specifically create Cache of PulsarClient objects to >> manage topics belonging to different usecases. If you use one channel for >> different tenants/usecases and if they get impacted then it's not a noisy >> neighbor but the application might need design improvement. >> For example: If client app use the same topic for different usecases then >> all usecases can be impacted by each other, and that doesn't mean Pulsar >> has a noisy neighbor issue but it needs a design change to use separate >> topics for each usecase. So, this challenge is easily achievable. >> >> >> Unaware clients - clients are completely unaware that they are being >> rate limited. This leads to all send calls taking super long time or >> simply >> timing out... they can either fail fast or induce back-pressure to their >> upstream. >> >> Well, even if we have throttle producer protocol, if client app is keep >> producing messages then client app will see high timeout and to fast fail >> this issue, Pulsar Client has internal producer queue and client can >> always >> tune that queue. once that queue is fail, client can configure to fast >> fail >> or wait by blocking client thread but in both ways, client application >> will >> be aware that publish is taking longer time and client app can always do >> backoff if needed, So, this is very well known issue and it's already >> solved in Pulsar. >> >> and we should have server side metrics for topic throttling which should >> give a clear picture of msgRate and throttling for any further debugging. >> >> So, I think every issue is already addressed and I don't see any specific >> need for these issue. >> >> Thanks, >> Rajan >> >> >> >> On Fri, Oct 4, 2024 at 3:45 PM Lari Hotari <lhot...@apache.org> wrote: >> >> > Great work on this proposal, Girish! >> > >> > This improvement addresses a crucial aspect of Pulsar's functionality. >> > You're effectively bridging an important gap in Pulsar's producer flow >> > control. This addition will improve the ability to set and meet SLAs >> across >> > various Pulsar use cases, which is invaluable for many of our users. >> > >> > Thank you for driving this important improvement. It's contributions >> like >> > these that continue to enhance Pulsar's robustness and flexibility. >> > >> > Looking forward to seeing this develop further. >> > >> > -Lari >> > >> > On 2024/10/04 14:48:09 Girish Sharma wrote: >> > > Hello Pulsar Community, >> > > >> > > I would like to propose a new improvement for Pulsar protocol related >> to >> > > rate limiting that the broker imposes to maintain quality of service. >> > This >> > > proposal adds a new binary protocol command pair and corresponding >> server >> > > and java client changes. With the new protocol command, clients would >> be >> > > able to understand that they are breaching the quota for a topic and >> take >> > > action accordingly. >> > > >> > > The full proposal can be found at >> > > https://github.com/apache/pulsar/pull/23398 >> > > Direct link to rendered markdown with mermaid flowcharts - >> > > https://github.com/grssam/pulsar/blob/rl-protocol/pip/pip-385.md >> > > >> > > Please share your thoughts on this proposal along with any concerns or >> > > suggestions. >> > > >> > > Regards >> > > -- >> > > Girish Sharma >> > > >> > >> > > > -- > Girish Sharma > -- Girish Sharma