Re: New Producer and acks configuration
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 performance than acks=0. Because in acks=0, when error occurred, broker will disconnect the connection. In that case, subsequent send from the producer needs to reconnect to the broker, that means it has to go through 3-way handshake, TCP slow-start, etc, etc. On the other hand, acks=1 will not have this issue. Maybe the major difference is still the delivery guarantee? acks=0 means send and forget while acks=1 means user still want to know if the messages were sent successfully or not. On Mon, Jul 27, 2015 at 10:57 AM, Ewen Cheslack-Postava e...@confluent.io wrote: 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 great presentation addressing that and a bunch of other issues with no data loss pipelines: http://www.slideshare.net/JiangjieQin/no-data-loss-pipeline-with-apache-kafka-49753844 Short version: as things are today, you have to *really* understand the producer settings, and some producer internals, to get the exact behavior you want. On Mon, Jul 27, 2015 at 9:44 AM, Gwen Shapira gshap...@cloudera.com wrote: 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 out of order? On Mon, Jul 27, 2015 at 9:41 AM, Guozhang Wang wangg...@gmail.com wrote: 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 happens) in the second case but not in the first. Given this difference, I feel there are still scenarios, though probably rare, that users would like to use acks = 0 even with new producer's callbacks. Guozhang On Mon, Jul 27, 2015 at 9:25 AM, Mayuresh Gharat gharatmayures...@gmail.com wrote: 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, with new producer with async mode, clients will not have to wait for the response since it will be handled in callbacks. So the use of acks = 0 sounds very rare to me and I am not able to think of an usecase around it. Thanks, Mayuresh On Sun, Jul 26, 2015 at 2:40 PM, Gwen Shapira gshap...@cloudera.com wrote: 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 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 ClientRequest). This means that the producer will invoke the callback resolve the future as soon as the request hits the TCP buffer on the client. At that point, the behavior of the broker wrt generating a response doesn't matter -- the client isn't waiting on that response anyway. This definitely is faster since you aren't waiting for the round trip, but it seems like it is of questionable value with the new producer as Jay explained. It is slightly better than just assuming records have been sent as soon as you call Producer.send() in this shouldn't trigger a callback until the records have made it through the internal KafkaProducer buffering. But since it still has to make it through the TCP buffers it doesn't really guarantee anything that useful. -Ewen On Sun, Jul 26, 2015 at 1:40 PM, Gwen Shapira gshap...@cloudera.com wrote: What bugs me is that even with acks = 0, the broker will append to local log before responding (unless I'm misreading the code), so I don't see why a client with acks = 0 will be any faster.
Re: New Producer and acks configuration
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 substantial since the latency of waiting for the ack blocks the client and reduces throughput. -Jay On Tue, Jul 28, 2015 at 10:21 AM, Jiangjie Qin j...@linkedin.com.invalid wrote: 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 performance than acks=0. Because in acks=0, when error occurred, broker will disconnect the connection. In that case, subsequent send from the producer needs to reconnect to the broker, that means it has to go through 3-way handshake, TCP slow-start, etc, etc. On the other hand, acks=1 will not have this issue. Maybe the major difference is still the delivery guarantee? acks=0 means send and forget while acks=1 means user still want to know if the messages were sent successfully or not. On Mon, Jul 27, 2015 at 10:57 AM, Ewen Cheslack-Postava e...@confluent.io wrote: 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 great presentation addressing that and a bunch of other issues with no data loss pipelines: http://www.slideshare.net/JiangjieQin/no-data-loss-pipeline-with-apache-kafka-49753844 Short version: as things are today, you have to *really* understand the producer settings, and some producer internals, to get the exact behavior you want. On Mon, Jul 27, 2015 at 9:44 AM, Gwen Shapira gshap...@cloudera.com wrote: 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 out of order? On Mon, Jul 27, 2015 at 9:41 AM, Guozhang Wang wangg...@gmail.com wrote: 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 happens) in the second case but not in the first. Given this difference, I feel there are still scenarios, though probably rare, that users would like to use acks = 0 even with new producer's callbacks. Guozhang On Mon, Jul 27, 2015 at 9:25 AM, Mayuresh Gharat gharatmayures...@gmail.com wrote: 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, with new producer with async mode, clients will not have to wait for the response since it will be handled in callbacks. So the use of acks = 0 sounds very rare to me and I am not able to think of an usecase around it. Thanks, Mayuresh On Sun, Jul 26, 2015 at 2:40 PM, Gwen Shapira gshap...@cloudera.com wrote: 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 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 ClientRequest). This means that the producer will invoke the callback resolve the future as soon as the request hits the TCP buffer on the client. At that point, the behavior of the broker wrt generating a response doesn't matter -- the client isn't waiting on that response anyway. This definitely is faster since you aren't waiting for the round trip, but it seems like it is of questionable value with the new producer as Jay explained. It is slightly better than just assuming records have been sent
Re: New Producer and acks configuration
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 correctly by clients. On Tue, Jul 28, 2015 at 11:40 AM, Jay Kreps j...@confluent.io wrote: 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 substantial since the latency of waiting for the ack blocks the client and reduces throughput. -Jay On Tue, Jul 28, 2015 at 10:21 AM, Jiangjie Qin j...@linkedin.com.invalid wrote: 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 performance than acks=0. Because in acks=0, when error occurred, broker will disconnect the connection. In that case, subsequent send from the producer needs to reconnect to the broker, that means it has to go through 3-way handshake, TCP slow-start, etc, etc. On the other hand, acks=1 will not have this issue. Maybe the major difference is still the delivery guarantee? acks=0 means send and forget while acks=1 means user still want to know if the messages were sent successfully or not. On Mon, Jul 27, 2015 at 10:57 AM, Ewen Cheslack-Postava e...@confluent.io wrote: 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 great presentation addressing that and a bunch of other issues with no data loss pipelines: http://www.slideshare.net/JiangjieQin/no-data-loss-pipeline-with-apache-kafka-49753844 Short version: as things are today, you have to *really* understand the producer settings, and some producer internals, to get the exact behavior you want. On Mon, Jul 27, 2015 at 9:44 AM, Gwen Shapira gshap...@cloudera.com wrote: 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 out of order? On Mon, Jul 27, 2015 at 9:41 AM, Guozhang Wang wangg...@gmail.com wrote: 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 happens) in the second case but not in the first. Given this difference, I feel there are still scenarios, though probably rare, that users would like to use acks = 0 even with new producer's callbacks. Guozhang On Mon, Jul 27, 2015 at 9:25 AM, Mayuresh Gharat gharatmayures...@gmail.com wrote: 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, with new producer with async mode, clients will not have to wait for the response since it will be handled in callbacks. So the use of acks = 0 sounds very rare to me and I am not able to think of an usecase around it. Thanks, Mayuresh On Sun, Jul 26, 2015 at 2:40 PM, Gwen Shapira gshap...@cloudera.com wrote: 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 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 ClientRequest). This means that the producer will invoke the callback resolve the future as soon as the request hits the TCP buffer on the client. At that point, the
Re: New Producer and acks configuration
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 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 great presentation addressing that and a bunch of other issues with no data loss pipelines: http://www.slideshare.net/JiangjieQin/no-data-loss-pipeline-with-apache-kafka-49753844 Short version: as things are today, you have to *really* understand the producer settings, and some producer internals, to get the exact behavior you want. On Mon, Jul 27, 2015 at 9:44 AM, Gwen Shapira gshap...@cloudera.com wrote: 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 out of order? On Mon, Jul 27, 2015 at 9:41 AM, Guozhang Wang wangg...@gmail.com wrote: 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 happens) in the second case but not in the first. Given this difference, I feel there are still scenarios, though probably rare, that users would like to use acks = 0 even with new producer's callbacks. Guozhang On Mon, Jul 27, 2015 at 9:25 AM, Mayuresh Gharat gharatmayures...@gmail.com wrote: 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, with new producer with async mode, clients will not have to wait for the response since it will be handled in callbacks. So the use of acks = 0 sounds very rare to me and I am not able to think of an usecase around it. Thanks, Mayuresh On Sun, Jul 26, 2015 at 2:40 PM, Gwen Shapira gshap...@cloudera.com wrote: 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 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 ClientRequest). This means that the producer will invoke the callback resolve the future as soon as the request hits the TCP buffer on the client. At that point, the behavior of the broker wrt generating a response doesn't matter -- the client isn't waiting on that response anyway. This definitely is faster since you aren't waiting for the round trip, but it seems like it is of questionable value with the new producer as Jay explained. It is slightly better than just assuming records have been sent as soon as you call Producer.send() in this shouldn't trigger a callback until the records have made it through the internal KafkaProducer buffering. But since it still has to make it through the TCP buffers it doesn't really guarantee anything that useful. -Ewen On Sun, Jul 26, 2015 at 1:40 PM, Gwen Shapira gshap...@cloudera.com wrote: What bugs me is that even with acks = 0, the broker will append to local log before responding (unless I'm misreading the code), so I don't see why a client with acks = 0 will be any faster. Unless the client chooses to not wait for response, which is orthogonal to acks parameter. On Mon, Jul 20, 2015 at 8:52 AM, Jay Kreps j...@confluent.io wrote: 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 will be a lot faster. It also makes some sense in terms of what is completed when the request is considered satisfied acks = 0 - message is written to the network (buffer)
Re: New Producer and acks configuration
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 happens) in the second case but not in the first. Given this difference, I feel there are still scenarios, though probably rare, that users would like to use acks = 0 even with new producer's callbacks. Guozhang On Mon, Jul 27, 2015 at 9:25 AM, Mayuresh Gharat gharatmayures...@gmail.com wrote: 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, with new producer with async mode, clients will not have to wait for the response since it will be handled in callbacks. So the use of acks = 0 sounds very rare to me and I am not able to think of an usecase around it. Thanks, Mayuresh On Sun, Jul 26, 2015 at 2:40 PM, Gwen Shapira gshap...@cloudera.com wrote: 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 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 ClientRequest). This means that the producer will invoke the callback resolve the future as soon as the request hits the TCP buffer on the client. At that point, the behavior of the broker wrt generating a response doesn't matter -- the client isn't waiting on that response anyway. This definitely is faster since you aren't waiting for the round trip, but it seems like it is of questionable value with the new producer as Jay explained. It is slightly better than just assuming records have been sent as soon as you call Producer.send() in this shouldn't trigger a callback until the records have made it through the internal KafkaProducer buffering. But since it still has to make it through the TCP buffers it doesn't really guarantee anything that useful. -Ewen On Sun, Jul 26, 2015 at 1:40 PM, Gwen Shapira gshap...@cloudera.com wrote: What bugs me is that even with acks = 0, the broker will append to local log before responding (unless I'm misreading the code), so I don't see why a client with acks = 0 will be any faster. Unless the client chooses to not wait for response, which is orthogonal to acks parameter. On Mon, Jul 20, 2015 at 8:52 AM, Jay Kreps j...@confluent.io wrote: 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 will be a lot faster. It also makes some sense in terms of what is completed when the request is considered satisfied acks = 0 - message is written to the network (buffer) acks = 1 - message is written to the leader log acks = -1 - message is committed -Jay On Sat, Jul 18, 2015 at 10:50 PM, Gwen Shapira gshap...@cloudera.com wrote: 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 with acks = 1, it sends an actual error message. Which seems to mean that with acks = 0, any failed produce will lead to metadata refresh and a retry (because network error), while acks = 1 will lead to either retries or abort, depending on the error. Not only this doesn't match the documentation, it doesn't even make much sense... acks = 0 was supposed to somehow makes things less safe but faster, and it doesn't seem to be doing that any more. I'm not even sure there's any case where the acks = 0 behavior is desirable. Is it my misunderstanding, or did we somehow screw up the logic here? Gwen -- Thanks, Ewen -- -Regards, Mayuresh R. Gharat (862) 250-7125 -- -- Guozhang
Re: New Producer and acks configuration
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, with new producer with async mode, clients will not have to wait for the response since it will be handled in callbacks. So the use of acks = 0 sounds very rare to me and I am not able to think of an usecase around it. Thanks, Mayuresh On Sun, Jul 26, 2015 at 2:40 PM, Gwen Shapira gshap...@cloudera.com wrote: 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 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 ClientRequest). This means that the producer will invoke the callback resolve the future as soon as the request hits the TCP buffer on the client. At that point, the behavior of the broker wrt generating a response doesn't matter -- the client isn't waiting on that response anyway. This definitely is faster since you aren't waiting for the round trip, but it seems like it is of questionable value with the new producer as Jay explained. It is slightly better than just assuming records have been sent as soon as you call Producer.send() in this shouldn't trigger a callback until the records have made it through the internal KafkaProducer buffering. But since it still has to make it through the TCP buffers it doesn't really guarantee anything that useful. -Ewen On Sun, Jul 26, 2015 at 1:40 PM, Gwen Shapira gshap...@cloudera.com wrote: What bugs me is that even with acks = 0, the broker will append to local log before responding (unless I'm misreading the code), so I don't see why a client with acks = 0 will be any faster. Unless the client chooses to not wait for response, which is orthogonal to acks parameter. On Mon, Jul 20, 2015 at 8:52 AM, Jay Kreps j...@confluent.io wrote: 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 will be a lot faster. It also makes some sense in terms of what is completed when the request is considered satisfied acks = 0 - message is written to the network (buffer) acks = 1 - message is written to the leader log acks = -1 - message is committed -Jay On Sat, Jul 18, 2015 at 10:50 PM, Gwen Shapira gshap...@cloudera.com wrote: 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 with acks = 1, it sends an actual error message. Which seems to mean that with acks = 0, any failed produce will lead to metadata refresh and a retry (because network error), while acks = 1 will lead to either retries or abort, depending on the error. Not only this doesn't match the documentation, it doesn't even make much sense... acks = 0 was supposed to somehow makes things less safe but faster, and it doesn't seem to be doing that any more. I'm not even sure there's any case where the acks = 0 behavior is desirable. Is it my misunderstanding, or did we somehow screw up the logic here? Gwen -- Thanks, Ewen -- -Regards, Mayuresh R. Gharat (862) 250-7125
Re: New Producer and acks configuration
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 out of order? On Mon, Jul 27, 2015 at 9:41 AM, Guozhang Wang wangg...@gmail.com wrote: 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 happens) in the second case but not in the first. Given this difference, I feel there are still scenarios, though probably rare, that users would like to use acks = 0 even with new producer's callbacks. Guozhang On Mon, Jul 27, 2015 at 9:25 AM, Mayuresh Gharat gharatmayures...@gmail.com wrote: 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, with new producer with async mode, clients will not have to wait for the response since it will be handled in callbacks. So the use of acks = 0 sounds very rare to me and I am not able to think of an usecase around it. Thanks, Mayuresh On Sun, Jul 26, 2015 at 2:40 PM, Gwen Shapira gshap...@cloudera.com wrote: 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 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 ClientRequest). This means that the producer will invoke the callback resolve the future as soon as the request hits the TCP buffer on the client. At that point, the behavior of the broker wrt generating a response doesn't matter -- the client isn't waiting on that response anyway. This definitely is faster since you aren't waiting for the round trip, but it seems like it is of questionable value with the new producer as Jay explained. It is slightly better than just assuming records have been sent as soon as you call Producer.send() in this shouldn't trigger a callback until the records have made it through the internal KafkaProducer buffering. But since it still has to make it through the TCP buffers it doesn't really guarantee anything that useful. -Ewen On Sun, Jul 26, 2015 at 1:40 PM, Gwen Shapira gshap...@cloudera.com wrote: What bugs me is that even with acks = 0, the broker will append to local log before responding (unless I'm misreading the code), so I don't see why a client with acks = 0 will be any faster. Unless the client chooses to not wait for response, which is orthogonal to acks parameter. On Mon, Jul 20, 2015 at 8:52 AM, Jay Kreps j...@confluent.io wrote: 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 will be a lot faster. It also makes some sense in terms of what is completed when the request is considered satisfied acks = 0 - message is written to the network (buffer) acks = 1 - message is written to the leader log acks = -1 - message is committed -Jay On Sat, Jul 18, 2015 at 10:50 PM, Gwen Shapira gshap...@cloudera.com wrote: 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 with acks = 1, it sends an actual error message. Which seems to mean that with acks = 0, any failed produce will lead to metadata refresh and a retry (because network error), while acks = 1 will lead to either retries or abort, depending on the error. Not only this doesn't match the documentation, it doesn't even make much sense... acks = 0 was supposed to somehow makes things less safe but faster, and it doesn't seem to be doing that any more. I'm not even sure there's any case where the acks = 0 behavior is
Re: New Producer and acks configuration
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 great presentation addressing that and a bunch of other issues with no data loss pipelines: http://www.slideshare.net/JiangjieQin/no-data-loss-pipeline-with-apache-kafka-49753844 Short version: as things are today, you have to *really* understand the producer settings, and some producer internals, to get the exact behavior you want. On Mon, Jul 27, 2015 at 9:44 AM, Gwen Shapira gshap...@cloudera.com wrote: 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 out of order? On Mon, Jul 27, 2015 at 9:41 AM, Guozhang Wang wangg...@gmail.com wrote: 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 happens) in the second case but not in the first. Given this difference, I feel there are still scenarios, though probably rare, that users would like to use acks = 0 even with new producer's callbacks. Guozhang On Mon, Jul 27, 2015 at 9:25 AM, Mayuresh Gharat gharatmayures...@gmail.com wrote: 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, with new producer with async mode, clients will not have to wait for the response since it will be handled in callbacks. So the use of acks = 0 sounds very rare to me and I am not able to think of an usecase around it. Thanks, Mayuresh On Sun, Jul 26, 2015 at 2:40 PM, Gwen Shapira gshap...@cloudera.com wrote: 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 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 ClientRequest). This means that the producer will invoke the callback resolve the future as soon as the request hits the TCP buffer on the client. At that point, the behavior of the broker wrt generating a response doesn't matter -- the client isn't waiting on that response anyway. This definitely is faster since you aren't waiting for the round trip, but it seems like it is of questionable value with the new producer as Jay explained. It is slightly better than just assuming records have been sent as soon as you call Producer.send() in this shouldn't trigger a callback until the records have made it through the internal KafkaProducer buffering. But since it still has to make it through the TCP buffers it doesn't really guarantee anything that useful. -Ewen On Sun, Jul 26, 2015 at 1:40 PM, Gwen Shapira gshap...@cloudera.com wrote: What bugs me is that even with acks = 0, the broker will append to local log before responding (unless I'm misreading the code), so I don't see why a client with acks = 0 will be any faster. Unless the client chooses to not wait for response, which is orthogonal to acks parameter. On Mon, Jul 20, 2015 at 8:52 AM, Jay Kreps j...@confluent.io wrote: 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 will be a lot faster. It also makes some sense in terms of what is completed when the request is considered satisfied acks = 0 - message is written to the network (buffer) acks = 1 - message is written to the leader log acks = -1 - message is committed -Jay On Sat, Jul 18, 2015 at 10:50 PM, Gwen Shapira gshap...@cloudera.com wrote: Hi, I was looking into the different between
Re: New Producer and acks configuration
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 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 ClientRequest). This means that the producer will invoke the callback resolve the future as soon as the request hits the TCP buffer on the client. At that point, the behavior of the broker wrt generating a response doesn't matter -- the client isn't waiting on that response anyway. This definitely is faster since you aren't waiting for the round trip, but it seems like it is of questionable value with the new producer as Jay explained. It is slightly better than just assuming records have been sent as soon as you call Producer.send() in this shouldn't trigger a callback until the records have made it through the internal KafkaProducer buffering. But since it still has to make it through the TCP buffers it doesn't really guarantee anything that useful. -Ewen On Sun, Jul 26, 2015 at 1:40 PM, Gwen Shapira gshap...@cloudera.com wrote: What bugs me is that even with acks = 0, the broker will append to local log before responding (unless I'm misreading the code), so I don't see why a client with acks = 0 will be any faster. Unless the client chooses to not wait for response, which is orthogonal to acks parameter. On Mon, Jul 20, 2015 at 8:52 AM, Jay Kreps j...@confluent.io wrote: 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 will be a lot faster. It also makes some sense in terms of what is completed when the request is considered satisfied acks = 0 - message is written to the network (buffer) acks = 1 - message is written to the leader log acks = -1 - message is committed -Jay On Sat, Jul 18, 2015 at 10:50 PM, Gwen Shapira gshap...@cloudera.com wrote: 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 with acks = 1, it sends an actual error message. Which seems to mean that with acks = 0, any failed produce will lead to metadata refresh and a retry (because network error), while acks = 1 will lead to either retries or abort, depending on the error. Not only this doesn't match the documentation, it doesn't even make much sense... acks = 0 was supposed to somehow makes things less safe but faster, and it doesn't seem to be doing that any more. I'm not even sure there's any case where the acks = 0 behavior is desirable. Is it my misunderstanding, or did we somehow screw up the logic here? Gwen -- Thanks, Ewen
Re: New Producer and acks configuration
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 ClientRequest). This means that the producer will invoke the callback resolve the future as soon as the request hits the TCP buffer on the client. At that point, the behavior of the broker wrt generating a response doesn't matter -- the client isn't waiting on that response anyway. This definitely is faster since you aren't waiting for the round trip, but it seems like it is of questionable value with the new producer as Jay explained. It is slightly better than just assuming records have been sent as soon as you call Producer.send() in this shouldn't trigger a callback until the records have made it through the internal KafkaProducer buffering. But since it still has to make it through the TCP buffers it doesn't really guarantee anything that useful. -Ewen On Sun, Jul 26, 2015 at 1:40 PM, Gwen Shapira gshap...@cloudera.com wrote: What bugs me is that even with acks = 0, the broker will append to local log before responding (unless I'm misreading the code), so I don't see why a client with acks = 0 will be any faster. Unless the client chooses to not wait for response, which is orthogonal to acks parameter. On Mon, Jul 20, 2015 at 8:52 AM, Jay Kreps j...@confluent.io wrote: 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 will be a lot faster. It also makes some sense in terms of what is completed when the request is considered satisfied acks = 0 - message is written to the network (buffer) acks = 1 - message is written to the leader log acks = -1 - message is committed -Jay On Sat, Jul 18, 2015 at 10:50 PM, Gwen Shapira gshap...@cloudera.com wrote: 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 with acks = 1, it sends an actual error message. Which seems to mean that with acks = 0, any failed produce will lead to metadata refresh and a retry (because network error), while acks = 1 will lead to either retries or abort, depending on the error. Not only this doesn't match the documentation, it doesn't even make much sense... acks = 0 was supposed to somehow makes things less safe but faster, and it doesn't seem to be doing that any more. I'm not even sure there's any case where the acks = 0 behavior is desirable. Is it my misunderstanding, or did we somehow screw up the logic here? Gwen -- Thanks, Ewen
Re: New Producer and acks configuration
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 will be a lot faster. It also makes some sense in terms of what is completed when the request is considered satisfied acks = 0 - message is written to the network (buffer) acks = 1 - message is written to the leader log acks = -1 - message is committed -Jay On Sat, Jul 18, 2015 at 10:50 PM, Gwen Shapira gshap...@cloudera.com wrote: 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 with acks = 1, it sends an actual error message. Which seems to mean that with acks = 0, any failed produce will lead to metadata refresh and a retry (because network error), while acks = 1 will lead to either retries or abort, depending on the error. Not only this doesn't match the documentation, it doesn't even make much sense... acks = 0 was supposed to somehow makes things less safe but faster, and it doesn't seem to be doing that any more. I'm not even sure there's any case where the acks = 0 behavior is desirable. Is it my misunderstanding, or did we somehow screw up the logic here? Gwen
New Producer and acks configuration
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 with acks = 1, it sends an actual error message. Which seems to mean that with acks = 0, any failed produce will lead to metadata refresh and a retry (because network error), while acks = 1 will lead to either retries or abort, depending on the error. Not only this doesn't match the documentation, it doesn't even make much sense... acks = 0 was supposed to somehow makes things less safe but faster, and it doesn't seem to be doing that any more. I'm not even sure there's any case where the acks = 0 behavior is desirable. Is it my misunderstanding, or did we somehow screw up the logic here? Gwen