Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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 {
Copy link
Copy Markdown
Member

@showuon showuon Mar 3, 2026

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.partitions and default.replication.factor set, and verify the default value 1 should takes place?

Copy link
Copy Markdown
Member Author

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!


@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();
}
}
}
12 changes: 10 additions & 2 deletions core/src/main/scala/kafka/server/AutoTopicCreationManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import org.apache.kafka.common.requests.{CreateTopicsRequest, CreateTopicsRespon
import org.apache.kafka.coordinator.group.GroupCoordinator
import org.apache.kafka.coordinator.share.ShareCoordinator
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.quota.ControllerMutationQuota
import org.apache.kafka.common.utils.Time
import org.apache.kafka.server.TopicCreator
Expand Down Expand Up @@ -270,10 +271,17 @@ class DefaultAutoTopicCreationManager(
.setReplicationFactor(config.shareCoordinatorConfig.shareCoordinatorStateTopicReplicationFactor())
.setConfigs(convertToTopicConfigCollections(shareCoordinator.shareGroupStateTopicConfigs()))
case topicName =>
val numPartitions: java.lang.Integer =
if (config.originals.containsKey(ServerLogConfigs.NUM_PARTITIONS_CONFIG)) config.numPartitions
else CreateTopicsRequest.NO_NUM_PARTITIONS
val replicationFactor: java.lang.Short =
if (config.originals.containsKey(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG)) config.defaultReplicationFactor.toShort
else CreateTopicsRequest.NO_REPLICATION_FACTOR

new CreatableTopic()
.setName(topicName)
.setNumPartitions(config.numPartitions)
.setReplicationFactor(config.defaultReplicationFactor.shortValue)
.setNumPartitions(numPartitions)
.setReplicationFactor(replicationFactor)
}
}

Expand Down
8 changes: 8 additions & 0 deletions core/src/main/scala/kafka/server/KafkaConfig.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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)) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Could you refactor warnIfConfigDefinedInWrongRole to ensure consistent warning messages?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The 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()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Nice catch! Why could we pass the tests with the wrong setting?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The 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))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,16 @@
public class ServerLogConfigs {
public static final String NUM_PARTITIONS_CONFIG = "num.partitions";
public static final int NUM_PARTITIONS_DEFAULT = 1;
public static final String NUM_PARTITIONS_DOC = "The default number of log partitions per topic";
public static final String NUM_PARTITIONS_DOC =
"The default number of log partitions per topic. This configuration affects the following paths:"
+ "<ul>"
+ " <li>1. Auto topic creation</li>"
+ " <li>2. Internal streams topic creation</li>"
+ " <li>3. Topic creation via <code>AdminClient#createTopics</code> when the number of partition is set to -1</li>"
+ "</ul>"
+ "<p>For (1), the value from the broker configuration is used only when it is explicitly set. "
+ "If it is not explicitly configured on the broker, the value from the controller configuration is used.<br/>"
+ "For (2) and (3), the value from the controller configuration is always used.</p>";

public static final String LOG_DIRS_CONFIG = LOG_PREFIX + "dirs";
public static final String LOG_DIR_CONFIG = LOG_PREFIX + "dir";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,8 +40,16 @@ public class ReplicationConfigs {

public static final String DEFAULT_REPLICATION_FACTOR_CONFIG = "default.replication.factor";
public static final int REPLICATION_FACTOR_DEFAULT = 1;
public static final String DEFAULT_REPLICATION_FACTOR_DOC = "The replication factor for automatically created topics," +
" and for topics created with -1 as the replication factor";
public static final String DEFAULT_REPLICATION_FACTOR_DOC =
"The default replication factor per topic. This configuration affects the following paths:"
+ "<ul>"
+ " <li>1. Auto topic creation</li>"
+ " <li>2. Internal streams topic creation</li>"
+ " <li>3. Topic creation via <code>AdminClient#createTopics</code> when the replication factor is set to -1</li>"
+ "</ul>"
+ "<p>For (1), the value from the broker configuration is used only when it is explicitly set. "
+ "If it is not explicitly configured on the broker, the value from the controller configuration is used.<br/>"
+ "For (2) and (3), the value from the controller configuration is always used.</p>";

public static final String REPLICA_LAG_TIME_MAX_MS_CONFIG = "replica.lag.time.max.ms";
public static final long REPLICA_LAG_TIME_MAX_MS_DEFAULT = 30000L;
Expand Down