Some responses to Jay's points. 1. Using commas - Cool.
2. Adding return flag - I'm inclined to agree with Joel that this is good to have in the initial implementation. 3. Config - +1. I'll remove it from the KIP. We can discuss this in parallel. 4. Purgatory vs Delay queue - I feel that it is simpler to reuse the existing purgatories for both delayed produce and fetch requests. IIUC, all we need for quotas is a minWait parameter for DelayedOperation (or something equivalent) since there is already a max wait. The completion criteria can check if minWait time has elapsed before declaring the operation complete. For this to impact performance, a significant number of clients may need to exceed their quota at the same time and even then I'm not very clear on the scope of the impact. Two layers of delays might add complexity to the implementation which I'm hoping to avoid. Aditya ________________________________________ From: Joel Koshy [jjkosh...@gmail.com] Sent: Friday, April 03, 2015 12:48 PM To: dev@kafka.apache.org Subject: Re: [KIP-DISCUSSION] KIP-13 Quotas Aditya, thanks for the updated KIP and Jay/Jun thanks for the comments. Couple of comments in-line: > 2. I would advocate for adding the return flag when we next bump the > request format version just to avoid proliferation. I agree this is a good > thing to know about, but at the moment I don't think we have a very well > flushed out idea of how the client would actually make use of this info. I I'm somewhat inclined to having something appropriate off the bat - mainly because (i) clients really should know that they have been throttled (ii) a smart producer/consumer implementation would want to know how much to back off. So perhaps this and config-management should be moved to a separate discussion, but it would be good to have this discussion going and incorporated into the first quota implementation. > 3. Config--I think we need to generalize the topic stuff so we can override > at multiple levels. We have topic and client, but I suspect "user" and > "broker" will also be important. I recommend we take config stuff out of > this KIP since we really need to fully think through a proposal that will > cover all these types of overrides. +1 - it is definitely orthogonal to the core quota implementation (although necessary for its operability). Having a config-related discussion in this KIP would only draw out the discussion and vote even if the core quota design looks good to everyone. So basically I think we can remove the portions on dynamic config as well as the response format but I really think we should close on those while the implementation is in progress and before quotas is officially released. > 4. Instead of using purgatories to implement the delay would it make more > sense to just use a delay queue? I think all the additional stuff in the > purgatory other than the delay queue doesn't make sense as the quota is a > hard N ms penalty with no chance of early eviction. If there is no perf > penalty for the full purgatory that may be fine (even good) to reuse, but I > haven't looked into that. A simple delay queue sounds good - I think Aditya was also trying to avoid adding a new quota purgatory. i.e., it may be possible to use the existing purgatory instances to enforce quotas. That may be simpler, but would be incur a slight perf penalty if too many clients are being throttled. Thanks, Joel > > -Jay > > On Fri, Apr 3, 2015 at 10:45 AM, Aditya Auradkar < > aaurad...@linkedin.com.invalid> wrote: > >> Update, I added a proposal on doing dynamic client based configuration >> that can be used for quotas. >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-13+-+Quotas >> >> Please take a look and let me know if there are any concerns. >> >> Thanks, >> Aditya >> ________________________________________ >> From: Aditya Auradkar >> Sent: Friday, April 03, 2015 10:10 AM >> To: dev@kafka.apache.org >> Subject: RE: [KIP-DISCUSSION] KIP-13 Quotas >> >> Thanks Jun. >> >> Some thoughts: >> >> 10) I think it is better we throttle regardless of the produce/fetch >> version. This is a nice feature where clients can tell if they are being >> throttled or not. If we only throttle newer clients, then we have >> inconsistent behavior across clients in a multi-tenant cluster. Having >> quota metrics on the client side is also a nice incentive to upgrade client >> versions. >> >> 11) I think we can call metric.record(fetchSize) before adding the >> delayedFetch request into the purgatory. This will give us the estimated >> delay of the request up-front. The timeout on the DelayedFetch is the >> Max(maxWait, quotaDelay). The DelayedFetch completion criteria can change a >> little to accomodate quotas. >> >> - I agree the quota code should return the estimated delay time in >> QuotaViolationException. >> >> Thanks, >> Aditya >> >> ________________________________________ >> From: Jun Rao [j...@confluent.io] >> Sent: Friday, April 03, 2015 9:16 AM >> To: dev@kafka.apache.org >> Subject: Re: [KIP-DISCUSSION] KIP-13 Quotas >> >> Thanks for the update. >> >> 10. About whether to return a new field in the response to indicate >> throttling. Earlier, the plan was to not change the response format and >> just have a metric on the broker to indicate whether a clientId is >> throttled or not. The issue is that we don't know whether a particular >> clientId instance is throttled or not (since there could be multiple >> clients with the same clientId). Your proposal of adding an isThrottled >> field in the response addresses and seems better. Then, do we just throttle >> the new version of produce/fetch request or both the old and the new >> versions? Also, we probably still need a separate metric on the broker side >> to indicate whether a clientId is throttled or not. >> >> 11. Just to clarify. For fetch requests, when will metric.record(fetchSize) >> be called? Is it when we are ready to send the fetch response (after >> minBytes and maxWait are satisfied)? >> >> As an implementation detail, it may be useful for the quota code to return >> an estimated delay time (to bring the measurement within the limit) in >> QuotaViolationException. >> >> Thanks, >> >> Jun >> >> On Wed, Apr 1, 2015 at 3:27 PM, Aditya Auradkar < >> aaurad...@linkedin.com.invalid> wrote: >> >> > Hey everyone, >> > >> > I've made changes to the KIP to capture our discussions over the last >> > couple of weeks. >> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-13+-+Quotas >> > >> > I'll start a voting thread after people have had a chance to >> read/comment. >> > >> > Thanks, >> > Aditya >> > >> > ________________________________________ >> > From: Steven Wu [stevenz...@gmail.com] >> > Sent: Friday, March 20, 2015 9:14 AM >> > To: dev@kafka.apache.org >> > Subject: Re: [KIP-DISCUSSION] KIP-13 Quotas >> > >> > +1 on Jun's suggestion of maintaining one set/style of metrics at broker. >> > In Netflix, we have to convert the yammer metrics to servo metrics at >> > broker. it will be painful to know some metrics are in a different style >> > and get to be handled differently. >> > >> > On Fri, Mar 20, 2015 at 8:17 AM, Jun Rao <j...@confluent.io> wrote: >> > >> > > Not so sure. People who use quota will definitely want to monitor the >> new >> > > metrics at the client id level. Then they will need to deal with those >> > > metrics differently from the rest of the metrics. It would be better if >> > we >> > > can hide this complexity from the users. >> > > >> > > Thanks, >> > > >> > > Jun >> > > >> > > On Thu, Mar 19, 2015 at 10:45 PM, Joel Koshy <jjkosh...@gmail.com> >> > wrote: >> > > >> > > > Actually thinking again - since these will be a few new metrics at >> the >> > > > client id level (bytes in and bytes out to start with) maybe it is >> fine >> > > to >> > > > have the two type of metrics coexist and we can migrate the existing >> > > > metrics in parallel. >> > > > >> > > > On Thursday, March 19, 2015, Joel Koshy <jjkosh...@gmail.com> wrote: >> > > > >> > > > > That is a valid concern but in that case I think it would be better >> > to >> > > > > just migrate completely to the new metrics package first. >> > > > > >> > > > > On Thursday, March 19, 2015, Jun Rao <j...@confluent.io >> > > > > <javascript:_e(%7B%7D,'cvml','j...@confluent.io');>> wrote: >> > > > > >> > > > >> Hmm, I was thinking a bit differently on the metrics stuff. I >> think >> > it >> > > > >> would be confusing to have some metrics defined in the new metrics >> > > > package >> > > > >> while some others defined in Coda Hale. Those metrics will look >> > > > different >> > > > >> (e.g., rates in Coda Hale will have special attributes such as >> > > > >> 1-min-average). People may need different ways to export the >> metrics >> > > to >> > > > >> external systems such as Graphite. So, instead of using the new >> > > metrics >> > > > >> package on the broker, I was thinking that we can just implement a >> > > > >> QuotaMetrics that wraps the Coda Hale metrics. The implementation >> > can >> > > be >> > > > >> the same as what's in the new metrics package. >> > > > >> >> > > > >> Thanks, >> > > > >> >> > > > >> Jun >> > > > >> >> > > > >> On Thu, Mar 19, 2015 at 8:09 PM, Jay Kreps <jay.kr...@gmail.com> >> > > wrote: >> > > > >> >> > > > >> > Yeah I was saying was that we are blocked on picking an approach >> > for >> > > > >> > metrics but not necessarily the full conversion. Clearly if we >> > pick >> > > > the >> > > > >> new >> > > > >> > metrics package we would need to implement the two metrics we >> want >> > > to >> > > > >> quota >> > > > >> > on. But the conversion of the remaining metrics can be done >> > > > >> asynchronously. >> > > > >> > >> > > > >> > -Jay >> > > > >> > >> > > > >> > On Thu, Mar 19, 2015 at 5:56 PM, Joel Koshy < >> jjkosh...@gmail.com> >> > > > >> wrote: >> > > > >> > >> > > > >> > > > in KAFKA-1930). I agree that this KIP doesn't need to block >> on >> > > the >> > > > >> > > > migration of the metrics package. >> > > > >> > > >> > > > >> > > Can you clarify the above? i.e., if we are going to quota on >> > > > something >> > > > >> > > then we would want to have migrated that metric over right? Or >> > do >> > > > you >> > > > >> > > mean we don't need to complete the migration of all metrics to >> > the >> > > > >> > > metrics package right? >> > > > >> > > >> > > > >> > > I think most of us now feel that the delay + no error is a >> good >> > > > >> > > approach, but it would be good to make sure everyone is on the >> > > same >> > > > >> > > page. >> > > > >> > > >> > > > >> > > As Aditya requested a couple of days ago I think we should go >> > over >> > > > >> > > this at the next KIP hangout. >> > > > >> > > >> > > > >> > > Joel >> > > > >> > > >> > > > >> > > On Thu, Mar 19, 2015 at 09:24:09AM -0700, Jun Rao wrote: >> > > > >> > > > 1. Delay + no error seems reasonable to me. However, I do >> feel >> > > > that >> > > > >> we >> > > > >> > > need >> > > > >> > > > to give the client an indicator that it's being throttled, >> > > instead >> > > > >> of >> > > > >> > > doing >> > > > >> > > > this silently. For that, we probably need to evolve the >> > > > >> produce/fetch >> > > > >> > > > protocol to include an extra status field in the response. >> We >> > > > >> probably >> > > > >> > > need >> > > > >> > > > to think more about whether we just want to return a simple >> > > status >> > > > >> code >> > > > >> > > > (e.g., 1 = throttled) or a value that indicates how much is >> > > being >> > > > >> > > throttled. >> > > > >> > > > >> > > > >> > > > 2. We probably need to improve the histogram support in the >> > new >> > > > >> metrics >> > > > >> > > > package before we can use it more widely on the server side >> > > (left >> > > > a >> > > > >> > > comment >> > > > >> > > > in KAFKA-1930). I agree that this KIP doesn't need to block >> on >> > > the >> > > > >> > > > migration of the metrics package. >> > > > >> > > > >> > > > >> > > > Thanks, >> > > > >> > > > >> > > > >> > > > Jun >> > > > >> > > > >> > > > >> > > > On Wed, Mar 18, 2015 at 4:02 PM, Aditya Auradkar < >> > > > >> > > > aaurad...@linkedin.com.invalid> wrote: >> > > > >> > > > >> > > > >> > > > > Hey everyone, >> > > > >> > > > > >> > > > >> > > > > Thanks for the great discussion. There are currently a few >> > > > points >> > > > >> on >> > > > >> > > this >> > > > >> > > > > KIP that need addressing and I want to make sure we are on >> > the >> > > > >> same >> > > > >> > > page >> > > > >> > > > > about those. >> > > > >> > > > > >> > > > >> > > > > 1. Append and delay response vs delay and return error >> > > > >> > > > > - I think we've discussed the pros and cons of each >> approach >> > > but >> > > > >> > > haven't >> > > > >> > > > > chosen an approach yet. Where does everyone stand on this >> > > issue? >> > > > >> > > > > >> > > > >> > > > > 2. Metrics Migration and usage in quotas >> > > > >> > > > > - The metrics library in clients has a notion of quotas >> that >> > > we >> > > > >> > should >> > > > >> > > > > reuse. For that to happen, we need to migrate the server >> to >> > > the >> > > > >> new >> > > > >> > > metrics >> > > > >> > > > > package. >> > > > >> > > > > - Need more clarification on how to compute throttling >> time >> > > and >> > > > >> > > windowing >> > > > >> > > > > for quotas. >> > > > >> > > > > >> > > > >> > > > > I'm going to start a new KIP to discuss metrics migration >> > > > >> separately. >> > > > >> > > That >> > > > >> > > > > will also contain a section on quotas. >> > > > >> > > > > >> > > > >> > > > > 3. Dynamic Configuration management - Being discussed in >> > > KIP-5. >> > > > >> > > Basically >> > > > >> > > > > we need something that will model default quotas and allow >> > > > >> per-client >> > > > >> > > > > overrides. >> > > > >> > > > > >> > > > >> > > > > Is there something else that I'm missing? >> > > > >> > > > > >> > > > >> > > > > Thanks, >> > > > >> > > > > Aditya >> > > > >> > > > > ________________________________________ >> > > > >> > > > > From: Jay Kreps [jay.kr...@gmail.com] >> > > > >> > > > > Sent: Wednesday, March 18, 2015 2:10 PM >> > > > >> > > > > To: dev@kafka.apache.org >> > > > >> > > > > Subject: Re: [KIP-DISCUSSION] KIP-13 Quotas >> > > > >> > > > > >> > > > >> > > > > Hey Steven, >> > > > >> > > > > >> > > > >> > > > > The current proposal is actually to enforce quotas at the >> > > > >> > > > > client/application level, NOT the topic level. So if you >> > have >> > > a >> > > > >> > service >> > > > >> > > > > with a few dozen instances the quota is against all of >> those >> > > > >> > instances >> > > > >> > > > > added up across all their topics. So actually the effect >> > would >> > > > be >> > > > >> the >> > > > >> > > same >> > > > >> > > > > either way but throttling gives the producer the choice of >> > > > either >> > > > >> > > blocking >> > > > >> > > > > or dropping. >> > > > >> > > > > >> > > > >> > > > > -Jay >> > > > >> > > > > >> > > > >> > > > > On Tue, Mar 17, 2015 at 10:08 AM, Steven Wu < >> > > > stevenz...@gmail.com >> > > > >> > >> > > > >> > > wrote: >> > > > >> > > > > >> > > > >> > > > > > Jay, >> > > > >> > > > > > >> > > > >> > > > > > let's say an app produces to 10 different topics. one of >> > the >> > > > >> topic >> > > > >> > is >> > > > >> > > > > sent >> > > > >> > > > > > from a library. due to whatever condition/bug, this lib >> > > starts >> > > > >> to >> > > > >> > > send >> > > > >> > > > > > messages over the quota. if we go with the delayed >> > response >> > > > >> > > approach, it >> > > > >> > > > > > will cause the whole shared RecordAccumulator buffer to >> be >> > > > >> filled >> > > > >> > up. >> > > > >> > > > > that >> > > > >> > > > > > will penalize other 9 topics who are within the quota. >> > that >> > > is >> > > > >> the >> > > > >> > > > > > unfairness point that Ewen and I were trying to make. >> > > > >> > > > > > >> > > > >> > > > > > if broker just drop the msg and return an error/status >> > code >> > > > >> > > indicates the >> > > > >> > > > > > drop and why. then producer can just move on and accept >> > the >> > > > >> drop. >> > > > >> > > shared >> > > > >> > > > > > buffer won't be saturated and other 9 topics won't be >> > > > penalized. >> > > > >> > > > > > >> > > > >> > > > > > Thanks, >> > > > >> > > > > > Steven >> > > > >> > > > > > >> > > > >> > > > > > >> > > > >> > > > > > >> > > > >> > > > > > On Tue, Mar 17, 2015 at 9:44 AM, Jay Kreps < >> > > > jay.kr...@gmail.com >> > > > >> > >> > > > >> > > wrote: >> > > > >> > > > > > >> > > > >> > > > > > > Hey Steven, >> > > > >> > > > > > > >> > > > >> > > > > > > It is true that hitting the quota will cause >> > back-pressure >> > > > on >> > > > >> the >> > > > >> > > > > > producer. >> > > > >> > > > > > > But the solution is simple, a producer that wants to >> > avoid >> > > > >> this >> > > > >> > > should >> > > > >> > > > > > stay >> > > > >> > > > > > > under its quota. In other words this is a contract >> > between >> > > > the >> > > > >> > > cluster >> > > > >> > > > > > and >> > > > >> > > > > > > the client, with each side having something to uphold. >> > > Quite >> > > > >> > > possibly >> > > > >> > > > > the >> > > > >> > > > > > > same thing will happen in the absence of a quota, a >> > client >> > > > >> that >> > > > >> > > > > produces >> > > > >> > > > > > an >> > > > >> > > > > > > unexpected amount of load will hit the limits of the >> > > server >> > > > >> and >> > > > >> > > > > > experience >> > > > >> > > > > > > backpressure. Quotas just allow you to set that same >> > limit >> > > > at >> > > > >> > > something >> > > > >> > > > > > > lower than 100% of all resources on the server, which >> is >> > > > >> useful >> > > > >> > > for a >> > > > >> > > > > > > shared cluster. >> > > > >> > > > > > > >> > > > >> > > > > > > -Jay >> > > > >> > > > > > > >> > > > >> > > > > > > On Mon, Mar 16, 2015 at 11:34 PM, Steven Wu < >> > > > >> > stevenz...@gmail.com> >> > > > >> > > > > > wrote: >> > > > >> > > > > > > >> > > > >> > > > > > > > wait. we create one kafka producer for each cluster. >> > > each >> > > > >> > > cluster can >> > > > >> > > > > > > have >> > > > >> > > > > > > > many topics. if producer buffer got filled up due to >> > > > delayed >> > > > >> > > response >> > > > >> > > > > > for >> > > > >> > > > > > > > one throttled topic, won't that penalize other >> topics >> > > > >> unfairly? >> > > > >> > > it >> > > > >> > > > > > seems >> > > > >> > > > > > > to >> > > > >> > > > > > > > me that broker should just return error without >> delay. >> > > > >> > > > > > > > >> > > > >> > > > > > > > sorry that I am chatting to myself :) >> > > > >> > > > > > > > >> > > > >> > > > > > > > On Mon, Mar 16, 2015 at 11:29 PM, Steven Wu < >> > > > >> > > stevenz...@gmail.com> >> > > > >> > > > > > > wrote: >> > > > >> > > > > > > > >> > > > >> > > > > > > > > I think I can answer my own question. delayed >> > response >> > > > >> will >> > > > >> > > cause >> > > > >> > > > > the >> > > > >> > > > > > > > > producer buffer to be full, which then result in >> > > either >> > > > >> > thread >> > > > >> > > > > > blocking >> > > > >> > > > > > > > or >> > > > >> > > > > > > > > message drop. >> > > > >> > > > > > > > > >> > > > >> > > > > > > > > On Mon, Mar 16, 2015 at 11:24 PM, Steven Wu < >> > > > >> > > stevenz...@gmail.com> >> > > > >> > > > > > > > wrote: >> > > > >> > > > > > > > > >> > > > >> > > > > > > > >> please correct me if I am missing sth here. I am >> > not >> > > > >> > > understanding >> > > > >> > > > > > how >> > > > >> > > > > > > > >> would throttle work without cooperation/back-off >> > from >> > > > >> > > producer. >> > > > >> > > > > new >> > > > >> > > > > > > Java >> > > > >> > > > > > > > >> producer supports non-blocking API. why would >> > delayed >> > > > >> > > response be >> > > > >> > > > > > able >> > > > >> > > > > > > > to >> > > > >> > > > > > > > >> slow down producer? producer will continue to >> fire >> > > > async >> > > > >> > > sends. >> > > > >> > > > > > > > >> >> > > > >> > > > > > > > >> On Mon, Mar 16, 2015 at 10:58 PM, Guozhang Wang < >> > > > >> > > > > wangg...@gmail.com >> > > > >> > > > > > > >> > > > >> > > > > > > > >> wrote: >> > > > >> > > > > > > > >> >> > > > >> > > > > > > > >>> I think we are really discussing two separate >> > issues >> > > > >> here: >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> 1. Whether we should a) >> > > > >> > > > > append-then-block-then-returnOKButThrottled >> > > > >> > > > > > > or >> > > > >> > > > > > > > b) >> > > > >> > > > > > > > >>> block-then-returnFailDuetoThrottled for quota >> > > actions >> > > > on >> > > > >> > > produce >> > > > >> > > > > > > > >>> requests. >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> Both these approaches assume some kind of >> > > > >> well-behaveness >> > > > >> > of >> > > > >> > > the >> > > > >> > > > > > > > clients: >> > > > >> > > > > > > > >>> option a) assumes the client sets an proper >> > timeout >> > > > >> value >> > > > >> > > while >> > > > >> > > > > can >> > > > >> > > > > > > > just >> > > > >> > > > > > > > >>> ignore "OKButThrottled" response, while option >> b) >> > > > >> assumes >> > > > >> > the >> > > > >> > > > > > client >> > > > >> > > > > > > > >>> handles the "FailDuetoThrottled" appropriately. >> > For >> > > > any >> > > > >> > > malicious >> > > > >> > > > > > > > clients >> > > > >> > > > > > > > >>> that, for example, just keep retrying either >> > > > >> intentionally >> > > > >> > or >> > > > >> > > > > not, >> > > > >> > > > > > > > >>> neither >> > > > >> > > > > > > > >>> of these approaches are actually effective. >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> 2. For "OKButThrottled" and "FailDuetoThrottled" >> > > > >> responses, >> > > > >> > > shall >> > > > >> > > > > > we >> > > > >> > > > > > > > >>> encode >> > > > >> > > > > > > > >>> them as error codes or augment the protocol to >> > use a >> > > > >> > separate >> > > > >> > > > > field >> > > > >> > > > > > > > >>> indicating "status codes". >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> Today we have already incorporated some status >> > code >> > > as >> > > > >> > error >> > > > >> > > > > codes >> > > > >> > > > > > in >> > > > >> > > > > > > > the >> > > > >> > > > > > > > >>> responses, e.g. ReplicaNotAvailable in >> > > > MetadataResponse, >> > > > >> > the >> > > > >> > > pros >> > > > >> > > > > > of >> > > > >> > > > > > > > this >> > > > >> > > > > > > > >>> is of course using a single field for response >> > > status >> > > > >> like >> > > > >> > > the >> > > > >> > > > > HTTP >> > > > >> > > > > > > > >>> status >> > > > >> > > > > > > > >>> codes, while the cons is that it requires >> clients >> > to >> > > > >> handle >> > > > >> > > the >> > > > >> > > > > > error >> > > > >> > > > > > > > >>> codes >> > > > >> > > > > > > > >>> carefully. >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> I think maybe we can actually extend the >> > single-code >> > > > >> > > approach to >> > > > >> > > > > > > > overcome >> > > > >> > > > > > > > >>> its drawbacks, that is, wrap the error codes >> > > semantics >> > > > >> to >> > > > >> > the >> > > > >> > > > > users >> > > > >> > > > > > > so >> > > > >> > > > > > > > >>> that >> > > > >> > > > > > > > >>> users do not need to handle the codes >> one-by-one. >> > > More >> > > > >> > > > > concretely, >> > > > >> > > > > > > > >>> following Jay's example the client could write >> > sth. >> > > > like >> > > > >> > > this: >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> ----------------- >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> if(error.isOK()) >> > > > >> > > > > > > > >>> // status code is good or the code can be >> > > simply >> > > > >> > > ignored for >> > > > >> > > > > > > this >> > > > >> > > > > > > > >>> request type, process the request >> > > > >> > > > > > > > >>> else if(error.needsRetry()) >> > > > >> > > > > > > > >>> // throttled, transient error, etc: retry >> > > > >> > > > > > > > >>> else if(error.isFatal()) >> > > > >> > > > > > > > >>> // non-retriable errors, etc: notify / >> > > terminate >> > > > / >> > > > >> > other >> > > > >> > > > > > > handling >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> ----------------- >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> Only when the clients really want to handle, for >> > > > example >> > > > >> > > > > > > > >>> FailDuetoThrottled >> > > > >> > > > > > > > >>> status code specifically, it needs to: >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> if(error.isOK()) >> > > > >> > > > > > > > >>> // status code is good or the code can be >> > > simply >> > > > >> > > ignored for >> > > > >> > > > > > > this >> > > > >> > > > > > > > >>> request type, process the request >> > > > >> > > > > > > > >>> else if(error == FailDuetoThrottled ) >> > > > >> > > > > > > > >>> // throttled: log it >> > > > >> > > > > > > > >>> else if(error.needsRetry()) >> > > > >> > > > > > > > >>> // transient error, etc: retry >> > > > >> > > > > > > > >>> else if(error.isFatal()) >> > > > >> > > > > > > > >>> // non-retriable errors, etc: notify / >> > > terminate >> > > > / >> > > > >> > other >> > > > >> > > > > > > handling >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> ----------------- >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> And for implementation we can probably group the >> > > codes >> > > > >> > > > > accordingly >> > > > >> > > > > > > like >> > > > >> > > > > > > > >>> HTTP status code such that we can do: >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> boolean Error.isOK() { >> > > > >> > > > > > > > >>> return code < 300 && code >= 200; >> > > > >> > > > > > > > >>> } >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> Guozhang >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> On Mon, Mar 16, 2015 at 10:24 PM, Ewen >> > > > Cheslack-Postava >> > > > >> < >> > > > >> > > > > > > > >>> e...@confluent.io> >> > > > >> > > > > > > > >>> wrote: >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> > Agreed that trying to shoehorn non-error codes >> > > into >> > > > >> the >> > > > >> > > error >> > > > >> > > > > > field >> > > > >> > > > > > > > is >> > > > >> > > > > > > > >>> a >> > > > >> > > > > > > > >>> > bad idea. It makes it *way* too easy to write >> > code >> > > > >> that >> > > > >> > > looks >> > > > >> > > > > > (and >> > > > >> > > > > > > > >>> should >> > > > >> > > > > > > > >>> > be) correct but is actually incorrect. If >> > > > necessary, I >> > > > >> > > think >> > > > >> > > > > it's >> > > > >> > > > > > > > much >> > > > >> > > > > > > > >>> > better to to spend a couple of extra bytes to >> > > encode >> > > > >> that >> > > > >> > > > > > > information >> > > > >> > > > > > > > >>> > separately (a "status" or "warning" section of >> > the >> > > > >> > > response). >> > > > >> > > > > An >> > > > >> > > > > > > > >>> indication >> > > > >> > > > > > > > >>> > that throttling is occurring is something I'd >> > > expect >> > > > >> to >> > > > >> > be >> > > > >> > > > > > > indicated >> > > > >> > > > > > > > >>> by a >> > > > >> > > > > > > > >>> > bit flag in the response rather than as an >> error >> > > > code. >> > > > >> > > > > > > > >>> > >> > > > >> > > > > > > > >>> > Gwen - I think an error code makes sense when >> > the >> > > > >> request >> > > > >> > > > > > actually >> > > > >> > > > > > > > >>> failed. >> > > > >> > > > > > > > >>> > Option B, which Jun was advocating, would have >> > > > >> appended >> > > > >> > the >> > > > >> > > > > > > messages >> > > > >> > > > > > > > >>> > successfully. If the rate-limiting case you're >> > > > talking >> > > > >> > > about >> > > > >> > > > > had >> > > > >> > > > > > > > >>> > successfully committed the messages, I would >> say >> > > > >> that's >> > > > >> > > also a >> > > > >> > > > > > bad >> > > > >> > > > > > > > use >> > > > >> > > > > > > > >>> of >> > > > >> > > > > > > > >>> > error codes. >> > > > >> > > > > > > > >>> > >> > > > >> > > > > > > > >>> > >> > > > >> > > > > > > > >>> > On Mon, Mar 16, 2015 at 10:16 PM, Gwen >> Shapira < >> > > > >> > > > > > > > gshap...@cloudera.com> >> > > > >> > > > > > > > >>> > wrote: >> > > > >> > > > > > > > >>> > >> > > > >> > > > > > > > >>> > > We discussed an error code for rate-limiting >> > > > (which >> > > > >> I >> > > > >> > > think >> > > > >> > > > > > made >> > > > >> > > > > > > > >>> > > sense), isn't it a similar case? >> > > > >> > > > > > > > >>> > > >> > > > >> > > > > > > > >>> > > On Mon, Mar 16, 2015 at 10:10 PM, Jay Kreps >> < >> > > > >> > > > > > jay.kr...@gmail.com >> > > > >> > > > > > > > >> > > > >> > > > > > > > >>> wrote: >> > > > >> > > > > > > > >>> > > > My concern is that as soon as you start >> > > encoding >> > > > >> > > non-error >> > > > >> > > > > > > > response >> > > > >> > > > > > > > >>> > > > information into error codes the next >> > question >> > > > is >> > > > >> > what >> > > > >> > > to >> > > > >> > > > > do >> > > > >> > > > > > if >> > > > >> > > > > > > > two >> > > > >> > > > > > > > >>> > such >> > > > >> > > > > > > > >>> > > > codes apply (i.e. you have a replica down >> > and >> > > > the >> > > > >> > > response >> > > > >> > > > > is >> > > > >> > > > > > > > >>> > quota'd). I >> > > > >> > > > > > > > >>> > > > think I am trying to argue that error >> should >> > > > mean >> > > > >> > "why >> > > > >> > > we >> > > > >> > > > > > > failed >> > > > >> > > > > > > > >>> your >> > > > >> > > > > > > > >>> > > > request", for which there will really only >> > be >> > > > one >> > > > >> > > reason, >> > > > >> > > > > and >> > > > >> > > > > > > any >> > > > >> > > > > > > > >>> other >> > > > >> > > > > > > > >>> > > > useful information we want to send back is >> > > just >> > > > >> > another >> > > > >> > > > > field >> > > > >> > > > > > > in >> > > > >> > > > > > > > >>> the >> > > > >> > > > > > > > >>> > > > response. >> > > > >> > > > > > > > >>> > > > >> > > > >> > > > > > > > >>> > > > -Jay >> > > > >> > > > > > > > >>> > > > >> > > > >> > > > > > > > >>> > > > On Mon, Mar 16, 2015 at 9:51 PM, Gwen >> > Shapira >> > > < >> > > > >> > > > > > > > >>> gshap...@cloudera.com> >> > > > >> > > > > > > > >>> > > wrote: >> > > > >> > > > > > > > >>> > > > >> > > > >> > > > > > > > >>> > > >> I think its not too late to reserve a set >> > of >> > > > >> error >> > > > >> > > codes >> > > > >> > > > > > > > >>> (200-299?) >> > > > >> > > > > > > > >>> > > >> for "non-error" codes. >> > > > >> > > > > > > > >>> > > >> >> > > > >> > > > > > > > >>> > > >> It won't be backward compatible (i.e. >> > clients >> > > > >> that >> > > > >> > > > > currently >> > > > >> > > > > > > do >> > > > >> > > > > > > > >>> "else >> > > > >> > > > > > > > >>> > > >> throw" will throw on non-errors), but >> > perhaps >> > > > its >> > > > >> > > > > > worthwhile. >> > > > >> > > > > > > > >>> > > >> >> > > > >> > > > > > > > >>> > > >> On Mon, Mar 16, 2015 at 9:42 PM, Jay >> Kreps >> > < >> > > > >> > > > > > > jay.kr...@gmail.com >> > > > >> > > > > > > > > >> > > > >> > > > > > > > >>> > wrote: >> > > > >> > > > > > > > >>> > > >> > Hey Jun, >> > > > >> > > > > > > > >>> > > >> > >> > > > >> > > > > > > > >>> > > >> > I'd really really really like to avoid >> > > that. >> > > > >> > Having >> > > > >> > > just >> > > > >> > > > > > > > spent a >> > > > >> > > > > > > > >>> > > bunch of >> > > > >> > > > > > > > >>> > > >> > time on the clients, using the error >> > codes >> > > to >> > > > >> > encode >> > > > >> > > > > other >> > > > >> > > > > > > > >>> > information >> > > > >> > > > > > > > >>> > > >> > about the response is super dangerous. >> > The >> > > > >> error >> > > > >> > > > > handling >> > > > >> > > > > > is >> > > > >> > > > > > > > >>> one of >> > > > >> > > > > > > > >>> > > the >> > > > >> > > > > > > > >>> > > >> > hardest parts of the client (Guozhang >> > chime >> > > > in >> > > > >> > > here). >> > > > >> > > > > > > > >>> > > >> > >> > > > >> > > > > > > > >>> > > >> > Generally the error handling looks like >> > > > >> > > > > > > > >>> > > >> > if(error == none) >> > > > >> > > > > > > > >>> > > >> > // good, process the request >> > > > >> > > > > > > > >>> > > >> > else if(error == KNOWN_ERROR_1) >> > > > >> > > > > > > > >>> > > >> > // handle known error 1 >> > > > >> > > > > > > > >>> > > >> > else if(error == KNOWN_ERROR_2) >> > > > >> > > > > > > > >>> > > >> > // handle known error 2 >> > > > >> > > > > > > > >>> > > >> > else >> > > > >> > > > > > > > >>> > > >> > throw >> > > Errors.forCode(error).exception(); >> > > > >> // >> > > > >> > or >> > > > >> > > some >> > > > >> > > > > > > other >> > > > >> > > > > > > > >>> > default >> > > > >> > > > > > > > >>> > > >> > behavior >> > > > >> > > > > > > > >>> > > >> > >> > > > >> > > > > > > > >>> > > >> > This works because we have a convention >> > > that >> > > > >> and >> > > > >> > > error >> > > > >> > > > > is >> > > > >> > > > > > > > >>> something >> > > > >> > > > > > > > >>> > > that >> > > > >> > > > > > > > >>> > > >> > prevented your getting the response so >> > the >> > > > >> default >> > > > >> > > > > > handling >> > > > >> > > > > > > > >>> case is >> > > > >> > > > > > > > >>> > > sane >> > > > >> > > > > > > > >>> > > >> > and forward compatible. It is tempting >> to >> > > use >> > > > >> the >> > > > >> > > error >> > > > >> > > > > > code >> > > > >> > > > > > > > to >> > > > >> > > > > > > > >>> > convey >> > > > >> > > > > > > > >>> > > >> > information in the success case. For >> > > example >> > > > we >> > > > >> > > could >> > > > >> > > > > use >> > > > >> > > > > > > > error >> > > > >> > > > > > > > >>> > codes >> > > > >> > > > > > > > >>> > > to >> > > > >> > > > > > > > >>> > > >> > encode whether quotas were enforced, >> > > whether >> > > > >> the >> > > > >> > > request >> > > > >> > > > > > was >> > > > >> > > > > > > > >>> served >> > > > >> > > > > > > > >>> > > out >> > > > >> > > > > > > > >>> > > >> of >> > > > >> > > > > > > > >>> > > >> > cache, whether the stock market is up >> > > today, >> > > > or >> > > > >> > > > > whatever. >> > > > >> > > > > > > The >> > > > >> > > > > > > > >>> > problem >> > > > >> > > > > > > > >>> > > is >> > > > >> > > > > > > > >>> > > >> > that since these are not errors as far >> as >> > > the >> > > > >> > > client is >> > > > >> > > > > > > > >>> concerned it >> > > > >> > > > > > > > >>> > > >> should >> > > > >> > > > > > > > >>> > > >> > not throw an exception but process the >> > > > >> response, >> > > > >> > > but now >> > > > >> > > > > > we >> > > > >> > > > > > > > >>> created >> > > > >> > > > > > > > >>> > an >> > > > >> > > > > > > > >>> > > >> > explicit requirement that that error be >> > > > handled >> > > > >> > > > > explicitly >> > > > >> > > > > > > > >>> since it >> > > > >> > > > > > > > >>> > is >> > > > >> > > > > > > > >>> > > >> > different. I really think that this >> kind >> > of >> > > > >> > > information >> > > > >> > > > > is >> > > > >> > > > > > > not >> > > > >> > > > > > > > >>> an >> > > > >> > > > > > > > >>> > > error, >> > > > >> > > > > > > > >>> > > >> it >> > > > >> > > > > > > > >>> > > >> > is just information, and if we want it >> in >> > > the >> > > > >> > > response >> > > > >> > > > > we >> > > > >> > > > > > > > >>> should do >> > > > >> > > > > > > > >>> > > the >> > > > >> > > > > > > > >>> > > >> > right thing and add a new field to the >> > > > >> response. >> > > > >> > > > > > > > >>> > > >> > >> > > > >> > > > > > > > >>> > > >> > I think you saw the Samza bug that was >> > > > >> literally >> > > > >> > an >> > > > >> > > > > > example >> > > > >> > > > > > > of >> > > > >> > > > > > > > >>> this >> > > > >> > > > > > > > >>> > > >> > happening and leading to an infinite >> > retry >> > > > >> loop. >> > > > >> > > > > > > > >>> > > >> > >> > > > >> > > > > > > > >>> > > >> > Further more I really want to emphasize >> > > that >> > > > >> > hitting >> > > > >> > > > > your >> > > > >> > > > > > > > quota >> > > > >> > > > > > > > >>> in >> > > > >> > > > > > > > >>> > the >> > > > >> > > > > > > > >>> > > >> > design that Adi has proposed is >> actually >> > > not >> > > > an >> > > > >> > > error >> > > > >> > > > > > > > condition >> > > > >> > > > > > > > >>> at >> > > > >> > > > > > > > >>> > > all. >> > > > >> > > > > > > > >>> > > >> It >> > > > >> > > > > > > > >>> > > >> > is totally reasonable in any bootstrap >> > > > >> situation >> > > > >> > to >> > > > >> > > > > > > > >>> intentionally >> > > > >> > > > > > > > >>> > > want to >> > > > >> > > > > > > > >>> > > >> > run at the limit the system imposes on >> > you. >> > > > >> > > > > > > > >>> > > >> > >> > > > >> > > > > > > > >>> > > >> > -Jay >> > > > >> > > > > > > > >>> > > >> > >> > > > >> > > > > > > > >>> > > >> > >> > > > >> > > > > > > > >>> > > >> > >> > > > >> > > > > > > > >>> > > >> > On Mon, Mar 16, 2015 at 4:27 PM, Jun >> Rao >> > < >> > > > >> > > > > > j...@confluent.io> >> > > > >> > > > > > > > >>> wrote: >> > > > >> > > > > > > > >>> > > >> > >> > > > >> > > > > > > > >>> > > >> >> It's probably useful for a client to >> > know >> > > > >> whether >> > > > >> > > its >> > > > >> > > > > > > > requests >> > > > >> > > > > > > > >>> are >> > > > >> > > > > > > > >>> > > >> >> throttled or not (e.g., for monitoring >> > and >> > > > >> > > alerting). >> > > > >> > > > > > From >> > > > >> > > > > > > > that >> > > > >> > > > > > > > >>> > > >> >> perspective, option B (delay the >> > requests >> > > > and >> > > > >> > > return an >> > > > >> > > > > > > > error) >> > > > >> > > > > > > > >>> > seems >> > > > >> > > > > > > > >>> > > >> >> better. >> > > > >> > > > > > > > >>> > > >> >> >> > > > >> > > > > > > > >>> > > >> >> Thanks, >> > > > >> > > > > > > > >>> > > >> >> >> > > > >> > > > > > > > >>> > > >> >> Jun >> > > > >> > > > > > > > >>> > > >> >> >> > > > >> > > > > > > > >>> > > >> >> On Wed, Mar 4, 2015 at 3:51 PM, Aditya >> > > > >> Auradkar < >> > > > >> > > > > > > > >>> > > >> >> aaurad...@linkedin.com.invalid> >> wrote: >> > > > >> > > > > > > > >>> > > >> >> >> > > > >> > > > > > > > >>> > > >> >> > Posted a KIP for quotas in kafka. >> > > > >> > > > > > > > >>> > > >> >> > >> > > > >> > > > > > > > >>> > >> > > > >> > > > > > >> > > > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-13+-+Quotas >> > > > >> > > > > > > > >>> > > >> >> > >> > > > >> > > > > > > > >>> > > >> >> > Appreciate any feedback. >> > > > >> > > > > > > > >>> > > >> >> > >> > > > >> > > > > > > > >>> > > >> >> > Aditya >> > > > >> > > > > > > > >>> > > >> >> > >> > > > >> > > > > > > > >>> > > >> >> >> > > > >> > > > > > > > >>> > > >> >> > > > >> > > > > > > > >>> > > >> > > > >> > > > > > > > >>> > >> > > > >> > > > > > > > >>> > >> > > > >> > > > > > > > >>> > >> > > > >> > > > > > > > >>> > -- >> > > > >> > > > > > > > >>> > Thanks, >> > > > >> > > > > > > > >>> > Ewen >> > > > >> > > > > > > > >>> > >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >>> -- >> > > > >> > > > > > > > >>> -- Guozhang >> > > > >> > > > > > > > >>> >> > > > >> > > > > > > > >> >> > > > >> > > > > > > > >> >> > > > >> > > > > > > > > >> > > > >> > > > > > > > >> > > > >> > > > > > > >> > > > >> > > > > > >> > > > >> > > > > >> > > > >> > > >> > > > >> > > >> > > > >> > >> > > > >> >> > > > > >> > > > > >> > > > > -- >> > > > > Sent from Gmail Mobile >> > > > > >> > > > >> > > > >> > > > -- >> > > > Sent from Gmail Mobile >> > > > >> > > >> > >>