Rob Young created KAFKA-19596:
---------------------------------

             Summary: Log auto topic creation failures more visibly
                 Key: KAFKA-19596
                 URL: https://issues.apache.org/jira/browse/KAFKA-19596
             Project: Kafka
          Issue Type: Improvement
          Components: logging
            Reporter: Rob Young


Hi, I was playing with Share groups on the 4.1.0-rc2 and found it a little 
opaque to detect a failure to auto create the `__share_group_state` topic due 
to the replication factor not being satisfiable.

I start up a cluster like this (taken from the dockerhub instructions):
{code:java}
podman run --rm \
  -p 9092:9092 \
  -e KAFKA_NODE_ID=1 \
  -e KAFKA_PROCESS_ROLES=broker,controller \
  -e KAFKA_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 \
  -e KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://localhost:9092 \
  -e KAFKA_CONTROLLER_LISTENER_NAMES=CONTROLLER \
  -e 
KAFKA_LISTENER_SECURITY_PROTOCOL_MAP=CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT \
  -e KAFKA_CONTROLLER_QUORUM_VOTERS=1@localhost:9093 \
  -e KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR=1 \
  -e KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR=1 \
  -e KAFKA_TRANSACTION_STATE_LOG_MIN_ISR=1 \
  -e KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS=0 \
  -e KAFKA_NUM_PARTITIONS=3 \
  apache/kafka:4.1.0-rc2{code}
and then run the commands:
{code:java}
robeyoun:kafka_2.13-4.1.0$ bin/kafka-features.sh --bootstrap-server 
localhost:9092 upgrade --feature share.version=1
share.version was upgraded to 1.
robeyoun:kafka_2.13-4.1.0$ bin/kafka-topics.sh --create --topic my_topic 
--bootstrap-server localhost:9092
WARNING: Due to limitations in metric names, topics with a period ('.') or 
underscore ('_') could collide. To avoid issues it is best to use either, but 
not both.
Created topic my_topic.
robeyoun:kafka_2.13-4.1.0$ bin/kafka-console-share-consumer.sh 
--bootstrap-server localhost:9092 --topic quickstart-events
[2025-08-12 09:50:13,130] WARN Share groups and KafkaShareConsumer are part of 
a preview feature introduced by KIP-932, and are not recommended for use in 
production. 
(org.apache.kafka.clients.consumer.internals.ShareConsumerDelegateCreator)
[2025-08-12 09:50:13,357] WARN [ShareConsumer clientId=console-share-consumer, 
groupId=console-share-consumer] The metadata response from the cluster reported 
a recoverable issue with correlation id 20 : 
{quickstart-events=UNKNOWN_TOPIC_OR_PARTITION} 
(org.apache.kafka.clients.NetworkClient)
{code}
the consumer now sits there, appearing to be consuming happily, but the broker 
is emitting a stream of logs like:
{code:java}
[2025-08-11 21:51:03,648] INFO Sent auto-creation request for 
Set(__share_group_state) to the active controller. 
(kafka.server.DefaultAutoTopicCreationManager)
[2025-08-11 21:51:03,648] WARN Received retriable error in find coordinator for 
InitializeStateHandler using key 
SharePartitionKey{groupId=console-share-consumer, 
topicIdPartition=sI9vYBBGSKW_3BfG9ZJWhg:null-1}: The coordinator is not 
available. 
(org.apache.kafka.server.share.persister.PersisterStateManager$InitializeStateHandler){code}
So there's a clue there that it's repeatedly trying to auto-create the topic.

the underlying problem is the default replication factor of 3 for the 
__share_group_state topic, but I have to crank up the log level to DEBUG 
(setting -e KAFKA_LOG4J_LOGGERS="kafka=DEBUG") to see the cause:
{code:java}
[2025-08-11 21:53:58,343] INFO Sent auto-creation request for 
Set(__share_group_state) to the active controller. 
(kafka.server.DefaultAutoTopicCreationManager)
[2025-08-11 21:53:58,343] WARN Received retriable error in find coordinator for 
InitializeStateHandler using key 
SharePartitionKey{groupId=console-share-consumer, 
topicIdPartition=8kjLkvNzSCy7geIIriJVCQ:null-2}: The coordinator is not 
available. 
(org.apache.kafka.server.share.persister.PersisterStateManager$InitializeStateHandler)
[2025-08-11 21:53:58,345] DEBUG 
[broker-1-to-controller-forwarding-channel-manager]: Request 
CreateTopicsRequestData(topics=[CreatableTopic(name='__share_group_state', 
numPartitions=50, replicationFactor=3, assignments=[], 
configs=[CreatableTopicConfig(name='compression.type', value='producer'), 
CreatableTopicConfig(name='cleanup.policy', value='delete'), 
CreatableTopicConfig(name='min.insync.replicas', value='2'), 
CreatableTopicConfig(name='segment.bytes', value='104857600'), 
CreatableTopicConfig(name='retention.ms', value='-1')])], timeoutMs=30000, 
validateOnly=false) received ClientResponse(receivedTimeMs=1754949238345, 
latencyMs=2, disconnected=false, timedOut=false, 
requestHeader=RequestHeader(apiKey=CREATE_TOPICS, apiVersion=7, clientId=1, 
correlationId=22, headerVersion=2), 
responseBody=CreateTopicsResponseData(throttleTimeMs=0, 
topics=[CreatableTopicResult(name='__share_group_state', 
topicId=AAAAAAAAAAAAAAAAAAAAAA, errorCode=38, errorMessage='Unable to replicate 
the partition 3 time(s): The target replication factor of 3 cannot be reached 
because only 1 broker(s) are registered.', topicConfigErrorCode=0, 
numPartitions=-1, replicationFactor=-1, configs=[])])) 
(kafka.server.NodeToControllerRequestThread)
[2025-08-11 21:53:58,345] DEBUG Cleared inflight topic creation state for 
HashMap(__share_group_state -> CreatableTopic(name='__share_group_state', 
numPartitions=50, replicationFactor=3, assignments=[], 
configs=[CreatableTopicConfig(name='compression.type', value='producer'), 
CreatableTopicConfig(name='cleanup.policy', value='delete'), 
CreatableTopicConfig(name='min.insync.replicas', value='2'), 
CreatableTopicConfig(name='segment.bytes', value='104857600'), 
CreatableTopicConfig(name='retention.ms', value='-1')])) 
(kafka.server.DefaultAutoTopicCreationManager)
[2025-08-11 21:53:58,345] DEBUG Auto topic creation completed for 
Set(__share_group_state) with response 
CreateTopicsResponseData(throttleTimeMs=0, 
topics=[CreatableTopicResult(name='__share_group_state', 
topicId=AAAAAAAAAAAAAAAAAAAAAA, errorCode=38, errorMessage='Unable to replicate 
the partition 3 time(s): The target replication factor of 3 cannot be reached 
because only 1 broker(s) are registered.', topicConfigErrorCode=0, 
numPartitions=-1, replicationFactor=-1, configs=[])]). 
(kafka.server.DefaultAutoTopicCreationManager)
 {code}
The point where we log the auto creation outcome has two cases where it warn 
logs, but then this case is being handled by a catch all debug log. 
[https://github.com/apache/kafka/blob/18045c6ac30921503deffbef1744bb365dc599fb/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala#L141]

If there's an error code set, maybe it should log at warn like the other cases? 
I'm happy to be assigned to this.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to