-
Notifications
You must be signed in to change notification settings - Fork 3.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Purge brokers no longer reported in metadata #4557
base: master
Are you sure you want to change the base?
Conversation
The Kafka protocol allows for brokers to have multiple host:port pairs for a given node Id, e.g. see UpdateMetadata request which contains a live_brokers list where each broker Id has a list of host:port pairs. It follows from this that the thing that uniquely identifies a broker is its Id, and not the host:port. The behaviour right now is that if we have multiple brokers with the same host:port but different Ids, the first broker in the list will be updated to have the Id of whatever broker we're looking at as we iterate through the brokers in the Metadata response in rd_kafka_parse_Metadata0(), e.g. Step 1. Broker[0] = Metadata.brokers[0] Step 2. Broker[0] = Metadata.brokers[1] Step 3. Broker[0] = Metadata.brokers[2] A typical situation where brokers have the same host:port pair but differ in their Id is if the brokers are behind a load balancer. The NODE_UPDATE mechanism responsible for this was originally added in b09ff60 ("Handle broker name and nodeid updates (issue confluentinc#343)") as a way to forcibly update a broker hostname if an Id is reused with a new host after the original one was decommissioned. But this isn't how the Java Kafka client works, so let's use the Metadata response as the source of truth instead of updating brokers if we can only match by their host:port.
Brokers that are not in the metadata should be purged from the internal client lists. This helps to avoid annoying "No route to host" and other connection failure messages. Fixes confluentinc#238.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hi @mfleming thanks a lot for this contribution and sorry for letting it wait for long. We want to include these fixes in next version. Both fixes are good, just on broker decommission we want to do some additional checks.
Here are some comments mainly for the first fix:
Hi @mfleming can I apply those changes or do you want to continue the PR? Thanks! |
Hey sorry for the delay — yeah you can apply those changes if you have the
time. If not I’ll get to them sometime this week.
…On Tue, 21 May 2024 at 09:10, Emanuele Sabellico ***@***.***> wrote:
Hi @mfleming <https://github.com/mfleming> can I apply those changes or
do you want to continue the PR? Thanks!
—
Reply to this email directly, view it on GitHub
<#4557 (comment)>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AAAXALUEWS5XRZGGJX6EN43ZDL6QDAVCNFSM6AAAAABASEOCUWVHI2DSMVQWIX3LMV43OSLTON2WKQ3PNVWWK3TUHMZDCMRSGAZDKMRXG4>
.
You are receiving this because you were mentioned.Message ID:
***@***.***>
|
Co-authored-by: Emanuele Sabellico <[email protected]>
Co-authored-by: Emanuele Sabellico <[email protected]>
rkb creation.
@mfleming thanks, sorry for the delay too, I'm checking it again |
as it's now set only on creation and not modified anymore
mock cluster
/sem-approve |
and move documentation
/sem-approve |
We're not going to merge this for 2.5.0 that is due in July as we need to do more checks on possible regressions but we want to merge it for a maintenance release in September |
Thanks for fixing things :) |
Thank you for this PR! |
Just wanted to say a big thank you to both of you—@mfleming for writing this and @emasab for reviewing. We just ran into a slow thread leak in a Kafka consumer at @MaterializeInc that will be fixed by this patch. |
While going through the PR, I found that there are a few issues in the PR.
We need to fix the above issues before releasing this PR and need of more testing. As a result, it won't be part of the upcoming 2.6 release. |
Closes #4881 |
We agreed KIP-899 and KIP-1102 aren't strictly necessary, in Java client they were disabled by default until recently. In any case the metadata response contains at least the majority of KRaft elegible brokers. If majority changes, it must contain at least one broker from previous set. There are a few things left to check, at least:
|
Brokers that are not in the metadata should be purged from the internal client lists. This helps to avoid annoying "No route to host" and other connection failure messages.
Fixes #238