You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Jay Kreps (JIRA)" <ji...@apache.org> on 2017/06/21 23:34:00 UTC

[jira] [Comment Edited] (KAFKA-5494) Idempotent producer should not require max.in.flight.requests.per.connection=1 and acks=all

    [ https://issues.apache.org/jira/browse/KAFKA-5494?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16058432#comment-16058432 ] 

Jay Kreps edited comment on KAFKA-5494 at 6/21/17 11:33 PM:
------------------------------------------------------------

Yeah I think it'd be a fantastic outcome to just be able to eliminate a lot of these configs as things you need to ever think about. Ideally by default:
# idempotent=true
# retries=infinite
# acks=all
# max.inflight=2 (or whatever)

Various chunks of work to make each of these possible:
# I think enabling idempotence and retries=infinite by default is just a comfort thing since this is new
# For max.inflight I vaguely recall that the reason i added that config was because it helper performance a small amount. With only 1 you end up potentially leaving the server unused once the response is sent but the client hasn't yet sent its next request. But it'd be good to sanity check that max.inflight > 1 actually helps perf.
# acks=all Currently I believe this is about 2x slower than acks=1. At least this was the case when we did producer benchmarking a few years back and no one has ever actually looked at _why_ that is the case. The knee-jerk reaction is that "of course acks=all is slower". But if you think about it it isn't really clear why. Intuitively if the requests take longer to process, but this should just lead to more batching, which should be able to make up for it. So there is a 50% chance that this is just some perf issue with purgatory or with the client and fixing that could enable us to just default to the stronger semantics (which would be great).


was (Author: jkreps):
Yeah I think it'd be a fantastic outcome to just be able to eliminate a lot of these configs as things you need to ever think about. Ideally by default:
# idempotent=true
# retries=infinite
# acks=all
# max.inflight=2 (or whatever)

Various chunks of work to make each of these possible:
# I think enabling idempotence and retries=infinite by default is just a comfort thing since this is new
# For max.inflight I vaguely recall that the reason i added that config was because it helper performance a small amount. With only 1 you end up potentially using the server unused once the response is sent but the client hasn't yet sent its next request. But it'd be good to sanity check that max.inflight > 1 actually helps perf.
# acks=all Currently I believe this is about 2x slower than acks=1. At least this was the case when we did producer benchmarking a few years back and no one has ever actually looked at _why_ that is the case. The knee-jerk reaction is that "of course acks=all is slower". But if you think about it it isn't really clear why. Intuitively if the requests take longer to process, but this should just lead to more batching, which should be able to make up for it. So there is a 50% chance that this is just some perf issue with purgatory or with the client and fixing that could enable us to just default to the stronger semantics (which would be great).

> Idempotent producer should not require max.in.flight.requests.per.connection=1 and acks=all
> -------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-5494
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5494
>             Project: Kafka
>          Issue Type: Improvement
>    Affects Versions: 0.11.0.0
>            Reporter: Apurva Mehta
>              Labels: exactly-once
>
> Currently, the idempotent producer (and hence transactional producer) requires max.in.flight.requests.per.connection=1.
> This was due to simplifying the implementation on the client and server. With some additional work, we can satisfy the idempotent guarantees even with any number of in flight requests. The changes on the client be summarized as follows:
>  
> # We increment sequence numbers when batches are drained.
> # If for some reason, a batch fails with a retriable error, we know that all future batches would fail with an out of order sequence exception. 
> # As such, the client should treat some OutOfOrderSequence errors as retriable. In particular, we should maintain the 'last acked sequnece'. If the batch succeeding the last ack'd sequence has an OutOfOrderSequence, that is a fatal error. If a future batch fails with OutOfOrderSequence they should be reenqeued.
> # With the changes above, the the producer queues should become priority queues ordered by the sequence numbers. 
> # The partition is not ready unless the front of the queue has the next expected sequence.
> With the changes above, we would get the benefits of multiple inflights in normal cases. When there are failures, we automatically constrain to a single inflight until we get back in sequence. 
> With multiple inflights, we now have the possibility of getting duplicates for batches other than the last appended batch. In order to return the record metadata (including offset) of the duplicates inside the log, we would require a log scan at the tail to get the metadata at the tail. This can be optimized by caching the metadata for the last 'n' batches. For instance, if the default max.inflight is 5, we could cache the record metadata of the last 5 batches, and fall back to a scan if the duplicate is not within those 5. 
> * *
> The reason to have acks=all is to protect against OutOfOrderSequence exceptions in the case where the leader fails before replication happens. In that case, the next batch sent by the producer would get an OutOfOrderSequence because the new leader would not have the last message. 
> This may be OK: for applications which really care about avoiding duplicates, they have to handle fatal errors of this sort anyway. In particular, the recommendation is to close the producer in the callback on a fatal error and then check the tail of the log for the last committed message, and then start sending from there. 
> By making acks=all, this application logic would just be exercised more frequently.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)