Status
Current state: Under Discussion
Discussion thread: TBD
JIRA:
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
Public Interfaces
config | current default value | proposed default value |
---|---|---|
enable.idempotence | false | true |
acks | 1 | all |
max.in.flight.requests.per.connection | 5 | 2 |
retries | 0 | MAX_INT |
There will be no other publicly visible changes required.
Proposed Changes
Code changes
Currently, the idempotent producer requires max.in.flight.requests.per.connection=1 for correctness reasons. We will need to make client and broker changes to support max.in.flight.requests.per.connection > 1 with the idempotent producer. The details of the changes required are in this ticket.
Performance considerations
Next we will delineate the reasons for choosing the proposed values for the configurations in question and the performance impact of the same. A summary of the performance tests we ran to understand the impact of these changes can be found here: An analysis of the impact of max.in.flight.requests.per.connection and acks on Producer performance.
For the same value of max.in.flight.requests, and acks, we have observed that enabling idempotence alone does not significantly impact performance. Detailed results supporting this statement are available here.
Why change max.in.flight.requests.per.connection from 5 to 2?
Why change retries from 0 to MAX_INT?
The retries config was defaulted to 0 to ensure that internal producer retries don't introduce duplicates. With the idempotent producer introduced in KIP-98 - Exactly Once Delivery and Transactional Messaging, internal producer retries can no longer introduce duplicates. Hence, with the current proposal to enable the idempotent producer by default, we should let the producer retry as much as possible since there is no correctness penalty for doing so.
Why change acks from 1 to all?
With acks=1, we only have an at most once delivery guarantee. In particular, with acks=1, the broker could crash after acknowledging a message but before replicating it. This results in an acknowledged message being lost. In other words, if we want exactly once delivery, we need acks=all along with enable.idempotence=true.
The performance analysis above shows that there is an impact to latency when moving from acks=1 to acks=all. This is not entirely surprising: a ProduceRequest with acks=all is blocked on more RPCs since the followers need to fetch the newly appended data before the request is acknowledged, and hence we expect a hit to latency. Nonetheless, further analysis is ongoing to figure out the exact impact across different workloads and if there is a way to improve the situation.
Regardless, we believe strong durability guarantees out of the box are worth the cost of increased latency.
Compatibility, Deprecation, and Migration Plan
This KIP only proposes changes to the default settings of some client configurations. Hence no migration plans are required.
Further, we don't propose to deprecate any configurations at this point.
As for compatibility, it is possible that the change in defaults might cause unexpected changes in behavior for users who upgrade from older versions. We will call this out the release notes for the 1.0.0 release.
Rejected Alternatives
This is a proposal to change the default semantics of the Kafka Producer. The specific values of the various variables are either necessary to achieve stronger semantics, or have been chosen through empirical observation. Hence there are no rejected alternatives at this point.