Hi, Rajini, Consider modeling as n * 100% unit. For 2), the question is what's causing the I/O threads to be saturated. It's unlikely that all users' utilization have increased at the same. A more likely case is that a few isolated users' utilization have increased. If so, after increasing the number of threads, the admin just needs to adjust the quota for a few isolated users, which is expected and is less work.
Consider modeling as 1 * 100% unit. For 1), all users' quota need to be adjusted, which is unexpected and is more work. So, to me, the n * 100% model seems more convenient. As for future extension to cover network thread utilization, I was thinking that one way is to simply model the capacity as (n + m) * 100% unit, where n and m are the number of network and i/o threads, respectively. Then, for each user, we can just add up the utilization in the network and the i/o thread. If we do this, we don't need a new type of quota. Thanks, Jun On Thu, Mar 2, 2017 at 12:27 PM, Rajini Sivaram <rajinisiva...@gmail.com> wrote: > Jun, > > If we use request.percentage as the percentage used in a single I/O thread, > the total percentage being allocated will be num.io.threads * 100 for I/O > threads and num.network.threads * 100 for network threads. A single quota > covering the two as a percentage wouldn't quite work if you want to > allocate the same proportion in both cases. If we want to treat threads as > separate units, won't we need two quota configurations regardless of > whether we use units or percentage? Perhaps I misunderstood your > suggestion. > > I think there are two cases: > > 1. The use case that you mentioned where an admin is adding more users > and decides to add more I/O threads and expects to find free quota to > allocate for new users. > 2. Admin adds more I/O threads because the I/O threads are saturated and > there are cores available to allocate, even though the number or > users/clients hasn't changed. > > If we allocated treated I/O threads as a single unit of 100%, all user > quotas need to be reallocated for 1). If we allocated I/O threads as n > units with n*100%, all user quotas need to be reallocated for 2), otherwise > some of the new threads may just not be used. Either way it should be easy > to write a script to decrease/increase quotas by a multiple for all users. > > So it really boils down to which quota unit is most intuitive in terms of > configuration. And from the discussion so far, it feels like opinion is > divided on whether quotas should be carved out of an absolute 100% (or 1 > unit) or be relative to the number of threads (n*100% or n units). > > > > On Thu, Mar 2, 2017 at 7:31 PM, Jun Rao <j...@confluent.io> wrote: > > > Another way to express an absolute limit is to use request.percentage, > but > > treat it as the percentage used in a single request handling thread. For > > now, the request handling threads can be just the io threads. In the > > future, they can cover the network threads as well. This is similar to > how > > top reports CPU usage and may be a bit easier for people to understand. > > > > Thanks, > > > > Jun > > > > On Fri, Feb 24, 2017 at 10:31 AM, Jun Rao <j...@confluent.io> wrote: > > > > > Hi, Jay, > > > > > > 2. Regarding request.unit vs request.percentage. I started with > > > request.percentage too. The reasoning for request.unit is the > following. > > > Suppose that the capacity has been reached on a broker and the admin > > needs > > > to add a new user. A simple way to increase the capacity is to increase > > the > > > number of io threads, assuming there are still enough cores. If the > limit > > > is based on percentage, the additional capacity automatically gets > > > distributed to existing users and we haven't really carved out any > > > additional resource for the new user. Now, is it easy for a user to > > reason > > > about 0.1 unit vs 10%. My feeling is that both are hard and have to be > > > configured empirically. Not sure if percentage is obviously easier to > > > reason about. > > > > > > Thanks, > > > > > > Jun > > > > > > On Fri, Feb 24, 2017 at 8:10 AM, Jay Kreps <j...@confluent.io> wrote: > > > > > >> A couple of quick points: > > >> > > >> 1. Even though the implementation of this quota is only using io > thread > > >> time, i think we should call it something like "request-time". This > will > > >> give us flexibility to improve the implementation to cover network > > threads > > >> in the future and will avoid exposing internal details like our thread > > >> pools on the server. > > >> > > >> 2. Jun/Roger, I get what you are trying to fix but the idea of > > >> thread/units > > >> is super unintuitive as a user-facing knob. I had to read the KIP like > > >> eight times to understand this. I'm not sure that your point that > > >> increasing the number of threads is a problem with a percentage-based > > >> value, it really depends on whether the user thinks about the > > "percentage > > >> of request processing time" or "thread units". If they think "I have > > >> allocated 10% of my request processing time to user x" then it is a > bug > > >> that increasing the thread count decreases that percent as it does in > > the > > >> current proposal. As a practical matter I think the only way to > actually > > >> reason about this is as a percent---I just don't believe people are > > going > > >> to think, "ah, 4.3 thread units, that is the right amount!". Instead I > > >> think they have to understand this thread unit concept, figure out > what > > >> they have set in number of threads, compute a percent and then come up > > >> with > > >> the number of thread units, and these will all be wrong if that thread > > >> count changes. I also think this ties us to throttling the I/O thread > > >> pool, > > >> which may not be where we want to end up. > > >> > > >> 3. For what it's worth I do think having a single throttle_ms field in > > all > > >> the responses that combines all throttling from all quotas is probably > > the > > >> simplest. There could be a use case for having separate fields for > each, > > >> but I think that is actually harder to use/monitor in the common case > so > > >> unless someone has a use case I think just one should be fine. > > >> > > >> -Jay > > >> > > >> On Fri, Feb 24, 2017 at 4:21 AM, Rajini Sivaram < > > rajinisiva...@gmail.com> > > >> wrote: > > >> > > >> > I have updated the KIP based on the discussions so far. > > >> > > > >> > > > >> > Regards, > > >> > > > >> > Rajini > > >> > > > >> > On Thu, Feb 23, 2017 at 11:29 PM, Rajini Sivaram < > > >> rajinisiva...@gmail.com> > > >> > wrote: > > >> > > > >> > > Thank you all for the feedback. > > >> > > > > >> > > Ismael #1. It makes sense not to throttle inter-broker requests > like > > >> > > LeaderAndIsr etc. The simplest way to ensure that clients cannot > use > > >> > these > > >> > > requests to bypass quotas for DoS attacks is to ensure that ACLs > > >> prevent > > >> > > clients from using these requests and unauthorized requests are > > >> included > > >> > > towards quotas. > > >> > > > > >> > > Ismael #2, Jay #1 : I was thinking that these quotas can return a > > >> > separate > > >> > > throttle time, and all utilization based quotas could use the same > > >> field > > >> > > (we won't add another one for network thread utilization for > > >> instance). > > >> > But > > >> > > perhaps it makes sense to keep byte rate quotas separate in > > >> produce/fetch > > >> > > responses to provide separate metrics? Agree with Ismael that the > > >> name of > > >> > > the existing field should be changed if we have two. Happy to > switch > > >> to a > > >> > > single combined throttle time if that is sufficient. > > >> > > > > >> > > Ismael #4, #5, #6: Will update KIP. Will use dot separated name > for > > >> new > > >> > > property. Replication quotas use dot separated, so it will be > > >> consistent > > >> > > with all properties except byte rate quotas. > > >> > > > > >> > > Radai: #1 Request processing time rather than request rate were > > chosen > > >> > > because the time per request can vary significantly between > requests > > >> as > > >> > > mentioned in the discussion and KIP. > > >> > > #2 Two separate quotas for heartbeats/regular requests feel like > > more > > >> > > configuration and more metrics. Since most users would set quotas > > >> higher > > >> > > than the expected usage and quotas are more of a safety net, a > > single > > >> > quota > > >> > > should work in most cases. > > >> > > #3 The number of requests in purgatory is limited by the number > of > > >> > active > > >> > > connections since only one request per connection will be > throttled > > >> at a > > >> > > time. > > >> > > #4 As with byte rate quotas, to use the full allocated quotas, > > >> > > clients/users would need to use partitions that are distributed > > across > > >> > the > > >> > > cluster. The alternative of using cluster-wide quotas instead of > > >> > per-broker > > >> > > quotas would be far too complex to implement. > > >> > > > > >> > > Dong : We currently have two ClientQuotaManagers for quota types > > Fetch > > >> > and > > >> > > Produce. A new one will be added for IOThread, which manages > quotas > > >> for > > >> > I/O > > >> > > thread utilization. This will not update the Fetch or Produce > > >> queue-size, > > >> > > but will have a separate metric for the queue-size. I wasn't > > >> planning to > > >> > > add any additional metrics apart from the equivalent ones for > > existing > > >> > > quotas as part of this KIP. Ratio of byte-rate to I/O thread > > >> utilization > > >> > > could be slightly misleading since it depends on the sequence of > > >> > requests. > > >> > > But we can look into more metrics after the KIP is implemented if > > >> > required. > > >> > > > > >> > > I think we need to limit the maximum delay since all requests are > > >> > > throttled. If a client has a quota of 0.001 units and a single > > request > > >> > used > > >> > > 50ms, we don't want to delay all requests from the client by 50 > > >> seconds, > > >> > > throwing the client out of all its consumer groups. The issue is > > only > > >> if > > >> > a > > >> > > user is allocated a quota that is insufficient to process one > large > > >> > > request. The expectation is that the units allocated per user will > > be > > >> > much > > >> > > higher than the time taken to process one request and the limit > > should > > >> > > seldom be applied. Agree this needs proper documentation. > > >> > > > > >> > > Regards, > > >> > > > > >> > > Rajini > > >> > > > > >> > > > > >> > > On Thu, Feb 23, 2017 at 8:04 PM, radai < > radai.rosenbl...@gmail.com> > > >> > wrote: > > >> > > > > >> > >> @jun: i wasnt concerned about tying up a request processing > thread, > > >> but > > >> > >> IIUC the code does still read the entire request out, which might > > >> add-up > > >> > >> to > > >> > >> a non-negligible amount of memory. > > >> > >> > > >> > >> On Thu, Feb 23, 2017 at 11:55 AM, Dong Lin <lindon...@gmail.com> > > >> wrote: > > >> > >> > > >> > >> > Hey Rajini, > > >> > >> > > > >> > >> > The current KIP says that the maximum delay will be reduced to > > >> window > > >> > >> size > > >> > >> > if it is larger than the window size. I have a concern with > this: > > >> > >> > > > >> > >> > 1) This essentially means that the user is allowed to exceed > > their > > >> > quota > > >> > >> > over a long period of time. Can you provide an upper bound on > > this > > >> > >> > deviation? > > >> > >> > > > >> > >> > 2) What is the motivation for cap the maximum delay by the > window > > >> > size? > > >> > >> I > > >> > >> > am wondering if there is better alternative to address the > > problem. > > >> > >> > > > >> > >> > 3) It means that the existing metric-related config will have a > > >> more > > >> > >> > directly impact on the mechanism of this io-thread-unit-based > > >> quota. > > >> > The > > >> > >> > may be an important change depending on the answer to 1) above. > > We > > >> > >> probably > > >> > >> > need to document this more explicitly. > > >> > >> > > > >> > >> > Dong > > >> > >> > > > >> > >> > > > >> > >> > On Thu, Feb 23, 2017 at 10:56 AM, Dong Lin < > lindon...@gmail.com> > > >> > wrote: > > >> > >> > > > >> > >> > > Hey Jun, > > >> > >> > > > > >> > >> > > Yeah you are right. I thought it wasn't because at LinkedIn > it > > >> will > > >> > be > > >> > >> > too > > >> > >> > > much pressure on inGraph to expose those per-clientId metrics > > so > > >> we > > >> > >> ended > > >> > >> > > up printing them periodically to local log. Never mind if it > is > > >> not > > >> > a > > >> > >> > > general problem. > > >> > >> > > > > >> > >> > > Hey Rajini, > > >> > >> > > > > >> > >> > > - I agree with Jay that we probably don't want to add a new > > field > > >> > for > > >> > >> > > every quota ProduceResponse or FetchResponse. Is there any > > >> use-case > > >> > >> for > > >> > >> > > having separate throttle-time fields for byte-rate-quota and > > >> > >> > > io-thread-unit-quota? You probably need to document this as > > >> > interface > > >> > >> > > change if you plan to add new field in any request. > > >> > >> > > > > >> > >> > > - I don't think IOThread belongs to quotaType. The existing > > quota > > >> > >> types > > >> > >> > > (i.e. Produce/Fetch/LeaderReplication/FollowerReplication) > > >> identify > > >> > >> the > > >> > >> > > type of request that are throttled, not the quota mechanism > > that > > >> is > > >> > >> > applied. > > >> > >> > > > > >> > >> > > - If a request is throttled due to this io-thread-unit-based > > >> quota, > > >> > is > > >> > >> > the > > >> > >> > > existing queue-size metric in ClientQuotaManager incremented? > > >> > >> > > > > >> > >> > > - In the interest of providing guide line for admin to decide > > >> > >> > > io-thread-unit-based quota and for user to understand its > > impact > > >> on > > >> > >> their > > >> > >> > > traffic, would it be useful to have a metric that shows the > > >> overall > > >> > >> > > byte-rate per io-thread-unit? Can we also show this a > > >> per-clientId > > >> > >> > metric? > > >> > >> > > > > >> > >> > > Thanks, > > >> > >> > > Dong > > >> > >> > > > > >> > >> > > > > >> > >> > > On Thu, Feb 23, 2017 at 9:25 AM, Jun Rao <j...@confluent.io> > > >> wrote: > > >> > >> > > > > >> > >> > >> Hi, Ismael, > > >> > >> > >> > > >> > >> > >> For #3, typically, an admin won't configure more io threads > > than > > >> > CPU > > >> > >> > >> cores, > > >> > >> > >> but it's possible for an admin to start with fewer io > threads > > >> than > > >> > >> cores > > >> > >> > >> and grow that later on. > > >> > >> > >> > > >> > >> > >> Hi, Dong, > > >> > >> > >> > > >> > >> > >> I think the throttleTime sensor on the broker tells the > admin > > >> > >> whether a > > >> > >> > >> user/clentId is throttled or not. > > >> > >> > >> > > >> > >> > >> Hi, Radi, > > >> > >> > >> > > >> > >> > >> The reasoning for delaying the throttled requests on the > > broker > > >> > >> instead > > >> > >> > of > > >> > >> > >> returning an error immediately is that the latter has no way > > to > > >> > >> prevent > > >> > >> > >> the > > >> > >> > >> client from retrying immediately, which will make things > > worse. > > >> The > > >> > >> > >> delaying logic is based off a delay queue. A separate > > expiration > > >> > >> thread > > >> > >> > >> just waits on the next to be expired request. So, it doesn't > > tie > > >> > up a > > >> > >> > >> request handler thread. > > >> > >> > >> > > >> > >> > >> Thanks, > > >> > >> > >> > > >> > >> > >> Jun > > >> > >> > >> > > >> > >> > >> On Thu, Feb 23, 2017 at 9:07 AM, Ismael Juma < > > ism...@juma.me.uk > > >> > > > >> > >> wrote: > > >> > >> > >> > > >> > >> > >> > Hi Jay, > > >> > >> > >> > > > >> > >> > >> > Regarding 1, I definitely like the simplicity of keeping a > > >> single > > >> > >> > >> throttle > > >> > >> > >> > time field in the response. The downside is that the > client > > >> > metrics > > >> > >> > >> will be > > >> > >> > >> > more coarse grained. > > >> > >> > >> > > > >> > >> > >> > Regarding 3, we have `leader.imbalance.per.broker. > > percentage` > > >> > and > > >> > >> > >> > `log.cleaner.min.cleanable.ratio`. > > >> > >> > >> > > > >> > >> > >> > Ismael > > >> > >> > >> > > > >> > >> > >> > On Thu, Feb 23, 2017 at 4:43 PM, Jay Kreps < > > j...@confluent.io> > > >> > >> wrote: > > >> > >> > >> > > > >> > >> > >> > > A few minor comments: > > >> > >> > >> > > > > >> > >> > >> > > 1. Isn't it the case that the throttling time > response > > >> field > > >> > >> > should > > >> > >> > >> > have > > >> > >> > >> > > the total time your request was throttled > irrespective > > of > > >> > the > > >> > >> > >> quotas > > >> > >> > >> > > that > > >> > >> > >> > > caused that. Limiting it to byte rate quota doesn't > > make > > >> > >> sense, > > >> > >> > >> but I > > >> > >> > >> > > also > > >> > >> > >> > > I don't think we want to end up adding new fields in > > the > > >> > >> response > > >> > >> > >> for > > >> > >> > >> > > every > > >> > >> > >> > > single thing we quota, right? > > >> > >> > >> > > 2. I don't think we should make this quota > specifically > > >> > about > > >> > >> io > > >> > >> > >> > > threads. Once we introduce these quotas people set > them > > >> and > > >> > >> > expect > > >> > >> > >> > them > > >> > >> > >> > > to > > >> > >> > >> > > be enforced (and if they aren't it may cause an > > outage). > > >> As > > >> > a > > >> > >> > >> result > > >> > >> > >> > > they > > >> > >> > >> > > are a bit more sensitive than normal configs, I > think. > > >> The > > >> > >> > current > > >> > >> > >> > > thread > > >> > >> > >> > > pools seem like something of an implementation detail > > and > > >> > not > > >> > >> the > > >> > >> > >> > level > > >> > >> > >> > > the > > >> > >> > >> > > user-facing quotas should be involved with. I think > it > > >> might > > >> > >> be > > >> > >> > >> better > > >> > >> > >> > > to > > >> > >> > >> > > make this a general request-time throttle with no > > >> mention in > > >> > >> the > > >> > >> > >> > naming > > >> > >> > >> > > about I/O threads and simply acknowledge the current > > >> > >> limitation > > >> > >> > >> (which > > >> > >> > >> > > we > > >> > >> > >> > > may someday fix) in the docs that this covers only > the > > >> time > > >> > >> after > > >> > >> > >> the > > >> > >> > >> > > thread is read off the network. > > >> > >> > >> > > 3. As such I think the right interface to the user > > would > > >> be > > >> > >> > >> something > > >> > >> > >> > > like percent_request_time and be in {0,...100} or > > >> > >> > >> request_time_ratio > > >> > >> > >> > > and be > > >> > >> > >> > > in {0.0,...,1.0} (I think "ratio" is the terminology > we > > >> used > > >> > >> if > > >> > >> > the > > >> > >> > >> > > scale > > >> > >> > >> > > is between 0 and 1 in the other metrics, right?) > > >> > >> > >> > > > > >> > >> > >> > > -Jay > > >> > >> > >> > > > > >> > >> > >> > > On Thu, Feb 23, 2017 at 3:45 AM, Rajini Sivaram < > > >> > >> > >> rajinisiva...@gmail.com > > >> > >> > >> > > > > >> > >> > >> > > wrote: > > >> > >> > >> > > > > >> > >> > >> > > > Guozhang/Dong, > > >> > >> > >> > > > > > >> > >> > >> > > > Thank you for the feedback. > > >> > >> > >> > > > > > >> > >> > >> > > > Guozhang : I have updated the section on co-existence > of > > >> byte > > >> > >> rate > > >> > >> > >> and > > >> > >> > >> > > > request time quotas. > > >> > >> > >> > > > > > >> > >> > >> > > > Dong: I hadn't added much detail to the metrics and > > >> sensors > > >> > >> since > > >> > >> > >> they > > >> > >> > >> > > are > > >> > >> > >> > > > going to be very similar to the existing metrics and > > >> sensors. > > >> > >> To > > >> > >> > >> avoid > > >> > >> > >> > > > confusion, I have now added more detail. All metrics > are > > >> in > > >> > the > > >> > >> > >> group > > >> > >> > >> > > > "quotaType" and all sensors have names starting with > > >> > >> "quotaType" > > >> > >> > >> (where > > >> > >> > >> > > > quotaType is Produce/Fetch/LeaderReplication/ > > >> > >> > >> > > > FollowerReplication/*IOThread*). > > >> > >> > >> > > > So there will be no reuse of existing metrics/sensors. > > The > > >> > new > > >> > >> > ones > > >> > >> > >> for > > >> > >> > >> > > > request processing time based throttling will be > > >> completely > > >> > >> > >> independent > > >> > >> > >> > > of > > >> > >> > >> > > > existing metrics/sensors, but will be consistent in > > >> format. > > >> > >> > >> > > > > > >> > >> > >> > > > The existing throttle_time_ms field in produce/fetch > > >> > responses > > >> > >> > will > > >> > >> > >> not > > >> > >> > >> > > be > > >> > >> > >> > > > impacted by this KIP. That will continue to return > > >> byte-rate > > >> > >> based > > >> > >> > >> > > > throttling times. In addition, a new field > > >> > >> > request_throttle_time_ms > > >> > >> > >> > will > > >> > >> > >> > > be > > >> > >> > >> > > > added to return request quota based throttling times. > > >> These > > >> > >> will > > >> > >> > be > > >> > >> > >> > > exposed > > >> > >> > >> > > > as new metrics on the client-side. > > >> > >> > >> > > > > > >> > >> > >> > > > Since all metrics and sensors are different for each > > type > > >> of > > >> > >> > quota, > > >> > >> > >> I > > >> > >> > >> > > > believe there is already sufficient metrics to monitor > > >> > >> throttling > > >> > >> > on > > >> > >> > >> > both > > >> > >> > >> > > > client and broker side for each type of throttling. > > >> > >> > >> > > > > > >> > >> > >> > > > Regards, > > >> > >> > >> > > > > > >> > >> > >> > > > Rajini > > >> > >> > >> > > > > > >> > >> > >> > > > > > >> > >> > >> > > > On Thu, Feb 23, 2017 at 4:32 AM, Dong Lin < > > >> > lindon...@gmail.com > > >> > >> > > > >> > >> > >> wrote: > > >> > >> > >> > > > > > >> > >> > >> > > > > Hey Rajini, > > >> > >> > >> > > > > > > >> > >> > >> > > > > I think it makes a lot of sense to use > io_thread_units > > >> as > > >> > >> metric > > >> > >> > >> to > > >> > >> > >> > > quota > > >> > >> > >> > > > > user's traffic here. LGTM overall. I have some > > questions > > >> > >> > regarding > > >> > >> > >> > > > sensors. > > >> > >> > >> > > > > > > >> > >> > >> > > > > - Can you be more specific in the KIP what sensors > > will > > >> be > > >> > >> > added? > > >> > >> > >> For > > >> > >> > >> > > > > example, it will be useful to specify the name and > > >> > >> attributes of > > >> > >> > >> > these > > >> > >> > >> > > > new > > >> > >> > >> > > > > sensors. > > >> > >> > >> > > > > > > >> > >> > >> > > > > - We currently have throttle-time and queue-size for > > >> > >> byte-rate > > >> > >> > >> based > > >> > >> > >> > > > quota. > > >> > >> > >> > > > > Are you going to have separate throttle-time and > > >> queue-size > > >> > >> for > > >> > >> > >> > > requests > > >> > >> > >> > > > > throttled by io_thread_unit-based quota, or will > they > > >> share > > >> > >> the > > >> > >> > >> same > > >> > >> > >> > > > > sensor? > > >> > >> > >> > > > > > > >> > >> > >> > > > > - Does the throttle-time in the ProduceResponse and > > >> > >> > FetchResponse > > >> > >> > >> > > > contains > > >> > >> > >> > > > > time due to io_thread_unit-based quota? > > >> > >> > >> > > > > > > >> > >> > >> > > > > - Currently kafka server doesn't not provide any log > > or > > >> > >> metrics > > >> > >> > >> that > > >> > >> > >> > > > tells > > >> > >> > >> > > > > whether any given clientId (or user) is throttled. > > This > > >> is > > >> > >> not > > >> > >> > too > > >> > >> > >> > bad > > >> > >> > >> > > > > because we can still check the client-side byte-rate > > >> metric > > >> > >> to > > >> > >> > >> > validate > > >> > >> > >> > > > > whether a given client is throttled. But with this > > >> > >> > io_thread_unit, > > >> > >> > >> > > there > > >> > >> > >> > > > > will be no way to validate whether a given client is > > >> slow > > >> > >> > because > > >> > >> > >> it > > >> > >> > >> > > has > > >> > >> > >> > > > > exceeded its io_thread_unit limit. It is necessary > for > > >> user > > >> > >> to > > >> > >> > be > > >> > >> > >> > able > > >> > >> > >> > > to > > >> > >> > >> > > > > know this information to figure how whether they > have > > >> > reached > > >> > >> > >> there > > >> > >> > >> > > quota > > >> > >> > >> > > > > limit. How about we add log4j log on the server side > > to > > >> > >> > >> periodically > > >> > >> > >> > > > print > > >> > >> > >> > > > > the (client_id, byte-rate-throttle-time, > > >> > >> > >> > io-thread-unit-throttle-time) > > >> > >> > >> > > so > > >> > >> > >> > > > > that kafka administrator can figure those users that > > >> have > > >> > >> > reached > > >> > >> > >> > their > > >> > >> > >> > > > > limit and act accordingly? > > >> > >> > >> > > > > > > >> > >> > >> > > > > Thanks, > > >> > >> > >> > > > > Dong > > >> > >> > >> > > > > > > >> > >> > >> > > > > > > >> > >> > >> > > > > > > >> > >> > >> > > > > > > >> > >> > >> > > > > > > >> > >> > >> > > > > On Wed, Feb 22, 2017 at 4:46 PM, Guozhang Wang < > > >> > >> > >> wangg...@gmail.com> > > >> > >> > >> > > > wrote: > > >> > >> > >> > > > > > > >> > >> > >> > > > > > Made a pass over the doc, overall LGTM except a > > minor > > >> > >> comment > > >> > >> > on > > >> > >> > >> > the > > >> > >> > >> > > > > > throttling implementation: > > >> > >> > >> > > > > > > > >> > >> > >> > > > > > Stated as "Request processing time throttling will > > be > > >> > >> applied > > >> > >> > on > > >> > >> > >> > top > > >> > >> > >> > > if > > >> > >> > >> > > > > > necessary." I thought that it meant the request > > >> > processing > > >> > >> > time > > >> > >> > >> > > > > throttling > > >> > >> > >> > > > > > is applied first, but continue reading I found it > > >> > actually > > >> > >> > >> meant to > > >> > >> > >> > > > apply > > >> > >> > >> > > > > > produce / fetch byte rate throttling first. > > >> > >> > >> > > > > > > > >> > >> > >> > > > > > Also the last sentence "The remaining delay if any > > is > > >> > >> applied > > >> > >> > to > > >> > >> > >> > the > > >> > >> > >> > > > > > response." is a bit confusing to me. Maybe > rewording > > >> it a > > >> > >> bit? > > >> > >> > >> > > > > > > > >> > >> > >> > > > > > > > >> > >> > >> > > > > > Guozhang > > >> > >> > >> > > > > > > > >> > >> > >> > > > > > > > >> > >> > >> > > > > > On Wed, Feb 22, 2017 at 3:24 PM, Jun Rao < > > >> > j...@confluent.io > > >> > >> > > > >> > >> > >> wrote: > > >> > >> > >> > > > > > > > >> > >> > >> > > > > > > Hi, Rajini, > > >> > >> > >> > > > > > > > > >> > >> > >> > > > > > > Thanks for the updated KIP. The latest proposal > > >> looks > > >> > >> good > > >> > >> > to > > >> > >> > >> me. > > >> > >> > >> > > > > > > > > >> > >> > >> > > > > > > Jun > > >> > >> > >> > > > > > > > > >> > >> > >> > > > > > > On Wed, Feb 22, 2017 at 2:19 PM, Rajini Sivaram > < > > >> > >> > >> > > > > rajinisiva...@gmail.com > > >> > >> > >> > > > > > > > > >> > >> > >> > > > > > > wrote: > > >> > >> > >> > > > > > > > > >> > >> > >> > > > > > > > Jun/Roger, > > >> > >> > >> > > > > > > > > > >> > >> > >> > > > > > > > Thank you for the feedback. > > >> > >> > >> > > > > > > > > > >> > >> > >> > > > > > > > 1. I have updated the KIP to use absolute > units > > >> > >> instead of > > >> > >> > >> > > > > percentage. > > >> > >> > >> > > > > > > The > > >> > >> > >> > > > > > > > property is called* io_thread_units* to align > > with > > >> > the > > >> > >> > >> thread > > >> > >> > >> > > count > > >> > >> > >> > > > > > > > property *num.io.threads*. When we implement > > >> network > > >> > >> > thread > > >> > >> > >> > > > > utilization > > >> > >> > >> > > > > > > > quotas, we can add another property > > >> > >> > *network_thread_units.* > > >> > >> > >> > > > > > > > > > >> > >> > >> > > > > > > > 2. ControlledShutdown is already listed under > > the > > >> > >> exempt > > >> > >> > >> > > requests. > > >> > >> > >> > > > > Jun, > > >> > >> > >> > > > > > > did > > >> > >> > >> > > > > > > > you mean a different request that needs to be > > >> added? > > >> > >> The > > >> > >> > >> four > > >> > >> > >> > > > > requests > > >> > >> > >> > > > > > > > currently exempt in the KIP are StopReplica, > > >> > >> > >> > ControlledShutdown, > > >> > >> > >> > > > > > > > LeaderAndIsr and UpdateMetadata. These are > > >> controlled > > >> > >> > using > > >> > >> > >> > > > > > ClusterAction > > >> > >> > >> > > > > > > > ACL, so it is easy to exclude and only > throttle > > if > > >> > >> > >> > unauthorized. > > >> > >> > >> > > I > > >> > >> > >> > > > > > wasn't > > >> > >> > >> > > > > > > > sure if there are other requests used only for > > >> > >> > inter-broker > > >> > >> > >> > that > > >> > >> > >> > > > > needed > > >> > >> > >> > > > > > > to > > >> > >> > >> > > > > > > > be excluded. > > >> > >> > >> > > > > > > > > > >> > >> > >> > > > > > > > 3. I was thinking the smallest change would be > > to > > >> > >> replace > > >> > >> > >> all > > >> > >> > >> > > > > > references > > >> > >> > >> > > > > > > to > > >> > >> > >> > > > > > > > *requestChannel.sendResponse()* with a local > > >> method > > >> > >> > >> > > > > > > > *sendResponseMaybeThrottle()* that does the > > >> > throttling > > >> > >> if > > >> > >> > >> any > > >> > >> > >> > > plus > > >> > >> > >> > > > > send > > >> > >> > >> > > > > > > > response. If we throttle first in > > >> > *KafkaApis.handle()*, > > >> > >> > the > > >> > >> > >> > time > > >> > >> > >> > > > > spent > > >> > >> > >> > > > > > > > within the method handling the request will > not > > be > > >> > >> > recorded > > >> > >> > >> or > > >> > >> > >> > > used > > >> > >> > >> > > > > in > > >> > >> > >> > > > > > > > throttling. We can look into this again when > the > > >> PR > > >> > is > > >> > >> > ready > > >> > >> > >> > for > > >> > >> > >> > > > > > review. > > >> > >> > >> > > > > > > > > > >> > >> > >> > > > > > > > Regards, > > >> > >> > >> > > > > > > > > > >> > >> > >> > > > > > > > Rajini > > >> > >> > >> > > > > > > > > > >> > >> > >> > > > > > > > > > >> > >> > >> > > > > > > > > > >> > >> > >> > > > > > > > On Wed, Feb 22, 2017 at 5:55 PM, Roger Hoover > < > > >> > >> > >> > > > > roger.hoo...@gmail.com> > > >> > >> > >> > > > > > > > wrote: > > >> > >> > >> > > > > > > > > > >> > >> > >> > > > > > > > > Great to see this KIP and the excellent > > >> discussion. > > >> > >> > >> > > > > > > > > > > >> > >> > >> > > > > > > > > To me, Jun's suggestion makes sense. If my > > >> > >> application > > >> > >> > is > > >> > >> > >> > > > > allocated > > >> > >> > >> > > > > > 1 > > >> > >> > >> > > > > > > > > request handler unit, then it's as if I > have a > > >> > Kafka > > >> > >> > >> broker > > >> > >> > >> > > with > > >> > >> > >> > > > a > > >> > >> > >> > > > > > > single > > >> > >> > >> > > > > > > > > request handler thread dedicated to me. > > That's > > >> the > > >> > >> > most I > > >> > >> > >> > can > > >> > >> > >> > > > use, > > >> > >> > >> > > > > > at > > >> > >> > >> > > > > > > > > least. That allocation doesn't change even > if > > >> an > > >> > >> admin > > >> > >> > >> later > > >> > >> > >> > > > > > increases > > >> > >> > >> > > > > > > > the > > >> > >> > >> > > > > > > > > size of the request thread pool on the > broker. > > >> > It's > > >> > >> > >> similar > > >> > >> > >> > to > > >> > >> > >> > > > the > > >> > >> > >> > > > > > CPU > > >> > >> > >> > > > > > > > > abstraction that VMs and containers get from > > >> > >> hypervisors > > >> > >> > >> or > > >> > >> > >> > OS > > >> > >> > >> > > > > > > > schedulers. > > >> > >> > >> > > > > > > > > While different client access patterns can > use > > >> > wildly > > >> > >> > >> > different > > >> > >> > >> > > > > > amounts > > >> > >> > >> > > > > > > > of > > >> > >> > >> > > > > > > > > request thread resources per request, a > given > > >> > >> > application > > >> > >> > >> > will > > >> > >> > >> > > > > > > generally > > >> > >> > >> > > > > > > > > have a stable access pattern and can figure > > out > > >> > >> > >> empirically > > >> > >> > >> > how > > >> > >> > >> > > > > many > > >> > >> > >> > > > > > > > > "request thread units" it needs to meet it's > > >> > >> > >> > throughput/latency > > >> > >> > >> > > > > > goals. > > >> > >> > >> > > > > > > > > > > >> > >> > >> > > > > > > > > Cheers, > > >> > >> > >> > > > > > > > > > > >> > >> > >> > > > > > > > > Roger > > >> > >> > >> > > > > > > > > > > >> > >> > >> > > > > > > > > On Wed, Feb 22, 2017 at 8:53 AM, Jun Rao < > > >> > >> > >> j...@confluent.io> > > >> > >> > >> > > > wrote: > > >> > >> > >> > > > > > > > > > > >> > >> > >> > > > > > > > > > Hi, Rajini, > > >> > >> > >> > > > > > > > > > > > >> > >> > >> > > > > > > > > > Thanks for the updated KIP. A few more > > >> comments. > > >> > >> > >> > > > > > > > > > > > >> > >> > >> > > > > > > > > > 1. A concern of request_time_percent is > that > > >> it's > > >> > >> not > > >> > >> > an > > >> > >> > >> > > > absolute > > >> > >> > >> > > > > > > > value. > > >> > >> > >> > > > > > > > > > Let's say you give a user a 10% limit. If > > the > > >> > admin > > >> > >> > >> doubles > > >> > >> > >> > > the > > >> > >> > >> > > > > > > number > > >> > >> > >> > > > > > > > of > > >> > >> > >> > > > > > > > > > request handler threads, that user now > > >> actually > > >> > has > > >> > >> > >> twice > > >> > >> > >> > the > > >> > >> > >> > > > > > > absolute > > >> > >> > >> > > > > > > > > > capacity. This may confuse people a bit. > So, > > >> > >> perhaps > > >> > >> > >> > setting > > >> > >> > >> > > > the > > >> > >> > >> > > > > > > quota > > >> > >> > >> > > > > > > > > > based on an absolute request thread unit > is > > >> > better. > > >> > >> > >> > > > > > > > > > > > >> > >> > >> > > > > > > > > > 2. ControlledShutdownRequest is also an > > >> > >> inter-broker > > >> > >> > >> > request > > >> > >> > >> > > > and > > >> > >> > >> > > > > > > needs > > >> > >> > >> > > > > > > > to > > >> > >> > >> > > > > > > > > > be excluded from throttling. > > >> > >> > >> > > > > > > > > > > > >> > >> > >> > > > > > > > > > 3. Implementation wise, I am wondering if > > it's > > >> > >> simpler > > >> > >> > >> to > > >> > >> > >> > > apply > > >> > >> > >> > > > > the > > >> > >> > >> > > > > > > > > request > > >> > >> > >> > > > > > > > > > time throttling first in > KafkaApis.handle(). > > >> > >> > Otherwise, > > >> > >> > >> we > > >> > >> > >> > > will > > >> > >> > >> > > > > > need > > >> > >> > >> > > > > > > to > > >> > >> > >> > > > > > > > > add > > >> > >> > >> > > > > > > > > > the throttling logic in each type of > > request. > > >> > >> > >> > > > > > > > > > > > >> > >> > >> > > > > > > > > > Thanks, > > >> > >> > >> > > > > > > > > > > > >> > >> > >> > > > > > > > > > Jun > > >> > >> > >> > > > > > > > > > > > >> > >> > >> > > > > > > > > > On Wed, Feb 22, 2017 at 5:58 AM, Rajini > > >> Sivaram < > > >> > >> > >> > > > > > > > rajinisiva...@gmail.com > > >> > >> > >> > > > > > > > > > > > >> > >> > >> > > > > > > > > > wrote: > > >> > >> > >> > > > > > > > > > > > >> > >> > >> > > > > > > > > > > Jun, > > >> > >> > >> > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > Thank you for the review. > > >> > >> > >> > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > I have reverted to the original KIP that > > >> > >> throttles > > >> > >> > >> based > > >> > >> > >> > on > > >> > >> > >> > > > > > request > > >> > >> > >> > > > > > > > > > handler > > >> > >> > >> > > > > > > > > > > utilization. At the moment, it uses > > >> percentage, > > >> > >> but > > >> > >> > I > > >> > >> > >> am > > >> > >> > >> > > > happy > > >> > >> > >> > > > > to > > >> > >> > >> > > > > > > > > change > > >> > >> > >> > > > > > > > > > to > > >> > >> > >> > > > > > > > > > > a fraction (out of 1 instead of 100) if > > >> > >> required. I > > >> > >> > >> have > > >> > >> > >> > > > added > > >> > >> > >> > > > > > the > > >> > >> > >> > > > > > > > > > examples > > >> > >> > >> > > > > > > > > > > from this discussion to the KIP. Also > > added > > >> a > > >> > >> > "Future > > >> > >> > >> > Work" > > >> > >> > >> > > > > > section > > >> > >> > >> > > > > > > > to > > >> > >> > >> > > > > > > > > > > address network thread utilization. The > > >> > >> > configuration > > >> > >> > >> is > > >> > >> > >> > > > named > > >> > >> > >> > > > > > > > > > > "request_time_percent" with the > > expectation > > >> > that > > >> > >> it > > >> > >> > >> can > > >> > >> > >> > > also > > >> > >> > >> > > > be > > >> > >> > >> > > > > > > used > > >> > >> > >> > > > > > > > as > > >> > >> > >> > > > > > > > > > the > > >> > >> > >> > > > > > > > > > > limit for network thread utilization > when > > >> that > > >> > is > > >> > >> > >> > > > implemented, > > >> > >> > >> > > > > so > > >> > >> > >> > > > > > > > that > > >> > >> > >> > > > > > > > > > > users have to set only one config for > the > > >> two > > >> > and > > >> > >> > not > > >> > >> > >> > have > > >> > >> > >> > > to > > >> > >> > >> > > > > > worry > > >> > >> > >> > > > > > > > > about > > >> > >> > >> > > > > > > > > > > the internal distribution of the work > > >> between > > >> > the > > >> > >> > two > > >> > >> > >> > > thread > > >> > >> > >> > > > > > pools > > >> > >> > >> > > > > > > in > > >> > >> > >> > > > > > > > > > > Kafka. > > >> > >> > >> > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > Regards, > > >> > >> > >> > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > Rajini > > >> > >> > >> > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > On Wed, Feb 22, 2017 at 12:23 AM, Jun > Rao > > < > > >> > >> > >> > > j...@confluent.io> > > >> > >> > >> > > > > > > wrote: > > >> > >> > >> > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > Hi, Rajini, > > >> > >> > >> > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > Thanks for the proposal. > > >> > >> > >> > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > The benefit of using the request > > >> processing > > >> > >> time > > >> > >> > >> over > > >> > >> > >> > the > > >> > >> > >> > > > > > request > > >> > >> > >> > > > > > > > > rate > > >> > >> > >> > > > > > > > > > is > > >> > >> > >> > > > > > > > > > > > exactly what people have said. I will > > just > > >> > >> expand > > >> > >> > >> that > > >> > >> > >> > a > > >> > >> > >> > > > bit. > > >> > >> > >> > > > > > > > > Consider > > >> > >> > >> > > > > > > > > > > the > > >> > >> > >> > > > > > > > > > > > following case. The producer sends a > > >> produce > > >> > >> > request > > >> > >> > >> > > with a > > >> > >> > >> > > > > > 10MB > > >> > >> > >> > > > > > > > > > message > > >> > >> > >> > > > > > > > > > > > but compressed to 100KB with gzip. The > > >> > >> > >> decompression of > > >> > >> > >> > > the > > >> > >> > >> > > > > > > message > > >> > >> > >> > > > > > > > > on > > >> > >> > >> > > > > > > > > > > the > > >> > >> > >> > > > > > > > > > > > broker could take 10-15 seconds, > during > > >> which > > >> > >> > time, > > >> > >> > >> a > > >> > >> > >> > > > request > > >> > >> > >> > > > > > > > handler > > >> > >> > >> > > > > > > > > > > > thread is completely blocked. In this > > >> case, > > >> > >> > neither > > >> > >> > >> the > > >> > >> > >> > > > > byte-in > > >> > >> > >> > > > > > > > quota > > >> > >> > >> > > > > > > > > > nor > > >> > >> > >> > > > > > > > > > > > the request rate quota may be > effective > > in > > >> > >> > >> protecting > > >> > >> > >> > the > > >> > >> > >> > > > > > broker. > > >> > >> > >> > > > > > > > > > > Consider > > >> > >> > >> > > > > > > > > > > > another case. A consumer group starts > > >> with 10 > > >> > >> > >> instances > > >> > >> > >> > > and > > >> > >> > >> > > > > > later > > >> > >> > >> > > > > > > > on > > >> > >> > >> > > > > > > > > > > > switches to 20 instances. The request > > rate > > >> > will > > >> > >> > >> likely > > >> > >> > >> > > > > double, > > >> > >> > >> > > > > > > but > > >> > >> > >> > > > > > > > > the > > >> > >> > >> > > > > > > > > > > > actually load on the broker may not > > double > > >> > >> since > > >> > >> > >> each > > >> > >> > >> > > fetch > > >> > >> > >> > > > > > > request > > >> > >> > >> > > > > > > > > > only > > >> > >> > >> > > > > > > > > > > > contains half of the partitions. > Request > > >> rate > > >> > >> > quota > > >> > >> > >> may > > >> > >> > >> > > not > > >> > >> > >> > > > > be > > >> > >> > >> > > > > > > easy > > >> > >> > >> > > > > > > > > to > > >> > >> > >> > > > > > > > > > > > configure in this case. > > >> > >> > >> > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > What we really want is to be able to > > >> prevent > > >> > a > > >> > >> > >> client > > >> > >> > >> > > from > > >> > >> > >> > > > > > using > > >> > >> > >> > > > > > > > too > > >> > >> > >> > > > > > > > > > much > > >> > >> > >> > > > > > > > > > > > of the server side resources. In this > > >> > >> particular > > >> > >> > >> KIP, > > >> > >> > >> > > this > > >> > >> > >> > > > > > > resource > > >> > >> > >> > > > > > > > > is > > >> > >> > >> > > > > > > > > > > the > > >> > >> > >> > > > > > > > > > > > capacity of the request handler > > threads. I > > >> > >> agree > > >> > >> > >> that > > >> > >> > >> > it > > >> > >> > >> > > > may > > >> > >> > >> > > > > > not > > >> > >> > >> > > > > > > be > > >> > >> > >> > > > > > > > > > > > intuitive for the users to determine > how > > >> to > > >> > set > > >> > >> > the > > >> > >> > >> > right > > >> > >> > >> > > > > > limit. > > >> > >> > >> > > > > > > > > > However, > > >> > >> > >> > > > > > > > > > > > this is not completely new and has > been > > >> done > > >> > in > > >> > >> > the > > >> > >> > >> > > > container > > >> > >> > >> > > > > > > world > > >> > >> > >> > > > > > > > > > > > already. For example, Linux cgroup ( > > >> > >> > >> > > > > https://access.redhat.com/ > > >> > >> > >> > > > > > > > > > > > documentation/en-US/Red_Hat_En > > >> > >> > >> terprise_Linux/6/html/ > > >> > >> > >> > > > > > > > > > > > Resource_Management_Guide/sec- > cpu.html) > > >> has > > >> > >> the > > >> > >> > >> > concept > > >> > >> > >> > > of > > >> > >> > >> > > > > > > > > > > > cpu.cfs_quota_us, > > >> > >> > >> > > > > > > > > > > > which specifies the total amount of > time > > >> in > > >> > >> > >> > microseconds > > >> > >> > >> > > > for > > >> > >> > >> > > > > > > which > > >> > >> > >> > > > > > > > > all > > >> > >> > >> > > > > > > > > > > > tasks in a cgroup can run during a one > > >> second > > >> > >> > >> period. > > >> > >> > >> > We > > >> > >> > >> > > > can > > >> > >> > >> > > > > > > > > > potentially > > >> > >> > >> > > > > > > > > > > > model the request handler threads in a > > >> > similar > > >> > >> > way. > > >> > >> > >> For > > >> > >> > >> > > > > > example, > > >> > >> > >> > > > > > > > each > > >> > >> > >> > > > > > > > > > > > request handler thread can be 1 > request > > >> > handler > > >> > >> > unit > > >> > >> > >> > and > > >> > >> > >> > > > the > > >> > >> > >> > > > > > > admin > > >> > >> > >> > > > > > > > > can > > >> > >> > >> > > > > > > > > > > > configure a limit on how many units > (say > > >> > 0.01) > > >> > >> a > > >> > >> > >> client > > >> > >> > >> > > can > > >> > >> > >> > > > > > have. > > >> > >> > >> > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > Regarding not throttling the internal > > >> broker > > >> > to > > >> > >> > >> broker > > >> > >> > >> > > > > > requests. > > >> > >> > >> > > > > > > We > > >> > >> > >> > > > > > > > > > could > > >> > >> > >> > > > > > > > > > > > do that. Alternatively, we could just > > let > > >> the > > >> > >> > admin > > >> > >> > >> > > > > configure a > > >> > >> > >> > > > > > > > high > > >> > >> > >> > > > > > > > > > > limit > > >> > >> > >> > > > > > > > > > > > for the kafka user (it may not be able > > to > > >> do > > >> > >> that > > >> > >> > >> > easily > > >> > >> > >> > > > > based > > >> > >> > >> > > > > > on > > >> > >> > >> > > > > > > > > > > clientId > > >> > >> > >> > > > > > > > > > > > though). > > >> > >> > >> > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > Ideally we want to be able to protect > > the > > >> > >> > >> utilization > > >> > >> > >> > of > > >> > >> > >> > > > the > > >> > >> > >> > > > > > > > network > > >> > >> > >> > > > > > > > > > > thread > > >> > >> > >> > > > > > > > > > > > pool too. The difficult is mostly what > > >> Rajini > > >> > >> > said: > > >> > >> > >> (1) > > >> > >> > >> > > The > > >> > >> > >> > > > > > > > mechanism > > >> > >> > >> > > > > > > > > > for > > >> > >> > >> > > > > > > > > > > > throttling the requests is through > > >> Purgatory > > >> > >> and > > >> > >> > we > > >> > >> > >> > will > > >> > >> > >> > > > have > > >> > >> > >> > > > > > to > > >> > >> > >> > > > > > > > > think > > >> > >> > >> > > > > > > > > > > > through how to integrate that into the > > >> > network > > >> > >> > >> layer. > > >> > >> > >> > > (2) > > >> > >> > >> > > > In > > >> > >> > >> > > > > > the > > >> > >> > >> > > > > > > > > > network > > >> > >> > >> > > > > > > > > > > > layer, currently we know the user, but > > not > > >> > the > > >> > >> > >> clientId > > >> > >> > >> > > of > > >> > >> > >> > > > > the > > >> > >> > >> > > > > > > > > request. > > >> > >> > >> > > > > > > > > > > So, > > >> > >> > >> > > > > > > > > > > > it's a bit tricky to throttle based on > > >> > clientId > > >> > >> > >> there. > > >> > >> > >> > > > Plus, > > >> > >> > >> > > > > > the > > >> > >> > >> > > > > > > > > > byteOut > > >> > >> > >> > > > > > > > > > > > quota can already protect the network > > >> thread > > >> > >> > >> > utilization > > >> > >> > >> > > > for > > >> > >> > >> > > > > > > fetch > > >> > >> > >> > > > > > > > > > > > requests. So, if we can't figure out > > this > > >> > part > > >> > >> > right > > >> > >> > >> > now, > > >> > >> > >> > > > > just > > >> > >> > >> > > > > > > > > focusing > > >> > >> > >> > > > > > > > > > > on > > >> > >> > >> > > > > > > > > > > > the request handling threads for this > > KIP > > >> is > > >> > >> > still a > > >> > >> > >> > > useful > > >> > >> > >> > > > > > > > feature. > > >> > >> > >> > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > Thanks, > > >> > >> > >> > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > Jun > > >> > >> > >> > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > On Tue, Feb 21, 2017 at 4:27 AM, > Rajini > > >> > >> Sivaram < > > >> > >> > >> > > > > > > > > > rajinisiva...@gmail.com > > >> > >> > >> > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > wrote: > > >> > >> > >> > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > Thank you all for the feedback. > > >> > >> > >> > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > Jay: I have removed exemption for > > >> consumer > > >> > >> > >> heartbeat > > >> > >> > >> > > etc. > > >> > >> > >> > > > > > Agree > > >> > >> > >> > > > > > > > > that > > >> > >> > >> > > > > > > > > > > > > protecting the cluster is more > > important > > >> > than > > >> > >> > >> > > protecting > > >> > >> > >> > > > > > > > individual > > >> > >> > >> > > > > > > > > > > apps. > > >> > >> > >> > > > > > > > > > > > > Have retained the exemption for > > >> > >> > >> > > StopReplicat/LeaderAndIsr > > >> > >> > >> > > > > > etc, > > >> > >> > >> > > > > > > > > these > > >> > >> > >> > > > > > > > > > > are > > >> > >> > >> > > > > > > > > > > > > throttled only if authorization > fails > > >> (so > > >> > >> can't > > >> > >> > be > > >> > >> > >> > used > > >> > >> > >> > > > for > > >> > >> > >> > > > > > DoS > > >> > >> > >> > > > > > > > > > attacks > > >> > >> > >> > > > > > > > > > > > in > > >> > >> > >> > > > > > > > > > > > > a secure cluster, but allows > > >> inter-broker > > >> > >> > >> requests to > > >> > >> > >> > > > > > complete > > >> > >> > >> > > > > > > > > > without > > >> > >> > >> > > > > > > > > > > > > delays). > > >> > >> > >> > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > I will wait another day to see if > > these > > >> is > > >> > >> any > > >> > >> > >> > > objection > > >> > >> > >> > > > to > > >> > >> > >> > > > > > > > quotas > > >> > >> > >> > > > > > > > > > > based > > >> > >> > >> > > > > > > > > > > > on > > >> > >> > >> > > > > > > > > > > > > request processing time (as opposed > to > > >> > >> request > > >> > >> > >> rate) > > >> > >> > >> > > and > > >> > >> > >> > > > if > > >> > >> > >> > > > > > > there > > >> > >> > >> > > > > > > > > are > > >> > >> > >> > > > > > > > > > > no > > >> > >> > >> > > > > > > > > > > > > objections, I will revert to the > > >> original > > >> > >> > proposal > > >> > >> > >> > with > > >> > >> > >> > > > > some > > >> > >> > >> > > > > > > > > changes. > > >> > >> > >> > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > The original proposal was only > > including > > >> > the > > >> > >> > time > > >> > >> > >> > used > > >> > >> > >> > > by > > >> > >> > >> > > > > the > > >> > >> > >> > > > > > > > > request > > >> > >> > >> > > > > > > > > > > > > handler threads (that made > calculation > > >> > >> easy). I > > >> > >> > >> think > > >> > >> > >> > > the > > >> > >> > >> > > > > > > > > suggestion > > >> > >> > >> > > > > > > > > > is > > >> > >> > >> > > > > > > > > > > > to > > >> > >> > >> > > > > > > > > > > > > include the time spent in the > network > > >> > >> threads as > > >> > >> > >> well > > >> > >> > >> > > > since > > >> > >> > >> > > > > > > that > > >> > >> > >> > > > > > > > > may > > >> > >> > >> > > > > > > > > > be > > >> > >> > >> > > > > > > > > > > > > significant. As Jay pointed out, it > is > > >> more > > >> > >> > >> > complicated > > >> > >> > >> > > > to > > >> > >> > >> > > > > > > > > calculate > > >> > >> > >> > > > > > > > > > > the > > >> > >> > >> > > > > > > > > > > > > total available CPU time and convert > > to > > >> a > > >> > >> ratio > > >> > >> > >> when > > >> > >> > >> > > > there > > >> > >> > >> > > > > > *m* > > >> > >> > >> > > > > > > > I/O > > >> > >> > >> > > > > > > > > > > > threads > > >> > >> > >> > > > > > > > > > > > > and *n* network threads. > > >> > >> > >> > ThreadMXBean#getThreadCPUTime( > > >> > >> > >> > > ) > > >> > >> > >> > > > > may > > >> > >> > >> > > > > > > > give > > >> > >> > >> > > > > > > > > us > > >> > >> > >> > > > > > > > > > > > what > > >> > >> > >> > > > > > > > > > > > > we want, but it can be very > expensive > > on > > >> > some > > >> > >> > >> > > platforms. > > >> > >> > >> > > > As > > >> > >> > >> > > > > > > > Becket > > >> > >> > >> > > > > > > > > > and > > >> > >> > >> > > > > > > > > > > > > Guozhang have pointed out, we do > have > > >> > several > > >> > >> > time > > >> > >> > >> > > > > > measurements > > >> > >> > >> > > > > > > > > > already > > >> > >> > >> > > > > > > > > > > > for > > >> > >> > >> > > > > > > > > > > > > generating metrics that we could > use, > > >> > though > > >> > >> we > > >> > >> > >> might > > >> > >> > >> > > > want > > >> > >> > >> > > > > to > > >> > >> > >> > > > > > > > > switch > > >> > >> > >> > > > > > > > > > to > > >> > >> > >> > > > > > > > > > > > > nanoTime() instead of > > >> currentTimeMillis() > > >> > >> since > > >> > >> > >> some > > >> > >> > >> > of > > >> > >> > >> > > > the > > >> > >> > >> > > > > > > > values > > >> > >> > >> > > > > > > > > > for > > >> > >> > >> > > > > > > > > > > > > small requests may be < 1ms. But > > rather > > >> > than > > >> > >> add > > >> > >> > >> up > > >> > >> > >> > the > > >> > >> > >> > > > > time > > >> > >> > >> > > > > > > > spent > > >> > >> > >> > > > > > > > > in > > >> > >> > >> > > > > > > > > > > I/O > > >> > >> > >> > > > > > > > > > > > > thread and network thread, wouldn't > it > > >> be > > >> > >> better > > >> > >> > >> to > > >> > >> > >> > > > convert > > >> > >> > >> > > > > > the > > >> > >> > >> > > > > > > > > time > > >> > >> > >> > > > > > > > > > > > spent > > >> > >> > >> > > > > > > > > > > > > on each thread into a separate > ratio? > > >> UserA > > >> > >> has > > >> > >> > a > > >> > >> > >> > > request > > >> > >> > >> > > > > > quota > > >> > >> > >> > > > > > > > of > > >> > >> > >> > > > > > > > > > 5%. > > >> > >> > >> > > > > > > > > > > > Can > > >> > >> > >> > > > > > > > > > > > > we take that to mean that UserA can > > use > > >> 5% > > >> > of > > >> > >> > the > > >> > >> > >> > time > > >> > >> > >> > > on > > >> > >> > >> > > > > > > network > > >> > >> > >> > > > > > > > > > > threads > > >> > >> > >> > > > > > > > > > > > > and 5% of the time on I/O threads? > If > > >> > either > > >> > >> is > > >> > >> > >> > > exceeded, > > >> > >> > >> > > > > the > > >> > >> > >> > > > > > > > > > response > > >> > >> > >> > > > > > > > > > > is > > >> > >> > >> > > > > > > > > > > > > throttled - it would mean > maintaining > > >> two > > >> > >> sets > > >> > >> > of > > >> > >> > >> > > metrics > > >> > >> > >> > > > > for > > >> > >> > >> > > > > > > the > > >> > >> > >> > > > > > > > > two > > >> > >> > >> > > > > > > > > > > > > durations, but would result in more > > >> > >> meaningful > > >> > >> > >> > ratios. > > >> > >> > >> > > We > > >> > >> > >> > > > > > could > > >> > >> > >> > > > > > > > > > define > > >> > >> > >> > > > > > > > > > > > two > > >> > >> > >> > > > > > > > > > > > > quota limits (UserA has 5% of > request > > >> > threads > > >> > >> > and > > >> > >> > >> 10% > > >> > >> > >> > > of > > >> > >> > >> > > > > > > network > > >> > >> > >> > > > > > > > > > > > threads), > > >> > >> > >> > > > > > > > > > > > > but that seems unnecessary and > harder > > to > > >> > >> explain > > >> > >> > >> to > > >> > >> > >> > > > users. > > >> > >> > >> > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > Back to why and how quotas are > applied > > >> to > > >> > >> > network > > >> > >> > >> > > thread > > >> > >> > >> > > > > > > > > utilization: > > >> > >> > >> > > > > > > > > > > > > a) In the case of fetch, the time > > >> spent in > > >> > >> the > > >> > >> > >> > network > > >> > >> > >> > > > > > thread > > >> > >> > >> > > > > > > > may > > >> > >> > >> > > > > > > > > be > > >> > >> > >> > > > > > > > > > > > > significant and I can see the need > to > > >> > include > > >> > >> > >> this. > > >> > >> > >> > Are > > >> > >> > >> > > > > there > > >> > >> > >> > > > > > > > other > > >> > >> > >> > > > > > > > > > > > > requests where the network thread > > >> > >> utilization is > > >> > >> > >> > > > > significant? > > >> > >> > >> > > > > > > In > > >> > >> > >> > > > > > > > > the > > >> > >> > >> > > > > > > > > > > case > > >> > >> > >> > > > > > > > > > > > > of fetch, request handler thread > > >> > utilization > > >> > >> > would > > >> > >> > >> > > > throttle > > >> > >> > >> > > > > > > > clients > > >> > >> > >> > > > > > > > > > > with > > >> > >> > >> > > > > > > > > > > > > high request rate, low data volume > and > > >> > fetch > > >> > >> > byte > > >> > >> > >> > rate > > >> > >> > >> > > > > quota > > >> > >> > >> > > > > > > will > > >> > >> > >> > > > > > > > > > > > throttle > > >> > >> > >> > > > > > > > > > > > > clients with high data volume. > Network > > >> > thread > > >> > >> > >> > > utilization > > >> > >> > >> > > > > is > > >> > >> > >> > > > > > > > > perhaps > > >> > >> > >> > > > > > > > > > > > > proportional to the data volume. I > am > > >> > >> wondering > > >> > >> > >> if we > > >> > >> > >> > > > even > > >> > >> > >> > > > > > need > > >> > >> > >> > > > > > > > to > > >> > >> > >> > > > > > > > > > > > throttle > > >> > >> > >> > > > > > > > > > > > > based on network thread utilization > or > > >> > >> whether > > >> > >> > the > > >> > >> > >> > data > > >> > >> > >> > > > > > volume > > >> > >> > >> > > > > > > > > quota > > >> > >> > >> > > > > > > > > > > > covers > > >> > >> > >> > > > > > > > > > > > > this case. > > >> > >> > >> > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > b) At the moment, we record and > check > > >> for > > >> > >> quota > > >> > >> > >> > > violation > > >> > >> > >> > > > > at > > >> > >> > >> > > > > > > the > > >> > >> > >> > > > > > > > > same > > >> > >> > >> > > > > > > > > > > > time. > > >> > >> > >> > > > > > > > > > > > > If a quota is violated, the response > > is > > >> > >> delayed. > > >> > >> > >> > Using > > >> > >> > >> > > > > Jay'e > > >> > >> > >> > > > > > > > > example > > >> > >> > >> > > > > > > > > > of > > >> > >> > >> > > > > > > > > > > > > disk reads for fetches happening in > > the > > >> > >> network > > >> > >> > >> > thread, > > >> > >> > >> > > > We > > >> > >> > >> > > > > > > can't > > >> > >> > >> > > > > > > > > > record > > >> > >> > >> > > > > > > > > > > > and > > >> > >> > >> > > > > > > > > > > > > delay a response after the disk > reads. > > >> We > > >> > >> could > > >> > >> > >> > record > > >> > >> > >> > > > the > > >> > >> > >> > > > > > time > > >> > >> > >> > > > > > > > > spent > > >> > >> > >> > > > > > > > > > > on > > >> > >> > >> > > > > > > > > > > > > the network thread when the response > > is > > >> > >> complete > > >> > >> > >> and > > >> > >> > >> > > > > > introduce > > >> > >> > >> > > > > > > a > > >> > >> > >> > > > > > > > > > delay > > >> > >> > >> > > > > > > > > > > > for > > >> > >> > >> > > > > > > > > > > > > handling a subsequent request > > (separate > > >> out > > >> > >> > >> recording > > >> > >> > >> > > and > > >> > >> > >> > > > > > quota > > >> > >> > >> > > > > > > > > > > violation > > >> > >> > >> > > > > > > > > > > > > handling in the case of network > thread > > >> > >> > overload). > > >> > >> > >> > Does > > >> > >> > >> > > > that > > >> > >> > >> > > > > > > make > > >> > >> > >> > > > > > > > > > sense? > > >> > >> > >> > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > Regards, > > >> > >> > >> > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > Rajini > > >> > >> > >> > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > On Tue, Feb 21, 2017 at 2:58 AM, > > Becket > > >> > Qin < > > >> > >> > >> > > > > > > > becket....@gmail.com> > > >> > >> > >> > > > > > > > > > > > wrote: > > >> > >> > >> > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > > Hey Jay, > > >> > >> > >> > > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > > Yeah, I agree that enforcing the > CPU > > >> time > > >> > >> is a > > >> > >> > >> > little > > >> > >> > >> > > > > > > tricky. I > > >> > >> > >> > > > > > > > > am > > >> > >> > >> > > > > > > > > > > > > thinking > > >> > >> > >> > > > > > > > > > > > > > that maybe we can use the existing > > >> > request > > >> > >> > >> > > statistics. > > >> > >> > >> > > > > They > > >> > >> > >> > > > > > > are > > >> > >> > >> > > > > > > > > > > already > > >> > >> > >> > > > > > > > > > > > > > very detailed so we can probably > see > > >> the > > >> > >> > >> > approximate > > >> > >> > >> > > > CPU > > >> > >> > >> > > > > > time > > >> > >> > >> > > > > > > > > from > > >> > >> > >> > > > > > > > > > > it, > > >> > >> > >> > > > > > > > > > > > > e.g. > > >> > >> > >> > > > > > > > > > > > > > something like (total_time - > > >> > >> > >> > > > request/response_queue_time > > >> > >> > >> > > > > - > > >> > >> > >> > > > > > > > > > > > remote_time). > > >> > >> > >> > > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > > I agree with Guozhang that when a > > >> user is > > >> > >> > >> throttled > > >> > >> > >> > > it > > >> > >> > >> > > > is > > >> > >> > >> > > > > > > > likely > > >> > >> > >> > > > > > > > > > that > > >> > >> > >> > > > > > > > > > > > we > > >> > >> > >> > > > > > > > > > > > > > need to see if anything has went > > wrong > > >> > >> first, > > >> > >> > >> and > > >> > >> > >> > if > > >> > >> > >> > > > the > > >> > >> > >> > > > > > > users > > >> > >> > >> > > > > > > > > are > > >> > >> > >> > > > > > > > > > > well > > >> > >> > >> > > > > > > > > > > > > > behaving and just need more > > >> resources, we > > >> > >> will > > >> > >> > >> have > > >> > >> > >> > > to > > >> > >> > >> > > > > bump > > >> > >> > >> > > > > > > up > > >> > >> > >> > > > > > > > > the > > >> > >> > >> > > > > > > > > > > > quota > > >> > >> > >> > > > > > > > > > > > > > for them. It is true that > > >> pre-allocating > > >> > >> CPU > > >> > >> > >> time > > >> > >> > >> > > quota > > >> > >> > >> > > > > > > > precisely > > >> > >> > >> > > > > > > > > > for > > >> > >> > >> > > > > > > > > > > > the > > >> > >> > >> > > > > > > > > > > > > > users is difficult. So in practice > > it > > >> > would > > >> > >> > >> > probably > > >> > >> > >> > > be > > >> > >> > >> > > > > > more > > >> > >> > >> > > > > > > > like > > >> > >> > >> > > > > > > > > > > first > > >> > >> > >> > > > > > > > > > > > > set > > >> > >> > >> > > > > > > > > > > > > > a relative high protective CPU > time > > >> quota > > >> > >> for > > >> > >> > >> > > everyone > > >> > >> > >> > > > > and > > >> > >> > >> > > > > > > > > increase > > >> > >> > >> > > > > > > > > > > > that > > >> > >> > >> > > > > > > > > > > > > > for some individual clients on > > demand. > > >> > >> > >> > > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > > Thanks, > > >> > >> > >> > > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > > Jiangjie (Becket) Qin > > >> > >> > >> > > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > > On Mon, Feb 20, 2017 at 5:48 PM, > > >> Guozhang > > >> > >> > Wang < > > >> > >> > >> > > > > > > > > wangg...@gmail.com > > >> > >> > >> > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > wrote: > > >> > >> > >> > > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > > > This is a great proposal, glad > to > > >> see > > >> > it > > >> > >> > >> > happening. > > >> > >> > >> > > > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > > > I am inclined to the CPU > > >> throttling, or > > >> > >> more > > >> > >> > >> > > > > specifically > > >> > >> > >> > > > > > > > > > > processing > > >> > >> > >> > > > > > > > > > > > > time > > >> > >> > >> > > > > > > > > > > > > > > ratio instead of the request > rate > > >> > >> throttling > > >> > >> > >> as > > >> > >> > >> > > well. > > >> > >> > >> > > > > > > Becket > > >> > >> > >> > > > > > > > > has > > >> > >> > >> > > > > > > > > > > very > > >> > >> > >> > > > > > > > > > > > > > well > > >> > >> > >> > > > > > > > > > > > > > > summed my rationales above, and > > one > > >> > >> thing to > > >> > >> > >> add > > >> > >> > >> > > here > > >> > >> > >> > > > > is > > >> > >> > >> > > > > > > that > > >> > >> > >> > > > > > > > > the > > >> > >> > >> > > > > > > > > > > > > former > > >> > >> > >> > > > > > > > > > > > > > > has a good support for both > > >> "protecting > > >> > >> > >> against > > >> > >> > >> > > rogue > > >> > >> > >> > > > > > > > clients" > > >> > >> > >> > > > > > > > > as > > >> > >> > >> > > > > > > > > > > > well > > >> > >> > >> > > > > > > > > > > > > as > > >> > >> > >> > > > > > > > > > > > > > > "utilizing a cluster for > > >> multi-tenancy > > >> > >> > usage": > > >> > >> > >> > when > > >> > >> > >> > > > > > > thinking > > >> > >> > >> > > > > > > > > > about > > >> > >> > >> > > > > > > > > > > > how > > >> > >> > >> > > > > > > > > > > > > to > > >> > >> > >> > > > > > > > > > > > > > > explain this to the end users, I > > >> find > > >> > it > > >> > >> > >> actually > > >> > >> > >> > > > more > > >> > >> > >> > > > > > > > natural > > >> > >> > >> > > > > > > > > > than > > >> > >> > >> > > > > > > > > > > > the > > >> > >> > >> > > > > > > > > > > > > > > request rate since as mentioned > > >> above, > > >> > >> > >> different > > >> > >> > >> > > > > requests > > >> > >> > >> > > > > > > > will > > >> > >> > >> > > > > > > > > > have > > >> > >> > >> > > > > > > > > > > > > quite > > >> > >> > >> > > > > > > > > > > > > > > different "cost", and Kafka > today > > >> > already > > >> > >> > have > > >> > >> > >> > > > various > > >> > >> > >> > > > > > > > request > > >> > >> > >> > > > > > > > > > > types > > >> > >> > >> > > > > > > > > > > > > > > (produce, fetch, admin, > metadata, > > >> etc), > > >> > >> > >> because > > >> > >> > >> > of > > >> > >> > >> > > > that > > >> > >> > >> > > > > > the > > >> > >> > >> > > > > > > > > > request > > >> > >> > >> > > > > > > > > > > > > rate > > >> > >> > >> > > > > > > > > > > > > > > throttling may not be as > effective > > >> > >> unless it > > >> > >> > >> is > > >> > >> > >> > set > > >> > >> > >> > > > > very > > >> > >> > >> > > > > > > > > > > > > conservatively. > > >> > >> > >> > > > > > > > > > > > > > > > > >> > >> > >> > > > > > > > > > > > > > > Regarding to user reactions when > > >> they > > >> > are > > >> > >> > >> > > throttled, > > >> > >> > >> > > > I > > >> > >> > >> > > > > > > think > > >> > >> > >> > > > > > > > it > > >> > >> > >> > > > > > > > > > may > > >> > >> > >> > > > > > > > > > > > > > differ > > >> > >> > >> > > > > > > > > > > > > > > case-by-case, and need to be > > >> > discovered / > > >> > >> > >> guided > > >> > >> > >> > by > > >> > >> > >> > > > > > looking > > >> > >> > >> > > > > > > > at > > >> > >> > >> > > > > > > > > > > > relative > > >> > >> > >> > > > > > > > > > > > > > > metrics. So in other words users > > >> would > > >> > >> not > > >> > >> > >> expect > > >> > >> > >> > > to > > >> > >> > >> > > > > get > > >> > >> > >> > > > > > > > > > additional > > >> > >> > >> > > > > > > > > > > > > > > information by simply being told > > >> "hey, > > >> > >> you > > >> > >> > are > > >> > >> > >> > > > > > throttled", > > >> > >> > >> > > > > > > > > which > > >> > >> > >> > > > > > > > > > is > > >> > >> > >> > > > > > > > > > > > all > > >> > >> > >> > > > > > > > > > > > > > > what throttling does; they need > to > > >> > take a > > >> > >> > >> > follow-up > > >> > >> > >> > > > > step > > >> > >> > >> > > > > > > and > > >> > >> > >> > > > > > > > > see > > >> > >> > >> > > > > > > > > > > > "hmm, > > >> > >> > >> > > > > > > > > > > > > > I'm > > >> > >> > >> > > > > > > > > > > > > > > throttled probably because of > ..", > > >> > which > > >> > >> is > > >> > >> > by > > >> > >> > >> > > > looking > > >> > >> > >> > > > > at > > >> > >> > >> > > > > > > > other > > >> > >> > >> > > > > > > > > > > > metric > > >> > >> > >> > > > > > > > > > > > > > > values: e.g. whether I'm > > bombarding > > >> the > > >> > >> > >> brokers > > >> > >> > >> > > with > > >> > >> > >> > > > > > > >> > > > ... > > > > > > [Message clipped] > > >