Lincong Li created KAFKA-10606:
----------------------------------

             Summary: Auto create non-existent topics when fetching metadata 
for all topics
                 Key: KAFKA-10606
                 URL: https://issues.apache.org/jira/browse/KAFKA-10606
             Project: Kafka
          Issue Type: Bug
            Reporter: Lincong Li


The "allow auto topic creation" flag is hardcoded to be true for the 
fetch-all-topic metadata request:

https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java#L37

In the below code, annotation claims that "*This never causes auto-creation*". 
It it NOT true and auto topic creation still gets triggered under some 
circumstances. So, this is a bug that needs to be fixed.

https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java#L68


For example, the bug could be manifested in the below situation:

A topic T is being deleted and a request to fetch metadata for all topics gets 
sent to one broker. The broker reads names of all topics from its metadata 
cache (shown below).

https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/KafkaApis.scala#L1196

Then the broker authorizes all topics and makes sure that they are allowed to 
be described. Then the broker tries to get metadata for every authorized topic 
by reading the metadata cache again, once for every topic (show below).

https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/KafkaApis.scala#L1240

However, the metadata cache could have been updated while the broker was 
authorizing all topics and topic T and its metadata no longer exist in the 
cache since the topic got deleted and metadata update requests eventually got 
propagated from the controller to all brokers. So, at this point, when the 
broker tries to get metadata for topic T from its cache, it realizes that it 
does not exist and the broker tries to "auto create" topic T since the 
allow-auto-topic-creation flag was set to true in all the fetch-all-topic 
metadata requests.

I think this bug exists since "*metadataRequest.allowAutoTopicCreation*" was 
introduced.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to