KAFKA-7077: Use default producer settings in Connect Worker#11475
KAFKA-7077: Use default producer settings in Connect Worker#11475showuon merged 1 commit intoapache:trunkfrom
Conversation
showuon
left a comment
There was a problem hiding this comment.
@LiamClarkeNZ , thanks for the PR. LGTM. But there's a bug cause the default idempotent producer is not enabled and get fixed recently. (ref: #11691 ) Could you rebase to the latest trunk and trigger jenkins build again?
Also, I was wrong in this comment: https://github.com/apache/kafka/pull/11691/files#r791886361 , I checked again and found there's only KafkaStatusBackingStore class cannot enable idempotent producer because it implement its own retry logic, to include deleted record here.
Anyway, looks like even if we don't have this PR, the kafka connect will use idempotent producer by default (with just 1 in-flight request only). cc @mimaison
…by default, remove explicit setting of acks and max in flight in Connect Worker producer Signed-off-by: Liam Clarke-Hutchinson <liam@steelsky.co.nz>
9c6275c to
5889cd3
Compare
Thanks @showuon, I have rebased onto trunk and force pushed. I appreciate the review :) |
|
Failed tests are unrelated: |
|
@kkonstantine @rhauch Is it OK to enable idempotence by default in this case? Are there any compatibility or behavior concerns? |
|
@showuon We should add a note to |
|
@ijuma , I see. Thanks for the reminder. Will do. |
|
Thanks for bring this to our attention @ijuma. This change should be fine from Connect's perspective. The property that would give someone pause is the value for Having said that, I think it would have been a good idea to leave a comment in the code that would say that Connect requires these configs which are now enabled by default with the idempotent producer. |
|
@kkonstantine , thanks for the comment. I agree we add comments in the code to say the idempotent producer is enabled by default. Thank you. |
|
Hi Luke, Ismael et al,
No worries on adding a comment to that effect in the upgrade docs - but I'm
not sure how to word it, as the idempotent producer was never explicitly
disabled by default in the Kafka Connect Worker. I realise, based on Luke's
recent work to fix KAFKA-13598, that the intended enabling of idempotence
by default with Kafka 3.0 didn't happen either.
So while I just removed the setting of max inflight requests to 1, allowing
the default of 5 to be used, . (I also removed the explicit setting of
acks=all for the same reason), I guess this will be the first version where
Kafka Connect uses idempotent producers by default. So I was thinking of
wording it like this?
*Kafka Connect now uses [idempotent producers](http://link.to.docs.here
<http://link.to.docs.here>) by default, and now defaults to a maximum of 5
inflight requests (five is the uppermost limit supported by idempotent
producers). You can override producer settings controlling this behaviour
using the properties producer.enable.idempotence and
producer.max.inflight.requests*
Actually, do we need to add similar documentation to the upgrade
documentation for 3.0.1, 3.1.1 also? As KC workers based on those version
will now be defaulting to idempotent enabled. The workers in those versions
will still have max.inflight.requests set to 1, but it could be set to 5 by
producer.* overrides, I could add commentary to that regard.
Thanks,
Liam
…On Tue, 22 Mar 2022 at 19:07, Luke Chen ***@***.***> wrote:
@kkonstantine <https://github.com/kkonstantine> , thanks for the comment.
I agree we add comments in the code to say the idempotent producer is
enabled by default.
cc @LiamClarkeNZ <https://github.com/LiamClarkeNZ>
Thank you.
—
Reply to this email directly, view it on GitHub
<#11475 (comment)>, or
unsubscribe
<https://github.com/notifications/unsubscribe-auth/AC2Q2JGE4E2XYPJ5LINUMNDVBFPQDANCNFSM5HRTSEYA>
.
You are receiving this because you were mentioned.Message ID:
***@***.***>
|
|
Thanks @LiamClarkeNZ ! The wording overall LGTM! Please submit a PR when available. We can comment on the wording there.
Yes, please add them to 3.0.1 and 3.1.1, too. |
|
Cheers Luke, will do :)
…On Tue, 22 Mar 2022 at 19:43, Luke Chen ***@***.***> wrote:
Thanks @LiamClarkeNZ <https://github.com/LiamClarkeNZ> !
The wording overall LGTM! Please submit a PR when available. We can
comment on the wording there.
Actually, do we need to add similar documentation to the upgrade
documentation for 3.0.1, 3.1.1 also?
Yes, please add them to 3.0.1 and 3.1.1, too.
Thanks.
—
Reply to this email directly, view it on GitHub
<#11475 (comment)>, or
unsubscribe
<https://github.com/notifications/unsubscribe-auth/AC2Q2JD2ZYB6NDBBCRAQLPDVBFTXNANCNFSM5HRTSEYA>
.
You are receiving this because you were mentioned.Message ID:
***@***.***>
|
|
@kkonstantine What is the expected broker version compatibility for connect? If it's older than 0.11, then we cannot rely on idempotence. |
|
One other thing: the IDEMPOTENT_WRITE ACL is required for the Connect worker principal if the kafka cluster is older than 2.8. |
|
@showuon I noticed that you recently disable idempotency for the status backing store. Now that we remove Also, re:
my intention was actually to add a comment in the code to say what are the configs that we definitely require for the producers in Connect to work as intended. And these are |
|
@Ismael you're bringing up a good point, which I missed in your latest comment before I send my recent reply. In light of the requirement to explicitly add the IDEMPOTENT_WRITE ACL for Connect workers when talking to Kafka brokers older 2.8 (which I wasn't aware of), I'd like to suggest changing course here as follows:
|
|
That sounds good to me. Let's make sure we include these changes in 3.0.x and 3.1.x as well. For the latter, we still have time to make it for 3.1.1. |
|
Sounds good to me, too. Thanks, @kkonstantine. |
…11475)" (#11932) This reverts commit 76cf7a5. Connect already allows users to enable idempotent producers for connectors and the Connect workers. Although Kafka producers enabled idempotency by default in 3.0, due to compatibility requirements and the fact that [KIP-318](https://cwiki.apache.org/confluence/display/KAFKA/KIP-318%3A+Make+Kafka+Connect+Source+idempotent) hasn't been explicitly approved, the changes here are reverted. A separate commit will explicitly disable idempotency in producers instantiated by Connect by default until KIP-318 is approved and scheduled for release.
…11475)" (#11932) This reverts commit 76cf7a5. Connect already allows users to enable idempotent producers for connectors and the Connect workers. Although Kafka producers enabled idempotency by default in 3.0, due to compatibility requirements and the fact that [KIP-318](https://cwiki.apache.org/confluence/display/KAFKA/KIP-318%3A+Make+Kafka+Connect+Source+idempotent) hasn't been explicitly approved, the changes here are reverted. A separate commit will explicitly disable idempotency in producers instantiated by Connect by default until KIP-318 is approved and scheduled for release.
|
Thanks @kkonstantine ! |
|
Hi Konstantine, please note that reverting KAFKA-7077 won't disable
producer idempotence in Kafka Connect, as it never explicitly enabled or
disabled it. You'll still need to make changes in code to explicitly
disable producer idempotence to achieve your aims, whether 7077 is reverted
or not.
…On Wed, 23 Mar 2022 at 11:15, Konstantine Karantasis < ***@***.***> wrote:
@Ismael <https://github.com/Ismael> you're bringing up a good point,
which I missed in your latest comment before I send my recent reply.
In light of the requirement to explicitly add the IDEMPOTENT_WRITE ACL for
Connect workers when talking to Kafka brokers older 2.8 (which I wasn't
aware of), I'd like to suggest changing course here as follows:
- Revert KAFKA-7077 <https://issues.apache.org/jira/browse/KAFKA-7077>
from all the branches that has been merged.
- Return to KIP-318
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-318%3A+Make+Kafka+Connect+Source+idempotent>,
update it and actually vote for it. @LiamClarkeNZ
<https://github.com/LiamClarkeNZ> you referred to this KIP in the
description of this PR but I missed that this KIP hasn't been approved and
is actually currently marked as inactive. I think we should raise it again
after we update it to include all the compatibility requirements and have
it target the next major version (4.0).
- Issue a new PR that will explicitly disable idempotency by default
in Connect and will allow users to override the setting via the worker
and/or the connector configs like we allow it today.
- In the same PR, update our docs to say that despite the Kafka
producer enabling idempotency by default in 3.0, due to compatibility
requirements Connect chooses to disable idempotency for all the producers
that instantiates by default.
—
Reply to this email directly, view it on GitHub
<#11475 (comment)>, or
unsubscribe
<https://github.com/notifications/unsubscribe-auth/AC2Q2JFYOJFUFYW7QRLZP23VBJA6JANCNFSM5HRTSEYA>
.
You are receiving this because you were mentioned.Message ID:
***@***.***>
|
In KAFKA-7077 and the associated KIP-318, there's a desire to a) enable producer idempotence and b) set max in flight requests to 5 to improve throughput, as opposed to the previous hardwired 1.
With the change of producer defaults in Kafka 3.0.0 (acks = all, enable.idempotence = true, max.in.flight.requests.per.connection=5) all this behaviour comes for free if the explicit producer configuration in the Worker is removed.
Committer Checklist (excluded from commit message)