Debezium connector for MySQL

How the connector works

An overview of the MySQL topologies that the connector supports is useful for planning your application. To optimally configure and run a Debezium MySQL connector, it is helpful to understand how the connector tracks the structure of tables, exposes schema changes, performs snapshots, and determines Kafka topic names.

Supported MySQL topologies

The Debezium MySQL connector supports the following MySQL topologies:

Standalone

When a single MySQL server is used, the server must have the binlog enabled (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 and may have been recorded before the tables’ schemas were changed.

To handle this, 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 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 history Kafka topic and parsing all DDL statements up to the point in the binlog where the connector is starting.

This database history topic is for connector use only. The connector can optionally See 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 include all DDL statements applied to databases in the MySQL server. The connector emits these events to a Kafka topic named serverName where serverName is the name of the connector as specified by the database.server.name connector configuration property.

If you choose to use schema change events, ensure that you consume records from the schema change topic. The database history topic is for connector use only.

A global order for events emitted to the schema change topic is vital. Therefore, you must not partition the database history topic. This means that you must specify a partition count of 1 when creating the database history topic. When relying on auto topic creation, make sure that Kafka’s num.partitions configuration option, which specifies the default number of partitions, is set to 1.

Each record that the connector emits to the schema change topic contains a message key that includes the name of the connected database when the DDL statement was applied, for example:

  1. {
  2. "schema": {
  3. "type": "struct",
  4. "name": "io.debezium.connector.mysql.SchemaChangeKey",
  5. "optional": false,
  6. "fields": [
  7. {
  8. "field": "databaseName",
  9. "type": "string",
  10. "optional": false
  11. }
  12. ]
  13. },
  14. "payload": {
  15. "databaseName": "inventory"
  16. }
  17. }

The schema change event record value contains a structure that includes the DDL statements, the name of the database to which the statements were applied, and the position in the binlog where the statements appeared, for example:

  1. {
  2. "schema": {
  3. "type": "struct",
  4. "name": "io.debezium.connector.mysql.SchemaChangeValue",
  5. "optional": false,
  6. "fields": [
  7. {
  8. "field": "databaseName",
  9. "type": "string",
  10. "optional": false
  11. },
  12. {
  13. "field": "ddl",
  14. "type": "string",
  15. "optional": false
  16. },
  17. {
  18. "field": "source",
  19. "type": "struct",
  20. "name": "io.debezium.connector.mysql.Source",
  21. "optional": false,
  22. "fields": [
  23. {
  24. "type": "string",
  25. "optional": true,
  26. "field": "version"
  27. },
  28. {
  29. "type": "string",
  30. "optional": false,
  31. "field": "name"
  32. },
  33. {
  34. "type": "int64",
  35. "optional": false,
  36. "field": "server_id"
  37. },
  38. {
  39. "type": "int64",
  40. "optional": false,
  41. "field": "ts_ms"
  42. },
  43. {
  44. "type": "string",
  45. "optional": true,
  46. "field": "gtid"
  47. },
  48. {
  49. "type": "string",
  50. "optional": false,
  51. "field": "file"
  52. },
  53. {
  54. "type": "int64",
  55. "optional": false,
  56. "field": "pos"
  57. },
  58. {
  59. "type": "int32",
  60. "optional": false,
  61. "field": "row"
  62. },
  63. {
  64. "type": "boolean",
  65. "optional": true,
  66. "default": false,
  67. "field": "snapshot"
  68. },
  69. {
  70. "type": "int64",
  71. "optional": true,
  72. "field": "thread"
  73. },
  74. {
  75. "type": "string",
  76. "optional": true,
  77. "field": "db"
  78. },
  79. {
  80. "type": "string",
  81. "optional": true,
  82. "field": "table"
  83. },
  84. {
  85. "type": "string",
  86. "optional": true,
  87. "field": "query"
  88. }
  89. ]
  90. }
  91. ]
  92. },
  93. "payload": {
  94. "databaseName": "inventory",
  95. "ddl": "CREATE TABLE products ( id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, name VARCHAR(255) NOT NULL, description VARCHAR(512), weight FLOAT ); ALTER TABLE products AUTO_INCREMENT = 101;",
  96. "source" : {
  97. "version": "1.4.2.Final",
  98. "name": "mysql-server-1",
  99. "server_id": 0,
  100. "ts_ms": 0,
  101. "gtid": null,
  102. "file": "mysql-bin.000003",
  103. "pos": 154,
  104. "row": 0,
  105. "snapshot": true,
  106. "thread": null,
  107. "db": null,
  108. "table": null,
  109. "query": null
  110. }
  111. }
  112. }

The ddl field might contain multiple DDL statements. Each statement applies to the database in the databaseName field. The statements appear in the order in which they were applied to the database. The source field is structured exactly as a standard data change event written to table-specific topics. This field is useful to correlate events on different topics.

  1. ....
  2. "payload": {
  3. "databaseName": "inventory",
  4. "ddl": "CREATE TABLE products ( id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,...)",
  5. "source" : {
  6. ...
  7. }
  8. }
  9. ....

A client can submit multiple DDL statements to be applied 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.

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.

Table 1. Workflow for performing an initial snapshot with a global read lock
StepAction

1

Grabs a global read lock that blocks 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 keeps the global read lock while it reads the binlog position, and releases the lock as described in a later step.

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 DROP…​ and CREATE…​ DDL statements.

7

Scans the database tables. For each row, the connector emits CREATE events to the relevant table-specific Kafka topics.

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.

Table 2. Workflow for performing an initial snapshot with table-level locks
StepAction

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 DROP…​ and CREATE…​ DDL statements.

7

Scans the database tables. For each row, the connector emits CREATE events to the relevant table-specific Kafka topics.

8

Commits the transaction.

9

Releases the table-level locks.

10

Records the completed snapshot in the connector offsets.

Operation type for snapshot events

The MySql connector emits snapshot events using the “r” operation type (READ). In case you want the connector to emit snapshot events as “c” events (CREATE, as done incorrectly in earlier versions), this can be achieved using a Simple Message Transforms (SMT). Configure the Debezium ReadToInsertEvent SMT by adding the SMT configuration details to your connector’s configuration.

An example of the configuration is this:

  1. transforms=snapshotasinsert,...
  2. transforms.snapshotasinsert.type=io.debezium.connector.mysql.transforms.ReadToInsertEvent

Topic names

The default behavior is that a Debezium MySQL connector writes events for all INSERT, UPDATE, and DELETE operations in one table to one Kafka topic. The Kafka topic naming convention is as follows:

serverName.databaseName.tableName

Suppose that fulfillment is the server name, 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:

  1. fulfillment.inventory.orders
  2. fulfillment.inventory.customers
  3. fulfillment.inventory.products

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. {
  2. "schema": { (1)
  3. ...
  4. },
  5. "payload": { (2)
  6. ...
  7. },
  8. "schema": { (3)
  9. ...
  10. },
  11. "payload": { (4)
  12. ...
  13. },
  14. }
Table 3. Overview of change event basic content
ItemField nameDescription

1

schema

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.

2

payload

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.

3

schema

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.

4

payload

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.

  1. CREATE TABLE customers (
  2. id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
  3. first_name VARCHAR(255) NOT NULL,
  4. last_name VARCHAR(255) NOT NULL,
  5. email VARCHAR(255) NOT NULL UNIQUE KEY
  6. ) 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. {
  2. "schema": { (1)
  3. "type": "struct",
  4. "name": "mysql-server-1.inventory.customers.Key", (2)
  5. "optional": false, (3)
  6. "fields": [ (4)
  7. {
  8. "field": "id",
  9. "type": "int32",
  10. "optional": false
  11. }
  12. ]
  13. },
  14. "payload": { (5)
  15. "id": 1001
  16. }
  17. }
Table 4. Description of change event key
ItemField nameDescription

1

schema

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

2

mysql-server-1.inventory.customers.Key

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.

3

optional

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.

4

fields

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

5

payload

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:

  1. CREATE TABLE customers (
  2. id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
  3. first_name VARCHAR(255) NOT NULL,
  4. last_name VARCHAR(255) NOT NULL,
  5. email VARCHAR(255) NOT NULL UNIQUE KEY
  6. ) 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. {
  2. "schema": { (1)
  3. "type": "struct",
  4. "fields": [
  5. {
  6. "type": "struct",
  7. "fields": [
  8. {
  9. "type": "int32",
  10. "optional": false,
  11. "field": "id"
  12. },
  13. {
  14. "type": "string",
  15. "optional": false,
  16. "field": "first_name"
  17. },
  18. {
  19. "type": "string",
  20. "optional": false,
  21. "field": "last_name"
  22. },
  23. {
  24. "type": "string",
  25. "optional": false,
  26. "field": "email"
  27. }
  28. ],
  29. "optional": true,
  30. "name": "mysql-server-1.inventory.customers.Value", (2)
  31. "field": "before"
  32. },
  33. {
  34. "type": "struct",
  35. "fields": [
  36. {
  37. "type": "int32",
  38. "optional": false,
  39. "field": "id"
  40. },
  41. {
  42. "type": "string",
  43. "optional": false,
  44. "field": "first_name"
  45. },
  46. {
  47. "type": "string",
  48. "optional": false,
  49. "field": "last_name"
  50. },
  51. {
  52. "type": "string",
  53. "optional": false,
  54. "field": "email"
  55. }
  56. ],
  57. "optional": true,
  58. "name": "mysql-server-1.inventory.customers.Value",
  59. "field": "after"
  60. },
  61. {
  62. "type": "struct",
  63. "fields": [
  64. {
  65. "type": "string",
  66. "optional": false,
  67. "field": "version"
  68. },
  69. {
  70. "type": "string",
  71. "optional": false,
  72. "field": "connector"
  73. },
  74. {
  75. "type": "string",
  76. "optional": false,
  77. "field": "name"
  78. },
  79. {
  80. "type": "int64",
  81. "optional": false,
  82. "field": "ts_ms"
  83. },
  84. {
  85. "type": "boolean",
  86. "optional": true,
  87. "default": false,
  88. "field": "snapshot"
  89. },
  90. {
  91. "type": "string",
  92. "optional": false,
  93. "field": "db"
  94. },
  95. {
  96. "type": "string",
  97. "optional": true,
  98. "field": "table"
  99. },
  100. {
  101. "type": "int64",
  102. "optional": false,
  103. "field": "server_id"
  104. },
  105. {
  106. "type": "string",
  107. "optional": true,
  108. "field": "gtid"
  109. },
  110. {
  111. "type": "string",
  112. "optional": false,
  113. "field": "file"
  114. },
  115. {
  116. "type": "int64",
  117. "optional": false,
  118. "field": "pos"
  119. },
  120. {
  121. "type": "int32",
  122. "optional": false,
  123. "field": "row"
  124. },
  125. {
  126. "type": "int64",
  127. "optional": true,
  128. "field": "thread"
  129. },
  130. {
  131. "type": "string",
  132. "optional": true,
  133. "field": "query"
  134. }
  135. ],
  136. "optional": false,
  137. "name": "io.debezium.connector.mysql.Source", (3)
  138. "field": "source"
  139. },
  140. {
  141. "type": "string",
  142. "optional": false,
  143. "field": "op"
  144. },
  145. {
  146. "type": "int64",
  147. "optional": true,
  148. "field": "ts_ms"
  149. }
  150. ],
  151. "optional": false,
  152. "name": "mysql-server-1.inventory.customers.Envelope" (4)
  153. },
  154. "payload": { (5)
  155. "op": "c", (6)
  156. "ts_ms": 1465491411815, (7)
  157. "before": null, (8)
  158. "after": { (9)
  159. "id": 1004,
  160. "first_name": "Anne",
  161. "last_name": "Kretchmar",
  162. "email": "annek@noanswer.org"
  163. },
  164. "source": { (10)
  165. "version": "1.4.2.Final",
  166. "connector": "mysql",
  167. "name": "mysql-server-1",
  168. "ts_ms": 0,
  169. "snapshot": false,
  170. "db": "inventory",
  171. "table": "customers",
  172. "server_id": 0,
  173. "gtid": null,
  174. "file": "mysql-bin.000003",
  175. "pos": 154,
  176. "row": 0,
  177. "thread": 7,
  178. "query": "INSERT INTO customers (first_name, last_name, email) VALUES ('Anne', 'Kretchmar', 'annek@noanswer.org')"
  179. }
  180. }
  181. }
Table 5. Descriptions of create event value fields
ItemField nameDescription

1

schema

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

name

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.

3

name

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.

4

name

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.

5

payload

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.

6

op

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)

7

ts_ms

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.

8

before

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.

9

after

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.

10

source

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

If the binlog_rows_query_log_events MySQL configuration option is enabled and the connector configuration include.query property is enabled, the source field also provides the query field, which contains the original SQL statement that caused the change event.

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:

  1. {
  2. "schema": { ... },
  3. "payload": {
  4. "before": { (1)
  5. "id": 1004,
  6. "first_name": "Anne",
  7. "last_name": "Kretchmar",
  8. "email": "annek@noanswer.org"
  9. },
  10. "after": { (2)
  11. "id": 1004,
  12. "first_name": "Anne Marie",
  13. "last_name": "Kretchmar",
  14. "email": "annek@noanswer.org"
  15. },
  16. "source": { (3)
  17. "version": "1.4.2.Final",
  18. "name": "mysql-server-1",
  19. "connector": "mysql",
  20. "name": "mysql-server-1",
  21. "ts_ms": 1465581029100,
  22. "snapshot": false,
  23. "db": "inventory",
  24. "table": "customers",
  25. "server_id": 223344,
  26. "gtid": null,
  27. "file": "mysql-bin.000003",
  28. "pos": 484,
  29. "row": 0,
  30. "thread": 7,
  31. "query": "UPDATE customers SET first_name='Anne Marie' WHERE id=1004"
  32. },
  33. "op": "u", (4)
  34. "ts_ms": 1465581029523 (5)
  35. }
  36. }
Table 6. Descriptions of update event value fields
ItemField nameDescription

1

before

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.

2

after

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.

3

source

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

If the binlog_rows_query_log_events MySQL configuration option is enabled and the connector configuration include.query property is enabled, the source field also provides the query field, which contains the original SQL statement that caused the change event.

4

op

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.

5

ts_ms

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.

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:

  1. {
  2. "schema": { ... },
  3. "payload": {
  4. "before": { (1)
  5. "id": 1004,
  6. "first_name": "Anne Marie",
  7. "last_name": "Kretchmar",
  8. "email": "annek@noanswer.org"
  9. },
  10. "after": null, (2)
  11. "source": { (3)
  12. "version": "1.4.2.Final",
  13. "connector": "mysql",
  14. "name": "mysql-server-1",
  15. "ts_ms": 1465581902300,
  16. "snapshot": false,
  17. "db": "inventory",
  18. "table": "customers",
  19. "server_id": 223344,
  20. "gtid": null,
  21. "file": "mysql-bin.000003",
  22. "pos": 805,
  23. "row": 0,
  24. "thread": 7,
  25. "query": "DELETE FROM customers WHERE id=1004"
  26. },
  27. "op": "d", (4)
  28. "ts_ms": 1465581902461 (5)
  29. }
  30. }
Table 7. Descriptions of delete event value fields
ItemField nameDescription

1

before

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.

2

after

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.

3

source

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

If the binlog_rows_query_log_events MySQL configuration option is enabled and the connector configuration include.query property is enabled, the source field also provides the query field, which contains the original SQL statement that caused the change event.

4

op

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

5

ts_ms

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.

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)

Basic types

The following table shows how the connector maps basic MySQL data types.

Table 8. Descriptions of basic type mappings
MySQL typeLiteral typeSemantic type

BOOLEAN, BOOL

BOOLEAN

n/a

BIT(1)

BOOLEAN

n/a

BIT(>1)

BYTES

io.debezium.data.Bits
The length schema parameter contains an integer that represents the number of bits. The byte[] contains the bits in little-endian form and is sized to contain the specified number of bits. For example, where n is bits:
numBytes = n/8 + (n%8== 0 ? 0 : 1)

TINYINT

INT16

n/a

SMALLINT[(M)]

INT16

n/a

MEDIUMINT[(M)]

INT32

n/a

INT, INTEGER[(M)]

INT32

n/a

BIGINT[(M)]

INT64

n/a

REAL[(M,D)]

FLOAT32

n/a

FLOAT[(M,D)]

FLOAT64

n/a

DOUBLE[(M,D)]

FLOAT64

n/a

CHAR(M)]

STRING

n/a

VARCHAR(M)]

STRING

n/a

BINARY(M)]

BYTES or STRING

n/a
Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the binary.handling.mode connector configuration property setting.

VARBINARY(M)]

BYTES or STRING

n/a
Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the binary.handling.mode connector configuration property setting.

TINYBLOB

BYTES or STRING

n/a
Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the binary.handling.mode connector configuration property setting.

TINYTEXT

STRING

n/a

BLOB

BYTES or STRING

n/a
Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the binary.handling.mode connector configuration property setting.

TEXT

STRING

n/a

MEDIUMBLOB

BYTES or STRING

n/a
Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the binary.handling.mode connector configuration property setting.

MEDIUMTEXT

STRING

n/a

LONGBLOB

BYTES or STRING

n/a
Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the binary.handling.mode connector configuration property setting.

LONGTEXT

STRING

n/a

JSON

STRING

io.debezium.data.Json
Contains the string representation of a JSON document, array, or scalar.

ENUM

STRING

io.debezium.data.Enum
The allowed schema parameter contains the comma-separated list of allowed values.

SET

STRING

io.debezium.data.EnumSet
The allowed schema parameter contains the comma-separated list of allowed values.

YEAR[(2|4)]

INT32

io.debezium.time.Year

TIMESTAMP[(M)]

STRING

io.debezium.time.ZonedTimestamp
In ISO 8601 format with microsecond precision. MySQL allows M to be in the range of 0-6.

Temporal types

Excluding the TIMESTAMP data type, MySQL temporal types depend on the value of the time.precision.mode connector configuration property. For TIMESTAMP columns whose default value is specified as CURRENT_TIMESTAMP or NOW, the value 1970-01-01 00:00:00 is used as the default value in the Kafka Connect schema.

MySQL allows zero-values for DATE, `DATETIME, and TIMESTAMP columns because zero-values are sometimes preferred over null values. The MySQL connector represents zero-values as null values when the column definition allows null values, or as the epoch day when the column does not allow null values.

Temporal values without time zones

The DATETIME type represents a local date and time such as “2018-01-13 09:48:27”. As you can see, there is no time zone information. Such columns are converted into epoch milliseconds or microseconds based on the column’s precision by using UTC. The TIMESTAMP type represents a timestamp without time zone information. It is converted by MySQL from the server (or session’s) current time zone into UTC when writing and from UTC into the server (or session’s) current time zone when reading back the value. For example:

  • DATETIME with a value of 2018-06-20 06:37:03 becomes 1529476623000.

  • TIMESTAMP with a value of 2018-06-20 06:37:03 becomes 2018-06-20T13:37:03Z.

Such columns are converted into an equivalent io.debezium.time.ZonedTimestamp in UTC based on the server (or session’s) current time zone. The time zone will be queried from the server by default. If this fails, it must be specified explicitly by the database serverTimezone MySQL configuration option. For example, if the database’s time zone (either globally or configured for the connector by means of the serverTimezone option) is “America/Los_Angeles”, the TIMESTAMP value “2018-06-20 06:37:03” is represented by a ZonedTimestamp with the value “2018-06-20T13:37:03Z”.

The time zone of the JVM running Kafka Connect and Debezium does not affect these conversions.

More details about properties related to termporal values are in the documentation for MySQL connector configuration properties.

time.precision.mode=adaptive_time_microseconds(default)

The MySQL connector determines the literal type and semantic type based on the column’s data type definition so that events represent exactly the values in the database. All time fields are in microseconds. Only positive TIME field values in the range of 00:00:00.000000 to 23:59:59.999999 can be captured correctly.

Table 9. Mappings when time.precision.mode=adaptive_time_microseconds
MySQL typeLiteral typeSemantic type

DATE

INT32

io.debezium.time.Date
Represents the number of days since the epoch.

TIME[(M)]

INT64

io.debezium.time.MicroTime
Represents the time value in microseconds and does not include time zone information. MySQL allows M to be in the range of 0-6.

DATETIME, DATETIME(0), DATETIME(1), DATETIME(2), DATETIME(3)

INT64

io.debezium.time.Timestamp
Represents the number of milliseconds past the epoch and does not include time zone information.

DATETIME(4), DATETIME(5), DATETIME(6)

INT64

io.debezium.time.MicroTimestamp
Represents the number of microseconds past the epoch and does not include time zone information.

time.precision.mode=connect

The MySQL connector uses defined Kafka Connect logical types. This approach is less precise than the default approach and the events could be less precise if the database column has a fractional second precision value of greater than 3. Values in only the range of 00:00:00.000 to 23:59:59.999 can be handled. Set time.precision.mode=connect only if you can ensure that the TIME values in your tables never exceed the supported ranges. The connect setting is expected to be removed in a future version of Debezium.

Table 10. Mappings when time.precision.mode=connect
MySQL typeLiteral typeSemantic type

DATE

INT32

org.apache.kafka.connect.data.Date
Represents the number of days since the epoch.

TIME[(M)]

INT64

org.apache.kafka.connect.data.Time
Represents the time value in microseconds since midnight and does not include time zone information.

DATETIME[(M)]

INT64

org.apache.kafka.connect.data.Timestamp
Represents the number of milliseconds since the epoch, and does not include time zone information.

Decimal types

Debezium connectors handle decimals according to the setting of the decimal.handling.mode connector configuration property.

decimal.handling.mode=precise

Table 11. Mappings when decimal.handing.mode=precise
MySQL typeLiteral typeSemantic type

NUMERIC[(M[,D])]

BYTES

org.apache.kafka.connect.data.Decimal
The scale schema parameter contains an integer that represents how many digits the decimal point shifted.

DECIMAL[(M[,D])]

BYTES

org.apache.kafka.connect.data.Decimal
The scale schema parameter contains an integer that represents how many digits the decimal point shifted.

decimal.handling.mode=double

Table 12. Mappings when decimal.handing.mode=double
MySQL typeLiteral typeSemantic type

NUMERIC[(M[,D])]

FLOAT64

n/a

DECIMAL[(M[,D])]

FLOAT64

n/a

decimal.handling.mode=string

Table 13. Mappings when decimal.handing.mode=string
MySQL typeLiteral typeSemantic type

NUMERIC[(M[,D])]

STRING

n/a

DECIMAL[(M[,D])]

STRING

n/a

Boolean values

MySQL handles the BOOLEAN value internally in a specific way. The BOOLEAN column is internally mapped to the TINYINT(1) data type. When the table is created during streaming then it uses proper BOOLEAN mapping as Debezium receives the original DDL. During snapshots, Debezium executes SHOW CREATE TABLE to obtain table definitions that return TINYINT(1) for both BOOLEAN and TINYINT(1) columns. Debezium then has no way to obtain the original type mapping and so maps to TINYINT(1).

The operator can configure the out-of-the-box TinyIntOneToBooleanConverter custom converter that would either map all TINYINT(1) columns to BOOLEAN or if the selector parameter is set then a subset of columns could be enumerated using comma-separated regular expressions.

Following is an example configuration:

  1. converters=boolean
  2. boolean.type=io.debezium.connector.mysql.converters.TinyIntOneToBooleanConverter
  3. boolean.selector=db1.table1.*, db1.table2.column1

Spatial types

Currently, the Debezium MySQL connector supports the following spatial data types.

Table 14. Description of spatial type mappings
MySQL typeLiteral typeSemantic type

GEOMETRY,
LINESTRING,
POLYGON,
MULTIPOINT,
MULTILINESTRING,
MULTIPOLYGON,
GEOMETRYCOLLECTION

STRUCT

io.debezium.data.geometry.Geometry
Contains a structure with two fields:

  • srid (INT32: spatial reference system ID that defines the type of geometry object stored in the structure

  • wkb (BYTES): binary representation of the geometry object encoded in the Well-Known-Binary (wkb) format. See the Open Geospatial Consortium for more details.

Set up

Some MySQL setup tasks are required before you can install and run a Debezium connector.

Creating a user

A Debezium MySQL connector requires a MySQL user account. This MySQL user must have appropriate permissions on all databases for which the Debezium MySQL connector captures changes.

Prerequisites

  • A MySQL server.

  • Basic knowledge of SQL commands.

Procedure

  1. Create the MySQL user:

    1. mysql> CREATE USER 'user'@'localhost' IDENTIFIED BY 'password';
  2. Grant the required permissions to the user:

    1. mysql> GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO 'user' IDENTIFIED BY 'password';

    The table below describes the permissions.

    If using a hosted option such as Amazon RDS or Amazon Aurora that does not allow a global read lock, table-level locks are used to create the consistent snapshot. In this case, you need to also grant LOCK TABLES permissions to the user that you create. See snapshots for more details.
  3. Finalize the user’s permissions:

    1. mysql> FLUSH PRIVILEGES;
Table 15. Descriptions of user permissions
KeywordDescription

SELECT

Enables the connector to select rows from tables in databases. This is used only when performing a snapshot.

RELOAD

Enables the connector the use of the FLUSH statement to clear or reload internal caches, flush tables, or acquire locks. This is used only when performing a snapshot.

SHOW DATABASES

Enables the connector to see database names by issuing the SHOW DATABASE statement. This is used only when performing a snapshot.

REPLICATION SLAVE

Enables the connector to connect to and read the MySQL server binlog.

REPLICATION CLIENT

Enables the connector the use of the following statements:

  • SHOW MASTER STATUS

  • SHOW SLAVE STATUS

  • SHOW BINARY LOGS

The connector always requires this.

ON

Identifies the database to which the permissions apply.

TO ‘user’

Specifies the user to grant the permissions to.

IDENTIFIED BY ‘password’

Specifies the user’s MySQL password.

Enabling the binlog

You must enable binary logging for MySQL replication. The binary logs record transaction updates for replication tools to propagate changes.

Prerequisites

  • A MySQL server.

  • Appropriate MySQL user privileges.

Procedure

  1. Check whether the log-bin option is already on:

    1. mysql> SELECT variable_value as "BINARY LOGGING STATUS (log-bin) ::"
    2. FROM information_schema.global_variables WHERE variable_name='log_bin';
  2. If it is OFF, configure your MySQL server configuration file with the following properties, which are described in the table below:

    1. server-id = 223344
    2. log_bin = mysql-bin
    3. binlog_format = ROW
    4. binlog_row_image = FULL
    5. expire_logs_days = 10
  3. Confirm your changes by checking the binlog status once more:

    1. mysql> SELECT variable_value as "BINARY LOGGING STATUS (log-bin) ::"
    2. FROM information_schema.global_variables WHERE variable_name='log_bin';
Table 16. Descriptions of MySQL binlog configuration properties
PropertyDescription

server-id

The value for the server-id must be unique for each server and replication client in the MySQL cluster. During MySQL connector set up, Debezium assigns a unique server ID to the connector.

log_bin

The value of log_bin is the base name of the sequence of binlog files.

binlog_format

The binlog-format must be set to ROW or row.

binlog_row_image

The binlog_row_image must be set to FULL or full.

expire_logs_days

This is the number of days for automatic binlog file removal. The default is 0, which means no automatic removal. Set the value to match the needs of your environment. See MySQL purges binlog files.

Enabling GTIDs

Global transaction identifiers (GTIDs) uniquely identify transactions that occur on a server within a cluster. Though not required for a Debezium MySQL connector, using GTIDs simplifies replication and enables you to more easily confirm if primary and replica servers are consistent.

GTIDs are available in MySQL 5.6.5 and later. See the MySQL documentation for more details.

Prerequisites

  • A MySQL server.

  • Basic knowledge of SQL commands.

  • Access to the MySQL configuration file.

Procedure

  1. Enable gtid_mode:

    1. mysql> gtid_mode=ON
  2. Enable enforce_gtid_consistency:

    1. mysql> enforce_gtid_consistency=ON
  3. Confirm the changes:

    1. mysql> show global variables like '%GTID%';

Result

  1. +--------------------------+-------+
  2. | Variable_name | Value |
  3. +--------------------------+-------+
  4. | enforce_gtid_consistency | ON |
  5. | gtid_mode | ON |
  6. +--------------------------+-------+
Table 17. Descriptions of GTID options
OptionDescription

gtid_mode

Boolean that specifies whether GTID mode of the MySQL server is enabled or not.

  • ON = enabled

  • OFF = disabled

enforce_gtid_consistency

Boolean that specifies whether the server enforces GTID consistency by allowing the execution of statements that can be logged in a transactionally safe manner. Required when using GTIDs.

  • ON = enabled

  • OFF = disabled

Configuring session timeouts

When an initial consistent snapshot is made for large databases, your established connection could timeout while the tables are being read. You can prevent this behavior by configuring interactive_timeout and wait_timeout in your MySQL configuration file.

Prerequisites

  • A MySQL server.

  • Basic knowledge of SQL commands.

  • Access to the MySQL configuration file.

Procedure

  1. Configure interactive_timeout:

    1. mysql> interactive_timeout=<duration-in-seconds>
  2. Configure wait_timeout:

    1. mysql> wait_timeout=<duration-in-seconds>
Table 18. Descriptions of MySQL session timeout options
OptionDescription

interactive_timeout

The number of seconds the server waits for activity on an interactive connection before closing it. See MySQL’s documentation for more details.

wait_timeout

The number of seconds the server waits for activity on a non-interactive connection before closing it. See MySQL’s documentation for more details.

Enabling query log events

You might want to see the original SQL statement for each binlog event. Enabling the binlog_rows_query_log_events option in the MySQL configuration file allows you to do this.

This option is available in MySQL 5.6 and later.

Prerequisites

  • A MySQL server.

  • Basic knowlede of SQL commands.

  • Access to the MySQL configuration file.

Procedure

  • Enable binlog_rows_query_log_events:

    1. mysql> binlog_rows_query_log_events=ON

    binlog_rows_query_log_events is set to a value that enables/disables support for including the original SQL statement in the binlog entry.

    • ON = enabled

    • OFF = disabled

Deployment

To deploy a Debezium MySQL connector, you install the Debezium MySQL connector archive, configure the connector, and start the connector by adding its configuration to Kafka Connect.

Prerequisites

Procedure

  1. Download the Debezium MySQL connector plug-in.

  2. Extract the files into your Kafka Connect environment.

  3. Add the directory with the JAR files to Kafka Connect’s plugin.path.

  4. Configure the connector and add the configuration to your Kafka Connect cluster.

  5. Restart your Kafka Connect process to pick up the new JAR files.

If you are working with immutable containers, see Debezium’s Container images for Apache Zookeeper, Apache Kafka, MySQL, and Kafka Connect with the MySQL connector already installed and ready to run.

You can also run Debezium on Kubernetes and OpenShift.

MySQL connector configuration example

Following is an example of the configuration for a connector instance that captures data from a MySQL server on port 3306 at 192.168.99.100, which we logically name fullfillment. Typically, you configure the Debezium MySQL connector in a JSON file by setting the configuration properties that are available for the connector.

You can choose to produce events for a subset of the schemas and tables in a database. Optionally, you can ignore, mask, or truncate columns that contain sensitive data, that are larger than a specified size, or that you do not need.

  1. {
  2. "name": "inventory-connector", (1)
  3. "config": {
  4. "connector.class": "io.debezium.connector.mysql.MySqlConnector", (2)
  5. "database.hostname": "192.168.99.100", (3)
  6. "database.port": "3306", (4)
  7. "database.user": "debezium-user", (5)
  8. "database.password": "debezium-user-pw", (6)
  9. "database.server.id": "184054", (7)
  10. "database.server.name": "fullfillment", (8)
  11. "database.include.list": "inventory", (9)
  12. "database.history.kafka.bootstrap.servers": "kafka:9092", (10)
  13. "database.history.kafka.topic": "dbhistory.fullfillment", (11)
  14. "include.schema.changes": "true" (12)
  15. }
  16. }
1Connector’s name when registered with the Kafka Connect service.
2Connector’s class name.
3MySQL server address.
4MySQL server port number.
5MySQL user with the appropriate privileges.
6MySQL user’s password.
7Unique ID of the connector.
8Logical name of the MySQL server or cluster.
9List of databases hosted by the specified server.
10List of Kafka brokers that the connector uses to write and recover DDL statements to the database history topic.
11Name of the database history topic. This topic is for internal use only and should not be used by consumers.
12Flag that specifies if the connector should generate events for DDL changes and emit them to the fulfillment schema change topic for use by consumers.

For the complete list of the configuration properties that you can set for the Debezium MySQL connector, see MySQL connector configuration properties.

You can send this configuration with a POST command to a running Kafka Connect service. The service records the configuration and starts one connector task that performs the following actions:

  • Connects to the MySQL database.

  • Reads change-data tables for tables in capture mode.

  • Streams change event records to Kafka topics.

Adding connector configuration

To start running a MySQL connector, configure a connector configuration, and add the configuration to your Kafka Connect cluster.

Prerequisites

Procedure

  1. Create a configuration for the MySQL connector.

  2. Use the Kafka Connect REST API to add that connector configuration to your Kafka Connect cluster.

Results

When the connector starts, it performs a consistent snapshot of the MySQL databases that the connector is configured for. The connector then starts generating data change events for row-level operations and streaming change event records to Kafka topics.

Connector properties

The Debezium MySQL connector has numerous configuration properties that you can use to achieve the right connector behavior for your application. Many properties have default values. Information about the properties is organized as follows:

The following configuration properties are required unless a default value is available.

Table 19. Required MySQL connector configuration properties
PropertyDefaultDescription

Unique name for the connector. Attempting to register again with the same name fails. This property is required by all Kafka Connect connectors.

The name of the Java class for the connector. Always specify io.debezium.connector.mysql.MySqlConnector for the MySQL connector.

1

The maximum number of tasks that should be created for this connector. The MySQL connector always uses a single task and therefore does not use this value, so the default is always acceptable.

IP address or host name of the MySQL database server.

3306

Integer port number of the MySQL database server.

Name of the MySQL user to use when connecting to the MySQL database server.

Password to use when connecting to the MySQL database server.

Logical name that identifies and provides a namespace for the particular MySQL database server/cluster in which Debezium is capturing changes. The logical name should be unique across all other connectors, since it is used as a prefix for all Kafka topic names that receive events emitted by this connector. Only alphanumeric characters and underscores are allowed in this name.

random

A numeric ID of this database client, which must be unique across all currently-running database processes in the MySQL cluster. This connector joins the MySQL database cluster as another server (with this unique ID) so it can read the binlog. By default, a random number between 5400 and 6400 is generated, though the recommendation is to explicitly set a value.

The full name of the Kafka topic where the connector stores the database schema history.

A list of host/port pairs that the connector uses for establishing an initial connection to the Kafka cluster. This connection is used for retrieving database schema history previously stored by the connector, and for writing each DDL statement read from the source database. Each pair should point to the same Kafka cluster used by the Kafka Connect process.

empty string

An optional, comma-separated list of regular expressions that match the names of the databases for which to capture changes. The connector does not capture changes in any database whose name is not in database.include.list. By default, the connector captures changes in all databases. Do not also set the database.exclude.list connector confiuration property.

empty string

An optional, comma-separated list of regular expressions that match the names of databases for which you do not want to capture changes. The connector captures changes in any database whose name is not in the database.exclude.list. Do not also set the database.include.list connector configuration property.

empty string

An optional, comma-separated list of regular expressions that match fully-qualified table identifiers of tables whose changes you want to capture. The connector does not capture changes in any table not included in table.include.list. Each identifier is of the form databaseName.tableName. By default, the connector captures changes in every non-system table in each database whose changes are being captured. Do not also specify the table.exclude.list connector configuration property.

empty string

An optional, comma-separated list of regular expressions that match fully-qualified table identifiers for tables whose changes you do not want to capture. The connector captures changes in any table not included in table.exclude.list. Each identifier is of the form databaseName.tableName. Do not also specify the table.include.list connector configuration property.

empty string

An optional, comma-separated list of regular expressions that match the fully-qualified names of columns to exclude from change event record values. Fully-qualified names for columns are of the form databaseName.tableName.columnName.

empty string

An optional, comma-separated list of regular expressions that match the fully-qualified names of columns to include in change event record values. Fully-qualified names for columns are of the form databaseName.tableName.columnName.

n/a

An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns whose values should be truncated in the change event record values if the field values are longer than the specified number of characters. You can configure multiple properties with different lengths in a single configuration. The length must be a positive integer. Fully-qualified names for columns are of the form databaseName.tableName.columnName.

n/a

An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns whose values should be replaced in the change event message values with a field value consisting of the specified number of asterisk (*) characters. You can configure multiple properties with different lengths in a single configuration. Each length must be a positive integer or zero. Fully-qualified names for columns are of the form databaseName.tableName.columnName.

n/a

An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns whose values should be pseudonyms in the change event record values. Pseudonyms consist of the hashed value obtained by applying the algorithm hashAlgorithm and salt salt.

Based on the hash function used, referential integrity is kept while data is pseudonymized. Supported hash functions are described in the MessageDigest section of the Java Cryptography Architecture Standard Algorithm Name Documentation. The hash result is automatically shortened to the length of the column.

You can configure multiple properties with different lengths in a single configuration. Each length must be a positive integer or zero. Fully-qualified names for columns are of the form databaseName.tableName.columnName. For example:

column.mask.hash.SHA-256.with.salt.CzQMA0cB5K = inventory.orders.customerName, inventory.shipment.customerName

CzQMA0cB5K is a randomly selected salt.
Depending on the configured hashAlgorithm, the selected salt, and the actual data set, the resulting masked data set might not be completely anonymized.

n/a

An optional, comma-separated list of regular expressions that match the fully-qualified names of columns whose original type and length should be added as a parameter to the corresponding field schemas in the emitted change event records. These schema parameters:

Debezium.source.column.type

Debezium.source.column.length

Debezium.source.column.scale

are used to propagate the original type name and length for variable-width types, respectively. This is useful to properly size corresponding columns in sink databases. Fully-qualified names for columns are of one of these forms:

databaseName.tableName.columnName

databaseName.schemaName.tableName.columnName

n/a

An optional, comma-separated list of regular expressions that match the database-specific data type name of columns whose original type and length should be added as a parameter to the corresponding field schemas in the emitted change event records. These schema parameters:

debezium.source.column.type

debezium.source.column.length

debezium.source.column.scale

are used to propagate the original type name and length for variable-width types, respectively. This is useful to properly size corresponding columns in sink databases. Fully-qualified data type names are of one of these forms:

databaseName.tableName.typeName

databaseName.schemaName.tableName.typeName

See how MySQL connectors map data types for the list of MySQL-specific data type names.

adaptive_time_microseconds

Time, date, and timestamps can be represented with different kinds of precision, including:

adaptive_time_microseconds (the default) captures the date, datetime and timestamp values exactly as in the database using either millisecond, microsecond, or nanosecond precision values based on the database column’s type, with the exception of TIME type fields, which are always captured as microseconds.

adaptive (deprecated) captures the time and timestamp values exactly as in the database using either millisecond, microsecond, or nanosecond precision values based on the database column’s type.

connect always represents time and timestamp values using Kafka Connect’s built-in representations for Time, Date, and Timestamp, which use millisecond precision regardless of the database columns’ precision.

precise

Specifies how the connector should handle values for DECIMAL and NUMERIC columns:

precise (the default) represents them precisely using java.math.BigDecimal values represented in change events in a binary form.

double represents them using double values, which may result in a loss of precision but is easier to use.

string encodes values as formatted strings, which is easy to consume but semantic information about the real type is lost.

long

Specifies how BIGINT UNSIGNED columns should be represented in change events. Possible settings are:

long represents values by using Java’s long, which might not offer the precision but which is easy to use in consumers. long is usually the preferred setting.

precise uses java.math.BigDecimal to represent values, which are encoded in the change events by using a binary representation and Kafka Connect’s org.apache.kafka.connect.data.Decimal type. Use this setting when working with values larger than 2^63, because these values cannot be conveyed by using long.

true

Boolean value that specifies whether the connector should publish changes in the database schema to a Kafka topic with the same name as the database server ID. Each schema change is recorded by using a key that contains the database name and whose value includes the DDL statement(s). This is independent of how the connector internally records database history.

false

Boolean value that specifies whether the connector should include the original SQL query that generated the change event.

If you set this option to true then you must also configure MySQL with the binlog_rows_query_log_events option set to ON. When include.query is true, the query is not present for events that the snapshot process generates.

Setting include.query to true might expose tables or fields that are explicitly excluded or masked by including the original SQL statement in the change event. For this reason, the default setting is false.

fail

Specifies how the connector should react to exceptions during deserialization of binlog events.

fail propagates the exception, which indicates the problematic event and its binlog offset, and causes the connector to stop.

warn logs the problematic event and its binlog offset and then skips the event.

skip passes over the problematic event and does not log anything.

fail

Specifies how the connector should react to binlog events that relate to tables that are not present in internal schema representation. That is, the internal representation is not consistent with the database.

fail throws an exception that indicates the problematic event and its binlog offset, and causes the connector to stop.

warn logs the problematic event and its binlog offset and skips the event.

skip passes over the problematic event and does not log anything.

8192

Positive integer value that specifies the maximum size of the blocking queue into which change events read from the database log are placed before they are written to Kafka. This queue can provide backpressure to the binlog reader when, for example, writes to Kafka are slow or if Kafka is not available. Events that appear in the queue are not included in the offsets periodically recorded by this connector. Defaults to 8192, and should always be larger than the maximum batch size specified by the max.batch.size property.

2048

Positive integer value that specifies the maximum size of each batch of events that should be processed during each iteration of this connector. Defaults to 2048.

0

Long value for the maximum size in bytes of the blocking queue. The feature is disabled by default, it will be active if it’s set with a positive long value.

1000

Positive integer value that specifies the number of milliseconds the connector should wait for new change events to appear before it starts processing a batch of events. Defaults to 1000 milliseconds, or 1 second.

30000

A positive integer value that specifies the maximum time in milliseconds this connector should wait after trying to connect to the MySQL database server before timing out. Defaults to 30 seconds.

A comma-separated list of regular expressions that match source UUIDs in the GTID set used to find the binlog position in the MySQL server. Only the GTID ranges that have sources that match one of these include patterns are used. Do not also specify a setting for gtid.source.excludes.

A comma-separated list of regular expressions that match source UUIDs in the GTID set used to find the binlog position in the MySQL server. Only the GTID ranges that have sources that do not match any of these exclude patterns are used. Do not also specify a value for gtid.source.includes.

gtid.new.channel.position
deprecated and scheduled for removal

earliest

When set to latest, when the connector sees a new GTID channel, it starts consuming from the last executed transaction in that GTID channel. If set to earliest (default), the connector starts reading that channel from the first available (not purged) GTID position. earliest is useful when you have an active-passive MySQL setup where Debezium is connected to the primary server. In this case, during failover, the replica with the new UUID (and GTID channel) starts receiving writes before Debezium is connected. These writes would be lost when using latest.

true

Controls whether a delete event is followed by a tombstone event.

true - a delete operation is represented by a delete event and a subsequent tombstone event.

false - only a delete event is emitted.

After a source record is deleted, emitting a tombstone event (the default behavior) allows Kafka to completely delete all events that pertain to the key of the deleted row.

n/a

A semicolon separated list of tables with regular expressions that match table column names. The connector maps values in matching columns to key fields in change event records that it sends to Kafka topics. This is useful when a table does not have a primary key, or when you want to order change event records in a Kafka topic according to a field that is not a primary key.

Separate entries with semicolons. Insert a colon between the fully-qualified table name and its regular expression. The format (shown with spaces for clarity only) is:

database-name . table-name : regexp ; …​

For example:

dbA.table_a:regex_1;dbB.table_b:regex_2;dbC.table_c:regex_3

If table_a has an id column, and regex_1 is ^i (matches any column that starts with i), the connector maps the value in the id column of table_a to a key field in change events that the connector sends to Kafka.

bytes

Specifies how binary columns, for example, blob, binary, varbinary, should be represented in change events. Possible settings:

bytes represents binary data as a byte array.

base64 represents binary data as a base64-encoded String.

hex represents binary data as a hex-encoded (base16) String.

Advanced MySQL connector configuration properties

The following table describes advanced MySQL connector properties. The default values for these properties rarely need to be changed. Therefore, you do not need to specify them in the connector configuration.

Table 20. Descriptions of MySQL connector advanced configuration properties
PropertyDefaultDescription

true

A Boolean value that specifies whether a separate thread should be used to ensure that the connection to the MySQL server/cluster is kept alive.

true

A Boolean value that specifies whether built-in system tables should be ignored. This applies regardless of the table include and exclude lists. By default, system tables are excluded from having their changes captured, and no events are generated when changes are made to any system tables.

100

An integer value that specifies the maximum number of milliseconds the connector should wait during startup/recovery while polling for persisted data. The default is 100ms.

4

The maximum number of times that the connector should try to read persisted history data before the connector recovery fails with an error. The maximum amount of time to wait after receiving no data is recovery.attempts x recovery.poll.interval.ms.

false

A Boolean value that specifies whether the connector should ignore malformed or unknown database statements or stop processing so a human can fix the issue. The safe default is false. Skipping should be used only with care as it can lead to data loss or mangling when the binlog is being processed.

false

A Boolean value that specifies whether the connector should record all DDL statements

true records only those DDL statements that are relevant to tables whose changes are being captured by Debezium. Set to true with care because missing data might become necessary if you change which tables have their changes captured.

The safe default is false.

disabled

Specifies whether to use an encrypted connection. Possible settings are:

disabled specifies the use of an unencrypted connection.

preferred establishes an encrypted connection if the server supports secure connections. If the server does not support secure connections, falls back to an unencrypted connection.

required establishes an encrypted connection or fails if one cannot be made for any reason.

verify_ca behaves like required but additionally it verifies the server TLS certificate against the configured Certificate Authority (CA) certificates and fails if the server TLS certificate does not match any valid CA certificates.

verify_identity behaves like verify_ca but additionally verifies that the server certificate matches the host of the remote connection.

0

The size of a look-ahead buffer used by the binlog reader. The default setting of 0 disables buffering.

Under specific conditions, it is possible that the MySQL binlog contains uncommitted data finished by a ROLLBACK statement. Typical examples are using savepoints or mixing temporary and regular table changes in a single transaction.

When a beginning of a transaction is detected then Debezium tries to roll forward the binlog position and find either COMMIT or ROLLBACK so it can determine whether to stream the changes from the transaction. The size of the binlog buffer defines the maximum number of changes in the transaction that Debezium can buffer while searching for transaction boundaries. If the size of the transaction is larger than the buffer then Debezium must rewind and re-read the events that have not fit into the buffer while streaming.

NOTE: This feature is incubating. Feedback is encouraged. It is expected that this feature is not completely polished.

initial

Specifies the criteria for running a snapshot when the connector starts. Possible settings are:

initial - the connector runs a snapshot only when no offsets have been recorded for the logical server name.

when_needed - the connector runs a snapshot upon startup whenever it deems it necessary. That is, when no offsets are available, or when a previously recorded offset specifies a binlog location or GTID that is not available in the server.

never - the connector never uses snapshots. Upon first startup with a logical server name, the connector reads from the beginning of the binlog. Configure this behavior with care. It is valid only when the binlog is guaranteed to contain the entire history of the database.

schema_only - the connector runs a snapshot of the schemas and not the data. This setting is useful when you do not need the topics to contain a consistent snapshot of the data but need them to have only the changes since the connector was started.

schema_only_recovery - this is a recovery setting for a connector that has already been capturing changes. When you restart the connector, this setting enables recovery of a corrupted or lost database history topic. You might set it periodically to “clean up” a database history topic that has been growing unexpectedly. Database history topics require infinite retention.

minimal

Controls whether and how long the connector holds the global MySQL read lock, which prevents any updates to the database, while the connector is performing a snapshot. Possible settings are:

minimal - the connector holds the global read lock for only the initial portion of the snapshot during which the connector reads the database schemas and other metadata. The remaining work in a snapshot involves selecting all rows from each table. The connector can do this in a consistent fashion by using a REPEATABLE READ transaction. This is the case even when the global read lock is no longer held and other MySQL clients are updating the database.

minimal_percona - the connector holds the global backup lock for only the initial portion of the snapshot during which the connector reads the database schemas and other metadata. The remaining work in a snapshot involves selecting all rows from each table. The connector can do this in a consistent fashion by using a REPEATABLE READ transaction. This is the case even when the global backup lock is no longer held and other MySQL clients are updating the database. This mode does not flush tables to disk, is not blocked by long-running reads, and is available only in Percona Server.

extended - blocks all writes for the duration of the snapshot. Use this setting if there are clients that are submitting operations that MySQL excludes from REPEATABLE READ semantics.

none - prevents the connector from acquiring any table locks during the snapshot. While this setting is allowed with all snapshot modes, it is safe to use if and only if no schema changes are happening while the snapshot is running. For tables defined with MyISAM engine, the tables would still be locked despite this property being set as MyISAM acquires a table lock. This behavior is unlike InnoDB engine, which acquires row level locks.

All tables specified in table.include.list

An optional, comma-separated list of regular expressions that match names of schemas specified in table.include.list for which you want to take the snapshot.

Controls which table rows are included in snapshots. This property affects snapshots only. It does not affect events captured from the binlog. Specify a comma-separated list of fully-qualified table names in the form databaseName.tableName.

For each table that you specify, also specify another configuration property: snapshot.select.statement.overrides.DB_NAME.TABLE_NAME. For example, the name of the other configuration property might be: snapshot.select.statement.overrides.customers.orders. Set this property to a SELECT statement that obtains only the rows that you want in the snapshot. When the connector performs a snapshot, it executes this SELECT statement to retrieve data from that table.

A possible use case for setting these properties is large, append-only tables. You can specify a SELECT statement that sets a specific point for where to start a snapshot, or where to resume a snapshot if a previous snapshot was interrupted.

1000

During a snapshot, the connector queries each table for which the connector is configured to capture changes. The connector uses each query result to produce a read event that contains data for all rows in that table. This property determines whether the MySQL connector puts results for a table into memory, which is fast but requires large amounts of memory, or streams the results, which can be slower but work for very large tables. The setting of this property specifies the minimum number of rows a table must contain before the connector streams results.

To skip all table size checks and always stream all results during a snapshot, set this property to 0.

0

Controls how frequently the connector sends heartbeat messages to a Kafka topic. The default behavior is that the connector does not send heartbeat messages.

Heartbeat messages are useful for monitoring whether the connector is receiving change events from the database. Heartbeat messages might help decrease the number of change events that need to be re-sent when a connector restarts. To send heartbeat messages, set this property to a positive integer, which indicates the number of milliseconds between heartbeat messages.

debezium-heartbeat

Controls the name of the topic to which the connector sends heartbeat messages. The topic name has this pattern:

heartbeat.topics.prefix.server.name

For example, if the database server name is fulfillment, the default topic name is debezium-heartbeat.fulfillment.

A semicolon separated list of SQL statements to be executed when a JDBC connection, not the connection that is reading the transaction log, to the database is established. To specify a semicolon as a character in a SQL statement and not as a delimiter, use two semicolons, (;;).

The connector might establish JDBC connections at its own discretion, so this property is ony for configuring session parameters. It is not for executing DML statements.

An interval in milliseconds that the connector should wait before performing a snapshot when the connector starts. If you are starting multiple connectors in a cluster, this property is useful for avoiding snapshot interruptions, which might cause re-balancing of connectors.

During a snapshot, the connector reads table content in batches of rows. This property specifies the maximum number of rows in a batch.

10000

Positive integer that specifies the maximum amount of time (in milliseconds) to wait to obtain table locks when performing a snapshot. If the connector cannot acquire table locks in this time interval, the snapshot fails. See how MySQL connectors perform database snapshots.

true

Boolean value that indicates whether the connector converts a 2-digit year specification to 4 digits. Set to false when conversion is fully delegated to the database.

MySQL allows users to insert year values with either 2-digits or 4-digits. For 2-digit values, the value gets mapped to a year in the range 1970 - 2069. The default behavior is that the connector does the conversion.

v2

Schema version for the source block in Debezium events. Debezium 0.10 introduced a few breaking changes to the structure of the source block in order to unify the exposed structure across all the connectors.

By setting this option to v1, the structure used in earlier versions can be produced. However, this setting is not recommended and is planned for removal in a future Debezium version.

true if connector configuration sets the key.converter or value.converter property to the Avro converter.
false if not.

Indicates whether field names are sanitized to adhere to Avro naming requirements.

Comma-separated list of operation types to skip during streaming. The following values are possible: c for inserts/create, u for updates, d for deletes. By default, no operations are skipped.

Pass-through configuration properties

The MySQL connector also supports pass-through configuration properties that are used when creating the Kafka producer and consumer. Specifically, all connector configuration properties that begin with the database.history.producer. prefix are used (without the prefix) when creating the Kafka producer that writes to the database history. All properties that begin with the prefix database.history.consumer. are used (without the prefix) when creating the Kafka consumer that reads the database history upon connector start-up.

For example, the following connector configuration properties can be used to secure connections to the Kafka broker:

  1. database.history.producer.security.protocol=SSL
  2. database.history.producer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks
  3. database.history.producer.ssl.keystore.password=test1234
  4. database.history.producer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks
  5. database.history.producer.ssl.truststore.password=test1234
  6. database.history.producer.ssl.key.password=test1234
  7. database.history.consumer.security.protocol=SSL
  8. database.history.consumer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks
  9. database.history.consumer.ssl.keystore.password=test1234
  10. database.history.consumer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks
  11. database.history.consumer.ssl.truststore.password=test1234
  12. database.history.consumer.ssl.key.password=test1234

See the Kafka documentation for more details about pass-through properties.

Pass-through properties for database drivers

In addition to the pass-through properties for the Kafka producer and consumer, there are pass-through properties for database drivers. These properties have the database. prefix. For example, database.tinyInt1isBit=false is passed to the JDBC URL.

Monitoring

The Debezium MySQL connector provides three types of metrics that are in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect provide.

  • Snapshot metrics provide information about connector operation while performing a snapshot.

  • Binlog metrics provide information about connector operation when the connector is reading the binlog.

  • Schema history metrics provide information about the status of the connector’s schema history.

Debezium monitoring documentation provides details for how to expose these metrics by using JMX.

Snapshot metrics

The MBean is debezium.mysql:type=connector-metrics,context=snapshot,server=*<database.server.name>*.

AttributesTypeDescription

string

The last snapshot event that the connector has read.

long

The number of milliseconds since the connector has read and processed the most recent event.

long

The total number of events that this connector has seen since last started or reset.

long

The number of events that have been filtered by include/exclude list filtering rules configured on the connector.

string[]

The list of tables that are monitored by the connector.

int

The length the queue used to pass events between the snapshotter and the main Kafka Connect loop.

int

The free capacity of the queue used to pass events between the snapshotter and the main Kafka Connect loop.

int

The total number of tables that are being included in the snapshot.

int

The number of tables that the snapshot has yet to copy.

boolean

Whether the snapshot was started.

boolean

Whether the snapshot was aborted.

boolean

Whether the snapshot completed.

long

The total number of seconds that the snapshot has taken so far, even if not complete.

Map<String, Long>

Map containing the number of rows scanned for each table in the snapshot. Tables are incrementally added to the Map during processing. Updates every 10,000 rows scanned and upon completing a table.

long

The maximum buffer of the queue in bytes. It will be enabled if max.queue.size.in.bytes is passed with a positive long value.

long

The current data of records in the queue in bytes.

The Debezium MySQL connector also provides the HoldingGlobalLock custom snapshot metric. This metric is set to a Boolean value that indicates whether the connector currently holds a global or table write lock.

Binlog metrics

The MBean is debezium.mysql:type=connector-metrics,context=binlog,server=<database.server.name>.

Transaction-related attributes are available only if binlog event buffering is enabled. See binlog.buffer.size in the advanced connector configuration properties for more details.

AttributesTypeDescription

string

The last streaming event that the connector has read.

long

The number of milliseconds since the connector has read and processed the most recent event.

long

The total number of events that this connector has seen since last started or reset.

long

The number of events that have been filtered by include/exclude list filtering rules configured on the connector.

string[]

The list of tables that are monitored by the connector.

int

The length the queue used to pass events between the streamer and the main Kafka Connect loop.

int

The free capacity of the queue used to pass events between the streamer and the main Kafka Connect loop.

boolean

Flag that denotes whether the connector is currently connected to the database server.

long

The number of milliseconds between the last change event’s timestamp and the connector processing it. The values will incoporate any differences between the clocks on the machines where the database server and the connector are running.

long

The number of processed transactions that were committed.

Map<String, String>

The coordinates of the last received event.

string

Transaction identifier of the last processed transaction.

long

The maximum buffer of the queue in bytes.

long

The current data of records in the queue in bytes.

The Debezium MySQL connector also provides the following custom binlog metrics:

Table 21. Descriptions of custom binlog metrics
AttributeTypeDescription

BinlogFilename

string

The name of the binlog file that the connector has most recently read.

BinlogPosition

long

The most recent position (in bytes) within the binlog that the connector has read.

IsGtidModeEnabled

boolean

Flag that denotes whether the connector is currently tracking GTIDs from MySQL server.

GtidSet

string

The string representation of the most recent GTID set processed by the connector when reading the binlog.

NumberOfSkippedEvents

long

The number of events that have been skipped by the MySQL connector. Typically events are skipped due to a malformed or unparseable event from MySQL’s binlog.

NumberOfDisconnects

long

The number of disconnects by the MySQL connector.

NumberOfRolledBackTransactions

long

The number of processed transactions that were rolled back and not streamed.

NumberOfNotWellFormedTransactions

long

The number of transactions that have not conformed to the expected protocol of BEGIN + COMMIT/ROLLBACK. This value should be 0 under normal conditions.

NumberOfLargeTransactions

long

The number of transactions that have not fit into the look-ahead buffer. For optimal performance, this value should be significantly smaller than NumberOfCommittedTransactions and NumberOfRolledBackTransactions.

Schema history metrics

The MBean is debezium.mysql:type=connector-metrics,context=schema-history,server=*<database.server.name>*.

AttributesTypeDescription

string

One of STOPPED, RECOVERING (recovering history from the storage), RUNNING describing the state of the database history.

long

The time in epoch seconds at what recovery has started.

long

The number of changes that were read during recovery phase.

long

the total number of schema changes applied during recovery and runtime.

long

The number of milliseconds that elapsed since the last change was recovered from the history store.

long

The number of milliseconds that elapsed since the last change was applied.

string

The string representation of the last change recovered from the history store.

string

The string representation of the last applied change.

Behavior when things go wrong

Debezium is a distributed system that captures all changes in multiple upstream databases; it never misses or loses an event. When the system is operating normally or being managed carefully then Debezium provides exactly once delivery of every change event record.

If a fault does happen then the system does not lose any events. However, while it is recovering from the fault, it might repeat some change events. In these abnormal situations, Debezium, like Kafka, provides at least once delivery of change events.

The rest of this section describes how Debezium handles various kinds of faults and problems.

Configuration and startup errors

In the following situations, the connector fails when trying to start, reports an error or exception in the log, and stops running:

  • The connector’s configuration is invalid.

  • The connector cannot successfully connect to the MySQL server by using the specified connection parameters.

  • The connector is attempting to restart at a position in the binlog for which MySQL no longer has the history available.

In these cases, the error message has details about the problem and possibly a suggested workaround. After you correct the configuration or address the MySQL problem, restart the connector.

MySQL becomes unavailable

If your MySQL server becomes unavailable, the Debezium MySQL connector fails with an error and the connector stops. When the server is available again, restart the connector.

However, if GTIDs are enabled for a highly available MySQL cluster, you can restart the connector immediately. It will connect to a different MySQL server in the cluster, find the location in the server’s binlog that represents the last transaction, and begin reading the new server’s binlog from that specific location.

If GTIDs are not enabled, the connector records the binlog position of only the MySQL server to which it was connected. To restart from the correct binlog position, you must reconnect to that specific server.

Kafka Connect stops gracefully

When Kafka Connect stops gracefully, there is a short delay while the Debezium MySQL connector tasks are stopped and restarted on new Kafka Connect processes.

Kafka Connect process crashes

If Kafka Connect crashes, the process stops and any Debezium MySQL connector tasks terminate without their most recently-processed offsets being recorded. In distributed mode, Kafka Connect restarts the connector tasks on other processes. However, the MySQL connector resumes from the last offset recorded by the earlier processes. This means that the replacement tasks might generate some of the same events processed prior to the crash, creating duplicate events.

Each change event message includes source-specific information that you can use to identify duplicate events, for example:

  • Event origin

  • MySQL server’s event time

  • The binlog file name and position

  • GTIDs (if used)

Kafka becomes unavailable

The Kafka Connect framework records Debezium change events in Kafka by using the Kafka producer API. If the Kafka brokers become unavailable, the Debezium MySQL connector pauses until the connection is reestablished and the connector resumes where it left off.

MySQL purges binlog files

If the Debezium MySQL connector stops for too long, the MySQL server purges older binlog files and the connector’s last position may be lost. When the connector is restarted, the MySQL server no longer has the starting point and the connector performs another initial snapshot. If the snapshot is disabled, the connector fails with an error.

See snapshots for details about how MySQL connectors perform initial snapshots.