Debezium Connector for MySQL

Overview of how the MySQL connector works

Table of Contents

The Debezium MySQL connector tracks the structure of the tables, performs snapshots, transforms binlog events into Debezium change events and records where those events are recorded in Kafka.

How the MySQL connector uses database schemas

When a database client queries a database, it uses the database’s current schema. As database schemas often change, the Debezium MySQL connector knows how the schema appeared for each INSERT, UPDATE, and DELETE operation.

MySQL includes both row-level changes and DDL statements in its binlog which the connector reads to parse and update the in-memory representation of each table’s schema. This is used to understand the table structure at the time of each operation, which produces accurate change events.

The connector records all DDL statements along with their position in the binlog in a separate database history so that when the connector restarts (after a possible crash or graceful shutdown), it continues reading the binlog from that specific point in time.
See The MySQL connector and Kafka topics for more on topic naming conventions.

Additional resources

How the MySQL connector performs database snapshots

When your Debezium MySQL connector is first started, it performs an initial consistent snapshot of your database. The following flow describes how this snapshot is completed.

This is the default snapshot mode which is set as initial in the snapshot.mode property. For other snapshots modes, please check out the MySQL connector configuration properties.

The connector…​

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 which might interfere with the connector’s attempt to read the binlog position and table schemas. The global read lock is kept while the binlog position is read before released 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 allowed by the connector’s configuration.

5

Releases the global read lock. This now allows other database clients to write to the database.

6

Writes the DDL changes to the schema change topic, including all necessary DROP…​ and CREATE…​ DDL statements.

This happens if applicable.

7

Scans the database tables and generates CREATE events on the relevant table-specific Kafka topics for each row.

8

Commits the transaction.

9

Records the completed snapshot in the connector offsets.

What happens if the connector fails?

If the connector fails, stops, or is rebalanced while making the initial snapshot, the connector creates a new snapshot once restarted. Once 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 MySQL connector common issues.

What if Global Read Locks are not allowed?

Some environments do not allow a global read lock. 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.

The user must have LOCK_TABLES privileges.

The connector…​

StepAction

1

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

2

Reads and filters the names of the databases and tables.

3

Reads the current binlog position.

4

Reads the schema of the databases and tables allowed by the connector’s configuration.

5

Writes the DDL changes to the schema change topic, including all necessary DROP…​ and CREATE…​ DDL statements.

This happens if applicable.

6

Scans the database tables and generates CREATE events on the relevant table-specific Kafka topics for each row.

7

Commits the transaction.

8

Releases the table-level locks.

9

Records the completed snapshot in the connector offsets.

How the MySQL connector handles schema change topics

You can configure the Debezium MySQL connector to produce schema change events that include all DDL statements applied to databases in the MySQL server. The connector writes all of these events to a Kafka topic named <serverName> where serverName is the name of the connector as specified in the database.server.name configuration property.

If you choose to use schema change events, use the schema change topic and do not consume the database history topic.
Make sure that the num.partitions configuration for Kafka is set to 1 to ensure schema changes are kept in the correct order.

Schema change topic structure

Each message that is written to the schema change topic contains a message key which includes the name of the connected database used when applying DDL statements:

  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 message value contains a structure that includes the DDL statements, the database to which the statements were applied, and the position in the binlog where the statements appeared:

  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_sec"
  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": "0.10.0.Beta4",
  98. "name": "mysql-server-1",
  99. "server_id": 0,
  100. "ts_sec": 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. }
Important tips regarding schema change topics

The ddl field may contain multiple DDL statements. Every statement applies to the database in the databaseName field and appears in the same order as they were applied in 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 topic.

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

What if a client submits DDL statements 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.

Additional resources

MySQL connector events

All data change events produced by the Debezium MySQL connector contain a key and a value. The change event key and the change event value each contain a schema and a payload where the schema describes the structure of the payload and the payload contains the data.

The MySQL connector ensures that all Kafka Connect schema names adhere to the Avro schema name format. This is important as any character that is not a latin letter or underscore is replaced by an underscore which can lead to unexpected conflicts in schema names when the logical server names, database names, and table names container other characters that are replaced with these underscores.

Change event key

For any given table, the change event’s key has a structure that contains a field for each column in the PRIMARY KEY (or unique constraint) at the time the event was created. Let us look at an example table and then how the schema and payload would appear for the table.

example 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;

example change event key

  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. }
1The schema describes what is in the payload.
2The mysql-server-1.inventory.customers.Key is the name of the schema which defines the structure where mysql-server-1 is the connector name, inventory is the database, and customers is the table.
3Denotes that the payload is not optional.
4Specifies the type of fields expected in the payload.
5The payload itself, which in this case only contains a single id field.

This key describes the row in the inventory.customers table which is out from the connector entitled mysql-server-1 whose id primary key column has a value of 1001.

Change event value

The change event value contains a schema and a payload section. There are three types of change event values which have an envelope structure. The fields in this structure are explained below and marked on each of the change event value examples.

ItemField nameDescription

1

name

mysql-server-1.inventory.customers.Key is the name of the schema which defines the structure where mysql-server-1 is the connector name, inventory is the database and customers is the table

2

op

A mandatory string that describes the type of operation.

values
  • c = create

  • u = update

  • d = delete

  • r = read (initial snapshot only)

3

before

An optional field that specifies the state of the row before the event occurred.

4

after

An optional field that specifies the state of the row after the event occurred.

5

source

A mandatory field that describes the source metadata for the event including:

  • the Debezium version

  • the connector name

  • the binlog name where the event was recorded

  • the binlog position

  • the row within the event

  • if the event was part of a snapshot

  • the name of the affected database and table

  • the id of the MySQL thread creating the event (non-snapshot only)

  • the MySQL server ID (if available)

  • timestamp

If the binlog_rows_query_log_events option is enabled and the connector has the include.query option enabled, a query field is displayed which contains the original SQL statement that generated the event.

6

ts_ms

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

Let us look at an example table and then how the schema and payload would appear for the table.

example 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;
Create change event value

This example shows a create event for 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",
  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.product.connector.mysql.Source",
  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"
  153. },
  154. "payload": { (2)
  155. "op": "c",
  156. "ts_ms": 1465491411815,
  157. "before": null,
  158. "after": {
  159. "id": 1004,
  160. "first_name": "Anne",
  161. "last_name": "Kretchmar",
  162. "email": "annek@noanswer.org"
  163. },
  164. "source": {
  165. "version": "1.0.3.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. }
1The schema portion of this event’s value shows the schema for the envelope, the schema for the source structure (which is specific to the MySQL connector and reused across all events), and the table-specific schemas for the before and after fields.

The names of the schemas for the before and after fields are of the form <logicalName>.<tableName>.Value, and thus are entirely independent from all other schemas for all other tables. This means that when using the Avro Converter, the resulting Avro schemas for each table in each logical source have their own evolution and history.

2The payload portion of this event’s value shows the information in the event, namely that it is describing that the row was created (because op=c), and that the after field value contains the values of the new inserted row’s id, first_name, last_name, and email columns.

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 dramatically decrease the size of the actual messages written to the Kafka topics.

Update change event value

The value of an update change event on the customers table has the exact same schema as a create event. The payload is structured the same, but holds different values. Here is an example (formatted for readability):

  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.0.3.Final",
  18. "name": "mysql-server-1",
  19. "connector": "mysql",
  20. "name": "mysql-server-1",
  21. "ts_ms": 1465581,
  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. }

Comparing this to the value in the insert event, you can see a couple of differences in the payload section:

1The before field now has the state of the row with the values before the database commit.
2The after field now has the updated state of the row, and the first_name value is now Anne Marie. You can compare the before and after structures to determine what actually changed in this row because of the commit.
3The source field structure has the same fields as before, but the values are different (this event is from a different position in the binlog). The source structure shows information about MySQL’s record of this change (providing traceability). It also has information you can use to compare to other events in this and other topics to know whether this event occurred before, after, or as part of the same MySQL commit as other events.
4The op field value is now u, signifying that this row changed because of an update.
5The ts_ms field shows the timestamp when Debezium processed this event.

When the columns for a row’s primary or unique key are updated, the value of the row’s key is changed and Debezium outputs three events: a DELETE event and tombstone event with the old key for the row, followed by an INSERT event with the new key for the row.

Delete change event value

The value of a delete change event on the customers table has the exact same schema as create and update events. The payload is structured the same, but holds different values. Here is an example (formatted for readability):

  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.0.3.Final",
  13. "connector": "mysql",
  14. "name": "mysql-server-1",
  15. "ts_ms": 1465581,
  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. }

Comparing the payload portion to the payloads in the create and update events, you can see some differences:

1The before field now has the state of the row that was deleted with the database commit.
2The after field is null, signifying that the row no longer exists.
3The source field structure has many of the same values as before, except the ts_sec and pos fields have changed (and the file might have changed in other scenarios).
4The op field value is now d, signifying that this row was deleted.
5The ts_ms shows the timestamp when Debezium processed this event.

This event provides a consumer with the information that 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.

The MySQL connector’s events are designed to work with Kafka log compaction, which allows for the removal of some older messages as long as at least the most recent message for every key is kept. This allows Kafka to reclaim storage space while ensuring the topic contains a complete data set and can be used for reloading key-based state.

When a row is deleted, the delete event value listed above still works with log compaction, because Kafka can still remove all earlier messages with that same key. If the message value is null, Kafka knows that it can remove all messages with that same key. To make this possible, Debezium’s MySQL connector always follows a delete event with a special tombstone event that has the same key but a null value.

How the MySQL connector maps data types

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 the value is represented 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 following table shows how the connector maps the 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)

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.
example (where n is bits)
  1. 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

n/a

VARBINARY(M)]

BYTES

n/a

TINYBLOB

BYTES

n/a

TINYTEXT

STRING

n/a

BLOB

BYTES

n/a

TEXT

STRING

n/a

MEDIUMBLOB

BYTES

n/a

MEDIUMTEXT

STRING

n/a

LONGBLOB

BYTES

n/a

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 values

Excluding the TIMESTAMP data type, MySQL temporal types depend on the value of the time.precision.mode configuration property.

See MySQL connector configuration properties for more details.

Temporal values without timezones are converted from UTC to milliseconds or microseconds (DATETIME) or to the configured database timezone (TIMESTAMP).

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

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

time.precision.mode=adaptive_time_microseconds(default)

The MySQL connector determins 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.

MySQL typeLiteral typeSemantic type

DATE

INT32

io.debezium.time.Date

Represents the number of days since epoch.

TIME[(M)]

INT64

io.debezium.time.MicroTime

Represents the time value in microseconds and does not include timezone 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 epoch and does not include timezone information.

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

INT64

io.debezium.time.MicroTimestamp

Represents the number of microseconds past epoch and does not include timezone information.

time.precision.mode=connect

The MySQL connector uses the predefined 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.

MySQL typeLiteral typeSemantic type

DATE

INT32

org.apache.kafka.connect.data.Date

Represents the number of days since epoch.

TIME[(M)]

INT64

org.apache.kafka.connect.data.Time

Represents the time value in microseconds since midnight and does not include timezone information.

DATETIME[(M)]

INT64

org.apache.kafka.connect.data.Timestamp

Represents the number of milliseconds since epoch, and does not include timezone information.

Decimal values

Decimals are handled via the decimal.handling.mode property.

See MySQL connector configuration properties for more details.

decimal.handling.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

MySQL typeLiteral typeSemantic type

NUMERIC[(M[,D])]

FLOAT64

n/a

DECIMAL[(M[,D])]

FLOAT64

n/a

decimal.handling.mode=string

MySQL typeLiteral typeSemantic type

NUMERIC[(M[,D])]

STRING

n/a

DECIMAL[(M[,D])]

STRING

n/a

Spatial data types

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

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: a spatial reference system id that defines the type of geometry object stored in the structure

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

The MySQL connector and Kafka topics

The Debezium MySQL connector writes events for all INSERT, UPDATE, and DELETE operations from a single table to a single Kafka topic. The Kafka topic naming convention is as follows:

format

  1. serverName.databaseName.tableName

Example 1. example

Let us say that fulfillment is the server name and inventory is the database which contains three tables of orders, customers, and products. The Debezium MySQL connector produces events on three Kafka topics, one for each table in the database:

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

MySQL supported 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.

Master and slave

The Debezium MySQL connector can follow one of the masters or one of the slaves (if that slave has its binlog enabled), but the connector only sees changes in the cluster that are visible to that server. Generally, this is not a problem except for the multi-master topologies.

The connector records its position in the server’s binlog, which is different on each server in the cluster. Therefore, the connector will need to follow just one MySQL server instance. If that server fails, it 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 far easier to tolerate and almost immediately recover from problems and failures. Most HA MySQL clusters use GTIDs so that slaves are able to keep track of all changes on any of the master.

Multi-master

A multi-master MySQL topology uses one or more MySQL slaves that each replicate from multiple masters. This is a powerful way to aggregate the replication of multiple MySQL clusters, and requires using GTIDs.

The Debezium MySQL connector can use these multi-master MySQL slaves as sources, and can fail over to different multi-master MySQL slaves as long as thew new slave is caught up to the old slave (e.g., the new slave has all of the transactions that were last seen on the first slave). This works even if the connector is only using 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-master MySQL slave 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.

Setting up MySQL server

Table of Contents

Creating a MySQL user for Debezium

You have to define a MySQL user with appropriate permissions on all databases that the Debezium MySQL connector monitors.

Prerequisites

  • You must have a MySQL server.

  • You must know basic SQL commands.

Procedure

  1. Create the MySQL user:
  1. mysql> CREATE USER 'user'@'localhost' IDENTIFIED BY 'password';
  1. Grant the required permissions to the user:
  1. mysql> GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO 'user' IDENTIFIED BY 'password';
See permissions explained for notes on each permission.
If using a hosted option such as Amazon RDS or Amazon Aurora that do 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 Overview of how the MySQL connector works for more details.
  1. Finalize the user’s permissions:
  1. mysql> FLUSH PRIVILEGES;

Permissions explained

Permission/itemDescription

SELECT

enables the connector to select rows from tables in databases

This is only used 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 only used when performing a snapshot.

SHOW DATABASES

enables the connector to see database names by issuing the SHOW DATABASE statement.

This is only used 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 following statements:

  • SHOW MASTER STATUS

  • SHOW SLAVE STATUS

  • SHOW BINARY LOGS

This is always required for the connector.

ON

Identifies the database to which the permission apply.

TO ‘user’

Specifies the user to which the permissions are granted.

IDENTIFIED BY ‘password’

Specifies the password for the user.

Enabling the MySQL binlog for Debezium

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

Prerequisites

  • You must have a MySQL server.

  • You should have appropriate MySQL user privileges.

Procedure

  1. Check if the log-bin option is already on or not.
  1. mysql> SELECT variable_value as "BINARY LOGGING STATUS (log-bin) ::"
  2. FROM information_schema.global_variables WHERE variable_name='log_bin';
  1. If OFF, configure your MySQL server configuration file with the following:
See Binlog config properties for notes on each property.
  1. server-id = 223344 (1)
  2. log_bin = mysql-bin (2)
  3. binlog_format = ROW (3)
  4. binlog_row_image = FULL (4)
  5. expire_logs_days = 10 (5)
  1. 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';

Binlog configuration properties

NumberPropertyDescription

1

server-id

The value for the server-id must be unique for each server and replication client within the MySQL cluster. When the MySQL connector is setup, we assign the connector a unique server ID.

2

log_bin

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

3

binlog_format

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

4

binlog_row_image

The binlog_row_image must be set to FULL or full.

5

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.

Enabling MySQL Global Transaction Identifiers for Debezium

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

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

Prerequisites

  • You must have a MySQL server.

  • You must know basic SQL commands.

  • You must have access to the MySQL configuration file.

Procedure

  1. Enable gtid_mode:
  1. mysql> gtid_mode=ON
  1. Enable enforce_gtid_consistency:
  1. mysql> enforce_gtid_consistency=ON
  1. Confirm the changes:
  1. mysql> show global variables like '%GTID%';

response

  1. +--------------------------+-------+
  2. | Variable_name | Value |
  3. +--------------------------+-------+
  4. | enforce_gtid_consistency | ON |
  5. | gtid_mode | ON |
  6. +--------------------------+-------+

Options explained

Permission/itemDescription

gtid_mode

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

  • ON = enabled

  • OFF = disabled

enforce_gtid_consistency

Boolean which instructs the server whether or not to enforce 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

Setting up session timeouts for Debezium

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

  • You must have a MySQL server.

  • You must know basic SQL commands.

  • You must have access to the MySQL configuration file.

Procedure

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

Options explained

Permission/itemDescription

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 noninteractive connection before closing it.

See MySQL’s documentation for more details.

Enabling query log events for Debezium

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 only available from MySQL 5.6 and later.

Prerequisites

  • You must have a MySQL server.

  • You must know basic SQL commands.

  • You must have access to the MySQL configuration file.

Procedure

  1. Enable binlog_rows_query_log_events:
  1. mysql> binlog_rows_query_log_events=ON

Options explained

Permission/itemDescription

binlog_rows_query_log_events`

Boolean which enables/disables support for including the original SQL statement in the binlog entry.

  • ON = enabled

  • OFF = disabled

Deploying the MySQL connector

Table of Contents

Installing the MySQL connector

Installing the Debezium MySQL connector is a simple process whereby you only need to download the JAR, extract it to your Kafka Connect environment, and ensure the plugin’s parent directory is specified in your Kafka Connect environment.

Prerequisites

Procedure

  1. Download the Debezium MySQL connector plugin.

  2. Extract the files into your Kafka Connect environment.

  3. Add the plugin’s parent directory to your Kafka Connect plugin path:

  1. plugin.path=/kafka/connect
The above example assumes you have extracted the Debezium MySQL connector to the /kafka/connect/Debezium-connector-mysql path.
  1. Restart your Kafka Connect process. This ensures the new JARs are picked up.

Configuring the MySQL connector

Typically, you configure the Debezium MySQL connector in a .json file using the configuration properties available for the connector.

Prerequisites

Procedure

  1. Set the "name" of the connector in the .json file.

  2. Set the configuration properties that you require for your Debezium MySQL connector.

For a complete list of configuration properties, see MySQL connector configuration properties.

Example 2. MySQL connector example configuration

  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.whitelist": "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. }

Example configuration properties explained

  1. The connector’s name when registered with the Kafka Connect service.

  2. The connector’s class name.

  3. The MySQL server address.

  4. The MySQL server port number.

  5. The MySQL user with the appropriate privileges.

  6. The MySQL user’s password.

  7. The unique ID of the connector.

  8. The logical name of the MySQL server or cluster.

  9. A list of all databases hosted by the specified server.

  10. A list of Kafka brokers that the connector uses to write and recover DDL statements to the database history topic.

  11. The name of the database history topic.

  12. The flag that specifies if the connector should generate on the schema change topic named fulfillment events with DDL changes that can be used by consumers.

MySQL connector configuration properties

The configuration properties listed here are required to run the Debezium MySQL connector. There are also advanced MySQL connector properties whose default value rarely needs to be changed and therefore, they do not need to be specified in the connector configuration.

The Debezium MySQL connector supports pass-through configuration when creating the Kafka producer and consumer. See the Kafka documentation for more details on pass-through properties.
PropertyDefaultDescription

name

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

connector.class

The name of the Java class for the connector. Always use a value of io.debezium​.connector.mysql.MySqlConnector for the MySQL connector.

tasks.max

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.

database.hostname

IP address or hostname of the MySQL database server.

database.port

3306

Integer port number of the MySQL database server.

database.user

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

database.password

Password to use when connecting to the MySQL database server.

database.server.name

Logical name that identifies and provides a namespace for the particular MySQL database server/cluster being monitored. The logical name should be unique across all other connectors, since it is used as a prefix for all Kafka topic names emanating from this connector. Only alphanumeric characters and underscores should be used.

database.server.id

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 is generated between 5400 and 6400, though we recommend setting an explicit value.

database.history.kafka.topic

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

database.history​.kafka.bootstrap.servers

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

database.whitelist

empty string

An optional comma-separated list of regular expressions that match database names to be monitored; any database name not included in the whitelist will be excluded from monitoring. By default all databases will be monitored. May not be used with database.blacklist.

database.blacklist

empty string

An optional comma-separated list of regular expressions that match database names to be excluded from monitoring; any database name not included in the blacklist will be monitored. May not be used with database.whitelist.

table.whitelist

empty string

An optional comma-separated list of regular expressions that match fully-qualified table identifiers for tables to be monitored; any table not included in the whitelist will be excluded from monitoring. Each identifier is of the form databaseName.tableName. By default the connector will monitor every non-system table in each monitored database. May not be used with table.blacklist.

table.blacklist

empty string

An optional comma-separated list of regular expressions that match fully-qualified table identifiers for tables to be excluded from monitoring; any table not included in the blacklist will be monitored. Each identifier is of the form databaseName.tableName. May not be used with table.whitelist.

column.blacklist

empty string

An optional comma-separated list of regular expressions that match the fully-qualified names of columns that should be excluded from change event message values. Fully-qualified names for columns are of the form databaseName.tableName.columnName, or databaseName.schemaName.tableName.columnName.

column.truncate.to.length.chars

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 message values if the field values are longer than the specified number of characters. Multiple properties with different lengths can be used in a single configuration, although in each the length must be a positive integer. Fully-qualified names for columns are of the form databaseName.tableName.columnName, or databaseName.schemaName.tableName.columnName.

column.mask.with.length.chars

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. Multiple properties with different lengths can be used in a single configuration, although in each the length must be a positive integer or zero. Fully-qualified names for columns are of the form databaseName.tableName.columnName, or databaseName.schemaName.tableName.columnName.

column.propagate.source.type

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 messages. The schema parameters Debezium.source.column.type, Debezium.source.column.length and _Debezium.source.column.scale will be used to propagate the original type name and length (for variable-width types), respectively. Useful to properly size corresponding columns in sink databases. Fully-qualified names for columns are of the form databaseName.tableName.columnName, or databaseName.schemaName.tableName.columnName.

time.precision.mode

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; or connect always represents time and timestamp values using Kafka Connect’s built-in representations for Time, Date, and Timestamp, which uses millisecond precision regardless of the database columns’ precision.

decimal.handling.mode

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; or double represents them using double values, which may result in a loss of precision but will be far easier to use. string option encodes values as formatted string which is easy to consume but a semantic information about the real type is lost.

bigint.unsigned.handling.mode

long

Specifies how BIGINT UNSIGNED columns should be represented in change events, including: precise uses java.math.BigDecimal to represent values, which are encoded in the change events using a binary representation and Kafka Connect’s org.apache.kafka.connect.data.Decimal type; long (the default) represents values using Java’s long, which may not offer the precision but will be far easier to use in consumers. long is usually the preferable setting. Only when working with values larger than 2^63, the precise setting should be used as those values can’t be conveyed using long.

include.schema.changes

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 will be recorded 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. The default is true.

include.query

false

Boolean value that specifies whether the connector should include the original SQL query that generated the change event.
Note: This option requires MySQL be configured with the binlog_rows_query_log_events option set to ON. Query will not be present for events generated from the snapshot process.
WARNING: Enabling this option may expose tables or fields explicitly blacklisted or masked by including the original SQL statement in the change event. For this reason this option is defaulted to ‘false’.

event.processing​.failure.handling.mode

fail

Specifies how the connector should react to exceptions during deserialization of binlog events. fail will propagate the exception (indicating the problematic event and its binlog offset), causing the connector to stop.
warn will cause the problematic event to be skipped and the problematic event and its binlog offset to be logged.
skip will cause problematic event will be skipped.

inconsistent.schema.handling.mode

fail

Specifies how the connector should react to binlog events that relate to tables that are not present in internal schema representation (i.e. internal representation is not consistent with database) fail will throw an exception (indicating the problematic event and its binlog offset), causing the connector to stop.
warn will cause the problematic event to be skipped and the problematic event and its binlog offset to be logged.
skip will cause the problematic event to be skipped.

max.queue.size

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 slower 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 in the max.batch.size property.

max.batch.size

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.

poll.interval.ms

1000

Positive integer value that specifies the number of milliseconds the connector should wait during each iteration for new change events to appear. Defaults to 1000 milliseconds, or 1 second.

connect.timeout.ms

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.

gtid.source.includes

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 matching one of these include patterns will be used. May not be used with gtid.source.excludes.

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 matching none of these exclude patterns will be used. May not be used with 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 will start 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 a active-passive MySQL setup where Debezium is connected to master, in this case during failover the slave with new UUID (and GTID channel) starts receiving writes before Debezium is connected. These writes would be lost when using latest.

tombstones.on.delete

true

Controls whether a tombstone event should be generated after a delete event.
When true the delete operations are represented by a delete event and a subsequent tombstone event. When false only a delete event is sent.
Emitting the tombstone event (the default behavior) allows Kafka to completely delete all events pertaining to the given key once the source record got deleted.

message.key.columns

empty string

A semi-colon list of regular expressions that match fully-qualified tables and columns to map a primary key.
Each item (regular expression) must match the <fully-qualified table>:<a comma-separated list of columns> representing the custom key.
Fully-qualified tables could be defined as DB_NAME.TABLE_NAME or SCHEMA_NAME.TABLE_NAME, depending on the specific connector.

Advanced MySQL connector properties

PropertyDefaultDescription

connect.keep.alive

true

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

table.ignore.builtin

true

Boolean value that specifies whether built-in system tables should be ignored. This applies regardless of the table whitelist or blacklists. By default system tables are excluded from monitoring, and no events are generated when changes are made to any of the system tables.

database.history.kafka.recovery.poll.interval.ms

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.

database.history.kafka.recovery.attempts

4

The maximum number of times that the connector should attempt 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.

database.history.skip.unparseable.ddl

false

Boolean value that specifies if connector should ignore malformed or unknown database statements or stop processing and let operator to 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 binlog is processed.

database.history.store.only.monitored.tables.ddl

false

Boolean value that specifies if connector should should record all DDL statements or (when true) only those that are relevant to tables that are monitored by Debezium (via filter configuration). The safe default is false. This feature should be used only with care as the missing data might be necessary when the filters are changed.

database.ssl.mode

disabled

Specifies whether to use an encrypted connection. The default is disabled, and specifies to use an unencrypted connection.

The preferred option establishes an encrypted connection if the server supports secure connections but falls back to an unencrypted connection otherwise.

The required option establishes an encrypted connection but will fail if one cannot be made for any reason.

The verify_ca option behaves like required but additionally it verifies the server TLS certificate against the configured Certificate Authority (CA) certificates and will fail if it doesn’t match any valid CA certificates.

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

binlog.buffer.size

0

The size of a look-ahead buffer used by the binlog reader.
Under specific conditions it is possible that 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 decide whether the changes from the transaction will be streamed or not. The size of the buffer defines the maximum number of changes in the transaction that Debezium can buffer while searching for transaction boundaries. If the size of transaction is larger than the buffer then Debezium needs to rewind and re-read the events that has not fit into the buffer while streaming. Value 0 disables buffering.
Disabled by default.
Note: This feature should be considered an incubating one. We need a feedback from customers but it is expected that it is not completely polished.

snapshot.mode

initial

Specifies the criteria for running a snapshot upon startup of the connector. The default is initial, and specifies the connector can run a snapshot only when no offsets have been recorded for the logical server name. The when_needed option specifies that the connector run a snapshot upon startup whenever it deems it necessary (when no offsets are available, or when a previously recorded offset specifies a binlog location or GTID that is not available in the server). The never option specifies that the connect should never use snapshots and that upon first startup with a logical server name the connector should read from the beginning of the binlog; this should be used with care, as it is only valid when the binlog is guaranteed to contain the entire history of the database. If you don’t need the topics to contain a consistent snapshot of the data but only need them to have the changes since the connector was started, you can use the schema_only option, where the connector only snapshots the schemas (not the data).

schema_only_recovery is a recovery option for an existing connector to recover a corrupted or lost database history topic, or to periodically “clean up” a database history topic (which requires infinite retention) that may be growing unexpectedly.

snapshot.locking.mode

minimal

Controls if and how long the connector holds onto the global MySQL read lock (preventing any updates to the database) while it is performing a snapshot. There are three possible values minimal, extended, and none.

minimal The connector holds the global read lock for just the initial portion of the snapshot while the connector reads the database schemas and other metadata. The remaining work in a snapshot involves selecting all rows from each table, and this can be done in a consistent fashion using the REPEATABLE READ transaction even when the global read lock is no longer held and while other MySQL clients are updating the database.

extended In some cases where clients are submitting operations that MySQL excludes from REPEATABLE READ semantics, it may be desirable to block all writes for the entire duration of the snapshot. For these such cases, use this option.

none Will prevent the connector from acquiring any table locks during the snapshot process. This value can be used with all snapshot modes but it is safe to use if and only if no schema changes are happening while the snapshot is taken. Note that for tables defined with MyISAM engine, the tables would still be locked despite this property being set as MyISAM acquires a table lock. This behaviour is unlike InnoDB engine which acquires row level locks.

snapshot.select.statement.overrides

Controls which rows from tables will be included in snapshot.
This property contains a comma-separated list of fully-qualified tables (DB_NAME.TABLE_NAME). Select statements for the individual tables are specified in further configuration properties, one for each table, identified by the id snapshot.select.statement.overrides.[DB_NAME].[TABLE_NAME]. The value of those properties is the SELECT statement to use when retrieving data from the specific table during snapshotting. A possible use case for large append-only tables is setting a specific point where to start (resume) snapshotting, in case a previous snapshotting was interrupted.
Note: This setting has impact on snapshots only. Events captured from binlog are not affected by it at all.

min.row.count.to.stream.results

1000

During a snapshot operation, the connector will query each included table to produce a read event for all rows in that table. This parameter determines whether the MySQL connection will pull all results for a table into memory (which is fast but requires large amounts of memory), or whether the results will instead be streamed (can be slower, but will work for very large tables). The value specifies the minimum number of rows a table must contain before the connector will stream results, and defaults to 1,000. Set this parameter to ‘0’ to skip all table size checks and always stream all results during a snapshot.

heartbeat.interval.ms

0

Controls how frequently the heartbeat messages are sent.
This property contains an interval in milli-seconds that defines how frequently the connector sends heartbeat messages into a heartbeat topic. Set this parameter to 0 to not send heartbeat messages at all.
Disabled by default.

heartbeat.topics.prefix

__debezium-heartbeat

Controls the naming of the topic to which heartbeat messages are sent.
The topic is named according to the pattern <heartbeat.topics.prefix>.<server.name>.

database.initial.statements

A semicolon separated list of SQL statements to be executed when a JDBC connection (not the transaction log reading connection) to the database is established. Use doubled semicolon (‘;;’) to use a semicolon as a character and not as a delimiter.
Note: The connector may establish JDBC connections at its own discretion, so this should typically be used for configuration of session parameters only, but not for executing DML statements.

snapshot.delay.ms

An interval in milli-seconds that the connector should wait before taking a snapshot after starting up;
Can be used to avoid snapshot interruptions when starting multiple connectors in a cluster, which may cause re-balancing of connectors.

snapshot.fetch.size

Specifies the maximum number of rows that should be read in one go from each table while taking a snapshot. The connector will read the table contents in multiple batches of this size.

snapshot.lock.timeout.ms

10000

Positive integer value that specifies the maximum amount of time (in milliseconds) to wait to obtain table locks when performing a snapshot. If table locks cannot be acquired in this time interval, the snapshot will fail. See How the MySQL connector performs database snapshots.

enable.time.adjuster

MySQL allows user to insert year value as either 2-digit or 4-digit. In case of two digits the value is automatically mapped to 1970 - 2069 range. This is usually done by database.
Set to true (the default) when Debezium should do the conversion.
Set to false when conversion is fully delegated to the database.

source.struct.version

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. Note that this setting is not recommended and is planned for removal in a future Debezium version.

sanitize.field.names

true when connector configuration explicitly specifies the key.converter or value.converter parameters to use Avro, otherwise defaults to false.

Whether field names will be sanitized to adhere to Avro naming requirements.

MySQL connector monitoring metrics

The Debezium MySQL connector has three metric types in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect have.

Snapshot metrics

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

AttributeTypeDescription

TotalTableCount

int

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

RemainingTableCount

int

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

HoldingGlobalLock

boolean

Whether the connector currently holds a global or table write lock.

SnapshotRunning

boolean

Whether the snapshot was started.

SnapshotAborted

boolean

Whether the snapshot was aborted.

SnapshotCompleted

boolean

Whether the snapshot completed.

SnapshotDurationInSeconds

long

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

RowsScanned

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.

LastEvent

string

The last snapshot event that the connector has read.

MilliSecondsSinceLastEvent

long

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

TotalNumberOfEventsSeen

long

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

NumberOfEventsFiltered

long

The number of events that have been filtered by whitelist or blacklist filtering rules configured on the connector.

MonitoredTables

string[]

The list of tables that are monitored by the connector.

QueueTotalCapcity

int

The length of the queue used to pass events between snapshot reader and the main Kafka Connect loop.

QueueRemainingCapcity

int

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

Binlog metrics

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

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

Connected

boolean

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

BinlogFilename

string

The name of the binlog filename 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 seen by the connector when reading the binlog.

LastEvent

string

The last binlog event that the connector has read.

SecondsSinceLastEvent (obsolete)

long

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

SecondsBehindMaster (obsolete)

long

The number of seconds between the last event’s MySQL timestamp and the connector processing it. The values will incorporate any differences between the clocks on the machines where the MySQL server and the MySQL connector are running.

MilliSecondsBehindSource

long

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

TotalNumberOfEventsSeen

long

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

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.

NumberOfEventsFiltered

long

The number of events that have been filtered by whitelist or blacklist filtering rules configured on the connector.

NumberOfDisconnects

long

The number of disconnects by the MySQL connector.

SourceEventPosition

map<string, string>

The coordinates of the last received event.

LastTransactionId

string

Transaction identifier of the last processed transaction.

LastEvent

string

The last binlog event that the connector has read.

MilliSecondsSinceLastEvent

long

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

MonitoredTables

string[]

The list of tables that are monitored by Debezium.

QueueTotalCapcity

int

The length of the queue used to pass events between binlog reader and the main Kafka Connect loop.

QueueRemainingCapcity

int

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

NumberOfCommittedTransactions

long

The number of processed transactions that were committed.

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 expected protocol BEGIN + COMMIT/ROLLBACK. Should be 0 under normal conditions.

NumberOfLargeTransactions

long

The number of transactions that have not fitted into the look-ahead buffer. Should be significantly smaller than NumberOfCommittedTransactions and NumberOfRolledBackTransactions for optimal performance.

Schema history metrics

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

AttributeTypeDescription

Status

string

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

RecoveryStartTime

long

The time in epoch seconds at what recovery has started.

ChangesRecovered

long

The number of changes that were read during recovery phase.

ChangesApplied

long

The total number of schema changes applie during recovery and runtime.

MilliSecondsSinceLastRecoveredChange

long

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

MilliSecondsSinceLastAppliedChange

long

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

LastRecoveredChange

string

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

LastAppliedChange

string

The string representation of the last applied change.

MySQL connector common issues

Table of Contents

Configuration and startup errors

The Debezium MySQL connector fails, reports an error, and stops running when the following startup errors occur:

  • The connector’s configuration is invalid.

  • The connector cannot connect to the MySQL server using the specified connectivity parameters.

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

If you receive any of these errors, you receive more details in the error message. The error message also contains workarounds where possible.

MySQL is unavailable

If your MySQL server becomes unavailable, the Debezium MySQL connector fails with an error and the connector stops. You simply need to restart the connector when the server is available.

Using GTIDs

If you have GTIDs enabled and a highly available MySQL cluster, restart the connector immediately as the connector will simply 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.

Not Using GTIDs

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

Kafka Connect stops

There are three scenarios that cause some issues when Kafka Connect stops:

Kafka Connect stops gracefully

When Kafka Connect stops gracefully, there is only 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 may generate some of the same events processed prior to the crash, creating duplicate events.

Each change event message includes source-specific information about:
  • the event origin

  • the MySQL server’s event time

  • the binlog filename and position

  • GTIDs (if used)

Kafka becomes unavailable

The Kafka Connect framework records Debezium change events in Kafka 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 last 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 How the MySQL connector performs database snapshots for more information on initial snapshots.