Fix for brokers with different Ids but same host:port#4494
Open
Matt Fleming (mfleming) wants to merge 2 commits intoconfluentinc:masterfrom
Open
Fix for brokers with different Ids but same host:port#4494Matt Fleming (mfleming) wants to merge 2 commits intoconfluentinc:masterfrom
Matt Fleming (mfleming) wants to merge 2 commits intoconfluentinc:masterfrom
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.
Contributor
Author
|
Emanuele Sabellico (@emasab) would you mind taking a look at this PR? |
|
🎉 All Contributor License Agreements have been signed. Ready to merge. |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
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].id = Metadata.brokers[0].id
Step 2. Broker[0].id = Metadata.brokers[1].id
Step 3. Broker[0].id = Metadata.brokers[2].id
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 #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 mimic the Java client behaviour and use the Metadata response as the source of truth instead of updating brokers if we can only match by their host:port.
Here's a screenshot of the traffic going to brokers sat behind a load balancer before (left) and after (right) this patch which illustrates that new connections are opened and the load is balanced correctly.

Fixes #4212