Skip to content

KAFKA-19655: Align the behavior of num.partitions and default.replication.factor for topic creation#21550

Merged
showuon merged 4 commits intoapache:trunkfrom
brandboat:KAFKA-19655
Mar 3, 2026
Merged

KAFKA-19655: Align the behavior of num.partitions and default.replication.factor for topic creation#21550
showuon merged 4 commits intoapache:trunkfrom
brandboat:KAFKA-19655

Conversation

@brandboat
Copy link
Member

@brandboat brandboat commented Feb 23, 2026

Currently, 'num.partitions' and 'default.replication.factor' are applied
inconsistently. Topic auto-creation relies on Broker configs, while
Streams and AdminClient rely on Controller configs. This leads to
confusing behavior where a Broker and Controller might have diverging
defaults.

This commit implements the 4.x transition phase:

  • Updated DefaultAutoTopicCreationManager to check if these configs are
    explicitly set in 'broker.properties'.
  • If NOT explicitly set, the Broker now sends 'NO_NUM_PARTITIONS' and
    'NO_REPLICATION_FACTOR' (-1) in the CreateTopicsRequest, allowing
    the Controller's configuration to take precedence.
  • Added deprecation warnings in KafkaConfig when these properties are
    defined in a Broker role, notifying users to migrate them to the
    Controller role before 5.0.
  • Updated documentation to clarify the precedence logic between
    Broker and Controller nodes.

Reviewers: Luke Chen showuon@gmail.com

…itions and default.replication.factor from the controller properties
@github-actions github-actions bot added core Kafka Broker clients labels Feb 23, 2026
@brandboat brandboat changed the title KAFKA-19655: The normal topic auto-creation paths should use num.partitions and default.replication.factor from the controller properties KAFKA-19655: Align the behavior of num.partitions and default.replication.factor for topic creation Feb 23, 2026
@brandboat
Copy link
Member Author

gentle ping @chia7712, @FrankYang0529, @lucasbru, @showuon, could you please take a look when you're available?

@showuon showuon self-assigned this Mar 2, 2026
Copy link
Member

@showuon showuon left a comment

Choose a reason for hiding this comment

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

Thanks for the PR! Overall LGTM! Left some comments.

Comment on lines -128 to +134
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)
Copy link
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
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 😆

import static org.junit.jupiter.api.Assertions.assertEquals;

@ClusterTestDefaults(types = {Type.KRAFT}, brokers = 2)
public class AutoTopicCreationTest {
Copy link
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
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!

Copy link
Member

@showuon showuon left a comment

Choose a reason for hiding this comment

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

LGTM!

@showuon showuon merged commit 288b479 into apache:trunk Mar 3, 2026
23 checks passed
@brandboat brandboat deleted the KAFKA-19655 branch March 4, 2026 03:32
// 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
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
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

nicktelford pushed a commit to nicktelford/kafka that referenced this pull request Mar 6, 2026
…tion.factor for topic creation (apache#21550)

Currently, 'num.partitions' and 'default.replication.factor' are applied
inconsistently. Topic auto-creation relies on Broker configs, while
Streams and AdminClient rely on Controller configs. This leads to
confusing behavior where a Broker and Controller might   have diverging
defaults.

This commit implements the 4.x transition phase:
- Updated DefaultAutoTopicCreationManager to check if these configs are
  explicitly set in 'broker.properties'.
- If NOT explicitly set, the Broker now sends 'NO_NUM_PARTITIONS' and
  'NO_REPLICATION_FACTOR' (-1) in the CreateTopicsRequest, allowing
  the Controller's configuration to take precedence.
- Added deprecation warnings in KafkaConfig when these properties are
  defined in a Broker role, notifying users to migrate them to the
  Controller role before 5.0.
- Updated documentation to clarify the precedence logic between
  Broker and Controller nodes.

Reviewers: Luke Chen <showuon@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

clients core Kafka Broker

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants