Debezium connector for MySQL

MySQL has a binary log (binlog) that records all operations in the order in which they are committed to the database. This includes changes to table schemas as well as changes to the data in tables. MySQL uses the binlog for replication and recovery.

The Debezium MySQL connector reads the binlog, produces change events for row-level INSERT , UPDATE , and DELETE operations, and emits the change events to Kafka topics. Client applications read those Kafka topics.

Because MySQL is typically set up to purge binlogs after a specified period of time, the MySQL connector performs an initial consistent snapshot of each of your databases. The MySQL connector reads the binlog from the point at which the snapshot was made.

For information about the MySQL Database versions that are compatible with this connector, see the Debezium release overview.

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.

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.

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

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 , 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:

The name of the schema change event message.

The type of the change event message.

The version of the schema. The version is an integer that is incremented each time the schema is changed.

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:

Provides the SQL CREATE , ALTER , or DROP statement that results in the schema change.

The name of the database to which the DDL statements are applied. The value of databaseName serves as the message key.

The position in the binlog where the statements appear.

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": (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) (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" >] > > ] > >

The source field is structured exactly as standard data change events that the connector writes to table-specific topics. This field is useful to correlate events on different topics.

ts_ms , ts_us , ts_ns

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.

Identifies the database and the schema that contains the change. The value of the databaseName field is used as the message key for the record.

This field contains the DDL that is responsible for the schema change. The ddl field can contain multiple DDL statements. Each statement applies to the database in the databaseName field. Multiple DDL statements appear in the order in which they were applied to the database.

Clients can submit multiple DDL statements that apply to multiple databases. If MySQL applies them atomically, the connector takes the DDL statements in order, groups them by database, and creates a schema change event for each group. If MySQL applies them individually, the connector creates a separate schema change event for each statement.

An array of one or more items that contain the schema changes generated by a DDL command.

Describes the kind of change. The value is one of the following:

Full identifier of the table that was created, altered, or dropped. In the case of a table rename, this identifier is a concatenation of , table names.

Represents table metadata after the applied change.

List of columns that compose the table’s primary key.

Metadata for each column in the changed table.

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.

Establish a connection to the database.

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 .

Obtain a global read lock on the tables to be captured to block writes by other database clients.

The snapshot itself does not prevent other clients from applying DDL that might interfere with the connector’s attempt to read the binlog position and table schemas. The connector retains the global read lock while it reads the binlog position, and releases the lock as described in a later step.

Start a transaction with repeatable read semantics to ensure that all subsequent reads within the transaction are done against the consistent snapshot.

The use of these isolation semantics can slow the progress of the snapshot. If the snapshot takes too long to complete, consider using a different isolation configuration, or skip the initial snapshot and run an incremental snapshot instead.

Read the current binlog position.

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 DROP…​ and CREATE…​ DDL statements.
The schema history provides information about the structure that is in effect when a change event occurs.

By default, the connector captures the schema of every table in the database, including tables that are not configured for capture. If tables are not configured for capture, the initial snapshot captures only their structure; it does not capture any table data.

For more information about why snapshots persist schema information for tables that you did not include in the initial snapshot, see Understanding why initial snapshots capture the schema for all tables.

Release the global read lock obtained in Step 3. Other database clients can now write to the database.

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:

  1. Confirms that the table was created before the snapshot began. If the table was created after the snapshot began, the connector skips the table. After the snapshot is complete, and the connector transitions to streaming, it emits change events for any tables that were created after the snapshot began.
  2. Produces a read event for each row that is captured from a table. All read events contain the same binlog position, which is the position that was obtained in step 5.
  3. Emits each read event to the Kafka topic for the source table.
  4. Releases data table locks, if applicable.

Commit the transaction.

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.

Establish a connection to the database.

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 include and exclude properties to filter the data, for example, table.include.list or table.exclude.list .

Obtain table-level locks.

Start a transaction with repeatable read semantics to ensure that all subsequent reads within the transaction are done against the consistent snapshot.

Read the current binlog position.

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 DROP…​ and CREATE…​ DDL statements.
The schema history provides information about the structure that is in effect when a change event occurs.

By default, the connector captures the schema of every table in the database, including tables that are not configured for capture. If tables are not configured for capture, the initial snapshot captures only their structure; it does not capture any table data.

For more information about why snapshots persist schema information for tables that you did not include in the initial snapshot, see Understanding why initial snapshots capture the schema for all tables.

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:

  1. Confirms that the table was created before the snapshot began. If the table was created after the snapshot began, the connector skips the table. After the snapshot is complete, and the connector transitions to streaming, it emits change events for any tables that were created after the snapshot began.
  2. Produces a read event for each row that is captured from a table. All read events contain the same binlog position, which is the position that was obtained in step 5.
  3. Emits each read event to the Kafka topic for the source table.
  4. Releases data table locks, if applicable.

Commit the transaction.

Release the table-level locks. Other database clients can now write to any previously locked tables.

Record the successful completion of the snapshot in the connector offsets.

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 no_data .

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 READ events to represent the data set at the point of the connector’s start-up (Step 7.2).

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.

Deprecated, see 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.

WARNING: Do not use this mode to perform a snapshot if schema changes were committed to the database after the last connector shutdown.

After the connector starts, it performs a snapshot only if it detects one of the following circumstances:

Set the snapshot mode to configuration_based to control snapshot behavior through the set of connector properties that have the prefix 'snapshot.mode.configuration.based'.

The custom snapshot mode lets you inject your own implementation of the io.debezium.spi.snapshot.Snapshotter interface. Set the snapshot.mode.custom.name configuration property to the name provided by the name() method of your implementation. The name is specified on the classpath of your Kafka Connect cluster. If you use the EmbeddedEngine , the name is included in the connector JAR file. For more information, see custom snapshotter SPI.

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.

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)

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
  1. Stop the connector.
  2. Remove the internal database schema history topic that is specified by the schema.history.internal.kafka.topic property .
  3. Apply the following changes to the connector configuration:
  1. Set the snapshot.mode to schema_only_recovery .
  2. Set the value of schema.history.internal.store.only.captured.tables.ddl to false .
  3. 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.

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

Initial snapshot captured the schema for all tables ( store.only.captured.tables.ddl was set to false )

  1. Edit the table.include.list property to specify the tables that you want to capture.
  2. Restart the connector.
  3. 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.

  1. Stop the connector.
  2. Remove the internal database schema history topic that is specified by the schema.history.internal.kafka.topic property .
  3. 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.

  1. Set the value of the snapshot.mode property to schema_only .
  2. Edit the table.include.list to add the tables that you want to capture.
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.

  1. Stop the connector.
  2. Remove the internal database schema history topic that is specified by the schema.history.internal.kafka.topic property .
  3. 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.

  1. Set the value of the snapshot.mode property to initial .
  2. (Optional) Set schema.history.internal.store.only.captured.tables.ddl to false .

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:

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:

Specifies the type of snapshot that you want to run.
Currently, you can request incremental or blocking snapshots.

An array that contains regular expressions matching the fully-qualified names of the tables to include in the snapshot.
For the MySQL connector, use the following format to specify the fully qualified name of a table: database.table .

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.
Each additional condition is an object that specifies the criteria for filtering the data that an ad hoc snapshot captures. You can set the following parameters for each additional condition:

data-collection

The fully-qualified name of the table that the filter applies to. You can apply different filters to each table.

Specifies column values that must be present in a database record for the snapshot to include it, for example, "color='blue'" .

The values that you assign to the filter parameter are the same types of values that you might specify in the WHERE clause of SELECT statements when you set the snapshot.select.statement.overrides property for a blocking snapshot.

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.

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:

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,

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 ( . ), a space, or some other non-alphanumeric character, you must escape the table name in double quotes.
For example, to include a table that exists in the db1 database, and that has the name My.Table , use the following format: "db1.\"My.Table\"" .

Prerequisites Using a source signaling channel to trigger an incremental snapshot
  1. Send a SQL query to add the ad hoc incremental snapshot request to the signaling table:
INSERT INTO (id, type, data) VALUES ('', '', ' "," "],"type":" ","additional-conditions":[ ", "filter": " ">]>');
For example,
INSERT INTO db1.debezium_signal (id, type, data) (1) values ('ad-hoc-1', (2) 'execute-snapshot', (3) ' (4) "type":"incremental", (5) "additional-conditions":[]>'); (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

Specifies the fully-qualified name of the signaling table on the source database.

The id parameter specifies an arbitrary string that is assigned as the id identifier 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 own id string as a watermarking signal.

The type parameter specifies the operation that the signal is intended to trigger.

A required component of the data field 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 that use the format database.table to match the fully-qualified names of the tables. This format is the same as the one that you use to specify the name of the connector’s signaling table.

An optional type component of the data field of a signal that specifies the type of snapshot operation to run.
Valid values are incremental and blocking .
If you do not specify a value, the connector defaults to performing an incremental snapshot.

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.
Each additional condition is an object with data-collection and filter properties. You can specify different filters for each data collection.
* The data-collection property is the fully-qualified name of the data collection that the filter applies to. For more information about the additional-conditions parameter, see Running an ad hoc incremental snapshots with additional-conditions .

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 . 

By adding an additional-conditions parameter, you append a WHERE condition to the SQL query, as in the following example:

SELECT * FROM WHERE . 

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 (id, type, data) VALUES ('', '', ' "," "],"type":" ","additional-conditions":[ ", "filter": " ">]>');

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', ']>');

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', '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": (1) >, "op":"r", (2) "ts_ms":"1620393591654", "ts_us":"1620393591654547", "ts_ns":"1620393591654547920", "transaction":null >

Specifies the type of snapshot operation to run.
Currently, the only valid options are blocking and incremental .
Specifying a type value 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.

Specifies the event type.
The value for snapshot events is r , signifying a READ operation.

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:

The type of the snapshot to be executed. Currently Debezium supports the incremental and blocking types.
See the next section for more details.

An array of comma-separated regular expressions that match the fully-qualified names of tables to include in the snapshot.
Specify the names by using the same format as is required for the signal.data.collection configuration option.

An optional array of additional conditions that specifies criteria that the connector evaluates to designate a subset of records to include in a snapshot.
Each additional condition is an object that specifies the criteria for filtering the data that an ad hoc snapshot captures. You can set the following parameters for each additional condition: data-collection :: The fully-qualified name of the table that the filter applies to. You can apply different filters to each table. filter :: Specifies column values that must be present in a database record for the snapshot to include it, for example, "color='blue'" .

The values that you assign to the filter parameter are the same types of values that you might specify in the WHERE clause of SELECT statements when you set the snapshot.select.statement.overrides property for a blocking snapshot.

Example 2. An execute-snapshot Kafka message
Key = `test_connector` Value = `.table1", ".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:

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 WHERE …​.

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 = `]>>`

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 = `]>>`

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
  1. Send a SQL query to stop the ad hoc incremental snapshot to the signaling table:
INSERT INTO (id, type, data) values ('', 'stop-snapshot', ' "," "],"type":"incremental">');
For example,
INSERT INTO db1.debezium_signal (id, type, data) (1) values ('ad-hoc-1', (2) 'stop-snapshot', (3) ' (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

Specifies the fully-qualified name of the signaling table on the source database.

The id parameter specifies an arbitrary string that is assigned as the id identifier for the signal request.
Use this string to identify logging messages to entries in the signaling table. Debezium does not use this string.

Specifies type parameter specifies the operation that the signal is intended to trigger.

An optional component of the data field 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 in the format database.table

If you omit this component from the data field, the signal stops the entire incremental snapshot that is in progress.

A required component of the data field of a signal that specifies the type of snapshot operation that is to be stopped.
Currently, the only valid option is incremental .
If you do not specify a type value, the signal fails to stop the incremental snapshot.

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:

The type of the snapshot to be executed. Currently Debezium supports only the incremental type.
See the next section for more details.

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.
Specify table names by using the format database.table .

The following example shows a typical stop-snapshot Kafka message: :leveloffset: +1

Key = `test_connector` Value = `>`

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
  • Enable MySQL GTIDs.
  • If the connector reads from a multi-threaded replica (that is, a replica for which the value of replica_parallel_workers is greater than 0 ) 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.

Controls how data is queried during a snapshot.

Controls whether the connector locks tables when taking a snapshot.

io.debezium.snapshot.spi.Snapshotter interface. All built-in snapshot modes implement this interface.

/** * is used to determine the following details about the snapshot process: * 

* - Whether a snapshot occurs.
* - Whether streaming continues during the snapshot.
* - Whether the snapshot includes schema (if supported).
* - Whether to snapshot data or schema following an error. *

* 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 when the connector has an offset context (i.e. restarted) * @param snapshotInProgress is when the connector is started, but a snapshot is already in progress * * @return if the snapshotter should take a data snapshot */ boolean shouldSnapshotData(boolean offsetExists, boolean snapshotInProgress); /** * @param offsetExists is when the connector has an offset context (i.e. restarted) * @param snapshotInProgress is when the connector is started, but a snapshot is already in progress * * @return if the snapshotter should take a schema snapshot */ boolean shouldSnapshotSchema(boolean offsetExists, boolean snapshotInProgress); /** * @return if the snapshotter should stream after taking a snapshot */ boolean shouldStream(); /** * @return whether the schema can be recovered if database schema history is corrupted. */ boolean shouldSnapshotOnSchemaError(); /** * @return whether the snapshot should be re-executed when there is a gap in data stream. */ boolean shouldSnapshotOnDataError(); /** * * @return if streaming should resume from the start of the snapshot * transaction, or 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.

/** * 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 * 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 snapshotQuery(String tableId, List snapshotSelectColumns); >

io.debezium.snapshot.spi.SnapshotLock interface. All built-in snapshot lock modes implement this interface.

/** * 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. */ OptionaltableLockingStatement(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 the snapshot.select.statement.overrides
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:

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.

The name of the schema in which the operation occurred.

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:

String representation of the unique transaction identifier.

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.

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:

String representation of unique transaction identifier.

The absolute position of the event among all events generated by the transaction.

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:

 (1) . >, "payload": (2) . >, "schema": (3) . >, "payload": (4) . >, >

The first schema field is part of the event key. It specifies a Kafka Connect schema that describes what is in the event key’s payload portion. In other words, the first schema field describes the structure of the primary key, or the unique key if the table does not have a primary key, for the table that was changed.

It is possible to override the table’s primary key by setting the message.key.columns connector configuration property. In this case, the first schema field describes the structure of the key identified by that property.

The first payload field is part of the event key. It has the structure described by the previous schema field, and it contains the key for the row that was changed.

The second schema field is part of the event value. It specifies the Kafka Connect schema that describes what is in the event value’s payload portion. In other words, the second schema describes the structure of the row that was changed. Typically, this schema contains nested schemas.

The second payload field is part of the event value. It has the structure described by the previous schema field and it contains the actual data for the row that was changed.

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:

 (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 > >

The schema portion of the key specifies a Kafka Connect schema that describes what is in the key’s payload portion.

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. Key . In this example:

  • mysql-server-1 is the name of the connector that generated this event.
  • inventory is the database that contains the table that was changed.
  • customers is the table that was updated.

Indicates whether the event key must contain a value in its payload field. In this example, a value in the key’s payload is required. A value in the key’s payload field is optional when a table does not have a primary key.

Specifies each field that is expected in the payload , including each field’s name, type, and whether it is required.

Contains the key for the row for which this change event was generated. In this example, the key, contains a single id field whose value is 1001 .

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:

 (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')" > > >

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.

In the schema section, each name field specifies the schema for a field in the value’s payload.

mysql-server-1.inventory.customers.Value is the schema for the payload’s before and after fields. This schema is specific to the customers table.

Names of schemas for before and after fields are of the form logicalName.tableName.Value , which ensures that the schema name is unique in the database. This means that when using the Avro converter, the resulting Avro schema for each table in each logical source has its own evolution and history.

io.debezium.connector.mysql.Source is the schema for the payload’s source field. This schema is specific to the MySQL connector. The connector uses it for all events that it generates.

mysql-server-1.inventory.customers.Envelope is the schema for the overall structure of the payload, where mysql-server-1 is the connector name, inventory is the database, and customers is the table.

The value’s actual data. This is the information that the change event is providing.

It may appear that the JSON representations of the events are much larger than the rows they describe. This is because the JSON representation must include the schema and the payload portions of the message. However, by using the Avro converter, you can significantly decrease the size of the messages that the connector streams to Kafka topics.

Mandatory string that describes the type of operation that caused the connector to generate the event. In this example, c indicates that the operation created a row. Valid values are:

  • c = create
  • u = update
  • d = delete
  • r = read (applies to only snapshots)

ts_ms , ts_us , ts_ns

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.

An optional field that specifies the state of the row before the event occurred. When the op field is c for create, as it is in this example, the before field is null since this change event is for new content.

An optional field that specifies the state of the row after the event occurred. In this example, the after field contains the values of the new row’s id , first_name , last_name , and email columns.

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:

  • Debezium version
  • Connector name
  • binlog name where the event was recorded
  • binlog position
  • Row within the event
  • If the event was part of a snapshot
  • Name of the database and table that contain the new row
  • ID of the MySQL thread that created the event (non-snapshot only)
  • MySQL server ID (if available)
  • Timestamp for when the change was made in the database

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": (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 >, "op": "u", (4) "ts_ms": 1465581029523, (5) "ts_ms": 1465581029523758, (6) "ts_ms": 1465581029523758914 (7) > >

An optional field that specifies the state of the row before the event occurred. In an update event value, the before field contains a field for each table column and the value that was in that column before the database commit. In this example, the first_name value is Anne.

An optional field that specifies the state of the row after the event occurred. You can compare the before and after structures to determine what the update to this row was. In the example, the first_name value is now Anne Marie .

Mandatory field that describes the source metadata for the event. The source field structure has the same fields as in a create event, but some values are different, for example, the sample update event is from a different position in the binlog. The source metadata includes:

  • Debezium version
  • Connector name
  • binlog name where the event was recorded
  • binlog position
  • Row within the event
  • If the event was part of a snapshot
  • Name of the database and table that contain the updated row
  • ID of the MySQL thread that created the event (non-snapshot only)
  • MySQL server ID (if available)
  • Timestamp for when the change was made in the database

Mandatory string that describes the type of operation. In an update event value, the op field value is u , signifying that this row changed because of an update.

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.

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.

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 DELETE event and a tombstone event with the old key for the row, followed by an event with the new key for the row. Details are in the next section.

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": (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 >, "op": "d", (4) "ts_ms": 1465581902461, (5) "ts_us": 1465581902461842, (6) "ts_ns": 1465581902461842579 (7) > >

Optional field that specifies the state of the row before the event occurred. In a delete event value, the before field contains the values that were in the row before it was deleted with the database commit.

Optional field that specifies the state of the row after the event occurred. In a delete event value, the after field is null , signifying that the row no longer exists.

Mandatory field that describes the source metadata for the event. In a delete event value, the source field structure is the same as for create and update events for the same table. Many source field values are also the same. In a delete event value, the ts_ms and pos field values, as well as other values, might have changed. But the source field in a delete event value provides the same metadata:

  • Debezium version
  • Connector name
  • binlog name where the event was recorded
  • binlog position
  • Row within the event
  • If the event was part of a snapshot
  • Name of the database and table that contain the updated row
  • ID of the MySQL thread that created the event (non-snapshot only)
  • MySQL server ID (if available)
  • Timestamp for when the change was made in the database

Mandatory string that describes the type of operation. The op field value is d , signifying that this row was deleted.

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.

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.

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": (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 >, "op": "t", (2) "ts_ms": 1465581029523, (3) "ts_us": 1465581029523468, (4) "ts_ns": 1465581029523468471 (5) > >

Mandatory field that describes the source metadata for the event. In a truncate event value, the source field structure is the same as for create, update, and delete events for the same table, provides this metadata:

  • Debezium version
  • Connector type and name
  • Binlog name where the event was recorded
  • Binlog position
  • Row within the event
  • If the event was part of a snapshot
  • Name of the database and table
  • ID of the MySQL thread that truncated the event (non-snapshot only)
  • MySQL server ID (if available)
  • Timestamp for when the change was made in the database

Mandatory string that describes the type of operation. The op field value is t , signifying that this table was truncated.

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.

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.

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.