Re: New Producer and acks configuration

2015-07-28 Thread Jiangjie Qin
I seems to me that performance wide, acks=1 and acks=0 will be pretty much the same if max.in.flight.request.per.connection is set to very high and does not cause a request sending to block. In new producer, if there are send failure from time to time, I would guess asks=1 would even have better

Re: New Producer and acks configuration

2015-07-28 Thread Jay Kreps
For the new java producer this doesn't result in a big perf difference (at least in my testing) so there isn't a good reason to use acks=0. However this is a protocol and server feature, not a client feature. For the scala client and many of the other blocking clients the perf difference is quite

Re: New Producer and acks configuration

2015-07-28 Thread Gwen Shapira
But on the server side, the code path for acks = 0 and acks = 1 is the same... its up to the client to implement acks = 0 and acks = 1 correctly (the way our java client fakes a response for acks = 0). So its not really a server feature, more of a protocol feature that needs to be implemented

Re: New Producer and acks configuration

2015-07-28 Thread Gwen Shapira
Would also be nice if max inflights requests was documented :) https://issues.apache.org/jira/browse/KAFKA-2255 This is one of those things that would be nice to mention in docs On Mon, Jul 27, 2015 at 10:57 AM, Ewen Cheslack-Postava e...@confluent.io wrote: If only we had some sort of

Re: New Producer and acks configuration

2015-07-27 Thread Guozhang Wang
I think there is still a subtle difference between async with acks = 0 and async with callback, that when the #.max-inflight-requests has reached the subsequent requests cannot be sent until previous responses are returned (which could happen, for example, when the broker is slow / network issue

Re: New Producer and acks configuration

2015-07-27 Thread Mayuresh Gharat
So basically this means that with acks = 0, their is no guarantee that the message has been received by Kafka broker. I am just wondering, why would anyone be using acks = 0, since anyone using kafka and doing producer.send() would want that, their message got to kafka brokers. Also as Jay said,

Re: New Producer and acks configuration

2015-07-27 Thread Gwen Shapira
Yeah, using acks=0 should result in higher throughput since we are not limited by the roundtrip time to the broker. Btw. regarding in-flight requests: With acks = 1 (or -1), can we send a message batch to a partition before the brokers acked a previous request? Doesn't it risk getting messages

Re: New Producer and acks configuration

2015-07-27 Thread Ewen Cheslack-Postava
If only we had some sort of system test framework with a producer performance test that we could parameterize with the different acks settings to validate these performance differences... wrt out of order: yes, with 1 in flight requests with retries, messages can get out of order. Becket had a

Re: New Producer and acks configuration

2015-07-26 Thread Gwen Shapira
Aha! Yes, I was missing the part with the dummy response. Thank you! Gwen On Sun, Jul 26, 2015 at 2:17 PM, Ewen Cheslack-Postava e...@confluent.io wrote: It's different because it changes whether the client waits for the response from the broker at all. Take a look at

Re: New Producer and acks configuration

2015-07-26 Thread Ewen Cheslack-Postava
It's different because it changes whether the client waits for the response from the broker at all. Take a look at NetworkClient.handleCompletedSends, which fills in dummy responses when a response is not expected (and that flag gets set via Sender.produceRequest using acks != 0 as a flag to

Re: New Producer and acks configuration

2015-07-20 Thread Jay Kreps
acks=0 is a one-way send, the client doesn't need to wait on the response. Whether this is useful sort of depends on the client implementation. The new java producer does all sends async so waiting on a response isn't really a thing. For a client that lacks this, though, as some of them do, acks=0

New Producer and acks configuration

2015-07-18 Thread Gwen Shapira
Hi, I was looking into the different between acks = 0 and acks = 1 in the new producer, and was a bit surprised at what I found. Basically, if I understand correctly, the only difference is that with acks = 0, if the leader fails to append locally, it closes the network connection silently and