Skip to content
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

KAFKA-18360: Remove zookeeper configurations #18566

Merged
merged 7 commits into from
Feb 6, 2025

Conversation

cmccabe
Copy link
Contributor

@cmccabe cmccabe commented Jan 15, 2025

Remove the zookeeper configurations in ZkConfig.java.

Remove broker.id.generation.enable and reserved.broker.max.id, which are not used in KRaft mode.

Remove inter.broker.protocol.version, which is not used in KRaft mode.

@github-actions github-actions bot added triage PRs from the community core Kafka Broker labels Jan 15, 2025
@github-actions github-actions bot added tools and removed triage PRs from the community labels Jan 16, 2025
@ijuma
Copy link
Member

ijuma commented Jan 17, 2025

Can you fix the conflicts?

@@ -221,8 +213,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
def quotaConfig: QuotaConfig = _quotaConfig

/** ********* General Configuration ***********/
val brokerIdGenerationEnable: Boolean = getBoolean(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG)
val maxReservedBrokerId: Int = getInt(ServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG)
var brokerId: Int = getInt(ServerConfigs.BROKER_ID_CONFIG)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request #18478 removes the broker.id configuration, but this pull request does not. Should we continue treating broker.id as synonymous with node.id in Kafka version 4.x? The documentation states that node.id is mandatory when running in KRaft mode. Therefore, it's likely safe to remove broker.id from version 4.x, as Kafka servers in KRaft mode should already be configured with node.id.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If the behavior in 3.x is that the broker fails to start in KRaft mode unless node.id is set, then it would be fine to remove broker.id. But we should double check to make sure we didn't make an incompatible change in 4.0/trunk.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If the behavior in 3.x is that the broker fails to start in KRaft mode unless node.id is set

There's a potential regression if we remove broker.id because users can currently set it instead of node.id. However, this conflicts with our documentation, which states that node.id is mandatory. To address this, we could deprecate broker.id in version 4.x and then remove it entirely in version 5.x.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ping @dajac if the approach of deprecating broker.id is acceptable.

Copy link
Member

@ijuma ijuma Jan 22, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Deprecating broker.id has to be done via a KIP. Since there is no rush for that one, let's focus on making sure we don't break the 3.x behavior for now.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankYang0529 Let's retain broker.id in issue #18478. Additionally, we should update the documentation to clarify that node.id and broker.id are synonymous. This will help prevent user confusion regarding their relationship.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the confirmation. I will update my PR.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@ijuma @chia7712 do you think that we should add a warning message when users use broker.id? To let them know the configuration will be removed in 5.0.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do you think that we should add a warning message when users use broker.id? To let them know the configuration will be removed in 5.0.

broker.id is not deprecated in 4.0 (as @ijuma mentioned it needs a KIP), so the warning message is unnecessary to me. I open https://issues.apache.org/jira/browse/KAFKA-18628 to discuss the broker.id

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, we should discuss the node.id / broker.id thing in a separate JIRA or PR, I think.

@chia7712
Copy link
Member

@cmccabe could you please fix the conflicts? I hope this is the last PR required to remove zk configuration

@clolov
Copy link
Contributor

clolov commented Feb 5, 2025

Heya @cmccabe, if you are short on cycles I can help getting this over the line?

@cmccabe
Copy link
Contributor Author

cmccabe commented Feb 5, 2025

@ijuma @clolov @chia7712 the conflicts should be fixed now. sorry for the delay!

Copy link
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM and I open some follow-ups

@@ -182,8 +182,7 @@ object KafkaRaftServer {
}

// Load the BootstrapMetadata.
val bootstrapDirectory = new BootstrapDirectory(config.metadataLogDir,
Optional.ofNullable(config.interBrokerProtocolVersionString))
val bootstrapDirectory = new BootstrapDirectory(config.metadataLogDir, Optional.empty())
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

open https://issues.apache.org/jira/browse/KAFKA-18740 to cleanup BootstrapDirectory as follow-up

@@ -127,13 +125,6 @@ public class ReplicationConfigs {
"thread to trigger election periodically (default is 5 minutes). Please run `kafka-leader-election.sh` with `unclean` option " +
"to trigger the unclean leader election immediately if needed.</p>";

public static final String INTER_BROKER_PROTOCOL_VERSION_CONFIG = "inter.broker.protocol.version";
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the failed test testFormatWithReleaseVersionDefault is using inter.broker.protocol.version so it should be removed by this PR.

testFormatWithReleaseVersionDefaultAndReleaseVersion also sets the inter.broker.protocol.version. We can remove the line as it does not impact the test

@@ -187,7 +186,7 @@ class KafkaConfigTest {
val advertisedHostName = "routable-host"
val advertisedPort = 1234

val props = TestUtils.createBrokerConfig(0)
val props = createDefaultConfig()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this createDefaultConfig does not return a valid config (https://github.com/apache/kafka/pull/18547/files#r1944140562)

@mingyen066 will fix it by a minor patch

@chia7712
Copy link
Member

chia7712 commented Feb 6, 2025

@cmccabe I push a commit to include following changes.

  1. remove inter.broker.protocol.version from testFormatWithReleaseVersionDefaultAndReleaseVersion
  2. remove testFormatWithReleaseVersionDefault

I apologize for making minor changes to your PR. These changes are necessary to unblock subsequent work for the 4.0 release. I will merge this PR once it passes QA.

@chia7712 chia7712 merged commit b2b2408 into apache:trunk Feb 6, 2025
9 checks passed
chia7712 pushed a commit that referenced this pull request Feb 6, 2025
Remove broker.id.generation.enable and reserved.broker.max.id, which are not used in KRaft mode.
Remove inter.broker.protocol.version, which is not used in KRaft mode.

Reviewers: PoAn Yang <[email protected]>, Ismael Juma <[email protected]>, Chia-Ping Tsai <[email protected]>
pdruley pushed a commit to pdruley/kafka that referenced this pull request Feb 12, 2025
Remove broker.id.generation.enable and reserved.broker.max.id, which are not used in KRaft mode.
Remove inter.broker.protocol.version, which is not used in KRaft mode.

Reviewers: PoAn Yang <[email protected]>, Ismael Juma <[email protected]>, Chia-Ping Tsai <[email protected]>
manoj-mathivanan pushed a commit to manoj-mathivanan/kafka that referenced this pull request Feb 19, 2025
Remove broker.id.generation.enable and reserved.broker.max.id, which are not used in KRaft mode.
Remove inter.broker.protocol.version, which is not used in KRaft mode.

Reviewers: PoAn Yang <[email protected]>, Ismael Juma <[email protected]>, Chia-Ping Tsai <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
core Kafka Broker tools
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants