-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-19655: Align the behavior of num.partitions and default.replication.factor for topic creation #21550
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-19655: Align the behavior of num.partitions and default.replication.factor for topic creation #21550
Changes from all commits
64b0833
66b9d30
355669d
9d22084
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,112 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.kafka.clients.admin; | ||
|
|
||
| import org.apache.kafka.clients.producer.Producer; | ||
| import org.apache.kafka.clients.producer.ProducerRecord; | ||
| import org.apache.kafka.common.test.ClusterInstance; | ||
| import org.apache.kafka.common.test.api.ClusterConfigProperty; | ||
| import org.apache.kafka.common.test.api.ClusterTest; | ||
| import org.apache.kafka.common.test.api.ClusterTestDefaults; | ||
| import org.apache.kafka.common.test.api.Type; | ||
|
|
||
| import java.util.List; | ||
|
|
||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||
|
|
||
| @ClusterTestDefaults(types = {Type.KRAFT}, brokers = 2) | ||
| public class AutoTopicCreationTest { | ||
|
|
||
| @ClusterTest( | ||
| serverProperties = { | ||
| @ClusterConfigProperty(id = 0, key = "num.partitions", value = "5"), | ||
| @ClusterConfigProperty(id = 0, key = "default.replication.factor", value = "2"), | ||
| @ClusterConfigProperty(id = 1, key = "num.partitions", value = "5"), | ||
| @ClusterConfigProperty(id = 1, key = "default.replication.factor", value = "2"), | ||
| } | ||
| ) | ||
| public void testAutoCreateTopicWithExplicitBrokerConfig(ClusterInstance cluster) throws Exception { | ||
| String topic = "explicit-broker-topic"; | ||
| triggerAutoCreateTopic(cluster, topic); | ||
| try (Admin admin = cluster.admin()) { | ||
| TopicDescription desc = admin.describeTopics(List.of(topic)).allTopicNames().get().get(topic); | ||
| assertEquals(5, desc.partitions().size(), | ||
| "num.partitions explicitly set on broker should be used"); | ||
| assertEquals(2, desc.partitions().get(0).replicas().size(), | ||
| "default.replication.factor explicitly set on broker should be used"); | ||
| } | ||
| } | ||
|
|
||
| @ClusterTest( | ||
| serverProperties = { | ||
| @ClusterConfigProperty(id = 3000, key = "num.partitions", value = "5"), | ||
| @ClusterConfigProperty(id = 3000, key = "default.replication.factor", value = "2"), | ||
| } | ||
| ) | ||
| public void testAutoCreateTopicWithImplicitBrokerConfig(ClusterInstance cluster) throws Exception { | ||
| String topic = "implicit-broker-topic"; | ||
| triggerAutoCreateTopic(cluster, topic); | ||
| try (Admin admin = cluster.admin()) { | ||
| TopicDescription desc = admin.describeTopics(List.of(topic)).allTopicNames().get().get(topic); | ||
| assertEquals(5, desc.partitions().size(), | ||
| "Controller num.partitions should be used when broker does not explicitly set it"); | ||
| assertEquals(2, desc.partitions().get(0).replicas().size(), | ||
| "Controller default.replication.factor should be used when broker does not explicitly set it"); | ||
| } | ||
| } | ||
|
|
||
| @ClusterTest( | ||
| serverProperties = { | ||
| @ClusterConfigProperty(id = 0, key = "num.partitions", value = "5"), | ||
| @ClusterConfigProperty(id = 1, key = "num.partitions", value = "5"), | ||
| @ClusterConfigProperty(id = 3000, key = "default.replication.factor", value = "2"), | ||
| } | ||
| ) | ||
| public void testAutoCreateTopicWithMixedConfig(ClusterInstance cluster) throws Exception { | ||
| String topic = "mixed-config-topic"; | ||
| triggerAutoCreateTopic(cluster, topic); | ||
| try (Admin admin = cluster.admin()) { | ||
| TopicDescription desc = admin.describeTopics(List.of(topic)).allTopicNames().get().get(topic); | ||
| assertEquals(5, desc.partitions().size(), | ||
| "num.partitions explicitly set on broker should be used"); | ||
| assertEquals(2, desc.partitions().get(0).replicas().size(), | ||
| "Controller default.replication.factor should be used when broker does not set it"); | ||
| } | ||
| } | ||
|
|
||
| @ClusterTest | ||
| public void testAutoCreateTopicWithDefaultConfig(ClusterInstance cluster) throws Exception { | ||
| String topic = "default-config-topic"; | ||
| triggerAutoCreateTopic(cluster, topic); | ||
| try (Admin admin = cluster.admin()) { | ||
| TopicDescription desc = admin.describeTopics(List.of(topic)).allTopicNames().get().get(topic); | ||
| assertEquals(1, desc.partitions().size(), | ||
| "Default num.partitions of 1 should be used when neither broker nor controller sets it"); | ||
| assertEquals(1, desc.partitions().get(0).replicas().size(), | ||
| "Default default.replication.factor of 1 should be used when neither broker nor controller sets it"); | ||
| } | ||
| } | ||
|
|
||
| private void triggerAutoCreateTopic(ClusterInstance cluster, String topic) throws Exception { | ||
| // Sends a produce request to a non-existent topic so that auto topic creation is triggered. | ||
| try (Producer<byte[], byte[]> producer = cluster.producer()) { | ||
| ProducerRecord<byte[], byte[]> record = | ||
| new ProducerRecord<>(topic, null, "key".getBytes(), "value".getBytes()); | ||
| producer.send(record).get(); | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -584,6 +584,14 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) | |
| // warn if create.topic.policy.class.name or alter.config.policy.class.name is defined in the broker role | ||
| warnIfConfigDefinedInWrongRole(ProcessRole.ControllerRole, ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG) | ||
| warnIfConfigDefinedInWrongRole(ProcessRole.ControllerRole, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG) | ||
| if (originals.containsKey(ServerLogConfigs.NUM_PARTITIONS_CONFIG)) { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could you refactor
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Here you go: #21661 |
||
| warn(s"${ServerLogConfigs.NUM_PARTITIONS_CONFIG} is defined in the broker role. This configuration will be ignored in 5.0. " + | ||
| s"Please set ${ServerLogConfigs.NUM_PARTITIONS_CONFIG} in the controller role instead.") | ||
| } | ||
| if (originals.containsKey(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG)) { | ||
| warn(s"${ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG} is defined in the broker role. This configuration will be ignored in 5.0. " + | ||
| s"Please set ${ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG} in the controller role instead.") | ||
| } | ||
| } else if (processRoles == Set(ProcessRole.ControllerRole)) { | ||
| // KRaft controller-only | ||
| validateQuorumVotersAndQuorumBootstrapServerForKRaft() | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -125,13 +125,13 @@ class AutoTopicCreationManagerTest { | |
| val props = TestUtils.createBrokerConfig(1) | ||
| props.setProperty(ServerConfigs.REQUEST_TIMEOUT_MS_CONFIG, requestTimeout.toString) | ||
|
|
||
| props.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, internalTopicPartitions.toString) | ||
| props.setProperty(TransactionLogConfig.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, internalTopicPartitions.toString) | ||
| props.setProperty(ShareCoordinatorConfig.STATE_TOPIC_REPLICATION_FACTOR_CONFIG , internalTopicPartitions.toString) | ||
| props.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, internalTopicReplicationFactor.toString) | ||
| props.setProperty(TransactionLogConfig.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, internalTopicReplicationFactor.toString) | ||
| props.setProperty(ShareCoordinatorConfig.STATE_TOPIC_REPLICATION_FACTOR_CONFIG , internalTopicReplicationFactor.toString) | ||
|
|
||
| props.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString) | ||
| props.setProperty(TransactionLogConfig.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString) | ||
| props.setProperty(ShareCoordinatorConfig.STATE_TOPIC_NUM_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString) | ||
| props.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, internalTopicPartitions.toString) | ||
| props.setProperty(TransactionLogConfig.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, internalTopicPartitions.toString) | ||
| props.setProperty(ShareCoordinatorConfig.STATE_TOPIC_NUM_PARTITIONS_CONFIG, internalTopicPartitions.toString) | ||
|
Comment on lines
-128
to
+134
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nice catch! Why could we pass the tests with the wrong setting?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Since they both point to the same value, so everything stay in peace 😆 |
||
|
|
||
| config = KafkaConfig.fromProps(props) | ||
| val aliveBrokers = util.List.of(new Node(0, "host0", 0), new Node(1, "host1", 1)) | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
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.
Could we add a test case that both broker and controller nodes have no
num.partitionsanddefault.replication.factorset, and verify the default value 1 should takes place?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.
Sure, let me add one, thanks!