KAFKA-13759: Disable idempotence by default in producers instantiated by Connect#11933
Conversation
f4ab563 to
eccb9c0
Compare
rhauch
left a comment
There was a problem hiding this comment.
Thanks, @kkonstantine. A pretty minor wording suggestion below.
|
Thanks for the first review @rhauch |
|
Tests passed on the latest run with a couple of failures that don't seem relevant. Yet, because one failure was on |
rhauch
left a comment
There was a problem hiding this comment.
Just a few minor suggestions to wordsmith the comments and the Upgrade user documentation. Otherwise, LGTM.
Co-authored-by: Randall Hauch <rhauch@gmail.com>
|
Thanks @rhauch. I've incorporated your suggestions in all places. The PR is ready for another pass |
rhauch
left a comment
There was a problem hiding this comment.
LGTM. Thanks, @kkonstantine!
… by Connect (#11933) With AK 3.0, idempotence was enabled by default in Kafka producers. However, if idempotence is enabled, Connect won't be able to communicate via its producers with Kafka brokers older than version 0.11. Perhaps more importantly, for brokers older than version 2.8 the IDEMPOTENT_WRITE ACL is required to be granted to the principal of the Connect worker. Therefore this commit disables producer idempotence by default to all the producers instantiated by Connect. Users can still choose to enable producer idempotence by explicitly setting the right worker and/or connector properties. The changes were tested via existing unit, integration and system tests. Reviewers: Randall Hauch <rhauch@gmail.com>
… by Connect (#11933) With AK 3.0, idempotence was enabled by default in Kafka producers. However, if idempotence is enabled, Connect won't be able to communicate via its producers with Kafka brokers older than version 0.11. Perhaps more importantly, for brokers older than version 2.8 the IDEMPOTENT_WRITE ACL is required to be granted to the principal of the Connect worker. Therefore this commit disables producer idempotence by default to all the producers instantiated by Connect. Users can still choose to enable producer idempotence by explicitly setting the right worker and/or connector properties. The changes were tested via existing unit, integration and system tests. Reviewers: Randall Hauch <rhauch@gmail.com>
… by Connect (#11933) With AK 3.0, idempotence was enabled by default in Kafka producers. However, if idempotence is enabled, Connect won't be able to communicate via its producers with Kafka brokers older than version 0.11. Perhaps more importantly, for brokers older than version 2.8 the IDEMPOTENT_WRITE ACL is required to be granted to the principal of the Connect worker. Therefore this commit disables producer idempotence by default to all the producers instantiated by Connect. Users can still choose to enable producer idempotence by explicitly setting the right worker and/or connector properties. The changes were tested via existing unit, integration and system tests. Reviewers: Randall Hauch <rhauch@gmail.com>
|
Thanks @rhauch. The fix has now been merged to trunk and cherry picked to 3.2, 3.1 and 3.0 with the appropriate adjustments to the upgrade notes. cc @cadonna @tombentley |
… by Connect (apache#11933) With AK 3.0, idempotence was enabled by default in Kafka producers. However, if idempotence is enabled, Connect won't be able to communicate via its producers with Kafka brokers older than version 0.11. Perhaps more importantly, for brokers older than version 2.8 the IDEMPOTENT_WRITE ACL is required to be granted to the principal of the Connect worker. Therefore this commit disables producer idempotence by default to all the producers instantiated by Connect. Users can still choose to enable producer idempotence by explicitly setting the right worker and/or connector properties. The changes were tested via existing unit, integration and system tests. Reviewers: Randall Hauch <rhauch@gmail.com>
…cs-12-may-2022 * apache-kafka/3.0: (14 commits) fix: make sliding window works without grace period (#kafka-13739) (apache#11980) KAFKA-13794: Follow up to fix producer batch comparator (apache#12006) KAFKA-13794; Fix comparator of `inflightBatchesBySequence` in `TransactionManager` (apache#11991) KAFKA-13748: Do not include file stream connectors in Connect's CLASSPATH and plugin.path by default (apache#11908) KAFKA-13418: Support key updates with TLS 1.3 (apache#11966) KAFKA-13770: Restore compatibility with KafkaBasedLog using older Kafka brokers (apache#11946) KAFKA-13761: KafkaLog4jAppender deadlocks when idempotence is enabled (apache#11939) KAFKA-13759: Disable idempotence by default in producers instantiated by Connect (apache#11933) MINOR: Fix `ConsumerConfig.ISOLATION_LEVEL_DOC` (apache#11915) KAFKA-13750; Client Compatability KafkaTest uses invalid idempotency configs (apache#11909) ...
With AK 3.0, idempotence was enabled by default in Kafka producers.
However, if idempotence is enabled, Connect won't be able to communicate via its producers with Kafka brokers older than version 0.11. Perhaps more importantly, for brokers older than version 2.8 the IDEMPOTENT_WRITE ACL is required to be granted to the principal of the Connect worker.
Therefore this commit disables producer idempotence by default to all the producers instantiated by Connect. Users can still choose to enable producer idempotence by explicitly setting the right worker and/or connector properties.
The changes were tested via existing unit, integration and system tests.
Committer Checklist (excluded from commit message)