Description
In KAFKA-13598, we enabled idempotent producer by default. When idempotence is enabled, there are some constraints:
- acks=all
- retries > 0
- max.in.flight.requests.per.connection <= 5
We found the default idempotence enabled setting will break some tests because the `acks`, `retries` config are overridden, and conflict with idempotence. And we believe there are many users also overriding these configs in current producers. We should avoid to break the existing producers after user upgrading to the newer version. So, we won't enable idempotence when the `enable.idempotence` is not set explicitly, and `acks`, `retries`, `max.in.flight.requests.per.connection` config is conflicting with idempotence setting. Below are some example cases the user will have:
- if user doesn't set enable.idempotence:
a. enable.idempotence unset && acks unset => enable idempotence (this was the intent behind the 3.0 change I think)
b. enable.idempotence unset && acks=all => enable idempotence
c. enable.idempotence unset && acks!=all => disable idempotence (the case we will add in the Jira to avoid break existing producers)
- if user enable/disable enable.idempotence explicitly:
a. enable.idempotence=true && acks=all => enable idempotence
b. enable.idempotence=false => disable idempotence
c. enable.idempotence=true && acks!=all => throw exception (still throw exception in this case)
PS. the above cases only use `acks` config for example. It also apply for `retries` and `max.in.flight.requests.per.connection` config.
Attachments
Issue Links
- is related to
-
KAFKA-13598 idempotence producer is not enabled by default if not set explicitly
- Resolved
- links to