Skip to content

Conversation

robobario
Copy link
Contributor

@robobario robobario commented Aug 12, 2025

Log a warning for each topic that failed to be created as a result of an automatic creation. This makes the underlying cause more visible so users can take action.

Previously, at the default log level, you could only see logs that the broker was attempting to autocreate topics. If the creation failed, then it was logged at debug.

@github-actions github-actions bot added triage PRs from the community core Kafka Broker small Small PRs labels Aug 12, 2025
@robobario
Copy link
Contributor Author

For example, if I:

  1. edit server.properties to share.coordinator.state.topic.replication.factor=3
  2. run up the broker as per the quickstart:
    KAFKA_CLUSTER_ID="$(bin/kafka-storage.sh random-uuid)"
    bin/kafka-storage.sh format --standalone -t $KAFKA_CLUSTER_ID -c config/server.properties
    bin/kafka-server-start.sh config/server.properties
    
  3. bin/kafka-features.sh --bootstrap-server localhost:9092 upgrade --feature share.version=1
  4. bin/kafka-topics.sh --create --topic my_topic --bootstrap-server localhost:9092
  5. bin/kafka-console-share-consumer.sh --bootstrap-server localhost:9092 --topic quickstart-events

Now I will see logs in the server like:

[2025-08-12 16:28:56,007] INFO Sent auto-creation request for Set(__share_group_state) to the active controller. (kafka.server.DefaultAutoTopicCreationManager)
[2025-08-12 16:28:56,009] WARN Auto topic creation failed for __share_group_state with error 'INVALID_REPLICATION_FACTOR': Unable to replicate the partition 3 time(s): The target replication factor of 3 cannot be reached because only 1 broker(s) are registered. (kafka.server.DefaultAutoTopicCreationManager)

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.

It looks like the AutoTopicCreateManager just wants to try to create the topics once without retry, which is fine to me because users can create the topic manually later. Left a comment. Thanks.

warn(s"Auto topic creation failed for ${topicResult.name()} with error '${error.name()}': ${topicResult.errorMessage()}")
}
})
case _ => // do nothing
Copy link
Member

Choose a reason for hiding this comment

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

Don't you think this should be also an error we should log?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

true, have added a warn log in this case

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! Thanks for the improvement.

Copy link
Member

@brandboat brandboat left a comment

Choose a reason for hiding this comment

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

LGTM, thanks for the enhancement!

createTopicsResponse.data().topics().forEach(topicResult => {
val error = Errors.forCode(topicResult.errorCode())
if (error != Errors.NONE) {
warn(s"Auto topic creation failed for ${topicResult.name()} with error '${error.name()}': ${topicResult.errorMessage()}")
Copy link
Member

Choose a reason for hiding this comment

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

tiny nit: we can skip () for getters

Suggested change
warn(s"Auto topic creation failed for ${topicResult.name()} with error '${error.name()}': ${topicResult.errorMessage()}")
warn(s"Auto topic creation failed for ${topicResult.name} with error '${error.name}': ${topicResult.errorMessage}")

Copy link
Contributor Author

Choose a reason for hiding this comment

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

thanks, have killed off ()

Log a warning for each topic that failed to be created as a result of
an automatic creation.

Previously, at the default log level, you could only see logs that the
broker was attempting to autocreate topics. If the creation failed it
was logged at debug. This makes it more visible.

Signed-off-by: Robert Young <[email protected]>
@github-actions github-actions bot removed the triage PRs from the community label Aug 13, 2025
@showuon showuon merged commit 3067f15 into apache:trunk Aug 14, 2025
20 checks passed
createTopicsResponse.data().topics().forEach(topicResult => {
val error = Errors.forCode(topicResult.errorCode)
if (error != Errors.NONE) {
warn(s"Auto topic creation failed for ${topicResult.name} with error '${error.name}': ${topicResult.errorMessage}")
Copy link
Member

Choose a reason for hiding this comment

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

Have you considered logging the request to provide a more useful debug message?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
ci-approved core Kafka Broker small Small PRs
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants