tet123/documentation/modules/ROOT/pages/configuration/topic-auto-create-config.adoc

521 lines
22 KiB
Plaintext
Raw Normal View History

// Category: debezium-using
// Type: assembly
// ModuleID: customization-of-kafka-connect-automatic-topic-creation
// Title: Customization of Kafka Connect automatic topic creation
[id="customizing-debezium-automatically-created-topics"]
= Customization of Kafka Connect automatic topic creation
:toc:
:toc-placement: macro
:linkattrs:
:icons: font
:source-highlighter: highlight.js
toc::[]
Kafka provides two mechanisms for creating topics automatically.
You can enable automatic topic creation for the Kafka broker, and, beginning with Kafka 2.6.0, you can also enable Kafka Connect to create topics.
The Kafka broker uses the `auto.create.topics.enable` property to control automatic topic creation.
In Kafka Connect, the `topic.creation.enable` property specifies whether Kafka Connect is permitted to create topics.
In both cases, the default settings for the properties enables automatic topic creation.
When automatic topic creation is enabled,
if a Debezium source connector emits a change event record for a table for which no target topic already exists,
the topic is created at runtime as the event record is ingested into Kafka.
.Differences between automatic topic creation at the broker and in Kafka Connect
Topics that the broker creates are limited to sharing a single default configuration.
The broker cannot apply unique configurations to different topics or sets of topics.
By contrast, Kafka Connect can apply any of several configurations when creating topics,
setting the replication factor, number of partitions, and other topic-specific settings as specified in the {prodname} connector configuration.
The connector configuration defines a set of topic creation groups, and associates a set of topic configuration properties with each group.
The broker configuration and the Kafka Connect configuration are independent of each other.
Kafka Connect can create topics regardless of whether you disable topic creation at the broker.
If you enable automatic topic creation at both the broker and in Kafka Connect, the Connect configuration takes precedence,
and the broker creates topics only if none of the settings in the Kafka Connect configuration apply.
ifdef::product[]
See the following topics for more information:
* xref:disabling-automatic-topic-creation-for-the-kafka-broker[]
* xref:configuring-automatic-topic-creation-in-kafka-connect[]
* xref:configuration-of-automatically-created-topics[]
* xref:debezium-connector-topic-creation-groups[]
* xref:debezium-connector-topic-creation-group-configuration-properties[]
* xref:specifying-the-configuration-for-the-debezium-default-topic-creation-group[]
* xref:specifying-the-configurations-for-debezium-custom-topic-creation-groups[]
* xref:registering-debezium-custom-topic-creation-groups[]
endif::product[]
// Type: procedure
// ModuleID: disabling-automatic-topic-creation-for-the-kafka-broker
[id="disabling-automatic-topic-creation-for-the-kafka-broker"]
== Disabling automatic topic creation for the Kafka broker
By default, the Kafka broker configuration enables the broker to create topics at runtime if the topics do not already exist.
Topics created by the broker cannot be configured with custom properties.
If you use a Kafka version earlier than 2.6.0, and you want to create topics with specific configurations, you must to disable automatic topic creation at the broker,
and then explicitly create the topics, either manually, or through a custom deployment process.
.Procedure
* In the broker configuration, set the value of `auto.create.topics.enable` to `false`.
// Type: procedure
// ModuleID: configuring-automatic-topic-creation-in-kafka-connect
// Title: Configuring automatic topic creation in Kafka Connect
[id="enabling-automatic-topic-creation"]
2020-09-18 12:00:05 +02:00
== Set up Kafka Connect
Automatic topic creation in Kafka Connect is controlled by the `topic.creation.enable` property.
The default value for the property is `true`, enabling automatic topic creation, as shown in the following example:
[source,shell,options="nowrap"]
----
topic.creation.enable = true
----
The setting for the `topic.creation.enable` property applies to all workers in the Connect cluster.
Kafka Connect automatic topic creation requires you to define the configuration properties that Kafka Connect applies when creating topics.
You specify topic configuration properties in the {prodname} connector configuration by defining topic groups, and then specifying the properties to apply to each group.
The connector configuration defines a default topic creation group, and, optionally, one or more custom topic creation groups.
Custom topic creation groups use lists of topic name patterns to specify the topics to which the group's settings apply.
For details about how Kafka Connect matches topics to topic creation groups, see xref:topic-creation-groups[Topic creation groups].
For more information about how configuration properties are assigned to groups, see xref:topic-creation-group-configuration-properties[Topic creation group configuration properties].
By default, topics that Kafka Connect creates are named based on the pattern `server.schema.table`, for example, `dbserver.myschema.inventory`.
ifdef::community[]
[NOTE]
====
If you don't want to allow automatic topic creation by Kafka Connect, set the value of `topic.creation.enable` to `false`
in the Kafka Connect configuration (_connect-distributed.properties_ file or via environment variable
_CONNECT_TOPIC_CREATION_ENABLE_ when using https://quay.io/repository/debezium/connect[{prodname}'s container image for Kafka Connect]).
====
endif::community[]
ifdef::product[]
.Procedure
* To prevent Kafka Connect from creating topics automatically, set the value of `topic.creation.enable` to `false`
in the Kafka Connect custom resource, as in the following example:
[source,yaml,options="nowrap"]
----
apiVersion: kafka.strimzi.io/v1beta1
kind: KafkaConnect
metadata:
name: my-connect-cluster
...
spec:
config:
topic.creation.enable: "false"
----
endif::product[]
[NOTE]
====
Kafka Connect automatic topic creation requires the `replication.factor` and `partitions` properties to be set for at least the `default` topic creation group.
It is valid for groups to obtain the values for the required properties from the default values for the Kafka broker.
====
// Type: concept
// ModuleID: configuration-of-automatically-created-topics
// Title: Configuration of automatically created topics
[id="configuration"]
== Configuration
For Kafka Connect to create topics automatically, it requires information from the source connector about the configuration properties to apply when creating topics.
You define the properties that control topic creation in the configuration for each {prodname} connector.
As Kafka Connect creates topics for event records that a connector emits, the resulting topics obtain their configuration from the applicable group.
The configuration applies to event records emitted by that connector only.
// Type: concept
// ModuleID: debezium-connector-topic-creation-groups
[id="topic-creation-groups"]
=== Topic creation groups
A set of topic properties is associated with a topic creation group.
Minimally, you must define a `default` topic creation group and specify its configuration properties.
Beyond that you can optionally define one or more custom topic creation groups and specify unique properties for each.
When you create custom topic creation groups, you define the member topics for each group based on topic name patterns.
You can specify naming patterns that describe the topics to include or exclude from each group.
The `include` and `exclude` properties contain comma-separated lists of regular expressions that define topic name patterns.
For example, if you want a group to include all topics that start with the string `dbserver1.inventory`, set the value of its `topic.creation.inventory.include` property to `dbserver1\\.inventory\\.*`.
[NOTE]
====
If you specify both `include` and `exclude` properties for a custom topic group, the exclusion rules
take precedence, and override the inclusion rules.
====
// Type: concept
// ModuleID: debezium-connector-topic-creation-group-configuration-properties
[id="topic-creation-group-configuration-properties"]
=== Topic creation group configuration properties
The `default` topic creation group and each custom group is associated with a unique set of configuration properties.
You can configure a group to include any of the {link-kafka-docs}/#topicconfigs[Kafka topic-level configuration properties].
For example, you can specify the {link-kafka-docs}/#topicconfigs_cleanup.policy[cleanup policy for old topic segments], {link-kafka-docs}/#topicconfigs_retention.ms[retention time], or the {link-kafka-docs}/#topicconfigs_compression.type[topic compression type] for a topic group.
You must define at least a minimum set of properties to describe the configuration of the topics to be created.
If no custom groups are registered, or if the `include` patterns for any registered groups don't match the names of any topics to be created,
then Kafka Connect uses the configuration of the `default` group to create topics.
ifdef::community[]
See {link-prefix}:{link-install-debezium}#configuring-debezium-topics[Configuring {prodname} topics] in the
2020-09-16 15:53:03 +02:00
{prodname} installation guide on generic topic configuration considerations.
endif::community[]
2020-09-16 15:53:03 +02:00
ifdef::product[]
For general information about configuring topics, see link:{LinkDebeziumInstallOpenShift}#kafka-topic-creation-recommendations[Kafka topic creation recommendations] in {NameDebeziumInstallOpenShift}.
endif::product[]
// Type: procedure
// ModuleID: specifying-the-configuration-for-the-debezium-default-topic-creation-group
// Title: Specifying the configuration for the {prodname} default topic creation group
[id="default-topic-creation-group-configuration"]
2020-09-18 12:00:05 +02:00
=== Default group configuration
Before you can use Kafka Connect automatic topic creation, you must create a default topic creation group and define a configuration for it.
The configuration for the default topic creation group is applied to any topics with names that do not match the `include` list pattern of a custom topic creation group.
ifdef::product[]
.Prerequisites
* In the Kafka Connect custom resource, the `use-connector-resources` value in `metadata.annotations` specifies that the cluster Operator uses KafkaConnector custom resources to configure connectors in the cluster.
For example:
+
[source,yaml]
----
...
metadata:
name: my-connect-cluster
annotations: strimzi.io/use-connector-resources: "true"
...
----
endif::product[]
.Procedure
ifdef::community[]
* To define properties for the `topic.creation.default` group, add them to the connector configuration JSON, as shown in the following example:
+
[source,json,options="nowrap"]
----
{
...
"topic.creation.default.replication.factor": 3, //<1>
"topic.creation.default.partitions": 10, //<2>
"topic.creation.default.cleanup.policy": "compact", //<3>
"topic.creation.default.compression.type": "lz4" //<4>
...
}
----
endif::community[]
ifdef::product[]
* To define properties for the `topic.creation.default` group, add them to `spec.config` in the connector custom resource, as shown in the following example:
+
[source,yaml]
----
apiVersion: kafka.strimzi.io/v1beta1
kind: KafkaConnector
metadata:
name: inventory-connector
labels:
strimzi.io/cluster: my-connect-cluster
spec:
...
config:
...
topic.creation.default.replication.factor: 3 //<1>
topic.creation.default.partitions: 10 //<2>
topic.creation.default.cleanup.policy: compact //<3>
topic.creation.default.compression.type: lz4 //<4>
...
----
endif::product[]
+
You can include any {link-kafka-docs}/#topicconfigs[Kafka topic-level configuration property] in the configuration for the `default` group.
.Connector configuration for the `default` topic creation group
[cols="1,9",options="header"]
|===
|Item |Description
|1
|`topic.creation.default.replication.factor` defines the replication factor for topics created by
the default group.{empty} +
`replication.factor` is mandatory for the `default` group but optional for custom groups. Custom
groups will fall back to the `default` group's value if not set. Use `-1` to use the Kafka
broker's default value.
|2
|`topic.creation.default.partitions` defines the number of partitions for topics created by
the default group.{empty} +
`partitions` is mandatory for the `default` group but optional for custom groups. Custom
groups will fall back to the `default` group's value if not set. Use `-1` to use the Kafka
broker's default value.
|3
2020-09-16 15:53:03 +02:00
|`topic.creation.default.cleanup.policy` is mapped to the {link-kafka-docs}/#cleanup.policy[`cleanup.policy`]
property of the {link-kafka-docs}/#topicconfigs[topic level configuration parameters] and
defines the log retention policy.
|4
2020-09-16 15:53:03 +02:00
|`topic.creation.default.compression.type` is mapped to the {link-kafka-docs}/#compression.type[`compression.type`]
property of the {link-kafka-docs}/#topicconfigs[topic level configuration parameters] and
defines how messages are compressed on hard disk.
|===
[NOTE]
====
Custom groups fall back to the `default` group settings only for the required `replication.factor` and `partitions` properties.
If the configuration for a custom topic group leaves other properties undefined, the values specified in the `default` group are not applied.
====
// Type: procedure
// ModuleID: specifying-the-configurations-for-debezium-custom-topic-creation-groups
// Title: Specifying the configuration for {prodname} custom topic creation groups
[id="custom-topic-creation-group-configuration"]
2020-09-18 12:00:05 +02:00
=== Custom group configuration
You can define multiple custom topic groups, each with its own configuration.
.Procedure
ifdef::community[]
* To define a custom topic group, add a `topic.creation._<group_name>_.include` property to the connector JSON, and
list the properties for the custom group after the group name.
+
The following example shows sample configurations for the `inventory` and `applicationlogs` custom topic creation groups:
+
[source,json,options="nowrap"]
----
{
...
//<1>
"topic.creation.inventory.include": "dbserver1\\.inventory\\.*", //<2>
"topic.creation.inventory.partitions": 20,
"topic.creation.inventory.cleanup.policy": "compact",
"topic.creation.inventory.delete.retention.ms": 7776000000,
//<3>
"topic.creation.applicationlogs.include": "dbserver1\\.logs\\.applog-.*", //<4>
"topic.creation.applicationlogs.exclude": "dbserver1\\.logs\\.applog-old-.*", //<5>
"topic.creation.applicationlogs.replication.factor": 1,
"topic.creation.applicationlogs.partitions": 20,
"topic.creation.applicationlogs.cleanup.policy": "delete",
"topic.creation.applicationlogs.retention.ms": 7776000000,
"topic.creation.applicationlogs.compression.type": "lz4",
...
}
----
endif::community[]
ifdef::product[]
* To define a custom topic group, add a `topic.creation._<group_name>_.include` property to `spec.config` in the connector custom resource, followed
by the configuration properties that you want to apply to topics in the custom group.
+
The following example shows an excerpt of a custom resource that defines the custom topic creation groups `inventory` and `applicationlogs`:
+
[source,yaml]
----
apiVersion: kafka.strimzi.io/v1beta1
kind: KafkaConnector
metadata:
name: inventory-connector
...
spec:
...
config:
... //<1>
topic.creation.inventory.include: dbserver1\\.inventory\\.* //<2>
topic.creation.inventory.partitions: 20
topic.creation.inventory.cleanup.policy: compact
topic.creation.inventory.delete.retention.ms: 7776000000
//<3>
topic.creation.applicationlogs.include: dbserver1\\.logs\\.applog-.* //<4>
topic.creation.applicationlogs.exclude": dbserver1\\.logs\\.applog-old-.* //<5>
topic.creation.applicationlogs.replication.factor: 1
topic.creation.applicationlogs.partitions: 20
topic.creation.applicationlogs.cleanup.policy: delete
topic.creation.applicationlogs.retention.ms: 7776000000
topic.creation.applicationlogs.compression.type: lz4
...
...
----
endif::product[]
.Connector configuration for custom `inventory` and `applicationlogs` topic creation groups
[cols="1,9",options="header"]
|===
|Item |Description
|1
|Defines the configuration for the `inventory` group.{empty} +
The `replication.factor` and `partitions` properties are optional for custom groups. If no value is set, custom
groups fall back to the value set for the `default` group. Set the value to `-1` to use the value that is set for the Kafka broker.
|2
|`topic.creation.inventory.include` defines a regular expression to match all topics that start with
`dbserver1.inventory.`. The configuration that is defined for the `inventory` group is applied only to
topics with names that match the specified regular expression.
|3
|Defines the configuration for the `applicationlogs` group.{empty} +
The `replication.factor` and `partitions` properties are optional for custom groups. If no value is set,
custom groups fall back to the value set for the `default` group. Set the value to `-1` to use the value that is set for the Kafka broker.
|4
|`topic.creation.applicationlogs.include` defines a regular expression to match all topics that start
with `dbserver1.logs.applog-`. The configuration that is defined for the `applicationlogs` group is
applied only to topics with names that match the specified regular expression. Because an `exclude`
property is also defined for this group, the topics that match the `include` regular expression might be
further restricted by the that `exclude` property.
|5
|`topic.creation.applicationlogs.exclude` defines a regular expression to match all topics that start
with `dbserver1.logs.applog-old-`. The configuration that is defined for the `applicationlogs` group is
applied only to topics with name that do *not* match the given regular expression. Because an
`include` property is also defined for this group, the configuration of the `applicationlogs` group is applied only to topics
with names that match the specified `include` regular expressions *and* that do _not_ match the specified `exclude` regular
expressions.
|===
// Type: procedure
// ModuleID: registering-debezium-custom-topic-creation-groups
// Title: Registering {prodname} custom topic creation groups
[id="registering-custom-topic-creation-groups"]
2020-09-18 12:00:05 +02:00
=== Registering custom groups
After you specify the configuration for any custom topic creation groups, register the groups.
.Procedure
ifdef::community[]
* Register custom groups by adding the `topic.creation.groups` property to the connector JSON, and specifying a comma-separated list of groups.
+
The following example registers the custom topic creation groups `inventory` and `applicationlogs`:
+
[source,json,options="nowrap"]
----
{
...
"topic.creation.groups": "inventory,applicationlogs",
...
}
----
endif::community[]
ifdef::product[]
* Register custom groups by adding the `topic.creation.groups` property to the connector custom resource, and specifying a comma-separated list of custom topic creation groups.
+
The following excerpt from a connector custom resource registers the custom topic creation groups `inventory` and `applicationlogs`:
+
[source,yaml]
----
apiVersion: kafka.strimzi.io/v1beta1
kind: KafkaConnector
metadata:
name: inventory-connector
...
spec:
...
config:
topic.creation.groups: inventory,applicationlogs
...
----
endif::product[]
.Completed configuration
The following example shows a completed configuration that includes the configuration for a `default` topic group,
along with the configurations for an `inventory` and an `applicationlogs` custom topic creation group:
.Example: Configuration for a default topic creation group and two custom groups
ifdef::community[]
[source,json,options="nowrap"]
----
{
...
"topic.creation.default.replication.factor": 3,
"topic.creation.default.partitions": 10,
"topic.creation.default.cleanup.policy": "compact",
2023-03-23 21:36:21 +01:00
"topic.creation.default.compression.type": "lz4",
"topic.creation.groups": "inventory,applicationlogs",
"topic.creation.inventory.include": "dbserver1\\.inventory\\.*",
"topic.creation.inventory.partitions": 20,
"topic.creation.inventory.cleanup.policy": "compact",
"topic.creation.inventory.delete.retention.ms": 7776000000,
"topic.creation.applicationlogs.include": "dbserver1\\.logs\\.applog-.*",
"topic.creation.applicationlogs.exclude": "dbserver1\\.logs\\.applog-old-.*",
"topic.creation.applicationlogs.replication.factor": 1,
"topic.creation.applicationlogs.partitions": 20,
"topic.creation.applicationlogs.cleanup.policy": "delete",
"topic.creation.applicationlogs.retention.ms": 7776000000,
"topic.creation.applicationlogs.compression.type": "lz4"
}
----
endif::community[]
ifdef::product[]
[source,yaml]
----
apiVersion: kafka.strimzi.io/v1beta1
kind: KafkaConnector
metadata:
name: inventory-connector
...
spec:
...
config:
...
topic.creation.default.replication.factor: 3,
topic.creation.default.partitions: 10,
topic.creation.default.cleanup.policy: compact
topic.creation.default.compression.type: lz4
topic.creation.groups: inventory,applicationlogs
topic.creation.inventory.include: dbserver1\\.inventory\\.*
topic.creation.inventory.partitions: 20
topic.creation.inventory.cleanup.policy: compact
topic.creation.inventory.delete.retention.ms: 7776000000
topic.creation.applicationlogs.include: dbserver1\\.logs\\.applog-.*
topic.creation.applicationlogs.exclude": dbserver1\\.logs\\.applog-old-.*
topic.creation.applicationlogs.replication.factor: 1
topic.creation.applicationlogs.partitions: 20
topic.creation.applicationlogs.cleanup.policy: delete
topic.creation.applicationlogs.retention.ms: 7776000000
topic.creation.applicationlogs.compression.type: lz4
...
----
endif::product[]
ifdef::community[]
== Additional resources
2020-09-16 15:53:03 +02:00
For more information on topic auto-creation you can have a look at these resources:
2020-09-18 12:00:05 +02:00
* Debezium Blog: https://debezium.io/blog/2020/09/15/debezium-auto-create-topics/[Auto-creating Debezium Change Data Topics]
* Kafka Improvement Proposal about adding topic auto-creation to Kafka Connect: https://cwiki.apache.org/confluence/display/KAFKA/KIP-158%3A+Kafka+Connect+should+allow+source+connectors+to+set+topic-specific+settings+for+new+topics[KIP-158 Kafka Connect should allow source connectors to set topic-specific settings for new topics]
2020-09-16 15:53:03 +02:00
endif::community[]