Status

Current state: Accepted

Discussion threadHere

JIRA Unable to render Jira issues macro, execution error.

Released: 2.3.0

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

When consumer subscribes to a non-existent topic, it does not have control over whether the topic is automatically created or not. We have the broker configuration allow.auto.create.topics that controls topic auto-creation globally. If there is a lag between the consumers and producers starting up, we usually do not want the consumer to be able to automatically create the topic as this will use the default number of partitions and replication factor. Consumers should thus have the ability to be able to disable auto topic creation, if needed.

To work around the lack of such a configuration, consumers currently avoid subscribing to specific topics and use regex-based subscription instead, like in Streams. This is not optimal as it means consumers will request for and receive metadata for all topics and partitions in the cluster, which could be large.

Public Interfaces

We will add a new consumer configuration:

Configuration Name: allow.auto.create.topics

Type: BOOLEAN
Explanation: This configuration controls whether the topic is automatically created when subscribing to a non-existent topic. A topic will be auto-created only if this configuration is set to true and auto-topic creation is enabled on the broker using auto.create.topics.enable; otherwise auto-topic creation is not permitted. This configuration must be set to true when using brokers older than 0.11.
Default Value: TRUE
Priority: MEDIUM

Broker Config auto.create.topics.enableConsumer Config allow.auto.create.topicsAuto-Topic Creation Permitted When Consumer Subscribes to Topic
truetrueYes
truefalseNo
falsetrueNo
falsefalseNo


Proposed Changes

Whether the topic should be auto-created will be included in MetadataRequest sent by the consumer. In Kafka 0.11.0, MetadataRequest v4 had introduced a way to specify if a topic should be auto-created when requesting metadata for specific topics.

If the broker is older than 0.11.0, allow.auto.create.topics must be set to true. If set to false in this case, the consumer will throw an InvalidConfigurationException at run-time. Specifically, the exception will be thrown on calling KafkaConsumer#poll after subscribing to a non-existent topic.

Compatibility, Deprecation, and Migration Plan

NA

Rejected Alternatives

None

  • No labels