- Debezium connector for MySQL
- How the connector works
- Supported MySQL topologies
- Schema history topic
- Schema change topic
- Snapshots
- Initial snapshots that use a global read lock
- Initial snapshots that use table-level locks
- Understanding why initial snapshots capture the schema history for all tables
- Capturing data from tables not captured by the initial snapshot (no schema change)
- Capturing data from tables not captured by the initial snapshot (schema change)
- Ad hoc snapshots
- Incremental snapshots
- Triggering an incremental snapshot
- Running an ad hoc incremental snapshots with
additional-conditions
- Using the Kafka signaling channel to trigger an incremental snapshot
- Stopping an incremental snapshot
- Using the Kafka signaling channel to stop an incremental snapshot
- Read-only incremental snapshots
- Ad hoc read-only incremental snapshots
- Operation type of snapshot events
- Custom snapshotter SPI
- Blocking snapshots
- Topic names
- Transaction metadata
- Data change events
- Data type mappings
- Custom converters
- Setting up MySQL
- Deployment
- MySQL connector configuration example
- Adding connector configuration
- Connector properties
- Required Debezium MySQL connector configuration properties
- Advanced Debezium MySQL connector configuration properties
- Debezium connector database schema history configuration properties
- Pass-through MySQL connector configuration properties
- Pass-through properties for configuring how producer and consumer clients interact with schema history topics
- Pass-through properties for configuring how the MySQL connector interacts with the Kafka signaling topic
- Pass-through properties for configuring the Kafka consumer client for the signaling channel
- Pass-through properties for configuring the MySQL connector sink notification channel
- Debezium connector pass-through database driver configuration properties
- Monitoring
- Behavior when things go wrong
- How the connector works
Debezium connector for MySQL
How the connector works
An overview of the MySQL topologies that the connector supports is useful for planning your application. To optimally configure and run a Debezium MySQL connector, it is helpful to understand how the connector tracks the structure of tables, exposes schema changes, performs snapshots, and determines Kafka topic names.
Supported MySQL topologies
The Debezium MySQL connector supports the following MySQL topologies:
Standalone
When a single MySQL server is used, the server must have the binlog enabled so the Debezium MySQL connector can monitor the server. This is often acceptable, since the binary log can also be used as an incremental link:https://dev.mysql.com/doc/refman/8.2/en/backup-methods.html [backup]. In this case, the MySQL connector always connects to and follows this standalone MySQL server instance.
Primary and replica
The Debezium MySQL connector can follow one of the primary servers, or one of the replicas (if that replica has its binlog enabled), but the connector detects changes only in the cluster that is visible to that server. Generally, this is not a problem except for the multi-primary topologies.
The connector records its position in the server’s binlog, which is different on each server in the cluster. Therefore, the connector must follow just one MySQL server instance. If that server fails, that server must be restarted or recovered before the connector can continue.
High available clusters
A variety of https://dev.mysql.com/doc/mysql-ha-scalability/en/ [high availability] solutions exist for MySQL, and they make it significantly easier to tolerate and almost immediately recover from problems and failures. Because most HA MySQL clusters use GTIDs, replicas are able to track all of the changes that occur on any primary server.
Multi-primary
Network Database (NDB) cluster replication uses one or more MySQL replica nodes that each replicate from multiple primary servers. Cluster replication provides a powerful way to aggregate the replication of multiple MySQL clusters. This topology requires the use of GTIDs.
A Debezium MySQL connector can use these multi-primary MySQL replicas as sources, and can fail over to different multi-primary MySQL replicas as long as the new replica is caught up to the old replica. That is, the new replica has all transactions that were seen on the first replica. This works even if the connector is using only a subset of databases and/or tables, because the connector can be configured to include or exclude specific GTID sources when attempting to reconnect to a new multi-primary MySQL replica and find the correct position in the binlog.
Hosted
The Debezium MySQL connector can use hosted database options such as Amazon RDS and Amazon Aurora.
Because these hosted options do not permit the use of global read locks, the connector uses table-level locks when it creates a consistent snapshot.
Schema history topic
When a database client queries a database, the client uses the database’s current schema. However, the database schema can be changed at any time, which means that the connector must be able to identify what the schema was at the time each insert, update, or delete operation was recorded. Also, a connector cannot necessarily apply the current schema to every event. If an event is relatively old, it’s possible that it was recorded before the current schema was applied.
To ensure correct processing of events that occur after a schema change, MySQL includes in the transaction log not only the row-level changes that affect the data, but also the DDL statements that are applied to the database. As the connector encounters these DDL statements in the binlog, it parses them and updates an in-memory representation of each table’s schema. The connector uses this schema representation to identify the structure of the tables at the time of each insert, update, or delete operation and to produce the appropriate change event. In a separate database schema history Kafka topic, the connector records all DDL statements along with the position in the binlog where each DDL statement appeared.
When the connector restarts after either a crash or a graceful stop, it starts reading the binlog from a specific position, that is, from a specific point in time. The connector rebuilds the table structures that existed at this point in time by reading the database schema history Kafka topic and parsing all DDL statements up to the point in the binlog where the connector is starting.
This database schema history topic is for internal connector use only. Optionally, the connector can also emit schema change events to a different topic that is intended for consumer applications.
When the MySQL connector captures changes in a table to which a schema change tool such as gh-ost
or pt-online-schema-change
is applied, there are helper tables created during the migration process. You must configure the connector to capture changes that occur in these helper tables. If consumers do not need the records the connector generates for helper tables, configure a single message transform (SMT) to remove these records from the messages that the connector emits.
Additional resources
- Default names for topics that receive Debezium event records.
Schema change topic
You can configure a Debezium MySQL connector to produce schema change events that describe schema changes that are applied to tables in the database. The connector writes schema change events to a Kafka topic named _<topicPrefix>_
, where _topicPrefix_
is the namespace specified in the topic.prefix connector configuration property. Messages that the connector sends to the schema change topic contain a payload, and, optionally, also contain the schema of the change event message.
The schema for the schema change event has the following elements:
name
The name of the schema change event message.
type
The type of the change event message.
version
The version of the schema. The version is an integer that is incremented each time the schema is changed.
fields
The fields that are included in the change event message.
Example: Schema of the MySQL connector schema change topic
The following example shows a typical schema in JSON format.
{
"schema": {
"type": "struct",
"fields": [
{
"type": "string",
"optional": false,
"field": "databaseName"
}
],
"optional": false,
"name": "io.debezium.connector.mysql.SchemaChangeKey",
"version": 1
},
"payload": {
"databaseName": "inventory"
}
}
The payload of a schema change event message includes the following elements:
ddl
Provides the SQL CREATE
, ALTER
, or DROP
statement that results in the schema change.
databaseName
The name of the database to which the DDL statements are applied. The value of databaseName
serves as the message key.
pos
The position in the binlog where the statements appear.
tableChanges
A structured representation of the entire table schema after the schema change. The tableChanges
field contains an array that includes entries for each column of the table. Because the structured representation presents data in JSON or Avro format, consumers can easily read messages without first processing them through a DDL parser.
For a table that is in capture mode, the connector not only stores the history of schema changes in the schema change topic, but also in an internal database schema history topic. The internal database schema history topic is for connector use only, and it is not intended for direct use by consuming applications. Ensure that applications that require notifications about schema changes consume that information only from the schema change topic. |
Never partition the database schema history topic. For the database schema history topic to function correctly, it must maintain a consistent, global order of the event records that the connector emits to it. To ensure that the topic is not split among partitions, set the partition count for the topic by using one of the following methods:
|
The format of the messages that a connector emits to its schema change topic is in an incubating state and is subject to change without notice. |
Example: Message emitted to the MySQL connector schema change topic
The following example shows a typical schema change message in JSON format. The message contains a logical representation of the table schema.
{
"schema": { },
"payload": {
"source": { (1)
"version": "2.7.2.Final",
"connector": "mysql",
"name": "mysql",
"ts_ms": 1651535750218, (2)
"ts_us": 1651535750218000, (2)
"ts_ns": 1651535750218000000, (2)
"snapshot": "false",
"db": "inventory",
"sequence": null,
"table": "customers",
"server_id": 223344,
"gtid": null,
"file": "mysql-bin.000003",
"pos": 570,
"row": 0,
"thread": null,
"query": null
},
"databaseName": "inventory", (3)
"schemaName": null,
"ddl": "ALTER TABLE customers ADD middle_name varchar(255) AFTER first_name", (4)
"tableChanges": [ (5)
{
"type": "ALTER", (6)
"id": "\"inventory\".\"customers\"", (7)
"table": { (8)
"defaultCharsetName": "utf8mb4",
"primaryKeyColumnNames": [ (9)
"id"
],
"columns": [ (10)
{
"name": "id",
"jdbcType": 4,
"nativeType": null,
"typeName": "INT",
"typeExpression": "INT",
"charsetName": null,
"length": null,
"scale": null,
"position": 1,
"optional": false,
"autoIncremented": true,
"generated": true
},
{
"name": "first_name",
"jdbcType": 12,
"nativeType": null,
"typeName": "VARCHAR",
"typeExpression": "VARCHAR",
"charsetName": "utf8mb4",
"length": 255,
"scale": null,
"position": 2,
"optional": false,
"autoIncremented": false,
"generated": false
},
{
"name": "middle_name",
"jdbcType": 12,
"nativeType": null,
"typeName": "VARCHAR",
"typeExpression": "VARCHAR",
"charsetName": "utf8mb4",
"length": 255,
"scale": null,
"position": 3,
"optional": true,
"autoIncremented": false,
"generated": false
},
{
"name": "last_name",
"jdbcType": 12,
"nativeType": null,
"typeName": "VARCHAR",
"typeExpression": "VARCHAR",
"charsetName": "utf8mb4",
"length": 255,
"scale": null,
"position": 4,
"optional": false,
"autoIncremented": false,
"generated": false
},
{
"name": "email",
"jdbcType": 12,
"nativeType": null,
"typeName": "VARCHAR",
"typeExpression": "VARCHAR",
"charsetName": "utf8mb4",
"length": 255,
"scale": null,
"position": 5,
"optional": false,
"autoIncremented": false,
"generated": false
}
],
"attributes": [ (11)
{
"customAttribute": "attributeValue"
}
]
}
}
]
}
}
Item | Field name | Description |
---|---|---|
1 |
| The |
2 |
| Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
3 |
| Identifies the database and the schema that contains the change. The value of the |
4 |
| This field contains the DDL that is responsible for the schema change. The |
5 |
| An array of one or more items that contain the schema changes generated by a DDL command. |
6 |
| Describes the kind of change. The value is one of the following:
|
7 |
| Full identifier of the table that was created, altered, or dropped. In the case of a table rename, this identifier is a concatenation of |
8 |
| Represents table metadata after the applied change. |
9 |
| List of columns that compose the table’s primary key. |
10 |
| Metadata for each column in the changed table. |
11 |
| Custom attribute metadata for each table change. |
For more information, see schema history topic.
Snapshots
When a Debezium MySQL connector is first started, it performs an initial consistent snapshot of your database. This snapshot enables the connector to establish a baseline for the current state of the database.
Debezium can use different modes when it runs a snapshot. The snapshot mode is determined by the snapshot.mode configuration property. The default value of the property is initial
. You can customize the way that the connector creates snapshots by changing the value of the snapshot.mode
property.
The connector completes a series of tasks when it performs the snapshot. The exact steps vary with the snapshot mode and with the table locking policy that is in effect for the database. The Debezium MySQL connector completes different steps when it performs an initial snapshot that uses a global read lock or table-level locks.
Initial snapshots that use a global read lock
You can customize the way that the connector creates snapshots by changing the value of the snapshot.mode
property. If you configure a different snapshot mode, the connector completes the snapshot by using a modified version of this workflow. For information about the snapshot process in environments that do not permit global read locks, see the snapshot workflow for table-level locks.
Default workflow that the Debezium MySQL connector uses to perform an initial snapshot with a global read lock
The following table shows the steps in the workflow that Debezium follows to create a snapshot with a global read lock.
Step | Action | ||
---|---|---|---|
1 | Establish a connection to the database. | ||
2 | Determine the tables to be captured. By default, the connector captures the data for all non-system tables. After the snapshot completes, the connector continues to stream data for the specified tables. If you want the connector to capture data only from specific tables you can direct the connector to capture the data for only a subset of tables or table elements by setting properties such as table.include.list or table.exclude.list. | ||
3 | Obtain a global read lock on the tables to be captured to block writes by other database clients. | ||
4 | Start a transaction with repeatable read semantics to ensure that all subsequent reads within the transaction are done against the consistent snapshot.
| ||
5 | Read the current binlog position. | ||
6 | Capture the structure of all tables in the database, or all tables that are designated for capture. The connector persists schema information in its internal database schema history topic, including all necessary
| ||
7 | Release the global read lock obtained in Step 3. Other database clients can now write to the database. | ||
8 | At the binlog position that the connector read in Step 5, the connector begins to scan the tables that are designated for capture. During the scan, the connector completes the following tasks:
| ||
9 | Commit the transaction. | ||
10 | Record the successful completion of the snapshot in the connector offsets. |
The resulting initial snapshot captures the current state of each row in the captured tables. From this baseline state, the connector captures subsequent changes as they occur.
After the snapshot process begins, if the process is interrupted due to connector failure, rebalancing, or other reasons, the process restarts after the connector restarts.
After the connector completes the initial snapshot, it continues streaming from the position that it read in Step 5 so that it does not miss any updates.
If the connector stops again for any reason, after it restarts, it resumes streaming changes from where it previously left off.
After the connector restarts, if the logs have been pruned, the connector’s position in the logs might no longer available. The connector then fails, and returns an error that indicates that a new snapshot is required. To configure the connector to automatically initiate a snapshot in this situation, set the value of the snapshot.mode property to when_needed
. For more tips on troubleshooting the Debezium MySQL connector, see behavior when things go wrong.
Initial snapshots that use table-level locks
In some database environments, administrators do not permit global read locks. If the Debezium MySQL connector detects that global read locks are not permitted, the connector uses table-level locks when it performs snapshots. For the connector to perform a snapshot that uses table-level locks, the database account that the Debezium connector uses to connect to MySQL must have LOCK TABLES
privileges.
Default workflow that the Debezium MySQL connector uses to perform an initial snapshot with table-level locks
The following table shows the steps in the workflow that Debezium follows to create a snapshot with table-level read locks. For information about the snapshot process in environments that do not permit global read locks, see the snapshot workflow for global read locks.
Step | Action | ||
---|---|---|---|
1 | Establish a connection to the database. | ||
2 | Determine the tables to be captured. By default, the connector captures all non-system tables. To have the connector capture a subset of tables or table elements, you can set a number of | ||
3 | Obtain table-level locks. | ||
4 | Start a transaction with repeatable read semantics to ensure that all subsequent reads within the transaction are done against the consistent snapshot. | ||
5 | Read the current binlog position. | ||
6 | Read the schema of the databases and tables for which the connector is configured to capture changes. The connector persists schema information in its internal database schema history topic, including all necessary
| ||
7 | At the binlog position that the connector read in Step 5, the connector begins to scan the tables that are designated for capture. During the scan, the connector completes the following tasks:
| ||
8 | Commit the transaction. | ||
9 | Release the table-level locks. Other database clients can now write to any previously locked tables. | ||
10 | Record the successful completion of the snapshot in the connector offsets. |
Setting | Description |
---|---|
| The connector performs a snapshot every time that it starts. The snapshot includes the structure and data of the captured tables. Specify this value to populate topics with a complete representation of the data from the captured tables every time that the connector starts. After the snapshot completes, the connector begins to stream event records for subsequent database changes. |
| The connector performs a database snapshot as described in the default workflow for creating an initial snapshot. After the snapshot completes, the connector begins to stream event records for subsequent database changes. |
| The connector performs a database snapshot. After the snapshot completes, the connector stops, and does not stream event records for subsequent database changes. |
| Deprecated, see |
| The connector captures the structure of all relevant tables, performing all the steps described in the default workflow for creating an initial snapshot, except that it does not create |
| When the connector starts, rather than performing a snapshot, it immediately begins to stream event records for subsequent database changes. This option is under consideration for future deprecation, in favor of the |
| Deprecated, see |
| Set this option to restore a database schema history topic that is lost or corrupted. After a restart, the connector runs a snapshot that rebuilds the topic from the source tables. You can also set the property to periodically prune a database schema history topic that experiences unexpected growth. |
| After the connector starts, it performs a snapshot only if it detects one of the following circumstances:
|
| Set the snapshot mode to |
| The |
For more information, see snapshot.mode in the table of connector configuration properties.
Understanding why initial snapshots capture the schema history for all tables
The initial snapshot that a connector runs captures two types of information:
Table data
Information about INSERT
, UPDATE
, and DELETE
operations in tables that are named in the connector’s table.include.list property.
Schema data
DDL statements that describe the structural changes that are applied to tables. Schema data is persisted to both the internal schema history topic, and to the connector’s schema change topic, if one is configured.
After you run an initial snapshot, you might notice that the snapshot captures schema information for tables that are not designated for capture. By default, initial snapshots are designed to capture schema information for every table that is present in the database, not only from tables that are designated for capture. Connectors require that the table’s schema is present in the schema history topic before they can capture a table. By enabling the initial snapshot to capture schema data for tables that are not part of the original capture set, Debezium prepares the connector to readily capture event data from these tables should that later become necessary. If the initial snapshot does not capture a table’s schema, you must add the schema to the history topic before the connector can capture data from the table.
In some cases, you might want to limit schema capture in the initial snapshot. This can be useful when you want to reduce the time required to complete a snapshot. Or when Debezium connects to the database instance through a user account that has access to multiple logical databases, but you want the connector to capture changes only from tables in a specific logic database.
Additional information
Capturing data from tables not captured by the initial snapshot (no schema change)
Capturing data from tables not captured by the initial snapshot (schema change)
Setting the schema.history.internal.store.only.captured.tables.ddl property to specify the tables from which to capture schema information.
Setting the schema.history.internal.store.only.captured.databases.ddl property to specify the logical databases from which to capture schema changes.
Capturing data from tables not captured by the initial snapshot (no schema change)
In some cases, you might want the connector to capture data from a table whose schema was not captured by the initial snapshot. Depending on the connector configuration, the initial snapshot might capture the table schema only for specific tables in the database. If the table schema is not present in the history topic, the connector fails to capture the table, and reports a missing schema error.
You might still be able to capture data from the table, but you must perform additional steps to add the table schema.
Prerequisites
You want to capture data from a table with a schema that the connector did not capture during the initial snapshot.
In the transaction log, all entries for the table use the same schema. For information about capturing data from a new table that has undergone structural changes, see Capturing data from tables not captured by the initial snapshot (schema change).
Procedure
Stop the connector.
Remove the internal database schema history topic that is specified by the schema.history.internal.kafka.topic property.
Apply the following changes to the connector configuration:
Set the snapshot.mode to
schema_only_recovery
.Set the value of schema.history.internal.store.only.captured.tables.ddl to
false
.Add the tables that you want the connector to capture to
table.include.list
. This guarantees that in the future, the connector can reconstruct the schema history for all tables.
Restart the connector. The snapshot recovery process rebuilds the schema history based on the current structure of the tables.
(Optional) After the snapshot completes, initiate an incremental snapshot to capture existing data for newly added tables along with changes to other tables that occurred while that connector was off-line.
(Optional) Reset the
snapshot.mode
back toschema_only
to prevent the connector from initiating recovery after a future restart.
Capturing data from tables not captured by the initial snapshot (schema change)
If a schema change is applied to a table, records that are committed before the schema change have different structures than those that were committed after the change. When Debezium captures data from a table, it reads the schema history to ensure that it applies the correct schema to each event. If the schema is not present in the schema history topic, the connector is unable to capture the table, and an error results.
If you want to capture data from a table that was not captured by the initial snapshot, and the schema of the table was modified, you must add the schema to the history topic, if it is not already available. You can add the schema by running a new schema snapshot, or by running an initial snapshot for the table.
Prerequisites
You want to capture data from a table with a schema that the connector did not capture during the initial snapshot.
A schema change was applied to the table so that the records to be captured do not have a uniform structure.
Procedure
Initial snapshot captured the schema for all tables (store.only.captured.tables.ddl
was set to false
)
Edit the table.include.list property to specify the tables that you want to capture.
Restart the connector.
Initiate an incremental snapshot if you want to capture existing data from the newly added tables.
Initial snapshot did not capture the schema for all tables (store.only.captured.tables.ddl
was set to true
)
If the initial snapshot did not save the schema of the table that you want to capture, complete one of the following procedures:
Procedure 1: Schema snapshot, followed by incremental snapshot
In this procedure, the connector first performs a schema snapshot. You can then initiate an incremental snapshot to enable the connector to synchronize data.
Stop the connector.
Remove the internal database schema history topic that is specified by the schema.history.internal.kafka.topic property.
Clear the offsets in the configured Kafka Connect offset.storage.topic. For more information about how to remove offsets, see the Debezium community FAQ.
Removing offsets should be performed only by advanced users who have experience in manipulating internal Kafka Connect data. This operation is potentially destructive, and should be performed only as a last resort.
Set values for properties in the connector configuration as described in the following steps:
Set the value of the snapshot.mode property to
schema_only
.Edit the table.include.list to add the tables that you want to capture.
Restart the connector.
Wait for Debezium to capture the schema of the new and existing tables. Data changes that occurred any tables after the connector stopped are not captured.
To ensure that no data is lost, initiate an incremental snapshot.
Procedure 2: Initial snapshot, followed by optional incremental snapshot
In this procedure the connector performs a full initial snapshot of the database. As with any initial snapshot, in a database with many large tables, running an initial snapshot can be a time-consuming operation. After the snapshot completes, you can optionally trigger an incremental snapshot to capture any changes that occur while the connector is off-line.
Stop the connector.
Remove the internal database schema history topic that is specified by the schema.history.internal.kafka.topic property.
Clear the offsets in the configured Kafka Connect offset.storage.topic. For more information about how to remove offsets, see the Debezium community FAQ.
Removing offsets should be performed only by advanced users who have experience in manipulating internal Kafka Connect data. This operation is potentially destructive, and should be performed only as a last resort.
Edit the table.include.list to add the tables that you want to capture.
Set values for properties in the connector configuration as described in the following steps:
Set the value of the snapshot.mode property to
initial
.(Optional) Set schema.history.internal.store.only.captured.tables.ddl to
false
.
Restart the connector. The connector takes a full database snapshot. After the snapshot completes, the connector transitions to streaming.
(Optional) To capture any data that changed while the connector was off-line, initiate an incremental snapshot.
Ad hoc snapshots
By default, a connector runs an initial snapshot operation only after it starts for the first time. Following this initial snapshot, under normal circumstances, the connector does not repeat the snapshot process. Any future change event data that the connector captures comes in through the streaming process only.
However, in some situations the data that the connector obtained during the initial snapshot might become stale, lost, or incomplete. To provide a mechanism for recapturing table data, Debezium includes an option to perform ad hoc snapshots. You might want to perform an ad hoc snapshot after any of the following changes occur in your Debezium environment:
The connector configuration is modified to capture a different set of tables.
Kafka topics are deleted and must be rebuilt.
Data corruption occurs due to a configuration error or some other problem.
You can re-run a snapshot for a table for which you previously captured a snapshot by initiating a so-called ad-hoc snapshot. Ad hoc snapshots require the use of signaling tables. You initiate an ad hoc snapshot by sending a signal request to the Debezium signaling table.
When you initiate an ad hoc snapshot of an existing table, the connector appends content to the topic that already exists for the table. If a previously existing topic was removed, Debezium can create a topic automatically if automatic topic creation is enabled.
Ad hoc snapshot signals specify the tables to include in the snapshot. The snapshot can capture the entire contents of the database, or capture only a subset of the tables in the database. Also, the snapshot can capture a subset of the contents of the table(s) in the database.
You specify the tables to capture by sending an execute-snapshot
message to the signaling table. Set the type of the execute-snapshot
signal to incremental
or blocking
, and provide the names of the tables to include in the snapshot, as described in the following table:
Field | Default | Value |
---|---|---|
|
| Specifies the type of snapshot that you want to run. |
| N/A | An array that contains regular expressions matching the fully-qualified names of the tables to include in the snapshot. |
| N/A | An optional array that specifies a set of additional conditions that the connector evaluates to determine the subset of records to include in a snapshot.
|
| N/A | An optional string that specifies the column name that the connector uses as the primary key of a table during the snapshot process. |
Triggering an ad hoc incremental snapshot
You initiate an ad hoc incremental snapshot by adding an entry with the execute-snapshot
signal type to the signaling table, or by sending a signal message to a Kafka signaling topic. After the connector processes the message, it begins the snapshot operation. The snapshot process reads the first and last primary key values and uses those values as the start and end point for each table. Based on the number of entries in the table, and the configured chunk size, Debezium divides the table into chunks, and proceeds to snapshot each chunk, in succession, one at a time.
For more information, see Incremental snapshots.
Triggering an ad hoc blocking snapshot
You initiate an ad hoc blocking snapshot by adding an entry with the execute-snapshot
signal type to the signaling table or signaling topic. After the connector processes the message, it begins the snapshot operation. The connector temporarily stops streaming, and then initiates a snapshot of the specified table, following the same process that it uses during an initial snapshot. After the snapshot completes, the connector resumes streaming.
For more information, see Blocking snapshots.
Incremental snapshots
To provide flexibility in managing snapshots, Debezium includes a supplementary snapshot mechanism, known as incremental snapshotting. Incremental snapshots rely on the Debezium mechanism for sending signals to a Debezium connector. Incremental snapshots are based on the DDD-3 design document.
In an incremental snapshot, instead of capturing the full state of a database all at once, as in an initial snapshot, Debezium captures each table in phases, in a series of configurable chunks. You can specify the tables that you want the snapshot to capture and the size of each chunk. The chunk size determines the number of rows that the snapshot collects during each fetch operation on the database. The default chunk size for incremental snapshots is 1024 rows.
As an incremental snapshot proceeds, Debezium uses watermarks to track its progress, maintaining a record of each table row that it captures. This phased approach to capturing data provides the following advantages over the standard initial snapshot process:
You can run incremental snapshots in parallel with streamed data capture, instead of postponing streaming until the snapshot completes. The connector continues to capture near real-time events from the change log throughout the snapshot process, and neither operation blocks the other.
If the progress of an incremental snapshot is interrupted, you can resume it without losing any data. After the process resumes, the snapshot begins at the point where it stopped, rather than recapturing the table from the beginning.
You can run an incremental snapshot on demand at any time, and repeat the process as needed to adapt to database updates. For example, you might re-run a snapshot after you modify the connector configuration to add a table to its table.include.list property.
Incremental snapshot process
When you run an incremental snapshot, Debezium sorts each table by primary key and then splits the table into chunks based on the configured chunk size. Working chunk by chunk, it then captures each table row in a chunk. For each row that it captures, the snapshot emits a READ
event. That event represents the value of the row when the snapshot for the chunk began.
As a snapshot proceeds, it’s likely that other processes continue to access the database, potentially modifying table records. To reflect such changes, INSERT
, UPDATE
, or DELETE
operations are committed to the transaction log as per usual. Similarly, the ongoing Debezium streaming process continues to detect these change events and emits corresponding change event records to Kafka.
How Debezium resolves collisions among records with the same primary key
In some cases, the UPDATE
or DELETE
events that the streaming process emits are received out of sequence. That is, the streaming process might emit an event that modifies a table row before the snapshot captures the chunk that contains the READ
event for that row. When the snapshot eventually emits the corresponding READ
event for the row, its value is already superseded. To ensure that incremental snapshot events that arrive out of sequence are processed in the correct logical order, Debezium employs a buffering scheme for resolving collisions. Only after collisions between the snapshot events and the streamed events are resolved does Debezium emit an event record to Kafka.
Snapshot window
To assist in resolving collisions between late-arriving READ
events and streamed events that modify the same table row, Debezium employs a so-called snapshot window. The snapshot window demarcates the interval during which an incremental snapshot captures data for a specified table chunk. Before the snapshot window for a chunk opens, Debezium follows its usual behavior and emits events from the transaction log directly downstream to the target Kafka topic. But from the moment that the snapshot for a particular chunk opens, until it closes, Debezium performs a de-duplication step to resolve collisions between events that have the same primary key..
For each data collection, the Debezium emits two types of events, and stores the records for them both in a single destination Kafka topic. The snapshot records that it captures directly from a table are emitted as READ
operations. Meanwhile, as users continue to update records in the data collection, and the transaction log is updated to reflect each commit, Debezium emits UPDATE
or DELETE
operations for each change.
As the snapshot window opens, and Debezium begins processing a snapshot chunk, it delivers snapshot records to a memory buffer. During the snapshot windows, the primary keys of the READ
events in the buffer are compared to the primary keys of the incoming streamed events. If no match is found, the streamed event record is sent directly to Kafka. If Debezium detects a match, it discards the buffered READ
event, and writes the streamed record to the destination topic, because the streamed event logically supersede the static snapshot event. After the snapshot window for the chunk closes, the buffer contains only READ
events for which no related transaction log events exist. Debezium emits these remaining READ
events to the table’s Kafka topic.
The connector repeats the process for each snapshot chunk.
Currently, you can use either of the following methods to initiate an incremental snapshot:
Triggering an incremental snapshot
To initiate an incremental snapshot, you can send an ad hoc snapshot signal to the signaling table on the source database. You submit snapshot signals as SQL INSERT
queries.
After Debezium detects the change in the signaling table, it reads the signal, and runs the requested snapshot operation.
The query that you submit specifies the tables to include in the snapshot, and, optionally, specifies the type of snapshot operation. Debezium currently supports the incremental
and blocking
snapshot types.
To specify the tables to include in the snapshot, provide a data-collections
array that lists the tables, or an array of regular expressions used to match tables, for example,
{"data-collections": ["public.MyFirstTable", "public.MySecondTable"]}
The data-collections
array for an incremental snapshot signal has no default value. If the data-collections
array is empty, Debezium interprets the empty array to mean that no action is required, and it does not perform a snapshot.
If the name of a table that you want to include in a snapshot contains a dot ( |
Prerequisites
-
A signaling data collection exists on the source database.
The signaling data collection is specified in the signal.data.collection property.
Using a source signaling channel to trigger an incremental snapshot
Send a SQL query to add the ad hoc incremental snapshot request to the signaling table:
INSERT INTO <signalTable> (id, type, data) VALUES ('<id>', '<snapshotType>', '{"data-collections": ["<fullyQualfiedTableName>","<fullyQualfiedTableName>"],"type":"<snapshotType>","additional-conditions":[{"data-collection": "<fullyQualfiedTableName>", "filter": "<additional-condition>"}]}');
For example,
INSERT INTO db1.debezium_signal (id, type, data) (1)
values ('ad-hoc-1', (2)
'execute-snapshot', (3)
'{"data-collections": ["db1.table1", "db1.table2"], (4)
"type":"incremental", (5)
"additional-conditions":[{"data-collection": "db1.table1" ,"filter":"color=\'blue\'"}]}'); (6)
+ The values of the id
,type
, and data
parameters in the command correspond to the fields of the signaling table.
+ The following table describes the parameters in the example:
+ .Descriptions of fields in a SQL command for sending an incremental snapshot signal to the signaling table
Item | Value | Description |
---|---|---|
1 |
| Specifies the fully-qualified name of the signaling table on the source database. |
2 |
| The |
3 |
| The |
4 |
| A required component of the |
5 |
| An optional |
6 |
| An optional array that specifies a set of additional conditions that the connector evaluates to determine the subset of records to include in a snapshot. |
Running an ad hoc incremental snapshots with additional-conditions
If you want a snapshot to include only a subset of the content in a table, you can modify the signal request by appending an additional-conditions
parameter to the snapshot signal.
The SQL query for a typical snapshot takes the following form:
SELECT * FROM <tableName> ....
By adding an additional-conditions
parameter, you append a WHERE
condition to the SQL query, as in the following example:
SELECT * FROM <data-collection> WHERE <filter> ....
The following example shows a SQL query to send an ad hoc incremental snapshot request with an additional condition to the signaling table:
INSERT INTO <signalTable> (id, type, data) VALUES ('<id>', '<snapshotType>', '{"data-collections": ["<fullyQualfiedTableName>","<fullyQualfiedTableName>"],"type":"<snapshotType>","additional-conditions":[{"data-collection": "<fullyQualfiedTableName>", "filter": "<additional-condition>"}]}');
For example, suppose you have a products
table that contains the following columns:
id
(primary key)color
quantity
If you want an incremental snapshot of the products
table to include only the data items where color=blue
, you can use the following SQL statement to trigger the snapshot:
INSERT INTO db1.debezium_signal (id, type, data) VALUES('ad-hoc-1', 'execute-snapshot', '{"data-collections": ["db1.products"],"type":"incremental", "additional-conditions":[{"data-collection": "db1.products", "filter": "color=blue"}]}');
The additional-conditions
parameter also enables you to pass conditions that are based on more than one column. For example, using the products
table from the previous example, you can submit a query that triggers an incremental snapshot that includes the data of only those items for which color=blue
and quantity>10
:
INSERT INTO db1.debezium_signal (id, type, data) VALUES('ad-hoc-1', 'execute-snapshot', '{"data-collections": ["db1.products"],"type":"incremental", "additional-conditions":[{"data-collection": "db1.products", "filter": "color=blue AND quantity>10"}]}');
The following example, shows the JSON for an incremental snapshot event that is captured by a connector.
Example 1. Incremental snapshot event message
{
"before":null,
"after": {
"pk":"1",
"value":"New data"
},
"source": {
...
"snapshot":"incremental" (1)
},
"op":"r", (2)
"ts_ms":"1620393591654",
"ts_us":"1620393591654547",
"ts_ns":"1620393591654547920",
"transaction":null
}
Item | Field name | Description |
---|---|---|
1 |
| Specifies the type of snapshot operation to run. |
2 |
| Specifies the event type. |
Using the Kafka signaling channel to trigger an incremental snapshot
You can send a message to the configured Kafka topic to request the connector to run an ad hoc incremental snapshot.
The key of the Kafka message must match the value of the topic.prefix
connector configuration option.
The value of the message is a JSON object with type
and data
fields.
The signal type is execute-snapshot
, and the data
field must have the following fields:
Field | Default | Value |
---|---|---|
|
| The type of the snapshot to be executed. Currently Debezium supports the |
| N/A | An array of comma-separated regular expressions that match the fully-qualified names of tables to include in the snapshot. |
| N/A | An optional array of additional conditions that specifies criteria that the connector evaluates to designate a subset of records to include in a snapshot. |
Example 2. An execute-snapshot
Kafka message
Key = `test_connector`
Value = `{"type":"execute-snapshot","data": {"data-collections": ["{collection-container}.table1", "{collection-container}.table2"], "type": "INCREMENTAL"}}`
Ad hoc incremental snapshots with additional-conditions
Debezium uses the additional-conditions
field to select a subset of a table’s content.
Typically, when Debezium runs a snapshot, it runs a SQL query such as:
SELECT * FROM _<tableName>_ ….
When the snapshot request includes an additional-conditions
property, the data-collection
and filter
parameters of the property are appended to the SQL query, for example:
SELECT * FROM _<data-collection>_ WHERE _<filter>_ ….
For example, given a products
table with the columns id
(primary key), color
, and brand
, if you want a snapshot to include only content for which color='blue'
, when you request the snapshot, you could add the additional-conditions
property to filter the content: :leveloffset: +1
Key = `test_connector`
Value = `{"type":"execute-snapshot","data": {"data-collections": ["db1.products"], "type": "INCREMENTAL", "additional-conditions": [{"data-collection": "db1.products" ,"filter":"color='blue'"}]}}`
You can also use the additional-conditions
property to pass conditions based on multiple columns. For example, using the same products
table as in the previous example, if you want a snapshot to include only the content from the products
table for which color='blue'
, and brand='MyBrand'
, you could send the following request: :leveloffset: +1
Key = `test_connector`
Value = `{"type":"execute-snapshot","data": {"data-collections": ["db1.products"], "type": "INCREMENTAL", "additional-conditions": [{"data-collection": "db1.products" ,"filter":"color='blue' AND brand='MyBrand'"}]}}`
Stopping an incremental snapshot
In some situations, it might be necessary to stop an incremental snapshot. For example, you might realize that snapshot was not configured correctly, or maybe you want to ensure that resources are available for other database operations. You can stop a snapshot that is already running by sending a signal to the signaling table on the source database.
You submit a stop snapshot signal to the signaling table by sending it in a SQL INSERT
query. The stop-snapshot signal specifies the type
of the snapshot operation as incremental
, and optionally specifies the tables that you want to omit from the currently running snapshot. After Debezium detects the change in the signaling table, it reads the signal, and stops the incremental snapshot operation if it’s in progress.
Additional resources
You can also stop an incremental snapshot by sending a JSON message to the Kafka signaling topic.
Prerequisites
-
A signaling data collection exists on the source database.
The signaling data collection is specified in the signal.data.collection property.
Using a source signaling channel to stop an incremental snapshot
Send a SQL query to stop the ad hoc incremental snapshot to the signaling table:
INSERT INTO <signalTable> (id, type, data) values ('<id>', 'stop-snapshot', '{"data-collections": ["<fullyQualfiedTableName>","<fullyQualfiedTableName>"],"type":"incremental"}');
For example,
INSERT INTO db1.debezium_signal (id, type, data) (1)
values ('ad-hoc-1', (2)
'stop-snapshot', (3)
'{"data-collections": ["db1.table1", "db1.table2"], (4)
"type":"incremental"}'); (5)
+ The values of the id
, type
, and data
parameters in the signal command correspond to the fields of the signaling table.
+ The following table describes the parameters in the example:
+ .Descriptions of fields in a SQL command for sending a stop incremental snapshot signal to the signaling table
Item | Value | Description |
---|---|---|
1 |
| Specifies the fully-qualified name of the signaling table on the source database. |
2 |
| The |
3 |
| Specifies |
4 |
| An optional component of the If you omit this component from the |
5 |
| A required component of the |
Using the Kafka signaling channel to stop an incremental snapshot
You can send a signal message to the configured Kafka signaling topic to stop an ad hoc incremental snapshot.
The key of the Kafka message must match the value of the topic.prefix
connector configuration option.
The value of the message is a JSON object with type
and data
fields.
The signal type is stop-snapshot
, and the data
field must have the following fields:
Field | Default | Value |
---|---|---|
|
| The type of the snapshot to be executed. Currently Debezium supports only the |
| N/A | An optional array of comma-separated regular expressions that match the fully-qualified names of the tables an array of table names or regular expressions to match table names to remove from the snapshot. |
The following example shows a typical stop-snapshot
Kafka message: :leveloffset: +1
Key = `test_connector`
Value = `{"type":"stop-snapshot","data": {"data-collections": ["db1.table1", "db1.table2"], "type": "INCREMENTAL"}}`
Read-only incremental snapshots
The Debezium MySQL connector allows for running incremental snapshots with a read-only connection to the database. To run an incremental snapshot with read-only access, the connector uses the executed global transaction IDs (GTID) set as high and low watermarks. The state of a chunk’s window is updated by comparing the GTIDs of binary log (binlog) events or the server’s heartbeats against low and high watermarks.
To switch to a read-only implementation, set the value of the read.only property to true
.
Prerequisites
If the connector reads from a multi-threaded replica (that is, a replica for which the value of
replica_parallel_workers
is greater than0
) you must set one of the following options:replica_preserve_commit_order=ON
slave_preserve_commit_order=ON
Ad hoc read-only incremental snapshots
When the MySQL connection is read-only, you can use any of the available signaling channels without the requirement to use the source
channel.
Operation type of snapshot events
The MySQL connector emits snapshot events as READ
operations ("op" : "r")
. If you prefer that the connector emits snapshot events as CREATE
(c
) events, configure the Debezium ReadToInsertEvent
single message transform (SMT) to modify the event type.
The following example shows how to configure the SMT:
Example: Using the ReadToInsertEvent
SMT to change the type of snapshot events
transforms=snapshotasinsert,...
transforms.snapshotasinsert.type=io.debezium.connector.mysql.transforms.ReadToInsertEvent
Custom snapshotter SPI
For more advanced uses, you can fine-tune control of the snapshot by implementing one of the following interfaces:
io.debezium.snapshot.spi.Snapshotter
Controls whether the connector takes a snapshot.
io.debezium.snapshot.spi.SnapshotQuery
Controls how data is queried during a snapshot.
io.debezium.snapshot.spi.SnapshotLock
Controls whether the connector locks tables when taking a snapshot.
io.debezium.snapshot.spi.Snapshotter interface. All built-in snapshot modes implement this interface.
/**
* {@link Snapshotter} is used to determine the following details about the snapshot process:
* <p>
* - Whether a snapshot occurs. <br>
* - Whether streaming continues during the snapshot. <br>
* - Whether the snapshot includes schema (if supported). <br>
* - Whether to snapshot data or schema following an error.
* <p>
* Although Debezium provides many default snapshot modes,
* to provide more advanced functionality, such as partial snapshots,
* you can customize implementation of the interface.
* For more information, see the documentation.
*
*
*
*/
@Incubating
public interface Snapshotter extends Configurable {
/**
* @return the name of the snapshotter.
*
*
*/
String name();
/**
* @param offsetExists is {@code true} when the connector has an offset context (i.e. restarted)
* @param snapshotInProgress is {@code true} when the connector is started, but a snapshot is already in progress
*
* @return {@code true} if the snapshotter should take a data snapshot
*/
boolean shouldSnapshotData(boolean offsetExists, boolean snapshotInProgress);
/**
* @param offsetExists is {@code true} when the connector has an offset context (i.e. restarted)
* @param snapshotInProgress is {@code true} when the connector is started, but a snapshot is already in progress
*
* @return {@code true} if the snapshotter should take a schema snapshot
*/
boolean shouldSnapshotSchema(boolean offsetExists, boolean snapshotInProgress);
/**
* @return {@code true} if the snapshotter should stream after taking a snapshot
*/
boolean shouldStream();
/**
* @return {@code true} whether the schema can be recovered if database schema history is corrupted.
*/
boolean shouldSnapshotOnSchemaError();
/**
* @return {@code true} whether the snapshot should be re-executed when there is a gap in data stream.
*/
boolean shouldSnapshotOnDataError();
/**
*
* @return {@code true} if streaming should resume from the start of the snapshot
* transaction, or {@code false} for when a connector resumes and takes a snapshot,
* streaming should resume from where streaming previously left off.
*/
default boolean shouldStreamEventsStartingFromSnapshot() {
return true;
}
/**
* Lifecycle hook called after the snapshot phase is successful.
*/
default void snapshotCompleted() {
// no operation
}
/**
* Lifecycle hook called after the snapshot phase is aborted.
*/
default void snapshotAborted() {
// no operation
}
}
io.debezium.snapshot.spi.SnapshotQuery interface. All built-in snapshot query modes implement this interface.
/**
* {@link SnapshotQuery} is used to determine the query used during a data snapshot
*
*
*/
public interface SnapshotQuery extends Configurable, Service {
/**
* @return the name of the snapshot lock.
*
*
*/
String name();
/**
* Generate a valid query string for the specified table, or an empty {@link Optional}
* to skip snapshotting this table (but that table will still be streamed from)
*
* @param tableId the table to generate a query for
* @param snapshotSelectColumns the columns to be used in the snapshot select based on the column
* include/exclude filters
* @return a valid query string, or none to skip snapshotting this table
*/
Optional<String> snapshotQuery(String tableId, List<String> snapshotSelectColumns);
}
io.debezium.snapshot.spi.SnapshotLock interface. All built-in snapshot lock modes implement this interface.
/**
* {@link SnapshotLock} is used to determine the table lock mode used during schema snapshot
*
*
*/
public interface SnapshotLock extends Configurable, Service {
/**
* @return the name of the snapshot lock.
*
*
*/
String name();
/**
* Returns a SQL statement for locking the given table during snapshotting, if required by the specific snapshotter
* implementation.
*/
Optional<String> tableLockingStatement(Duration lockTimeout, String tableId);
}
Blocking snapshots
To provide more flexibility in managing snapshots, Debezium includes a supplementary ad hoc snapshot mechanism, known as a blocking snapshot. Blocking snapshots rely on the Debezium mechanism for sending signals to a Debezium connector.
A blocking snapshot behaves just like an initial snapshot, except that you can trigger it at run time.
You might want to run a blocking snapshot rather than use the standard initial snapshot process in the following situations:
You add a new table and you want to complete the snapshot while the connector is running.
You add a large table, and you want the snapshot to complete in less time than is possible with an incremental snapshot.
Blocking snapshot process
When you run a blocking snapshot, Debezium stops streaming, and then initiates a snapshot of the specified table, following the same process that it uses during an initial snapshot. After the snapshot completes, the streaming is resumed.
Configure snapshot
You can set the following properties in the data
component of a signal:
data-collections: to specify which tables must be snapshot
additional-conditions: You can specify different filters for different table.
The
data-collection
property is the fully-qualified name of the table for which the filter will be applied.The
filter
property will have the same value used in thesnapshot.select.statement.overrides
For example:
{"type": "blocking", "data-collections": ["schema1.table1", "schema1.table2"], "additional-conditions": [{"data-collection": "schema1.table1", "filter": "SELECT * FROM [schema1].[table1] WHERE column1 = 0 ORDER BY column2 DESC"}, {"data-collection": "schema1.table2", "filter": "SELECT * FROM [schema1].[table2] WHERE column2 > 0"}]}
Possible duplicates
A delay might exist between the time that you send the signal to trigger the snapshot, and the time when streaming stops and the snapshot starts. As a result of this delay, after the snapshot completes, the connector might emit some event records that duplicate records captured by the snapshot.
Topic names
By default, the MySQL connector writes change events for all of the INSERT
, UPDATE
, and DELETE
operations that occur in a table to a single Apache Kafka topic that is specific to that table.
The connector uses the following convention to name change event topics:
topicPrefix.databaseName.tableName
Suppose that fulfillment
is the topic prefix, inventory
is the database name, and the database contains tables named orders
, customers
, and products
. The Debezium MySQL connector emits events to three Kafka topics, one for each table in the database:
fulfillment.inventory.orders
fulfillment.inventory.customers
fulfillment.inventory.products
The following list provides definitions for the components of the default name:
topicPrefix
The topic prefix as specified by the topic.prefix connector configuration property.
schemaName
The name of the schema in which the operation occurred.
tableName
The name of the table in which the operation occurred.
The connector applies similar naming conventions to label its internal database schema history topics, schema change topics, and transaction metadata topics.
If the default topic name do not meet your requirements, you can configure custom topic names. To configure custom topic names, you specify regular expressions in the logical topic routing SMT. For more information about using the logical topic routing SMT to customize topic naming, see Topic routing.
Transaction metadata
Debezium can generate events that represent transaction boundaries and that enrich data change event messages.
Limits on when Debezium receives transaction metadata Debezium registers and receives metadata only for transactions that occur after you deploy the connector. Metadata for transactions that occur before you deploy the connector is not available. |
Debezium generates transaction boundary events for the BEGIN
and END
delimiters in every transaction. Transaction boundary events contain the following fields:
status
BEGIN
or END
.
id
String representation of the unique transaction identifier.
ts_ms
The time of a transaction boundary event (BEGIN
or END
event) at the data source. If the data source does not provide Debezium with the event time, then the field instead represents the time at which Debezium processes the event.
event_count
(for END
events)
Total number of events emitted by the transaction.
data_collections
(for END
events)
An array of pairs of data_collection
and event_count
elements that indicates the number of events that the connector emits for changes that originate from a data collection.
Example
{
"status": "BEGIN",
"id": "0e4d5dcd-a33b-11ea-80f1-02010a22a99e:10",
"ts_ms": 1486500577125,
"event_count": null,
"data_collections": null
}
{
"status": "END",
"id": "0e4d5dcd-a33b-11ea-80f1-02010a22a99e:10",
"ts_ms": 1486500577691,
"event_count": 2,
"data_collections": [
{
"data_collection": "s1.a",
"event_count": 1
},
{
"data_collection": "s2.a",
"event_count": 1
}
]
}
Unless overridden via the topic.transaction option, the connector emits transaction events to the .transaction
topic.
Change data event enrichment
When transaction metadata is enabled the data message Envelope
is enriched with a new transaction
field. This field provides information about every event in the form of a composite of fields:
id
String representation of unique transaction identifier.
total_order
The absolute position of the event among all events generated by the transaction.
data_collection_order
The per-data collection position of the event among all events that were emitted by the transaction.
Following is an example of a message:
{
"before": null,
"after": {
"pk": "2",
"aa": "1"
},
"source": {
...
},
"op": "c",
"ts_ms": "1580390884335",
"ts_us": "1580390884335472",
"ts_ns": "1580390884335472987",
"transaction": {
"id": "0e4d5dcd-a33b-11ea-80f1-02010a22a99e:10",
"total_order": "1",
"data_collection_order": "1"
}
}
Data change events
The Debezium MySQL connector generates a data change event for each row-level INSERT
, UPDATE
, and DELETE
operation. Each event contains a key and a value. The structure of the key and the value depends on the table that was changed.
Debezium and Kafka Connect are designed around continuous streams of event messages. However, the structure of these events may change over time, which can be difficult for consumers to handle. To address this, each event contains the schema for its content or, if you are using a schema registry, a schema ID that a consumer can use to obtain the schema from the registry. This makes each event self-contained.
The following skeleton JSON shows the basic four parts of a change event. However, how you configure the Kafka Connect converter that you choose to use in your application determines the representation of these four parts in change events. A schema
field is in a change event only when you configure the converter to produce it. Likewise, the event key and event payload are in a change event only if you configure a converter to produce it. If you use the JSON converter and you configure it to produce all four basic change event parts, change events have this structure:
{
"schema": { (1)
...
},
"payload": { (2)
...
},
"schema": { (3)
...
},
"payload": { (4)
...
},
}
Item | Field name | Description |
---|---|---|
1 |
| The first |
2 |
| The first |
3 |
| The second |
4 |
| The second |
By default, the connector streams change event records to topics with names that are the same as the event’s originating table. See topic names.
The MySQL connector ensures that all Kafka Connect schema names adhere to the Avro schema name format. This means that the logical server name must start with a Latin letter or an underscore, that is, a-z, A-Z, or . Each remaining character in the logical server name and each character in the database and table names must be a Latin letter, a digit, or an underscore, that is, a-z, A-Z, 0-9, or . If there is an invalid character it is replaced with an underscore character. This can lead to unexpected conflicts if the logical server name, a database name, or a table name contains invalid characters, and the only characters that distinguish names from one another are invalid and thus replaced with underscores. |
Change event keys
A change event’s key contains the schema for the changed table’s key and the changed row’s actual key. Both the schema and its corresponding payload contain a field for each column in the changed table’s PRIMARY KEY
(or unique constraint) at the time the connector created the event.
Consider the following customers
table, which is followed by an example of a change event key for this table.
CREATE TABLE customers (
id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
first_name VARCHAR(255) NOT NULL,
last_name VARCHAR(255) NOT NULL,
email VARCHAR(255) NOT NULL UNIQUE KEY
) AUTO_INCREMENT=1001;
Every change event that captures a change to the customers
table has the same event key schema. For as long as the customers
table has the previous definition, every change event that captures a change to the customers
table has the following key structure. In JSON, it looks like this:
{
"schema": { (1)
"type": "struct",
"name": "mysql-server-1.inventory.customers.Key", (2)
"optional": false, (3)
"fields": [ (4)
{
"field": "id",
"type": "int32",
"optional": false
}
]
},
"payload": { (5)
"id": 1001
}
}
Item | Field name | Description |
---|---|---|
1 |
| The schema portion of the key specifies a Kafka Connect schema that describes what is in the key’s |
2 |
| Name of the schema that defines the structure of the key’s payload. This schema describes the structure of the primary key for the table that was changed. Key schema names have the format connector-name.database-name.table-name.
|
3 |
| Indicates whether the event key must contain a value in its |
4 |
| Specifies each field that is expected in the |
5 |
| Contains the key for the row for which this change event was generated. In this example, the key, contains a single |
Change event values
The value in a change event is a bit more complicated than the key. Like the key, the value has a schema
section and a payload
section. The schema
section contains the schema that describes the Envelope
structure of the payload
section, including its nested fields. Change events for operations that create, update or delete data all have a value payload with an envelope structure.
Consider the same sample table that was used to show an example of a change event key:
CREATE TABLE customers (
id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
first_name VARCHAR(255) NOT NULL,
last_name VARCHAR(255) NOT NULL,
email VARCHAR(255) NOT NULL UNIQUE KEY
) AUTO_INCREMENT=1001;
The value portion of a change event for a change to this table is described for:
create events
The following example shows the value portion of a change event that the connector generates for an operation that creates data in the customers
table:
{
"schema": { (1)
"type": "struct",
"fields": [
{
"type": "struct",
"fields": [
{
"type": "int32",
"optional": false,
"field": "id"
},
{
"type": "string",
"optional": false,
"field": "first_name"
},
{
"type": "string",
"optional": false,
"field": "last_name"
},
{
"type": "string",
"optional": false,
"field": "email"
}
],
"optional": true,
"name": "mysql-server-1.inventory.customers.Value", (2)
"field": "before"
},
{
"type": "struct",
"fields": [
{
"type": "int32",
"optional": false,
"field": "id"
},
{
"type": "string",
"optional": false,
"field": "first_name"
},
{
"type": "string",
"optional": false,
"field": "last_name"
},
{
"type": "string",
"optional": false,
"field": "email"
}
],
"optional": true,
"name": "mysql-server-1.inventory.customers.Value",
"field": "after"
},
{
"type": "struct",
"fields": [
{
"type": "string",
"optional": false,
"field": "version"
},
{
"type": "string",
"optional": false,
"field": "connector"
},
{
"type": "string",
"optional": false,
"field": "name"
},
{
"type": "int64",
"optional": false,
"field": "ts_ms"
},
{
"type": "int64",
"optional": false,
"field": "ts_us"
},
{
"type": "int64",
"optional": false,
"field": "ts_ns"
},
{
"type": "boolean",
"optional": true,
"default": false,
"field": "snapshot"
},
{
"type": "string",
"optional": false,
"field": "db"
},
{
"type": "string",
"optional": true,
"field": "table"
},
{
"type": "int64",
"optional": false,
"field": "server_id"
},
{
"type": "string",
"optional": true,
"field": "gtid"
},
{
"type": "string",
"optional": false,
"field": "file"
},
{
"type": "int64",
"optional": false,
"field": "pos"
},
{
"type": "int32",
"optional": false,
"field": "row"
},
{
"type": "int64",
"optional": true,
"field": "thread"
},
{
"type": "string",
"optional": true,
"field": "query"
}
],
"optional": false,
"name": "io.debezium.connector.mysql.Source", (3)
"field": "source"
},
{
"type": "string",
"optional": false,
"field": "op"
},
{
"type": "int64",
"optional": true,
"field": "ts_ms"
},
{
"type": "int64",
"optional": true,
"field": "ts_us"
},
{
"type": "int64",
"optional": true,
"field": "ts_ns"
}
],
"optional": false,
"name": "mysql-server-1.inventory.customers.Envelope" (4)
},
"payload": { (5)
"op": "c", (6)
"ts_ms": 1465491411815, (7)
"ts_us": 1465491411815437, (7)
"ts_ns": 1465491411815437158, (7)
"before": null, (8)
"after": { (9)
"id": 1004,
"first_name": "Anne",
"last_name": "Kretchmar",
"email": "annek@noanswer.org"
},
"source": { (10)
"version": "2.7.2.Final",
"connector": "mysql",
"name": "mysql-server-1",
"ts_ms": 0,
"ts_us": 0,
"ts_ns": 0,
"snapshot": false,
"db": "inventory",
"table": "customers",
"server_id": 0,
"gtid": null,
"file": "mysql-bin.000003",
"pos": 154,
"row": 0,
"thread": 7,
"query": "INSERT INTO customers (first_name, last_name, email) VALUES ('Anne', 'Kretchmar', 'annek@noanswer.org')"
}
}
}
Item | Field name | Description |
---|---|---|
1 |
| The value’s schema, which describes the structure of the value’s payload. A change event’s value schema is the same in every change event that the connector generates for a particular table. |
2 |
| In the |
3 |
|
|
4 |
|
|
5 |
| The value’s actual data. This is the information that the change event is providing. |
6 |
| Mandatory string that describes the type of operation that caused the connector to generate the event. In this example,
|
7 |
| Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
8 |
| An optional field that specifies the state of the row before the event occurred. When the |
9 |
| An optional field that specifies the state of the row after the event occurred. In this example, the |
10 |
| Mandatory field that describes the source metadata for the event. This field contains information that you can use to compare this event with other events, with regard to the origin of the events, the order in which the events occurred, and whether events were part of the same transaction. The source metadata includes:
|
update events
The value of a change event for an update in the sample customers
table has the same schema as a create event for that table. Likewise, the event value’s payload has the same structure. However, the event value payload contains different values in an update event. Here is an example of a change event value in an event that the connector generates for an update in the customers
table:
{
"schema": { ... },
"payload": {
"before": { (1)
"id": 1004,
"first_name": "Anne",
"last_name": "Kretchmar",
"email": "annek@noanswer.org"
},
"after": { (2)
"id": 1004,
"first_name": "Anne Marie",
"last_name": "Kretchmar",
"email": "annek@noanswer.org"
},
"source": { (3)
"version": "2.7.2.Final",
"name": "mysql-server-1",
"connector": "mysql",
"name": "mysql-server-1",
"ts_ms": 1465581029100,
"ts_ms": 1465581029100000,
"ts_ms": 1465581029100000000,
"snapshot": false,
"db": "inventory",
"table": "customers",
"server_id": 223344,
"gtid": null,
"file": "mysql-bin.000003",
"pos": 484,
"row": 0,
"thread": 7,
"query": "UPDATE customers SET first_name='Anne Marie' WHERE id=1004"
},
"op": "u", (4)
"ts_ms": 1465581029523, (5)
"ts_ms": 1465581029523758, (6)
"ts_ms": 1465581029523758914 (7)
}
}
Item | Field name | Description |
---|---|---|
1 |
| An optional field that specifies the state of the row before the event occurred. In an update event value, the |
2 |
| An optional field that specifies the state of the row after the event occurred. You can compare the |
3 |
| Mandatory field that describes the source metadata for the event. The
|
4 |
| Mandatory string that describes the type of operation. In an update event value, the |
5 |
| Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
6 |
| Optional field that displays the time at which the connector processed the event, in microseconds. The time is based on the system clock in the JVM running the Kafka Connect task. |
7 |
| Optional field that displays the time at which the connector processed the event, in nanoseconds. The time is based on the system clock in the JVM running the Kafka Connect task. |
Updating the columns for a row’s primary/unique key changes the value of the row’s key. When a key changes, Debezium outputs three events: a |
Primary key updates
An UPDATE
operation that changes a row’s primary key field(s) is known as a primary key change. For a primary key change, in place of an UPDATE
event record, the connector emits a DELETE
event record for the old key and a CREATE
event record for the new (updated) key. These events have the usual structure and content, and in addition, each one has a message header related to the primary key change:
The
DELETE
event record has__debezium.newkey
as a message header. The value of this header is the new primary key for the updated row.The
CREATE
event record has__debezium.oldkey
as a message header. The value of this header is the previous (old) primary key that the updated row had.
delete events
The value in a delete change event has the same schema
portion as create and update events for the same table. The payload
portion in a delete event for the sample customers
table looks like this:
{
"schema": { ... },
"payload": {
"before": { (1)
"id": 1004,
"first_name": "Anne Marie",
"last_name": "Kretchmar",
"email": "annek@noanswer.org"
},
"after": null, (2)
"source": { (3)
"version": "2.7.2.Final",
"connector": "mysql",
"name": "mysql-server-1",
"ts_ms": 1465581902300,
"ts_us": 1465581902300000,
"ts_ns": 1465581902300000000,
"snapshot": false,
"db": "inventory",
"table": "customers",
"server_id": 223344,
"gtid": null,
"file": "mysql-bin.000003",
"pos": 805,
"row": 0,
"thread": 7,
"query": "DELETE FROM customers WHERE id=1004"
},
"op": "d", (4)
"ts_ms": 1465581902461, (5)
"ts_us": 1465581902461842, (6)
"ts_ns": 1465581902461842579 (7)
}
}
Item | Field name | Description |
---|---|---|
1 |
| Optional field that specifies the state of the row before the event occurred. In a delete event value, the |
2 |
| Optional field that specifies the state of the row after the event occurred. In a delete event value, the |
3 |
| Mandatory field that describes the source metadata for the event. In a delete event value, the
|
4 |
| Mandatory string that describes the type of operation. The |
5 |
| Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
6 |
| Optional field that displays the time at which the connector processed the event, in microseconds. The time is based on the system clock in the JVM running the Kafka Connect task. |
7 |
| Optional field that displays the time at which the connector processed the event, in nanoseconds. The time is based on the system clock in the JVM running the Kafka Connect task. |
A delete change event record provides a consumer with the information it needs to process the removal of this row. The old values are included because some consumers might require them in order to properly handle the removal.
MySQL connector events are designed to work with Kafka log compaction. Log compaction enables removal of some older messages as long as at least the most recent message for every key is kept. This lets Kafka reclaim storage space while ensuring that the topic contains a complete data set and can be used for reloading key-based state.
Tombstone events
When a row is deleted, the delete event value still works with log compaction, because Kafka can remove all earlier messages that have that same key. However, for Kafka to remove all messages that have that same key, the message value must be null
. To make this possible, after the Debezium MySQL connector emits a delete event, the connector emits a special tombstone event that has the same key but a null
value.
truncate events
A truncate change event signals that a table has been truncated. The message key of a truncate event is null
. The message value resembles the following example:
{
"schema": { ... },
"payload": {
"source": { (1)
"version": "2.7.2.Final",
"name": "mysql-server-1",
"connector": "mysql",
"name": "mysql-server-1",
"ts_ms": 1465581029100,
"ts_us": 1465581029100000,
"ts_ns": 1465581029100000000,
"snapshot": false,
"db": "inventory",
"table": "customers",
"server_id": 223344,
"gtid": null,
"file": "mysql-bin.000003",
"pos": 484,
"row": 0,
"thread": 7,
"query": "UPDATE customers SET first_name='Anne Marie' WHERE id=1004"
},
"op": "t", (2)
"ts_ms": 1465581029523, (3)
"ts_us": 1465581029523468, (4)
"ts_ns": 1465581029523468471 (5)
}
}
Item | Field name | Description |
---|---|---|
1 |
| Mandatory field that describes the source metadata for the event. In a truncate event value, the
|
2 |
| Mandatory string that describes the type of operation. The |
3 |
| Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
4 |
| Optional field that displays the time at which the connector processed the event, in microseconds. The time is based on the system clock in the JVM running the Kafka Connect task. |
5 |
| Optional field that displays the time at which the connector processed the event, in nanoseconds. The time is based on the system clock in the JVM running the Kafka Connect task. |
In case a single TRUNCATE
statement applies to multiple tables, one truncate change event record for each truncated table will be emitted.
A truncate event represents a change that applies to an entire table, and it does not have a message key. In topics that span multiple partition, the order of change events that apply to an entire table is is not guaranteed. That is, there is no ordering guarantee for (create, update, etc.), or for the truncate events for that table. When a consumer reads events from different partition, it might read an update event for a table from one partition only after it reads a truncate event for the same table from a second partition. |
Data type mappings
The Debezium MySQL connector represents changes to rows with events that are structured like the table in which the row exists. The event contains a field for each column value. The MySQL data type of that column dictates how Debezium represents the value in the event.
Columns that store strings are defined in MySQL with a character set and collation. The MySQL connector uses the column’s character set when reading the binary representation of the column values in the binlog events.
The connector can map MySQL data types to both literal and semantic types.
Literal type: how the value is represented using Kafka Connect schema types.
Semantic type: how the Kafka Connect schema captures the meaning of the field (schema name).
If the default data type conversions do not meet your needs, you can create a custom converter for the connector.
Basic types
The following table shows how the connector maps basic MySQL data types.
MySQL type | Literal type | Semantic type |
---|---|---|
|
| n/a |
|
| n/a |
|
|
|
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| The precision is used only to determine storage size. A precision |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
.
Temporal types
Excluding the TIMESTAMP
data type, MySQL temporal types depend on the value of the time.precision.mode
connector configuration property. For TIMESTAMP
columns whose default value is specified as CURRENT_TIMESTAMP
or NOW
, the value 1970-01-01 00:00:00
is used as the default value in the Kafka Connect schema.
MySQL allows zero-values for DATE
, DATETIME
, and TIMESTAMP
columns because zero-values are sometimes preferred over null values. The MySQL connector represents zero-values as null values when the column definition allows null values, or as the epoch day when the column does not allow null values.
Temporal values without time zones
The DATETIME
type represents a local date and time such as “2018-01-13 09:48:27”. As you can see, there is no time zone information. Such columns are converted into epoch milliseconds or microseconds based on the column’s precision by using UTC. The TIMESTAMP
type represents a timestamp without time zone information. It is converted by MySQL from the server (or session’s) current time zone into UTC when writing and from UTC into the server (or session’s) current time zone when reading back the value. For example:
DATETIME
with a value of2018-06-20 06:37:03
becomes1529476623000
.TIMESTAMP
with a value of2018-06-20 06:37:03
becomes2018-06-20T13:37:03Z
.
Such columns are converted into an equivalent io.debezium.time.ZonedTimestamp
in UTC based on the server (or session’s) current time zone. The time zone will be queried from the server by default.
The time zone of the JVM running Kafka Connect and Debezium does not affect these conversions.
More details about properties related to temporal values are in the documentation for MySQL connector configuration properties.
time.precision.mode=adaptive_time_microseconds(default)
The MySQL connector determines the literal type and semantic type based on the column’s data type definition so that events represent exactly the values in the database. All time fields are in microseconds. Only positive TIME
field values in the range of 00:00:00.000000
to 23:59:59.999999
can be captured correctly.
MySQL type | Literal type | Semantic type |
---|---|---|
|
|
|
|
|
|
|
|
|
|
|
|
time.precision.mode=connect
The MySQL connector uses defined Kafka Connect logical types. This approach is less precise than the default approach and the events could be less precise if the database column has a fractional second precision value of greater than 3
. Values in only the range of 00:00:00.000
to 23:59:59.999
can be handled. Set time.precision.mode=connect
only if you can ensure that the TIME
values in your tables never exceed the supported ranges. The connect
setting is expected to be removed in a future version of Debezium.
MySQL type | Literal type | Semantic type |
---|---|---|
|
|
|
|
|
|
|
|
|
Decimal types
Debezium connectors handle decimals according to the setting of the decimal.handling.mode connector configuration property.
decimal.handling.mode=precise
MySQL type | Literal type | Semantic type |
---|---|---|
|
|
|
|
|
|
decimal.handling.mode=double
MySQL type | Literal type | Semantic type |
---|---|---|
|
| n/a |
|
| n/a |
decimal.handling.mode=string
MySQL type | Literal type | Semantic type |
---|---|---|
|
| n/a |
|
| n/a |
Boolean values
MySQL handles the BOOLEAN
value internally in a specific way. The BOOLEAN
column is internally mapped to the TINYINT(1)
data type. When the table is created during streaming then it uses proper BOOLEAN
mapping as Debezium receives the original DDL. During snapshots, Debezium executes SHOW CREATE TABLE
to obtain table definitions that return TINYINT(1)
for both BOOLEAN
and TINYINT(1)
columns. Debezium then has no way to obtain the original type mapping and so maps to TINYINT(1)
.
To enable you to convert source columns to Boolean data types, Debezium provides a TinyIntOneToBooleanConverter
custom converter that you can use in one of the following ways:
Map all
TINYINT(1)
orTINYINT(1) UNSIGNED
columns toBOOLEAN
types.Enumerate a subset of columns by using a comma-separated list of regular expressions.
To use this type of conversion, you must set the converters configuration property with theselector
parameter, as shown in the following example:converters=boolean
boolean.type=io.debezium.connector.binlog.converters.TinyIntOneToBooleanConverter
boolean.selector=db1.table1.*, db1.table2.column1
NOTE: In some cases, the database may not show the length of
tinyint unsigned
when the snapshot executesSHOW CREATE TABLE
, which means this converter doesn’t work. The new optionlength.checker
can solve this issue, the default value istrue
. Disable thelength.checker
and specify the columns that need to be converted toselected
property instead of converting all columns based on type, as shown in the following example:converters=boolean
boolean.type=io.debezium.connector.binlog.converters.TinyIntOneToBooleanConverter
boolean.length.checker=false
boolean.selector=db1.table1.*, db1.table2.column1
Spatial types
Currently, the Debezium MySQL connector supports the following spatial data types.
MySQL type | Literal type | Semantic type |
---|---|---|
|
|
|
Custom converters
By default, the Debezium MySQL connector provides several CustomConverter
implementations for MySQL data types. These custom converters provide alternative mappings for specific data types based on the connector configuration. To add a CustomConverter
to the connector, follow the instructions in the Custom Converters documentation.
TINYINT(1)
to Boolean
By default, during a connector snapshot, the Debezium MySQL connector obtains column types from the JDBC driver, which assigns the TINYINT(1)
type to BOOLEAN
columns. Debezium then uses these JDBC column types to define the schema for the snapshot events. After the connector transitions from the snapshot to the streaming phase, the change event schema that results from the default mapping can lead to inconsistent mappings for BOOLEAN
columns. To help ensure that MySQL emits BOOLEAN
columns uniformly, you can apply the custom TinyIntOneToBooleanConverter
, as shown in the following configuration example.
Example: TinyIntOneToBooleanConverter
configuration
converters=tinyint-one-to-boolean
tinyint-one-to-boolean.type=io.debezium.connector.binlog.converters.TinyIntOneToBooleanConverter
tinyint-one-to-boolean.selector=.*.MY_TABLE.DATA
tinyint-one-to-boolean.length.checker=false
In the preceding example, the selector
and length.checker
properties are optional. By default, the converter checks that TINYINT
data types conform to a length of 1
. If length.checker
to false
, the converter does not explicitly confirm that the TINYINT
data type conforms to a length of 1
. The selector
designates the tables or columns to convert, based on the supplied regular expression. If you omit the selector
property, the converter maps all TINYINT
columns to logical BOOL
field types. If you do not configure a selector
option, and you want to map TINYINT
columns to TINYINT(1)
, omit the length.checker
property, or set its value to true
.
JDBC sink data types
If you integrate the Debezium JDBC sink connector with a Debezium MySQL source connector, the MySQL connector emits some column attributes differently during the snapshot and streaming phases. For the JDBC sink connector to consistently consume changes from both the snapshot and streaming phase, you must include the JdbcSinkDataTypesConverter
converter as part of the MySQL source connector configuration, as shown in the following example:
Example: JdbcSinkDataTypesConverter
configuration
converters=jdbc-sink
jdbc-sink.type=io.debezium.connector.binlog.converters.JdbcSinkDataTypesConverter
jdbc-sink.selector.boolean=.*.MY_TABLE.BOOL_COL
jdbc-sink.selector.real=.*.MY_TABLE.REAL_COL
jdbc-sink.selector.string=.*.MY_TABLE.STRING_COL
jdbc-sink.treat.real.as.double=true
In the preceding example, the selector.*
and treat.real.as.double
configuration properties are optional.
The selector.*
properties specify comma-separated lists of regular expressions that specify which tables and columns that the converter applies to. By default, the converter applies the following rules apply to all Boolean, real, and string-based column data types, across all tables:
BOOLEAN
data types are always emitted asINT16
logical types, with1
representingtrue
and0
representingfalse
REAL
data types are always emitted asFLOAT64
logical types.String-based columns always include the
__debezium.source.column.character_set
schema parameter that contains the column’s character set.
For each data type, you can configure a selector rule to override the default scope and apply the selector to specific tables and columns only. For example, to set the scope of the Boolean converter, add the following rule to the connector configuration, as in the preceding example: converters.jdbc-sink.selector.boolean=.*.MY_TABLE.BOOL_COL
Setting up MySQL
Some MySQL setup tasks are required before you can install and run a Debezium connector.
Creating a user
A Debezium MySQL connector requires a MySQL user account. This MySQL user must have appropriate permissions on all databases for which the Debezium MySQL connector captures changes.
Prerequisites
A MySQL server.
Basic knowledge of SQL commands.
Procedure
Create the MySQL user:
mysql> CREATE USER 'user'@'localhost' IDENTIFIED BY 'password';
Grant the required permissions to the user:
mysql> GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO 'user' IDENTIFIED BY 'password';
For a description of the required permissions, see Descriptions of user permissions.
If using a hosted option such as Amazon RDS or Amazon Aurora that does not allow a global read lock, table-level locks are used to create the consistent snapshot. In this case, you need to also grant LOCK TABLES
permissions to the user that you create. See snapshots for more details.Finalize the user’s permissions:
mysql> FLUSH PRIVILEGES;
Table 20. Descriptions of user permissions Keyword Description SELECT
Enables the connector to select rows from tables in databases. This is used only when performing a snapshot.
RELOAD
Enables the connector the use of the
FLUSH
statement to clear or reload internal caches, flush tables, or acquire locks. This is used only when performing a snapshot.SHOW DATABASES
Enables the connector to see database names by issuing the
SHOW DATABASE
statement. This is used only when performing a snapshot.REPLICATION SLAVE
Enables the connector to connect to and read the MySQL server binlog.
REPLICATION CLIENT
Enables the connector the use of the following statements:
SHOW MASTER STATUS
SHOW SLAVE STATUS
SHOW BINARY LOGS
The connector always requires this.
ON
Identifies the database to which the permissions apply.
TO ‘user’
Specifies the user to grant the permissions to.
IDENTIFIED BY ‘password’
Specifies the user’s MySQL password.
Enabling the binlog
You must enable binary logging for MySQL replication. The binary logs record transaction updates in a way that enables replicas to propagate those changes.
Prerequisites
A MySQL server.
Appropriate MySQL user privileges.
Procedure
Check whether the
log-bin
option is enabled:// for MySQL 5.x
mysql> SELECT variable_value as "BINARY LOGGING STATUS (log-bin) ::"
FROM information_schema.global_variables WHERE variable_name='log_bin';
// for MySQL 8.x
mysql> SELECT variable_value as "BINARY LOGGING STATUS (log-bin) ::"
FROM performance_schema.global_variables WHERE variable_name='log_bin';
If the binlog is
OFF
, add the properties in the following table to the configuration file for the MySQL server:server-id = 223344 # Querying variable is called server_id, e.g. SELECT variable_value FROM information_schema.global_variables WHERE variable_name='server_id';
log_bin = mysql-bin
binlog_format = ROW
binlog_row_image = FULL
binlog_expire_logs_seconds = 864000
Confirm your changes by checking the binlog status once more:
// for MySQL 5.x
{context}> SELECT variable_value as "BINARY LOGGING STATUS (log-bin) ::"
FROM information_schema.global_variables WHERE variable_name='log_bin';
// for MySQL 8.x
{context}> SELECT variable_value as "BINARY LOGGING STATUS (log-bin) ::"
FROM performance_schema.global_variables WHERE variable_name='log_bin';
If you run MySQL on Amazon RDS, you must enable automated backups for your database instance for binary logging to occur. If the database instance is not configured to perform automated backups, the binlog is disabled, even if you apply the settings described in the previous steps.
Table 21. Descriptions of MySQL binlog configuration properties Property Description server-id
The value for the
server-id
must be unique for each server and replication client in the MySQL cluster.log_bin
The value of
log_bin
is the base name of the sequence of binlog files.binlog_format
The
binlog-format
must be set toROW
orrow
.binlog_row_image
The
binlog_row_image
must be set toFULL
orfull
.binlog_expire_logs_seconds
The
binlog_expire_logs_seconds
corresponds to deprecated system variableexpire_logs_days
. This is the number of seconds for automatic binlog file removal. The default value is2592000
, which equals 30 days. Set the value to match the needs of your environment. For more information, see MySQL purges binlog files.
Enabling GTIDs
Global transaction identifiers (GTIDs) uniquely identify transactions that occur on a server within a cluster. Though not required for a Debezium MySQL connector, using GTIDs simplifies replication and enables you to more easily confirm if primary and replica servers are consistent.
GTIDs are available in MySQL 5.6.5 and later. See the MySQL documentation for more details.
Prerequisites
A MySQL server.
Basic knowledge of SQL commands.
Access to the MySQL configuration file.
Procedure
Enable
gtid_mode
:mysql> gtid_mode=ON
Enable
enforce_gtid_consistency
:mysql> enforce_gtid_consistency=ON
Confirm the changes:
mysql> show global variables like '%GTID%';
Result
+--------------------------+-------+
| Variable_name | Value |
+--------------------------+-------+
| enforce_gtid_consistency | ON |
| gtid_mode | ON |
+--------------------------+-------+
Table 22. Descriptions of GTID options Option Description gtid_mode
Boolean that specifies whether GTID mode of the MySQL server is enabled or not.
ON
= enabledOFF
= disabled
enforce_gtid_consistency
Boolean that specifies whether the server enforces GTID consistency by allowing the execution of statements that can be logged in a transactionally safe manner. Required when using GTIDs.
ON
= enabledOFF
= disabled
Configuring session timeouts
When an initial consistent snapshot is made for large databases, your established connection could timeout while the tables are being read. You can prevent this behavior by configuring interactive_timeout
and wait_timeout
in your MySQL configuration file.
Prerequisites
A MySQL server.
Basic knowledge of SQL commands.
Access to the MySQL configuration file.
Procedure
Configure
interactive_timeout
:mysql> interactive_timeout=<duration-in-seconds>
Configure
wait_timeout
:mysql> wait_timeout=<duration-in-seconds>
Table 23. Descriptions of MySQL session timeout options Option Description interactive_timeout
The number of seconds the server waits for activity on an interactive connection before closing it. For more information see the: MySQL documentation.
wait_timeout
The number of seconds that the server waits for activity on a non-interactive connection before closing it. For more information, see the MySQL documentation.
Enabling query log events
You might want to see the original SQL
statement for each binlog event. Enabling the binlog_rows_query_log_events
option in the MySQL configuration file allows you to do this.
This option is available in MySQL 5.6 and later.
Prerequisites
A MySQL server.
Basic knowledge of SQL commands.
Access to the MySQL configuration file.
Procedure
Enable
binlog_rows_query_log_events
in MySQL:mysql> binlog_rows_query_log_events=ON
binlog_rows_query_log_events
is set to a value that enables/disables support for including the originalSQL
statement in the binlog entry.ON
= enabledOFF
= disabled
Validating binlog row value options
Verify the setting of the binlog_row_value_options
variable in the database. To enable the connector to consume UPDATE events, this variable must be set to a value other than PARTIAL_JSON
.
Prerequisites
A MySQL server.
Basic knowledge of SQL commands.
Access to the MySQL configuration file.
Procedure
Check current variable value
mysql> show global variables where variable_name = 'binlog_row_value_options';
Result
+--------------------------+-------+
| Variable_name | Value |
+--------------------------+-------+
| binlog_row_value_options | |
+--------------------------+-------+
If the value of the variable is set to
PARTIAL_JSON
, run the following command to unset it:mysql> set @@global.binlog_row_value_options="" ;
Deployment
To deploy a Debezium MySQL connector, you install the Debezium MySQL connector archive, configure the connector, and start the connector by adding its configuration to Kafka Connect.
Prerequisites
Apache Zookeeper, Apache Kafka, and Kafka Connect are installed.
MySQL Server is installed and is set up to work with the Debezium connector.
Procedure
Download the Debezium MySQL connector plug-in.
Extract the files into your Kafka Connect environment.
Add the directory with the JAR files to Kafka Connect’s plugin.path.
Configure the connector and add the configuration to your Kafka Connect cluster.
Restart your Kafka Connect process to pick up the new JAR files.
If you are working with immutable containers, see Debezium’s Container images for Apache Zookeeper, Apache Kafka, MySQL, and Kafka Connect with the MySQL connector already installed and ready to run.
You can also run Debezium on Kubernetes and OpenShift.
MySQL connector configuration example
Following is an example of the configuration for a connector instance that captures data from a MySQL server on port 3306 at 192.168.99.100, which we logically name fullfillment
. Typically, you configure the Debezium MySQL connector in a JSON file by setting the configuration properties that are available for the connector.
You can choose to produce events for a subset of the schemas and tables in a database. Optionally, you can ignore, mask, or truncate columns that contain sensitive data, that are larger than a specified size, or that you do not need.
{
"name": "inventory-connector", (1)
"config": {
"connector.class": "io.debezium.connector.{context}.{connector-name}Connector", (2)
"database.hostname": "192.168.99.100", (3)
"database.port": "3306", (4)
"database.user": "debezium-user", (5)
"database.password": "debezium-user-pw", (6)
"database.server.id": "184054", (7)
"topic.prefix": "fullfillment", (8)
"database.include.list": "inventory", (9)
"schema.history.internal.kafka.bootstrap.servers": "kafka:9092", (10)
"schema.history.internal.kafka.topic": "schemahistory.fullfillment", (11)
"include.schema.changes": "true" (12)
}
}
1 | Connector’s name when registered with the Kafka Connect service. |
2 | Connector’s class name. |
3 | MySQL server address. |
4 | MySQL server port number. |
5 | MySQL user with the appropriate privileges. |
6 | MySQL user’s password. |
7 | Unique ID of the connector. |
8 | Topic prefix for the MySQL server or cluster. |
9 | List of databases hosted by the specified server. |
10 | List of Kafka brokers that the connector uses to write and recover DDL statements to the database schema history topic. |
11 | Name of the database schema history topic. This topic is for internal use only and should not be used by consumers. |
12 | Flag that specifies if the connector should generate events for DDL changes and emit them to the fulfillment schema change topic for use by consumers. |
For the complete list of the configuration properties that you can set for the Debezium MySQL connector, see MySQL connector configuration properties.
You can send this configuration with a POST
command to a running Kafka Connect service. The service records the configuration and starts one connector task that performs the following actions:
Connects to the MySQL database.
Reads change-data tables for tables in capture mode.
Streams change event records to Kafka topics.
Adding connector configuration
To start running a MySQL connector, configure a connector configuration, and add the configuration to your Kafka Connect cluster.
Prerequisites
The Debezium MySQL connector is installed.
Procedure
Create a configuration for the MySQL connector.
Use the Kafka Connect REST API to add that connector configuration to your Kafka Connect cluster.
Results
After the connector starts, it performs a consistent snapshot of the MySQL databases that the connector is configured for. The connector then starts generating data change events for row-level operations and streaming change event records to Kafka topics.
Connector properties
The Debezium MySQL connector has numerous configuration properties that you can use to achieve the right connector behavior for your application. Many properties have default values. Information about the properties is organized as follows:
Database schema history connector configuration properties that control how Debezium processes events that it reads from the database schema history topic.
Required Debezium MySQL connector configuration properties
The following configuration properties are required unless a default value is available.
Default value: long
Specifies how the connector represents BIGINT UNSIGNED columns in change events. Set one of the following options:
long
Uses Java
long
data types to represent BIGINT UNSIGNED column values. Although thelong
type does not offer the greatest precision, it is easy implement in most consumers. In most environments, this is the preferred setting.precise
Uses
java.math.BigDecimal
data types to represent values. The connector uses the Kafka Connectorg.apache.kafka.connect.data.Decimal
data type to represent values in encoded binary format. Set this option if the connector typically works with values larger than 2^63. Thelong
data type cannot convey values of that size.
Default value: bytes
Specifies how the connector represents values for binary columns, such as, blob
, binary
, varbinary
, in change events.
Set one of the following options:
bytes
Represents binary data as a byte array.
base64
Represents binary data as a base64-encoded String.
base64-url-safe
Represents binary data as a base64-url-safe-encoded String.
hex
Represents binary data as a hex-encoded (base16) String.
Default value: empty string
An optional, comma-separated list of regular expressions that match the fully-qualified names of columns to exclude from change event record values. Other columns in the source record are captured as usual. Fully-qualified names for columns are of the form databaseName.tableName.columnName.
To match the name of a column, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the column; it does not match substrings that might be present in a column name. If you include this property in the configuration, do not also set the column.include.list
property.
Default value: empty string
An optional, comma-separated list of regular expressions that match the fully-qualified names of columns to include in change event record values. Other columns are omitted from the event record. Fully-qualified names for columns are of the form databaseName.tableName.columnName.
To match the name of a column, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the column; it does not match substrings that might be present in a column name.
If you include this property in the configuration, do not set the column.exclude.list
property.
column.mask.hash.hashAlgorithm.with.salt.salt
column.mask.hash.v2.hashAlgorithm.with.salt.salt
Default value: No default
An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns. Fully-qualified names for columns are of the form _<databaseName>_._<tableName>_._<columnName>_
.
To match the name of a column Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the column; the expression does not match substrings that might be present in a column name. In the resulting change event record, the values for the specified columns are replaced with pseudonyms.
A pseudonym consists of the hashed value that results from applying the specified hashAlgorithm and salt. Based on the hash function that is used, referential integrity is maintained, while column values are replaced with pseudonyms. Supported hash functions are described in the MessageDigest section of the Java Cryptography Architecture Standard Algorithm Name Documentation.
In the following example, CzQMA0cB5K
is a randomly selected salt.
column.mask.hash.SHA-256.with.salt.CzQMA0cB5K = inventory.orders.customerName, inventory.shipment.customerName
If necessary, the pseudonym is automatically shortened to the length of the column. The connector configuration can include multiple properties that specify different hash algorithms and salts.
Depending on the hashAlgorithm used, the salt selected, and the actual data set, the resulting data set might not be completely masked.
Hashing strategy version 2 ensures fidelity of values that are hashed in different places or systems.
Default value: No default
An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns. Set this property if you want the connector to mask the values for a set of columns, for example, if they contain sensitive data. Set _length_
to a positive integer to replace data in the specified columns with the number of asterisk (*
) characters specified by the length in the property name. Set length to 0
(zero) to replace data in the specified columns with an empty string.
The fully-qualified name of a column observes the following format: databaseName.tableName.columnName. To match the name of a column, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the column; the expression does not match substrings that might be present in a column name.
You can specify multiple properties with different lengths in a single configuration.
Default value: No default
An optional, comma-separated list of regular expressions that match the fully-qualified names of columns for which you want the connector to emit extra parameters that represent column metadata. When this property is set, the connector adds the following fields to the schema of event records:
__debezium.source.column.type
__debezium.source.column.length
__debezium.source.column.scale
These parameters propagate a column’s original type name and length (for variable-width types), respectively.
Enabling the connector to emit this extra data can assist in properly sizing specific numeric or character-based columns in sink databases.The fully-qualified name of a column observes one of the following formats: databaseName.tableName.columnName, or databaseName.schemaName.tableName.columnName.
To match the name of a column, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the column; the expression does not match substrings that might be present in a column name.
column.truncate.to.length.chars
Default value: No default
An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns. Set this property if you want to truncate the data in a set of columns when it exceeds the number of characters specified by the length in the property name. Set length
to a positive integer value, for example, column.truncate.to.20.chars
.
The fully-qualified name of a column observes the following format: databaseName.tableName.columnName. To match the name of a column, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the column; the expression does not match substrings that might be present in a column name.
You can specify multiple properties with different lengths in a single configuration.
Default value: 30000
(30 seconds)
A positive integer value that specifies the maximum time in milliseconds that the connector waits to establish a connection to the MySQL database server before the connection request times out.
Default value: No default
The name of the Java class for the connector. Always specify io.debezium.connector.mysql.MySqlConnector
for the MySQL connector.
Default value: empty string
An optional, comma-separated list of regular expressions that match the names of databases from which you do not want the connector to capture changes. The connector captures changes in any database that is not named in the database.exclude.list
.
To match the name of a database, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the database; it does not match substrings that might be present in a database name.
If you include this property in the configuration, do not also set the database.include.list
property.
Default value: No default
The IP address or hostname of the MySQL database server.
Default value: empty string
An optional, comma-separated list of regular expressions that match the names of the databases from which the connector captures changes. The connector does not capture changes in any database whose name is not in database.include.list
. By default, the connector captures changes in all databases.
To match the name of a database, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the database; it does not match substrings that might be present in a database name.
If you include this property in the configuration, do not also set the database.exclude.list
property.
Default value: com.mysql.cj.jdbc.Driver
Specifies the name of the driver class that the connector uses. You can use this setting to specify a driver other than the one that is packaged with the connector.
Default value: No default
The password of the MySQL user that the connector uses to connect to the MySQL database server.
Default value: 3306
Integer port number of the MySQL database server.
Default value: jdbc:mysql
Specifies the JDBC protocol that the driver connection string uses to connect to the database.
Default value: No default
The numeric ID of this database client. The specified ID must be unique across all currently running database processes in the MySQL cluster. To enable it to read the binlog, the connector uses this unique ID to join the MySQL database cluster as another server.
Default value: No default
The name of the MySQL user that the connector uses to connect to the MySQL database server.
Default value: precise
Specifies how the connector handles values for DECIMAL
and NUMERIC
columns in change events.
Set one of the following options:
precise
(default)Uses
java.math.BigDecimal
values in binary form to represent values precisely.double
Uses the
double
data type to represent values. This option can result in a loss of precision, but it is easier for most consumers to use.string
Encodes values as formatted strings. This option is easy to consume, but can result in the loss of semantic information about the real type.
event.deserialization.failure.handling.mode
Default value: fail
Specifies how the connector reacts after an exception occurs during deserialization of binlog events. This option is deprecated, please use event.processing.failure.handling.mode option instead.
fail
Propagates the exception, which indicates the problematic event and its binlog offset, and causes the connector to stop.
warn
Logs the problematic event and its binlog offset and then skips the event.
ignore
Passes over the problematic event and does not log anything.
Default value: No default
Specifies how field names should be adjusted for compatibility with the message converter used by the connector. Set one of the following options:
none
No adjustment.
avro
Replaces characters that are not valid in Avro names with underscore characters.
avro_unicode
Replaces underscore characters or characters that cannot be used in Avro names with corresponding unicode, such as
_uxxxx
._
is an escape sequence, similar to a backslash in Java
For more information, see: Avro naming.
Default value: No default
A comma-separated list of regular expressions that match source domain IDs in the GTID set that the connector uses to find the binlog position on the MySQL server. When this property is set, the connector uses only the GTID ranges that have source UUIDs that do not match any of the specified exclude
patterns.
To match the value of a GTID, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the GTID’s domain identifier.
If you include this property in the configuration, do not also set the gtid.source.includes
property.
Default value: No default
A comma-separated list of regular expressions that match source domain IDs in the GTID set used that the connector uses to find the binlog position on the MySQL server. When this property is set, the connector uses only the GTID ranges that have source UUIDs that match one of the specified include
patterns.
To match the value of a GTID, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the GTID’s domain identifier.
If you include this property in the configuration, do not also set the gtid.source.excludes
property.
Default value: false
Boolean value that specifies whether the connector should include the original SQL query that generated the change event.
If you set this option to true
then you must also configure MySQL with the binlog_annotate_row_events
option set to ON
. When include.query
is true
, the query is not present for events that the snapshot process generates.
Setting include.query
to true
might expose tables or fields that are explicitly excluded or masked by including the original SQL statement in the change event. For this reason, the default setting is false
.
For more information about configuring the database to return the original SQL
statement for each log event, see Enabling query log events.
Default value: true
Boolean value that specifies whether the connector publishes changes that occur to the database schema to a Kafka topic with the name of the database server ID. Each schema change event that the connector captures uses a key that contains the database name and a value that includes the DDL statements that describe the change. This setting does not affect how the connector records schema changes in its internal database schema history.
Default value: false
Boolean value that specifies whether the connector parses and publishes table and column comments on metadata objects.
When you set this option to true , the schema comments that the connector includes can add a significant amount of string data to each schema object. Increasing the number and size of logical schema objects increases the amount of memory that the connector uses. |
inconsistent.schema.handling.mode
Default value: fail
Specifies how the connector responds to binlog events that refer to tables that are not present in the internal schema representation. That is, the internal representation is not consistent with the database.
Set one of the following options:
fail
The connector throws an exception that reports the problematic event and its binlog offset. The connector then stops.
warn
The connector logs the problematic event and its binlog offset, and then skips the event.
skip
The connector skips the problematic event and does not report it in the log.
Default value: No default
A list of expressions that specify the columns that the connector uses to form custom message keys for change event records that it publishes to the Kafka topics for specified tables.
By default, Debezium uses the primary key column of a table as the message key for records that it emits. In place of the default, or to specify a key for tables that lack a primary key, you can configure custom message keys based on one or more columns.
To establish a custom message key for a table, list the table, followed by the columns to use as the message key. Each list entry takes the following format:
_<fully-qualified_tableName>_:_<keyColumn>_,_<keyColumn>_
To base a table key on multiple column names, insert commas between the column names.
Each fully-qualified table name is a regular expression in the following format:_<databaseName>_._<tableName>_
The property can include entries for multiple tables. Use a semicolon to separate table entries in the list.
The following example sets the message key for the tables inventory.customers
and purchase.orders
:
inventory.customers:pk1,pk2;(.*).purchaseorders:pk3,pk4
For the table inventory.customer
, the columns pk1
and pk2
are specified as the message key. For the purchaseorders
tables in any database, the columns pk3
and pk4
server as the message key.
There is no limit to the number of columns that you use to create custom message keys. However, it’s best to use the minimum number that are required to specify a unique key.
Default value: No default
Unique name for the connector. If you attempt to use the same name to register another connector, registration fails. This property is required by all Kafka Connect connectors.
Default value: No default
Specifies how the connector adjusts schema names for compatibility with the message converter used by the connector. Set one of the following options:
none
No adjustment.
avro
Replaces characters that are not valid in Avro names with underscore characters.
avro_unicode
Replaces underscore characters or characters that cannot be used in Avro names with corresponding unicode, such as
_uxxxx.
NOTE:
_
is an escape sequence, similar to a backslash in Java
Default value: false
Specifies whether the connector emits messages for records when it does not detect a change in the included columns. Columns are considered to be included if they are listed in the column.include.list
, or are not listed in the column.exclude.list
. Set the value to true
to prevent the connector from capturing records when no changes are present in the included columns.
Default value: empty string
An optional, comma-separated list of regular expressions that match fully-qualified table identifiers of tables from which you do not want the connector to capture changes. The connector captures changes in any table that is not included in table.exclude.list
. Each identifier is of the form databaseName.tableName.
To match the name of a column, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the table; it does not match substrings that might be present in a table name.
If you include this property in the configuration, do not also set the table.include.list
property.
Default value: empty string
An optional, comma-separated list of regular expressions that match fully-qualified table identifiers of tables whose changes you want to capture. The connector does not capture changes in any table that is not included in table.include.list
. Each identifier is of the form databaseName.tableName. By default, the connector captures changes in all non-system tables in every database from which it is configured to captures changes.
To match the name of a table, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the table; it does not match substrings that might be present in a table name.
If you include this property in the configuration, do not also set the table.exclude.list
property.
Default value: 1
The maximum number of tasks to create for this connector. Because the MySQL connector always uses a single task, changing the default value has no effect.
Default value: adaptive_time_microseconds
Specifies the type of precision that the connector uses to represent time, date, and timestamps values. Set one of the following options:
adaptive_time_microseconds
(default)The connector captures the date, datetime and timestamp values exactly as in the database using either millisecond, microsecond, or nanosecond precision values based on the database column’s type, with the exception of TIME type fields, which are always captured as microseconds.
adaptive
(deprecated)The connector captures time and timestamp values exactly as in the database using either millisecond, microsecond, or nanosecond precision values based on the data type of the column.
connect
The connector always represents time and timestamp values using Kafka Connect’s built-in representations for Time, Date, and Timestamp, which use millisecond precision regardless of the database columns’ precision.
Default value: true
Specifies whether a delete event is followed by a tombstone event. After a source record is deleted, the connector can emit a tombstone event (the default behavior) to enable Kafka to completely delete all events that pertain to the key of the deleted row in case log compaction is enabled for the topic. Set one of the following options:
true
(default)The connector represents delete operations by emitting a delete event and a subsequent tombstone event.
false
The connector emits only delete events.
Default value: No default
Topic prefix that provides a namespace for the particular MySQL database server or cluster in which Debezium is capturing changes. Because the topic prefix is used to name all of the Kafka topics that receive events that this connector emits, it’s important that the topic prefix is unique across all connectors. Values must contain only alphanumeric characters, hyphens, dots, and underscores.
After you set this property, do not change its value. If you change the value, after the connector restarts, instead of continuing to emit events to the original topics, the connector emits subsequent events to topics whose names are based on the new value. The connector is also unable to recover its database schema history topic. |
Advanced Debezium MySQL connector configuration properties
The following list describes advanced MySQL connector configuration properties. The default values for these properties rarely require changes. Therefore, you do not need to specify them in the connector configuration.
Default value: 0
The size of a look-ahead buffer used by the binlog reader. The default setting of 0
disables buffering.
Under specific conditions, it is possible that the MySQL binlog contains uncommitted data finished by a ROLLBACK
statement. Typical examples are using savepoints or mixing temporary and regular table changes in a single transaction.
When a beginning of a transaction is detected then Debezium tries to roll forward the binlog position and find either COMMIT
or ROLLBACK
so it can determine whether to stream the changes from the transaction. The size of the binlog buffer defines the maximum number of changes in the transaction that Debezium can buffer while searching for transaction boundaries. If the size of the transaction is larger than the buffer then Debezium must rewind and re-read the events that have not fit into the buffer while streaming.
This feature is incubating. Feedback is encouraged. It is expected that this feature is not completely polished. |
Default value: true
A Boolean value that specifies whether a separate thread should be used to ensure that the connection to the MySQL server or cluster is kept alive.
Default value: No default
Enumerates a comma-separated list of the symbolic names of the custom converter instances that the connector can use.
For example, boolean
.
This property is required to enable the connector to use a custom converter.
For each converter that you configure for a connector, you must also add a .type
property, which specifies the fully-qualified name of the class that implements the converter interface. The .type
property uses the following format:
_<converterSymbolicName>_.type
For example,
boolean.type: io.debezium.connector.binlog.converters.TinyIntOneToBooleanConverter
If you want to further control the behavior of a configured converter, you can add one or more configuration parameters to pass values to the converter. To associate these additional configuration parameter with a converter, prefix the parameter name with the symbolic name of the converter.
For example, to define a selector
parameter that specifies the subset of columns that the boolean
converter processes, add the following property:
boolean.selector=db1.table1.*, db1.table2.column1
Default value: No default
Defines tags that customize MBean object names by adding metadata that provides contextual information. Specify a comma-separated list of key-value pairs. Each key represents a tag for the MBean object name, and the corresponding value represents a value for the key, for example,k1=v1,k2=v2
The connector appends the specified tags to the base MBean object name. Tags can help you to organize and categorize metrics data. You can define tags to identify particular application instances, environments, regions, versions, and so forth. For more information, see Customized MBean names.
Default value: No default
A semicolon separated list of SQL statements to be executed when a JDBC connection, not the connection that is reading the transaction log, to the database is established. To specify a semicolon as a character in a SQL statement and not as a delimiter, use two semicolons, (;;
).
The connector might establish JDBC connections at its own discretion, so this property is ony for configuring session parameters. It is not for executing DML statements.
Default value: 600000
(10 minutes)
Specifies the time, in milliseconds, that the connector waits for a query to complete. Set the value to 0
(zero) to remove the timeout limit.
Default value: No default
An optional setting that specifies the location of the key store file. A key store file can be used for two-way authentication between the client and the MySQL server.
database.ssl.keystore.password
Default value: No default
The password for the key store file. Specify a password only if the database.ssl.keystore is configured.
Default value: preferred
Specifies whether the connector uses an encrypted connection. The following settings are available:
disabled
Specifies the use of an unencrypted connection.
preferred
(Default)The connector establishes an encrypted connection if the server supports secure connections. If the server does not support secure connections, the connector falls back to using an unencrypted connection.
required
The connector establishes an encrypted connection. If it is unable to establish an encrypted connection, the connector fails.
verify_ca
The connector behaves as when you set the
required
option, but it also verifies the server TLS certificate against the configured Certificate Authority (CA) certificates. If the server TLS certificate does not match any valid CA certificates, the connector fails.verify_identity
The connector behaves as when you set the
verify_ca
option, but it also verifies that the server certificate matches the host of the remote connection.
Default value: No default
The location of the trust store file for the server certificate verification.
database.ssl.truststore.password
Default value: No default
The password for the trust store file. Used to check the integrity of the truststore, and unlock the truststore.
Default value: true
Boolean value that indicates whether the connector converts a 2-digit year specification to 4 digits. Set the value to false
when conversion is fully delegated to the database.
MySQL users can insert year values with either 2-digits or 4-digits. 2-digit values are mapped to a year in the range 1970 - 2069. By default, the connector performs the conversion.
Default value: -1
Specifies how the connector responds after an operation that results in a retriable error, such as a connection error.
Set one of the following options:
-1
No limit. The connector always restarts automatically, and retries the operation, regardless of the number of previous failures.
0
Disabled. The connector fails immediately, and never retries the operation. User intervention is required to restart the connector.
> 0
The connector restarts automatically until it reaches the specified maximum number of retries. After the next failure, the connector stops, and user intervention is required to restart it.
event.converting.failure.handling.mode
Default value: warn
Specifies how the connector responds when it cannot convert a table record due to a mismatch between the data type of a column and the type specified by the Debezium internal schema.
Set one of the following options:
fail
An exception reports that conversion failed because the data type of the field did not match the schema type, and indicates that it might be necessary to restart the connector in
schema _only_recovery
mode to enable a successful conversion.warn
The connector writes a
null
value to the event field for the column that failed conversion, writes a message to the warning log .skip
The connector writes a
null
value to the event field for the column that failed conversion, and writes a message to the debug log.
event.processing.failure.handling.mode
Default value: fail
Specifies how the connector handles failures that occur when processing events, for example, if it encounters a corrupted event. The following settings are available:
fail
The connector raises an exception that reports the problematic event and its position. The connector then stops.
warn
The connector does not raise an exception. Instead, it logs the problematic event and its position, and then skips the event.
ignore
The connector ignores the problematic event, and does not generate a log entry.
Default value: No default
Specifies a query that the connector executes on the source database when the connector sends a heartbeat message.
For example, the following query periodically captures the state of the executed GTID set in the source database.
INSERT INTO gtid_history_table (select @gtid_executed)
Default value: 0
Specifies how frequently the connector sends heartbeat messages to a Kafka topic. By default, the connector does not send heartbeat messages.
Heartbeat messages are useful for monitoring whether the connector is receiving change events from the database. Heartbeat messages might help decrease the number of change events that need to be re-sent when a connector restarts. To send heartbeat messages, set this property to a positive integer, which indicates the number of milliseconds between heartbeat messages.
incremental.snapshot.allow.schema.changes
Default value: false
Specifies whether the connector allows schema changes during an incremental snapshot. When the value is set to true
, the connector detects schema change during an incremental snapshot, and re-select a current chunk to avoid locking DDLs.
Changes to a primary key are not supported. Changing the primary during an incremental snapshot, can lead to incorrect results. A further limitation is that if a schema change affects only the default values of columns, then the change is not detected until the DDL is processed from the binlog stream. This does not affect the values of snapshot events, but the schema of these snapshot events may have outdated defaults.
incremental.snapshot.chunk.size
Default value: 1024
The maximum number of rows that the connector fetches and reads into memory when it retrieves an incremental snapshot chunk. Increasing the chunk size provides greater efficiency, because the snapshot runs fewer snapshot queries of a greater size. However, larger chunk sizes also require more memory to buffer the snapshot data. Adjust the chunk size to a value that provides the best performance in your environment.
incremental.snapshot.watermarking.strategy
Default value: insert_insert
Specifies the watermarking mechanism that the connector uses during an incremental snapshot to deduplicate events that might be captured by an incremental snapshot and then recaptured after streaming resumes.
You can specify one of the following options:
insert_insert
(default)When you send a signal to initiate an incremental snapshot, for every chunk that Debezium reads during the snapshot, it writes an entry to the signaling data collection to record the signal to open the snapshot window. After the snapshot completes, Debezium inserts a second entry that records the signal to close the window.
insert_delete
When you send a signal to initiate an incremental snapshot, for every chunk that Debezium reads, it writes a single entry to the signaling data collection to record the signal to open the snapshot window. After the snapshot completes, this entry is removed. No entry is created for the signal to close the snapshot window. Set this option to prevent rapid growth of the signaling data collection.
Default value: 2048
Positive integer value that specifies the maximum size of each batch of events that should be processed during each iteration of this connector.
Default value: 8192
A positive integer value that specifies the maximum number of records that the blocking queue can hold. When Debezium reads events streamed from the database, it places the events in the blocking queue before it writes them to Kafka. The blocking queue can provide backpressure for reading change events from the database in cases where the connector ingests messages faster than it can write them to Kafka, or when Kafka becomes unavailable. Events that are held in the queue are disregarded when the connector periodically records offsets. Always set max.queue.size
to a value that is larger than the value of max.batch.size.
Default value: 0
A long integer value that specifies the maximum volume of the blocking queue in bytes. By default, volume limits are not specified for the blocking queue. To specify the number of bytes that the queue can consume, set this property to a positive long value.
If max.queue.size is also set, writing to the queue is blocked when the size of the queue reaches the limit specified by either property. For example, if you set max.queue.size=1000
, and max.queue.size.in.bytes=5000
, writing to the queue is blocked after the queue contains 1000 records, or after the volume of the records in the queue reaches 5000 bytes.
min.row.count.to.stream.results
Default value: 1000
During a snapshot, the connector queries each table for which the connector is configured to capture changes. The connector uses each query result to produce a read event that contains data for all rows in that table. This property determines whether the MySQL connector puts results for a table into memory, which is fast but requires large amounts of memory, or streams the results, which can be slower but work for very large tables. The setting of this property specifies the minimum number of rows a table must contain before the connector streams results.
To skip all table size checks and always stream all results during a snapshot, set this property to 0
.
Default value: No default
List of notification channel names that are enabled for the connector. By default, the following channels are available:
sink
log
jmx
Optionally, you can also implement a custom notification channel.
Default value: 500
(0.5 seconds)
Positive integer value that specifies the number of milliseconds the connector waits for new change events to appear before it starts processing a batch of events.
Default value: false
Determines whether the connector generates events with transaction boundaries and enriches change event envelopes with transaction metadata. Specify true
if you want the connector to do this. For more information, see Transaction metadata.
Default value: false
Specifies whether a connector writes watermarks to the signal data collection to track the progress of an incremental snapshot. Set the value to true
to enable a connector that has a read-only connection to the database to use an incremental snapshot watermarking strategy that does not require writing to the signal data collection.
Default value: No default
Fully-qualified name of the data collection that is used to send signals to the connector.
Use the following format to specify the collection name:_<databaseName>_._<tableName>_
Default value: No default
List of the signaling channel names that are enabled for the connector. By default, the following channels are available:
source
kafka
file
jmx
Optionally, you can also implement a custom signaling channel.
Default value: t
A comma-separated list of operation types that will be skipped during streaming. The operations include: c
for inserts/create, u
for updates, d
for deletes, t
for truncates, and none
to not skip any operations. By default, truncate operations are skipped.
Default value: No default
An interval in milliseconds that the connector should wait before performing a snapshot when the connector starts. If you are starting multiple connectors in a cluster, this property is useful for avoiding snapshot interruptions, which might cause re-balancing of connectors.
Default value: No default
During a snapshot, the connector reads table content in batches of rows. This property specifies the maximum number of rows in a batch.
snapshot.include.collection.list
Default value: All tables specified in the table.include.list
.
An optional, comma-separated list of regular expressions that match the fully-qualified names (_<databaseName>.<tableName>_
) of the tables to include in a snapshot. The specified items must be named in the connector’s table.include.list property. This property takes effect only if the connector’s snapshot.mode property is set to a value other than never
.
This property does not affect the behavior of incremental snapshots.
To match the name of a table, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the table; it does not match substrings that might be present in a table name.
Default value: 10000
Positive integer that specifies the maximum amount of time (in milliseconds) to wait to obtain table locks when performing a snapshot. If the connector cannot acquire table locks in this time interval, the snapshot fails. For more information, see how MySQL connectors perform database snapshots.
Default value: minimal
Specifies whether and for how long the connector holds the global MySQL read lock, which prevents any updates to the database while the connector is performing a snapshot. The following settings are available:
minimal
The connector holds the global read lock for only the initial phase of the snapshot during which it reads the database schemas and other metadata. During the next phase of the snapshot, the connector releases the lock as it selects all rows from each table. To perform the SELECT operation in a consistent fashion, the connector uses a REPEATABLE READ transaction. Although the release of the global read lock permits other MySQL clients to update the database, use of REPEATABLE READ isolation ensures a consistent snapshot, because the connector continues to read the same data for the duration of the transaction.
extended
Blocks all write operations for the duration of the snapshot. Use this setting if clients submit concurrent operations that are incompatible with the REPEATABLE READ isolation level in MySQL.
none
Prevents the connector from acquiring any table locks during the snapshot. Although this option is allowed with all snapshot modes, it is safe to use only if no schema changes occur while the snapshot is running. Tables that are defined with the MyISAM engine always acquire a table lock. As a result, such tables are locked even if you set this option. This behavior differs from tables that are defined by the InnoDB engine, which acquire row-level locks.
custom
The connector performs a snapshot according to the implementation specified by the snapshot.locking.mode.custom.name property, which is a custom implementation of the
io.debezium.spi.snapshot.SnapshotLock
interface.
snapshot.locking.mode.custom.name
Default value: No default
When snapshot.locking.mode is set to custom
, use this setting to specify the name of the custom implementation provided in the name()
method that is defined by the ‘io.debezium.spi.snapshot.SnapshotLock’ interface. For more information, see custom snapshotter SPI.
Default value: 1
Specifies the number of threads that the connector uses when performing an initial snapshot. To enable parallel initial snapshots, set the property to a value greater than 1. In a parallel initial snapshot, the connector processes multiple tables concurrently.
Parallel initial snapshots is an incubating feature. |
Default value: initial
Specifies the criteria for running a snapshot when the connector starts. The following settings are available:
always
The connector performs a snapshot every time that it starts. The snapshot includes the structure and data of the captured tables. Specify this value to populate topics with a complete representation of the data from the captured tables every time that the connector starts.
initial
(default)The connector runs a snapshot only when no offsets have been recorded for the logical server name, or if it detects that an earlier snapshot failed to complete. After the snapshot completes, the connector begins to stream event records for subsequent database changes.
initial_only
The connector runs a snapshot only when no offsets have been recorded for the logical server name. After the snapshot completes, the connector stops. It does not transition to streaming to read change events from the binlog.
schema_only
Deprecated, see
no_data
.no_data
The connector runs a snapshot that captures only the schema, but not any table data. Set this option if you do not need the topics to contain a consistent snapshot of the data, but you want to capture any schema changes that were applied after the last connector restart.
schema_only_recovery
Deprecated, see
recovery
.recovery
Set this option to restore a database schema history topic that is lost or corrupted. After a restart, the connector runs a snapshot that rebuilds the topic from the source tables. You can also set the property to periodically prune a database schema history topic that experiences unexpected growth.
Do not use this mode to perform a snapshot if schema changes were committed to the database after the last connector shutdown.
never
When the connector starts, rather than performing a snapshot, it immediately begins to stream event records for subsequent database changes. This option is under consideration for future deprecation, in favor of the
no_data
option.when_needed
After the connector starts, it performs a snapshot only if it detects one of the following circumstances:
It cannot detect any topic offsets.
A previously recorded offset specifies a binlog position or GTID that is not available on the server.
configuration_based
With this option, you control snapshot behavior through a set of connector properties that have the prefix ‘snapshot.mode.configuration.based’.
custom
The connector performs a snapshot according to the implementation specified by the snapshot.mode.custom.name property, which defines a custom implementation of the
io.debezium.spi.snapshot.Snapshotter
interface.
snapshot.mode.configuration.based.snapshot.data
Default value: false
If the snapshot.mode
is set to configuration_based
, set this property to specify whether the connector includes table data when it performs a snapshot.
snapshot.mode.configuration.based.snapshot.on.data.error
Default value: false
If the snapshot.mode
is set to configuration_based
, set this property to specify whether the connector includes table data in a snapshot in the event that data is no longer available in the transaction log.
snapshot.mode.configuration.based.snapshot.on.schema.error
Default value: false
If the snapshot.mode
is set to configuration_based
, set this property to specify whether the connector includes table schema in a snapshot if the schema history topic is not available.
snapshot.mode.configuration.based.snapshot.schema
Default value: false
If the snapshot.mode
is set to configuration_based
, set this property to specify whether the connector includes the table schema when it performs a snapshot.
snapshot.mode.configuration.based.start.stream
Default value: false
If the snapshot.mode
is set to configuration_based
, set this property to specify whether the connector begins to stream change events after a snapshot completes.
Default value: No default
If snapshot.mode
is set to custom
, use this setting to specify the name of the custom implementation that is provided in the name()
method that is defined in the ‘io.debezium.spi.snapshot.Snapshotter’ interface. After a connector restart, Debezium calls the specified custom implementation to determine whether to perform a snapshot. For more information, see custom snapshotter SPI.
Default value: select_all
Specifies how the connector queries data while performing a snapshot.
Set one of the following options:
select_all
(default)The connector uses a
select all
query to retrieve rows from captured tables, optionally adjusting the columns selected based on the columninclude
andexclude
list configurations.custom
The connector performs a snapshot query according to the implementation specified by the snapshot.query.mode.custom.name property, which defines a custom implementation of the
io.debezium.spi.snapshot.SnapshotQuery
interface.This setting enables you to manage snapshot content in a more flexible manner compared to using the snapshot.select.statement.overrides property.
snapshot.query.mode.custom.name
Default value: No default
When snapshot.query.mode is set as custom
, use this setting to specify the name of the custom implementation provided in the name()
method that is defined by the ‘io.debezium.spi.snapshot.SnapshotQuery’ interface. For more information, see custom snapshotter SPI.
snapshot.select.statement.overrides
Default value: No default
Specifies the table rows to include in a snapshot. Use the property if you want a snapshot to include only a subset of the rows in a table. This property affects snapshots only. It does not apply to events that the connector reads from the log.
The property contains a comma-separated list of fully-qualified table names in the form _<databaseName>.<tableName>_
. For example,
"snapshot.select.statement.overrides": "inventory.products,customers.orders"
For each table in the list, add a further configuration property that specifies the SELECT
statement for the connector to run on the table when it takes a snapshot. The specified SELECT
statement determines the subset of table rows to include in the snapshot. Use the following format to specify the name of this SELECT
statement property:
snapshot.select.statement.overrides._<databaseName>_._<tableName>_
For example, snapshot.select.statement.overrides.customers.orders
From a customers.orders
table that includes the soft-delete column, delete_flag
, add the following properties if you want a snapshot to include only those records that are not soft-deleted:
"snapshot.select.statement.overrides": "customer.orders",
"snapshot.select.statement.overrides.customer.orders": "SELECT * FROM [customers].[orders] WHERE delete_flag = 0 ORDER BY id DESC"
In the resulting snapshot, the connector includes only the records for which delete_flag = 0
.
snapshot.tables.order.by.row.count
Default value: disabled
Specifies the order in which the connector processes tables when it performs an initial snapshot. Set one of the following options:
descending
The connector snapshots tables in order, based on the number of rows from the highest to the lowest.
ascending
The connector snapshots tables in order, based on the number of rows, from lowest to highest.
disabled
The connector disregards row count when performing an initial snapshot.
Default value: v2
Schema version for the source
block in Debezium events. Debezium 0.10 introduced a few breaking changes to the structure of the source
block in order to unify the exposed structure across all the connectors.
By setting this option to v1
, the structure used in earlier versions can be produced. However, this setting is not recommended and is planned for removal in a future Debezium version.
Default value: 0
Specifies the time, in milliseconds, that the connector delays the start of the streaming process after it completes a snapshot. Setting a delay interval helps to prevent the connector from restarting snapshots in the event that a failure occurs immediately after the snapshot completes, but before the streaming process begins. Set a delay value that is higher than the value of the offset.flush.interval.ms property that is set for the Kafka Connect worker.
Default value: true
A Boolean value that specifies whether built-in system tables should be ignored. This applies regardless of the table include and exclude lists. By default, changes that occur to the values in system tables are excluded from capture, and Debezium does not generate events for system table changes.
Default value: 10000
Specifies the number of topic names that can be stored in memory in a bounded concurrent hash map. The connector uses the cache to help determine the topic name that corresponds to a data collection.
Default value: .
Specifies the delimiter that the connector inserts between components of the topic name.
Default value: __debezium-heartbeat
Specifies the name of the topic to which the connector sends heartbeat messages. The topic name takes the following format:
topic.heartbeat.prefix.topic.prefix
For example, if the topic prefix is fulfillment
, the default topic name is __debezium-heartbeat.fulfillment
.
Default value: io.debezium.schema.DefaultTopicNamingStrategy
The name of the TopicNamingStrategy
class that the connector uses. The specified strategy determines how the connector names the topics that store event records for data changes, schema changes, transactions, heartbeats, and so forth.
Default value: transaction
Specifies the name of the topic to which the connector sends transaction metadata messages. The topic name takes the following pattern:
topic.prefix.topic.transaction
For example, if the topic prefix is fulfillment
, the default topic name is fulfillment.transaction
.
Default value: false
A Boolean value that specifies whether the binary log client’s keepalive thread sets the SO_LINGER
socket option to 0
to immediately close stale TCP connections.
Set the value to true
if the connector experiences deadlocks in SSLSocketImpl.close
.
For more information, see Issue 133 in the mysql-binlog-connector-java GitHub repository.
Debezium connector database schema history configuration properties
Debezium provides a set of schema.history.internal.*
properties that control how the connector interacts with the schema history topic.
The following table describes the schema.history.internal
properties for configuring the Debezium connector.
Property | Default | Description |
---|---|---|
No default | The full name of the Kafka topic where the connector stores the database schema history. | |
No default | A list of host/port pairs that the connector uses for establishing an initial connection to the Kafka cluster. This connection is used for retrieving the database schema history previously stored by the connector, and for writing each DDL statement read from the source database. Each pair should point to the same Kafka cluster used by the Kafka Connect process. | |
| An integer value that specifies the maximum number of milliseconds the connector should wait during startup/recovery while polling for persisted data. The default is 100ms. | |
| An integer value that specifies the maximum number of milliseconds the connector should wait while fetching cluster information using Kafka admin client. | |
| An integer value that specifies the maximum number of milliseconds the connector should wait while create kafka history topic using Kafka admin client. | |
| The maximum number of times that the connector should try to read persisted history data before the connector recovery fails with an error. The maximum amount of time to wait after receiving no data is | |
| A Boolean value that specifies whether the connector should ignore malformed or unknown database statements or stop processing so a human can fix the issue. The safe default is | |
| A Boolean value that specifies whether the connector records schema structures from all tables in a schema or database, or only from tables that are designated for capture.
| |
| A Boolean value that specifies whether the connector records schema structures from all logical databases in the database instance.
|
Pass-through MySQL connector configuration properties
You can set pass-through properties in the connector configuration to customize the behavior of the Apache Kafka producer and consumer. For information about the full range of configuration properties for Kafka producers and consumers, see the Kafka documentation.
Pass-through properties for configuring how producer and consumer clients interact with schema history topics
Debezium relies on an Apache Kafka producer to write schema changes to database schema history topics. Similarly, it relies on a Kafka consumer to read from database schema history topics when a connector starts. You define the configuration for the Kafka producer and consumer clients by assigning values to a set of pass-through configuration properties that begin with the schema.history.internal.producer.*
and schema.history.internal.consumer.*
prefixes. The pass-through producer and consumer database schema history properties control a range of behaviors, such as how these clients secure connections with the Kafka broker, as shown in the following example:
schema.history.internal.producer.security.protocol=SSL
schema.history.internal.producer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks
schema.history.internal.producer.ssl.keystore.password=test1234
schema.history.internal.producer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks
schema.history.internal.producer.ssl.truststore.password=test1234
schema.history.internal.producer.ssl.key.password=test1234
schema.history.internal.consumer.security.protocol=SSL
schema.history.internal.consumer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks
schema.history.internal.consumer.ssl.keystore.password=test1234
schema.history.internal.consumer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks
schema.history.internal.consumer.ssl.truststore.password=test1234
schema.history.internal.consumer.ssl.key.password=test1234
Debezium strips the prefix from the property name before it passes the property to the Kafka client.
For more information about Kafka producer configuration properties and Kafka consumer configuration properties, see the Apache Kafka documentation .
Pass-through properties for configuring how the MySQL connector interacts with the Kafka signaling topic
Debezium provides a set of signal.*
properties that control how the connector interacts with the Kafka signals topic.
The following table describes the Kafka signal
properties.
Property | Default | Description | ||
---|---|---|---|---|
<topic.prefix>-signal | The name of the Kafka topic that the connector monitors for ad hoc signals.
| |||
kafka-signal | The name of the group ID that is used by Kafka consumers. | |||
No default | A list of the host and port pairs that the connector uses to establish its initial connection to the Kafka cluster. Each pair references the Kafka cluster that is used by the Debezium Kafka Connect process. | |||
| An integer value that specifies the maximum number of milliseconds that the connector waits when polling signals. | |||
| Specifies whether the Kafka consumer writes an offset commit after it reads a message from the signaling topic. The value that you assign to this property determines whether the connector can process requests that the signaling topic receives while the connector is offline. Choose one of the following settings:
|
Pass-through properties for configuring the Kafka consumer client for the signaling channel
The Debezium connector provides for pass-through configuration of the signals Kafka consumer. Pass-through signals properties begin with the prefix signals.consumer.*
. For example, the connector passes properties such as signal.consumer.security.protocol=SSL
to the Kafka consumer.
Debezium strips the prefixes from the properties before it passes the properties to the Kafka signals consumer.
Pass-through properties for configuring the MySQL connector sink notification channel
The following table describes properties that you can use to configure the Debezium sink notification
channel.
Property | Default | Description |
---|---|---|
No default | The name of the topic that receives notifications from Debezium. This property is required when you configure the notification.enabled.channels property to include |
Debezium connector pass-through database driver configuration properties
The Debezium connector provides for pass-through configuration of the database driver. Pass-through database properties begin with the prefix driver.*
. For example, the connector passes properties such as driver.foobar=false
to the JDBC URL.
Debezium strips the prefixes from the properties before it passes the properties to the database driver.
Monitoring
The Debezium MySQL connector provides three types of metrics that are in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect provide.
Snapshot metrics provide information about connector operation while performing a snapshot.
Streaming metrics provide information about connector operation when the connector is reading the binlog.
Schema history metrics provide information about the status of the connector’s schema history.
Debezium monitoring documentation provides details for how to expose these metrics by using JMX.
Customized MBean names
Debezium connectors expose metrics via the MBean name for the connector. These metrics, which are specific to each connector instance, provide data about the behavior of the connector’s snapshot, streaming, and schema history processes.
By default, when you deploy a correctly configured connector, Debezium generates a unique MBean name for each of the different connector metrics. To view the metrics for a connector process, you configure your observability stack to monitor its MBean. But these default MBean names depend on the connector configuration; configuration changes can result in changes to the MBean names. A change to the MBean name breaks the linkage between the connector instance and the MBean, disrupting monitoring activity. In this scenario, you must reconfigure the observability stack to use the new MBean name if you want to resume monitoring.
To prevent monitoring disruptions that result from MBean name changes, you can configure custom metrics tags. You configure custom metrics by adding the custom.metric.tags
property to the connector configuration. The property accepts key-value pairs in which each key represents a tag for the MBean object name, and the corresponding value represents the value of that tag. For example: k1=v1,k2=v2
. Debezium appends the specified tags to the MBean name of the connector.
After you configure the custom.metric.tags
property for a connector, you can configure the observability stack to retrieve metrics associated with the specified tags. The observability stack then uses the specified tags, rather than the mutable MBean names to uniquely identify connectors. Later, if Debezium redefines how it constructs MBean names, or if the topic.prefix
in the connector configuration changes, metrics collection is uninterrupted, because the metrics scrape task uses the specified tag patterns to identify the connector.
A further benefit of using custom tags, is that you can use tags that reflect the architecture of your data pipeline, so that metrics are organized in a way that suits you operational needs. For example, you might specify tags with values that declare the type of connector activity, the application context, or the data source, for example, db1-streaming-for-application-abc
. If you specify multiple key-value pairs, all of the specified pairs are appended to the connector’s MBean name.
The following example illustrates how tags modify the default MBean name.
Example 3. How custom tags modify the connector MBean name
By default, the MySQL connector uses the following MBean name for streaming metrics:
debezium.mysql:type=connector-metrics,context=streaming,server=<topic.prefix>
If you set the value of custom.metric.tags
to database=salesdb-streaming,table=inventory
, Debezium generates the following custom MBean name:
debezium.mysql:type=connector-metrics,context=streaming,server=<topic.prefix>,database=salesdb-streaming,table=inventory
Snapshot metrics
The MBean is debezium.mysql:type=connector-metrics,context=snapshot,server=_<topic.prefix>_
.
Snapshot metrics are not exposed unless a snapshot operation is active, or if a snapshot has occurred since the last connector start.
The following table lists the snapshot metrics that are available.
Attributes | Type | Description |
---|---|---|
| The last snapshot event that the connector has read. | |
| The number of milliseconds since the connector has read and processed the most recent event. | |
| The total number of events that this connector has seen since last started or reset. | |
| The number of events that have been filtered by include/exclude list filtering rules configured on the connector. | |
| The list of tables that are captured by the connector. | |
| The length the queue used to pass events between the snapshotter and the main Kafka Connect loop. | |
| The free capacity of the queue used to pass events between the snapshotter and the main Kafka Connect loop. | |
| The total number of tables that are being included in the snapshot. | |
| The number of tables that the snapshot has yet to copy. | |
| Whether the snapshot was started. | |
| Whether the snapshot was paused. | |
| Whether the snapshot was aborted. | |
| Whether the snapshot completed. | |
| The total number of seconds that the snapshot has taken so far, even if not complete. Includes also time when snapshot was paused. | |
| The total number of seconds that the snapshot was paused. If the snapshot was paused several times, the paused time adds up. | |
| Map containing the number of rows scanned for each table in the snapshot. Tables are incrementally added to the Map during processing. Updates every 10,000 rows scanned and upon completing a table. | |
| The maximum buffer of the queue in bytes. This metric is available if max.queue.size.in.bytes is set to a positive long value. | |
| The current volume, in bytes, of records in the queue. |
The connector also provides the following additional snapshot metrics when an incremental snapshot is executed:
Attributes | Type | Description |
---|---|---|
| The identifier of the current snapshot chunk. | |
| The lower bound of the primary key set defining the current chunk. | |
| The upper bound of the primary key set defining the current chunk. | |
| The lower bound of the primary key set of the currently snapshotted table. | |
| The upper bound of the primary key set of the currently snapshotted table. |
The Debezium MySQL connector also provides the HoldingGlobalLock
custom snapshot metric. This metric is set to a Boolean value that indicates whether the connector currently holds a global or table write lock.
Streaming metrics
The Debezium MySQL connector provides three types of metrics that are in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect provide.
Snapshot metrics provide information about connector operation while performing a snapshot.
Streaming metrics provide information about connector operation when the connector is reading the binlog.
Schema history metrics provide information about the status of the connector’s schema history.
Debezium monitoring documentation provides details for how to expose these metrics by using JMX.
The MBean is debezium.mysql:type=connector-metrics,context=streaming,server=_<topic.prefix>_
.
The following table lists the streaming metrics that are available.
Attributes | Type | Description |
---|---|---|
| The last streaming event that the connector has read. | |
| The number of milliseconds since the connector has read and processed the most recent event. | |
| The total number of data change events reported by the source database since the last connector start, or since a metrics reset. Represents the data change workload for Debezium to process. | |
| The total number of create events processed by the connector since its last start or metrics reset. | |
| The total number of update events processed by the connector since its last start or metrics reset. | |
| The total number of delete events processed by the connector since its last start or metrics reset. | |
| The number of events that have been filtered by include/exclude list filtering rules configured on the connector. | |
| The list of tables that are captured by the connector. | |
| The length the queue used to pass events between the streamer and the main Kafka Connect loop. | |
| The free capacity of the queue used to pass events between the streamer and the main Kafka Connect loop. | |
| Flag that denotes whether the connector is currently connected to the database server. | |
| The number of milliseconds between the last change event’s timestamp and the connector processing it. The values will incorporate any differences between the clocks on the machines where the database server and the connector are running. | |
| The number of processed transactions that were committed. | |
| The coordinates of the last received event. | |
| Transaction identifier of the last processed transaction. | |
| The maximum buffer of the queue in bytes. This metric is available if max.queue.size.in.bytes is set to a positive long value. | |
| The current volume, in bytes, of records in the queue. |
The Debezium MySQL connector also provides the following additional streaming metrics:
Attribute | Type | Description |
---|---|---|
| The name of the binlog file that the connector has most recently read. | |
| The most recent position (in bytes) within the binlog that the connector has read. | |
| Flag that denotes whether the connector is currently tracking GTIDs from MySQL server. | |
| The string representation of the most recent GTID set processed by the connector when reading the binlog. | |
| The number of events that have been skipped by the MySQL connector. Typically events are skipped due to a malformed or unparseable event from MySQL’s binlog. | |
| The number of disconnects by the MySQL connector. | |
| The number of processed transactions that were rolled back and not streamed. | |
| The number of transactions that have not conformed to the expected protocol of | |
| The number of transactions that have not fit into the look-ahead buffer. For optimal performance, this value should be significantly smaller than |
Schema history metrics
The MBean is debezium.mysql:type=connector-metrics,context=schema-history,server=_<topic.prefix>_
.
The following table lists the schema history metrics that are available.
Attributes | Type | Description |
---|---|---|
| One of | |
| The time in epoch seconds at what recovery has started. | |
| The number of changes that were read during recovery phase. | |
| the total number of schema changes applied during recovery and runtime. | |
| The number of milliseconds that elapsed since the last change was recovered from the history store. | |
| The number of milliseconds that elapsed since the last change was applied. | |
| The string representation of the last change recovered from the history store. | |
| The string representation of the last applied change. |
Behavior when things go wrong
Debezium is a distributed system that captures all changes in multiple upstream databases; it never misses or loses an event. When the system is operating normally or being managed carefully then Debezium provides exactly once delivery of every change event record.
If a fault does occur, the system does not lose any events. However, while Debezium is recovering from a fault, it might repeat some change events. In these abnormal situations, Debezium, like Kafka, provides at least once delivery of change events.
The rest of this section describes how Debezium handles various kinds of faults and problems.
Configuration and startup errors
In the following situations, the connector fails when trying to start, reports an error or exception in the log, and stops running:
The connector’s configuration is invalid.
The connector cannot successfully connect to the MySQL server by using the specified connection parameters.
The connector is attempting to restart at a position in the binlog for which MySQL no longer has the history available.
In these cases, the error message has details about the problem and possibly a suggested workaround. After you correct the configuration or address the MySQL problem, restart the connector.
MySQL becomes unavailable
If your MySQL server becomes unavailable, the Debezium MySQL connector fails with an error and the connector stops. When the server is available again, restart the connector.
However, if you are connecting to a highly available MySQL cluster, you can restart the connector immediately. It will connect to a different MySQL server in the cluster, find the location in the server’s binlog that represents the last transaction, and begin reading the new server’s binlog from that specific location.
Kafka Connect stops gracefully
When Kafka Connect stops gracefully, there is a short delay while the Debezium MySQL connector tasks are stopped and restarted on new Kafka Connect processes.
Kafka Connect process crashes
If Kafka Connect crashes, the process stops and any Debezium MySQL connector tasks terminate without their most recently-processed offsets being recorded. In distributed mode, Kafka Connect restarts the connector tasks on other processes. However, the MySQL connector resumes from the last offset recorded by the earlier processes. As a result, the replacement tasks might regenerate some events that were processed before the crash, creating duplicate events.
Each change event message includes source-specific information that you can use to identify duplicate events, for example:
Event origin
MySQL server’s event time
The binlog file name and position
GTIDs (if used)
Kafka becomes unavailable
The Kafka Connect framework records Debezium change events in Kafka by using the Kafka producer API. If the Kafka brokers become unavailable, the Debezium MySQL connector pauses until the connection is reestablished, and the connector then resumes where it left off.
MySQL purges binlog files
If the Debezium MySQL connector stops for too long, the MySQL server purges older binlog files and the connector’s last position may be lost. When the connector is restarted, the MySQL server no longer has the starting point and the connector performs another initial snapshot. If the snapshot is disabled, the connector fails with an error.
See snapshots for details about how MySQL connectors perform initial snapshots.