Ewen, I see your point regarding the shared buffer. yes, a bad/slow broker could potentially consume up all buffer. On the other hand, I do like the batching behavior of shared RecordAccumulator buffer.
On Tue, Mar 17, 2015 at 8:25 AM, Guozhang Wang <wangg...@gmail.com> wrote: > Ewen, > > 1. I think we are on the same page as per "malicious clients", that it > should not be the target of either approach. I was just trying to separate > the discussion from "what if user just keep retrying" and maybe I was not > clear. > > 2. I was not advocating option A on the wiki, in my previous email I > actually assume that option is already dropped and we are only considering > option B (which is my option b) in the email) and C (option a) in my > email), and I think with some proper wrapping of "status codes" (today we > still call them error codes) option B in the wiki may not necessarily > require people who implement clients to handle each status code one-by-one. > > Guozhang > > On Tue, Mar 17, 2015 at 12:22 AM, Ewen Cheslack-Postava <e...@confluent.io > > > wrote: > > > Steven - that's a reasonable concern. I think I've mentioned the same > sort > > of issue in the issues about the new producer's RecordAccumulator not > > timing out sends, e.g. in > https://issues.apache.org/jira/browse/KAFKA-1788 > > . > > The shared buffer causes problems if one broker isn't available for > awhile > > since messages to that broker end up consuming the entire buffer. You can > > end up with a similar problem here due to the effective rate limiting > > caused by delaying responses. > > > > Guozhang - I think only option A from the KIP is actually an error. If we > > want to look to HTTP for examples, there's an RFC that defines the Too > Many > > Requests response to handle rate limiting: > > http://tools.ietf.org/html/rfc6585#page-3 In this case, it actually is > an > > error, specifically a client error since its in the 400 range.The > > implication from the status code (429), name of the response, and the > > example given is that is is an error and no real data is returned, which > > would correspond to option A from the KIP. Note that the protocol > provides > > a mechanism for giving extra (optional) information about when you should > > retry (via headers). I'd guess that even despite that, most systems that > > encounter a 429 use some ad hoc backoff mechanism because they only try > to > > detect anything in the 400 range... > > > > One additional point -- I think "malicious clients" shouldn't be our > target > > here, they can do a lot worse than what's been addressed in this thread. > > But I do agree that any proposal should have a clear explanation of how > > existing clients that are ignorant of quotas would behave (which is why > > options b and c make a lot of sense -- they rate limit without requiring > an > > update to normally-behaving clients). > > > > > > 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 > > > >>> > > > >> > > > >> > > > > > > > > > > > > > > > -- > > Thanks, > > Ewen > > > > > > -- > -- Guozhang >