- Debezium connector for MySQL
- How the connector works
- Data change events
- Data type mappings
- Setting up MySQL
- Deployment
- MySQL connector configuration example
- Adding connector configuration
- Connector properties
- Required Debezium MySQL connector configuration properties
- Advanced MySQL connector configuration properties
- Debezium connector database schema history configuration properties
- Debezium connector Kafka signals configuration properties
- Debezium connector pass-through signals Kafka consumer client configuration properties
- Debezium connector pass-through database driver configuration properties
- Monitoring
- Behavior when things go wrong
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.
The Debezium MySQL connector has yet to be tested with MariaDB, but multiple reports from the community indicate successful usage of the connector with this database. Official support for MariaDB is planned for a future Debezium version. |
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 (and optionally GTIDs 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 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 sees changes in only 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 high availability solutions exist for MySQL, and they make it significantly easier to tolerate and almost immediately recover from problems and failures. Most HA MySQL clusters use GTIDs so that replicas are able to keep track of all changes on any of the primary servers.
Multi-primary
Network Database (NDB) cluster replication uses one or more MySQL replica nodes that each replicate from multiple primary servers. This is 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, as 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
There is support for the Debezium MySQL connector to use hosted options such as Amazon RDS and Amazon Aurora.
Because these hosted options do not allow a global read lock, table-level locks are used to create the 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 just use the current schema because the connector might be processing events that are relatively old that were recorded before the tables’ schemas were changed.
To ensure correct processing of changes that occur after a schema change, MySQL includes in the binlog not only the row-level changes to the data, but also the DDL statements that are applied to the database. As the connector reads the binlog and comes across these DDL statements, 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 having crashed or been stopped gracefully, the connector 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 connector use only. The connector can optionally 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. The connector needs to be configured to capture change to these helper tables. If consumers do not need the records generated for helper tables, then a single message transform can be applied to filter them out.
See 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 captured 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 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.0.0.Final","connector": "mysql","name": "mysql","ts_ms": 1651535750218, (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. In the source object, ts_ms indicates the time that the change was made in the database. By comparing the value for payload.source.ts_ms with the value for payload.ts_ms, you can determine the lag between the source database update and Debezium. |
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. |
See also: schema history topic.
Snapshots
When a Debezium MySQL connector is first started, it performs an initial consistent snapshot of your database. The following flow describes how the connector creates this snapshot. This flow is for the default snapshot mode, which is initial. For information about other snapshot modes, see the MySQL connector snapshot.mode configuration property.
| Step | Action |
|---|---|
1 | Grabs a global read lock that blocks writes by other database clients. |
2 | Starts a transaction with repeatable read semantics to ensure that all subsequent reads within the transaction are done against the consistent snapshot. |
3 | Reads the current binlog position. |
4 | Reads the schema of the databases and tables for which the connector is configured to capture changes. |
5 | Releases the global read lock. Other database clients can now write to the database. |
6 | If applicable, writes the DDL changes to the schema change topic, including all necessary |
7 | Scans the database tables. For each row, the connector emits |
8 | Commits the transaction. |
9 | Records the completed snapshot in the connector offsets. |
Connector restarts
If the connector fails, stops, or is rebalanced while performing the initial snapshot, then after the connector restarts, it performs a new snapshot. After that intial snapshot is completed, the Debezium MySQL connector restarts from the same position in the binlog so it does not miss any updates.
If the connector stops for long enough, MySQL could purge old binlog files and the connector’s position would be lost. If the position is lost, the connector reverts to the initial snapshot for its starting position. For more tips on troubleshooting the Debezium MySQL connector, see behavior when things go wrong.
Global read locks not allowed
Some environments do not allow global read locks. If the Debezium MySQL connector detects that global read locks are not permitted, the connector uses table-level locks instead and performs a snapshot with this method. This requires the database user for the Debezium connector to have LOCK TABLES privileges.
| Step | Action |
|---|---|
1 | Obtains table-level locks. |
2 | Starts a transaction with repeatable read semantics to ensure that all subsequent reads within the transaction are done against the consistent snapshot. |
3 | Reads and filters the names of the databases and tables. |
4 | Reads the current binlog position. |
5 | Reads the schema of the databases and tables for which the connector is configured to capture changes. |
6 | If applicable, writes the DDL changes to the schema change topic, including all necessary |
7 | Scans the database tables. For each row, the connector emits |
8 | Commits the transaction. |
9 | Releases the table-level locks. |
10 | Records the completed snapshot in the connector offsets. |
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. The following changes in a database might be cause for performing an ad hoc snapshot:
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, 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 table to be snapshotted. |
| Optional.Empty | An optional string, which specifies a condition based on the column(s) of the table(s), to capture a subset of the contents of the table(s). |
Triggering an ad hoc snapshot
You initiate an ad hoc snapshot by adding an entry with the execute-snapshot signal type to the signaling table. 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.
Currently, the execute-snapshot action type triggers incremental snapshots only. For more information, see Incremental 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 1 KB.
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 windows 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.
Triggering an incremental snapshot
Currently, the only way to initiate an incremental snapshot is to send an ad hoc snapshot signal to the signaling table on the source database. You submit signals to the table 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 kind of snapshot operation. Currently, the only valid option for snapshots operations is the default value, incremental.
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 detects that no action is required and 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.
Procedure
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": ["_<tableName>_","_<tableName>_"],"type":"_<snapshotType>_"}');
For example,
INSERT INTO myschema.debezium_signal (id, type, data) VALUES('ad-hoc-1', 'execute-snapshot', '{"data-collections": ["schema1.table1", "schema2.table2"],"type":"incremental"}');
The values of the
id,type, anddataparameters in the command correspond to the fields of the signaling table.The following table describes the these parameters:
Table 5. Descriptions of fields in a SQL command for sending an incremental snapshot signal to the signaling table Value Description myschema.debezium_signalSpecifies the fully-qualified name of the signaling table on the source database
ad-hoc-1The
idparameter specifies an arbitrary string that is assigned as theididentifier for the signal request.
Use this string to identify logging messages to entries in the signaling table. Debezium does not use this string. Rather, during the snapshot, Debezium generates its ownidstring as a watermarking signal.execute-snapshotSpecifies
typeparameter specifies the operation that the signal is intended to trigger.data-collectionsA required component of the
datafield of a signal that specifies an array of table names or regular expressions to match table names to include in the snapshot.
The array lists regular expressions which match tables by their fully-qualified names, using the same format as you use to specify the name of the connector’s signaling table in the signal.data.collection configuration property.incrementalAn optional
typecomponent of thedatafield of a signal that specifies the kind of snapshot operation to run.
Currently, the only valid option is the default value,incremental.
Specifying atypevalue in the SQL query that you submit to the signaling table is optional.
If you do not specify a value, the connector runs an incremental snapshot.additional-conditionAn optional string, which specifies a condition based on the column(s) of the table(s), to capture a subset of the contents of the tables.
Ad hoc incremental snapshots with additional-condition
additional-conditionis used to select a subset of a table’s content.To give an analogy how
additional-conditionis used:For a snapshot, the SQL query executed behind the scenes is something like:
SELECT * FROM <tableName> ….
For a snapshot with a
additional-condition, theadditional-conditionis appended to the SQL query, something like:SELECT * FROM <tableName> WHERE <additional-condition> ….
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": ["_<tableName>_","_<tableName>_"],"type":"_<snapshotType>_","additional-condition":"_<additional-condition>_"}');
Suppose there is a
productstable with columnsid(primary key),colorandbrand.To snapshot just the content of the
productstable wherecolor=blueINSERT INTO myschema.debezium_signal (id, type, data) VALUES('ad-hoc-1', 'execute-snapshot', '{"data-collections": ["schema1.products"],"type":"incremental", "additonal-condition":"color=blue"}');
additional-conditioncan be used to pass condition based on multiple columns. Using the sameproductstable, to snapshot content of theproductstable wherecolor=blueandbrand=fooINSERT INTO myschema.debezium_signal (id, type, data) VALUES('ad-hoc-1', 'execute-snapshot', '{"data-collections": ["schema1.products"],"type":"incremental", "additonal-condition":"color=blue AND brand=foo"}');
The following example, shows the JSON for an incremental snapshot event that is captured by a connector.
Example: Incremental snapshot event message
{"before":null,"after": {"pk":"1","value":"New data"},"source": {..."snapshot":"incremental" (1)},"op":"r", (2)"ts_ms":"1620393591654","transaction":null}
| Item | Field name | Description |
|---|---|---|
1 |
| Specifies the type of snapshot operation to run. |
2 |
| Specifies the event type. |
Stopping an incremental snapshot
Incremental snapshots can also be stopped by sending a signal to the table on the source database. You submit signals to the table as SQL INSERT queries. After Debezium detects the change in the signaling table, it reads the signal, and stops the incremental snapshot operation if it’s in progress.
The query that you submit specifies the snapshot operation of incremental, and, optionally, the tables of the current running snapshot to be removed.
Prerequisites
-
A signaling data collection exists on the source database.
The signaling data collection is specified in the signal.data.collection property.
Procedure
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": ["_<tableName>_","_<tableName>_"],"type":"incremental"}');
For example,
INSERT INTO myschema.debezium_signal (id, type, dat) values ('ad-hoc-1', 'stop-snapshot', '{"data-collections": ["schema1.table1", "schema2.table2"],"type":"incremental"}');
The values of the
id,type, anddataparameters in the command correspond to the fields of the signaling table.The following table describes these parameters:
Table 6. Descriptions of fields in a SQL command for sending a stop incremental snapshot signal to the signaling table Value Description myschema.debezium_signalSpecifies the fully-qualified name of the signaling table on the source database
ad-hoc-1The
idparameter specifies an arbitrary string that is assigned as theididentifier for the signal request.
Use this string to identify logging messages to entries in the signaling table. Debezium does not use this string.stop-snapshotSpecifies
typeparameter specifies the operation that the signal is intended to trigger.data-collectionsAn optional component of the
datafield of a signal that specifies an array of table names or regular expressions to match table names to remove from the snapshot.
The array lists regular expressions which match tables by their fully-qualified names, using the same format as you use to specify the name of the connector’s signaling table in the signal.data.collection configuration property. If this component of thedatafield is omitted, the signal stops the entire incremental snapshot that is in progress.incrementalA required component of the
datafield of a signal that specifies the kind of snapshot operation that is to be stopped.
Currently, the only valid option isincremental.
Specifying atypevalue in the SQL query that you submit to the signaling table is required.
If you do not specify a value, the signal will not stop the incremental snapshot.
Read-only incremental snapshots
The 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_workersis greater than0) you must set one of the following options:replica_preserve_commit_order=ONslave_preserve_commit_order=ON
Ad hoc read-only incremental snapshots
When the MySQL connection is read-only, the signaling table mechanism can also run a snapshot by sending a message to the Kafka topic that is specified in the signal.kafka.topic property.
The key of the Kafka message must match the value of the topic.prefix connector configuration option.
The value 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 only |
| N/A | An array of comma-separated regular expressions that match fully-qualified names of tables to be snapshotted. |
| Optional.Empty | An optional string, which specifies a condition based on the column(s) of the table(s), to capture a subset of the contents of the table(s). |
An example of the execute-snapshot Kafka message:
Key = `test_connector`Value = `{"type":"execute-snapshot","data": {"data-collections": ["schema1.table1", "schema1.table2"], "type": "INCREMENTAL"}}`
Ad hoc read-only incremental snapshots with additional-condition
additional-conditionis used to select a subset of a table’s content.To give an analogy how
additional-conditionis used:For a snapshot, the SQL query executed behind the scenes is something like:
SELECT * FROM <tableName> ….
For a snapshot with a
additional-condition, theadditional-conditionis appended to the SQL query, something like:SELECT * FROM <tableName> WHERE <additional-condition> ….
Suppose there is a
productstable with columnsid(primary key),colorandbrand.To snapshot just the content of the
productstable wherecolor=blueKey = `test_connector`Value = `{"type":"execute-snapshot","data": {"data-collections": ["schema1.products"], "type": "INCREMENTAL", "additional-condition":"color=blue"}}`
additional-conditioncan be used to pass condition based on multiple columns. Using the sameproductstable, to snapshot content of theproductstable wherecolor=blueandbrand=fooKey = `test_connector`Value = `{"type":"execute-snapshot","data": {"data-collections": ["schema1.products"], "type": "INCREMENTAL", "additional-condition":"color=blue AND brand=foo"}}`
Stopping an Ad hoc read-only incremental snapshot
When the MySQL connection is read-only, the signaling table mechanism can also stop a snapshot by sending a message to the Kafka topic that is specified in the signal.kafka.topic property.
The key of the Kafka message must match the value of the topic.prefix connector configuration option.
The value 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 only |
| N/A | An optional array of comma-separated regular expressions that match fully-qualified names of tables to be snapshotted. |
An example of the stop-snapshot Kafka message:
Key = `test_connector`Value = `{"type":"stop-snapshot","data": {"data-collections": ["schema1.table1", "schema1.table2"], "type": "INCREMENTAL"}}`
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
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.ordersfulfillment.inventory.customersfulfillment.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","transaction": {"id": "0e4d5dcd-a33b-11ea-80f1-02010a22a99e:10","total_order": "1","data_collection_order": "1"}}
For systems which don’t have GTID enabled, the transaction identifier is constructed using the combination of binlog filename and binlog position. For example, if the binlog filename and position corresponding to the transaction BEGIN event are mysql-bin.000002 and 1913 respectively then the Debezium constructed transaction identifier would be file=mysql-bin.000002,pos=1913.
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": "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"}],"optional": false,"name": "mysql-server-1.inventory.customers.Envelope" (4)},"payload": { (5)"op": "c", (6)"ts_ms": 1465491411815, (7)"before": null, (8)"after": { (9)"id": 1004,"first_name": "Anne","last_name": "Kretchmar","email": "annek@noanswer.org"},"source": { (10)"version": "2.0.0.Final","connector": "mysql","name": "mysql-server-1","ts_ms": 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:
If the binlog_rows_query_log_events MySQL configuration option is enabled and the connector configuration |
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.0.0.Final","name": "mysql-server-1","connector": "mysql","name": "mysql-server-1","ts_ms": 1465581029100,"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)}}
| 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
If the binlog_rows_query_log_events MySQL configuration option is enabled and the connector configuration |
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. |
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
DELETEevent record has__debezium.newkeyas a message header. The value of this header is the new primary key for the updated row.The
CREATEevent record has__debezium.oldkeyas 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.0.0.Final","connector": "mysql","name": "mysql-server-1","ts_ms": 1465581902300,"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)}}
| 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
If the binlog_rows_query_log_events MySQL configuration option is enabled and the connector configuration |
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. |
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 Debezium’s MySQL connector emits a delete event, the connector emits a special tombstone event that has the same key but a null value.
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 |
|
| As of MySQL 8.0.17, the nonstandard FLOAT(M,D) and DOUBLE(M,D) syntax is deprecated, and should expect support for it be removed in a future version of MySQL, set |
|
| 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:
DATETIMEwith a value of2018-06-20 06:37:03becomes1529476623000.TIMESTAMPwith a value of2018-06-20 06:37:03becomes2018-06-20T13:37:03Z.
Such columns are converted into an equivalent io.debezium.time.