KAFKA-19655: Align the behavior of num.partitions and default.replication.factor for topic creation#21550
Conversation
…itions and default.replication.factor from the controller properties
|
gentle ping @chia7712, @FrankYang0529, @lucasbru, @showuon, could you please take a look when you're available? |
showuon
left a comment
There was a problem hiding this comment.
Thanks for the PR! Overall LGTM! Left some comments.
| 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) |
There was a problem hiding this comment.
Nice catch! Why could we pass the tests with the wrong setting?
There was a problem hiding this comment.
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 { |
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
Sure, let me add one, thanks!
| // 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)) { |
There was a problem hiding this comment.
Could you refactor warnIfConfigDefinedInWrongRole to ensure consistent warning messages?
…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>
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:
explicitly set in 'broker.properties'.
'NO_REPLICATION_FACTOR' (-1) in the CreateTopicsRequest, allowing
the Controller's configuration to take precedence.
defined in a Broker role, notifying users to migrate them to the
Controller role before 5.0.
Broker and Controller nodes.
Reviewers: Luke Chen showuon@gmail.com