Debezium Connector for MySQL
MySQL has a binary log (binlog) that records all operations in the order in which they are committed to the database. This includes changes to table schemas and the data within tables. MySQL uses the binlog for replication and recovery.
The Debezium MySQL connector reads the binlog and produces change events for row-level INSERT
, UPDATE
, and DELETE
operations and records the change events in a Kafka topic. Client applications read those Kafka topics.
As MySQL is typically set up to purge binlogs after a specified period of time, the MySQL connector performs and initial consistent snapshot of each of your databases. The MySQL connector reads the binlog from the point at which the snapshot was made.
The following sections provide more detailed information on how the Debezium MySQL connector functions, instructs you on how to set it up, as well as deploy and troubleshoot the connector.
Overview of how the MySQL connector works
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, 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 the row-level changes to the data and 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 and to produce the appropriate change event. In a separate database history Kafka topic, the connector also 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 generate schema change events on a different topic that is intended for consumer applications. This is described in how the MySQL connector exposes schema changes.
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 then helper tables created during the migration process need to be included among whitelisted tables.
If downstream systems do not need the messages generated by the temporary table then a simple message transform can be written and applied to filter them out.
For information about topic naming conventions, see MySQL connector and Kafka topics.
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…
Step | Action | ||
---|---|---|---|
|
Grabs a global read lock that blocks writes by other database clients.
|
||
|
Starts a transaction with repeatable read semantics to ensure that all subsequent reads within the transaction are done against the consistent snapshot. |
||
|
Reads the current binlog position. |
||
|
Reads the schema of the databases and tables allowed by the connector’s configuration. |
||
|
Releases the global read lock. This now allows other database clients to write to the database. |
||
|
Writes the DDL changes to the schema change topic, including all necessary
|
||
|
Scans the database tables and generates |
||
|
Commits the transaction. |
||
|
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…
Step | Action | ||
---|---|---|---|
|
Starts a transaction with repeatable read semantics to ensure that all subsequent reads within the transaction are done against the consistent snapshot. |
||
|
Reads and filters the names of the databases and tables. |
||
|
Reads the current binlog position. |
||
|
Reads the schema of the databases and tables allowed by the connector’s configuration. |
||
|
Writes the DDL changes to the schema change topic, including all necessary
|
||
|
Scans the database tables and generates |
||
|
Commits the transaction. |
||
|
Releases the table-level locks. |
||
|
Records the completed snapshot in the connector offsets. |
How the MySQL connector exposes schema changes
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. |
It is vital that there is a global order of the events in the database schema history.
Therefore, the database history topic must not be partitioned. This means that a partition
count of 1 must be specified when creating this topic. When relying on auto topic
creation, make sure that Kafka’s num.partitions configuration option
(the default number of partitions) is set to 1 .
|
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:
{
"schema": {
"type": "struct",
"name": "io.debezium.connector.mysql.SchemaChangeKey",
"optional": false,
"fields": [
{
"field": "databaseName",
"type": "string",
"optional": false
}
]
},
"payload": {
"databaseName": "inventory"
}
}
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:
{
"schema": {
"type": "struct",
"name": "io.debezium.connector.mysql.SchemaChangeValue",
"optional": false,
"fields": [
{
"field": "databaseName",
"type": "string",
"optional": false
},
{
"field": "ddl",
"type": "string",
"optional": false
},
{
"field": "source",
"type": "struct",
"name": "io.debezium.connector.mysql.Source",
"optional": false,
"fields": [
{
"type": "string",
"optional": true,
"field": "version"
},
{
"type": "string",
"optional": false,
"field": "name"
},
{
"type": "int64",
"optional": false,
"field": "server_id"
},
{
"type": "int64",
"optional": false,
"field": "ts_sec"
},
{
"type": "string",
"optional": true,
"field": "gtid"
},
{
"type": "string",
"optional": false,
"field": "file"
},
{
"type": "int64",
"optional": false,
"field": "pos"
},
{
"type": "int32",
"optional": false,
"field": "row"
},
{
"type": "boolean",
"optional": true,
"default": false,
"field": "snapshot"
},
{
"type": "int64",
"optional": true,
"field": "thread"
},
{
"type": "string",
"optional": true,
"field": "db"
},
{
"type": "string",
"optional": true,
"field": "table"
},
{
"type": "string",
"optional": true,
"field": "query"
}
]
}
]
},
"payload": {
"databaseName": "inventory",
"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;",
"source" : {
"version": "0.10.0.Beta4",
"name": "mysql-server-1",
"server_id": 0,
"ts_sec": 0,
"gtid": null,
"file": "mysql-bin.000003",
"pos": 154,
"row": 0,
"snapshot": true,
"thread": null,
"db": null,
"table": null,
"query": null
}
}
}
Important tips about the schema change topic
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.
....
"payload": {
"databaseName": "inventory",
"ddl": "CREATE TABLE products ( id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,...
"source" : {
....
}
}
....
- 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.
-
-
If you do not use the schema change topics detailed here, check out the database history topic.
MySQL connector 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 converver and you configure it to produce all four basic change event parts, change events have this structure:
{
"schema": { (1)
...
},
"payload": { (2)
...
},
"schema": { (3)
...
},
"payload": { (4)
...
},
}
Item | Field name | Description |
---|---|---|
1 |
|
The first |
2 |
|
The first |
3 |
|
The second |
4 |
|
The second |
By default, the connector streams change event records to topics with names that are the same as the event’s originating table. See MySQL connector and Kafka topics.
The MySQL connector ensures that all Kafka Connect schema names adhere to the Avro schema name format. This means that the logical server name must start with a Latin letter or an underscore, that is, a-z, A-Z, or _. Each remaining character in the logical server name and each character in the database and table names must be a Latin letter, a digit, or an underscore, that is, a-z, A-Z, 0-9, or \_. If there is an invalid character it is replaced with an underscore character. This can lead to unexpected conflicts if the logical server name, a database name, or a table name contains invalid characters, and the only characters that distinguish names from one another are invalid and thus replaced with underscores. |
Change event keys
A change event’s key contains the schema for the changed table’s key and the changed row’s actual key. Both the schema and its corresponding payload contain a field for each column in the changed table’s PRIMARY KEY
(or unique constraint) at the time the connector created the event.
Consider the following customers
table, which is followed by an example of a change event key for this table.
CREATE TABLE customers (
id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
first_name VARCHAR(255) NOT NULL,
last_name VARCHAR(255) NOT NULL,
email VARCHAR(255) NOT NULL UNIQUE KEY
) AUTO_INCREMENT=1001;
Every change event that captures a change to the customers
table has the same event key schema. For as long as the customers
table has the previous definition, every change event that captures a change to the customers
table has the following key structure. In JSON, it looks like this:
{
"schema": { (1)
"type": "struct",
"name": "mysql-server-1.inventory.customers.Key", (2)
"optional": false, (3)
"fields": [ (4)
{
"field": "id",
"type": "int32",
"optional": false
}
]
},
"payload": { (5)
"id": 1001
}
}
Item | Field name | Description |
---|---|---|
1 |
|
The schema portion of the key specifies a Kafka Connect schema that describes what is in the key’s |
2 |
|
Name of the schema that defines the structure of the key’s payload. This schema describes the structure of the primary key for the table that was changed. Key schema names have the format connector-name.database-name.table-name.
|
3 |
|
Indicates whether the event key must contain a value in its |
4 |
|
Specifies each field that is expected in the |
5 |
|
Contains the key for the row for which this change event was generated. In this example, the key, contains a single |
Change event values
The value in a change event is a bit more complicated than the key. Like the key, the value has a schema
section and a payload
section. The schema
section contains the schema that describes the Envelope
structure of the payload
section, including its nested fields. Change events for operations that create, update or delete data all have a value payload with an envelope structure.
Consider the same sample table that was used to show an example of a change event key:
CREATE TABLE customers (
id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
first_name VARCHAR(255) NOT NULL,
last_name VARCHAR(255) NOT NULL,
email VARCHAR(255) NOT NULL UNIQUE KEY
) AUTO_INCREMENT=1001;
The value portion of a change event for a change to this table is described for each event type:
create events
The following example shows the value portion of a change event that the connector generates for an operation that creates data in the customers
table:
{
"schema": { (1)
"type": "struct",
"fields": [
{
"type": "struct",
"fields": [
{
"type": "int32",
"optional": false,
"field": "id"
},
{
"type": "string",
"optional": false,
"field": "first_name"
},
{
"type": "string",
"optional": false,
"field": "last_name"
},
{
"type": "string",
"optional": false,
"field": "email"
}
],
"optional": true,
"name": "mysql-server-1.inventory.customers.Value", (2)
"field": "before"
},
{
"type": "struct",
"fields": [
{
"type": "int32",
"optional": false,
"field": "id"
},
{
"type": "string",
"optional": false,
"field": "first_name"
},
{
"type": "string",
"optional": false,
"field": "last_name"
},
{
"type": "string",
"optional": false,
"field": "email"
}
],
"optional": true,
"name": "mysql-server-1.inventory.customers.Value",
"field": "after"
},
{
"type": "struct",
"fields": [
{
"type": "string",
"optional": false,
"field": "version"
},
{
"type": "string",
"optional": false,
"field": "connector"
},
{
"type": "string",
"optional": false,
"field": "name"
},
{
"type": "int64",
"optional": false,
"field": "ts_sec"
},
{
"type": "boolean",
"optional": true,
"default": false,
"field": "snapshot"
},
{
"type": "string",
"optional": false,
"field": "db"
},
{
"type": "string",
"optional": true,
"field": "table"
},
{
"type": "int64",
"optional": false,
"field": "server_id"
},
{
"type": "string",
"optional": true,
"field": "gtid"
},
{
"type": "string",
"optional": false,
"field": "file"
},
{
"type": "int64",
"optional": false,
"field": "pos"
},
{
"type": "int32",
"optional": false,
"field": "row"
},
{
"type": "int64",
"optional": true,
"field": "thread"
},
{
"type": "string",
"optional": true,
"field": "query"
}
],
"optional": false,
"name": "io.debezium.connector.mysql.Source", (3)
"field": "source"
},
{
"type": "string",
"optional": false,
"field": "op"
},
{
"type": "int64",
"optional": true,
"field": "ts_ms"
}
],
"optional": false,
"name": "mysql-server-1.inventory.customers.Envelope" (4)
},
"payload": { (5)
"op": "c", (6)
"ts_ms": 1465491411815, (7)
"before": null, (8)
"after": { (9)
"id": 1004,
"first_name": "Anne",
"last_name": "Kretchmar",
"email": "annek@noanswer.org"
},
"source": { (10)
"version": "1.2.5.Final",
"connector": "mysql",
"name": "mysql-server-1",
"ts_sec": 0,
"snapshot": false,
"db": "inventory",
"table": "customers",
"server_id": 0,
"gtid": null,
"file": "mysql-bin.000003",
"pos": 154,
"row": 0,
"thread": 7,
"query": "INSERT INTO customers (first_name, last_name, email) VALUES ('Anne', 'Kretchmar', 'annek@noanswer.org')"
}
}
}
Item | Field name | Description |
---|---|---|
1 |
|
The value’s schema, which describes the structure of the value’s payload. A change event’s value schema is the same in every change event that the connector generates for a particular table. |
2 |
|
In the |
3 |
|
|
4 |
|
|
5 |
|
The value’s actual data. This is the information that the change event is providing. |
6 |
|
Mandatory string that describes the type of operation that caused the connector to generate the event. In this example,
|
7 |
|
Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
8 |
|
An optional field that specifies the state of the row before the event occurred. When the |
9 |
|
An optional field that specifies the state of the row after the event occurred. In this example, the |
10 |
|
Mandatory field that describes the source metadata for the event. This field contains information that you can use to compare this event with other events, with regard to the origin of the events, the order in which the events occurred, and whether events were part of the same transaction. The source metadata includes:
If the |
update events
The value of a change event for an update in the sample customers
table has the same schema as a create event for that table. Likewise, the event value’s payload has the same structure. However, the event value payload contains different values in an update event. Here is an example of a change event value in an event that the connector generates for an update in the customers
table:
{
"schema": { ... },
"payload": {
"before": { (1)
"id": 1004,
"first_name": "Anne",
"last_name": "Kretchmar",
"email": "annek@noanswer.org"
},
"after": { (2)
"id": 1004,
"first_name": "Anne Marie",
"last_name": "Kretchmar",
"email": "annek@noanswer.org"
},
"source": { (3)
"version": "1.2.5.Final",
"name": "mysql-server-1",
"connector": "mysql",
"name": "mysql-server-1",
"ts_sec": 1465581,
"snapshot": false,
"db": "inventory",
"table": "customers",
"server_id": 223344,
"gtid": null,
"file": "mysql-bin.000003",
"pos": 484,
"row": 0,
"thread": 7,
"query": "UPDATE customers SET first_name='Anne Marie' WHERE id=1004"
},
"op": "u", (4)
"ts_ms": 1465581029523
}
}
Item | Field name | Description |
---|---|---|
1 |
|
An optional field that specifies the state of the row before the event occurred. In an update event value, the |
2 |
|
An optional field that specifies the state of the row after the event occurred. You can compare the |
3 |
|
Mandatory field that describes the source metadata for the event. The
If the |
4 |
|
Mandatory string that describes the type of operation. In an update event value, the |
Updating the columns for a row’s primary/unique key changes the value of the row’s key. When a key changes, Debezium outputs three events: a |
Primary key updates
An UPDATE
operation that changes a row’s primary key field(s) is known
as a primary key change. For a primary key change, in place of an UPDATE
event record, the connector emits a DELETE
event record for the old key and a CREATE
event record for the new (updated) key. These events have the usual structure and content, and in addition, each one has a message header related to the primary key change:
-
The
DELETE
event record has__debezium.newkey
as a message header. The value of this header is the new primary key for the updated row. -
The
CREATE
event record has__debezium.oldkey
as a message header. The value of this header is the previous (old) primary key that the updated row had.
delete events
The value in a delete change event has the same schema
portion as create and update events for the same table. The payload
portion in a delete event for the sample customers
table looks like this:
{
"schema": { ... },
"payload": {
"before": { (1)
"id": 1004,
"first_name": "Anne Marie",
"last_name": "Kretchmar",
"email": "annek@noanswer.org"
},
"after": null, (2)
"source": { (3)
"version": "1.2.5.Final",
"connector": "mysql",
"name": "mysql-server-1",
"ts_sec": 1465581,
"snapshot": false,
"db": "inventory",
"table": "customers",
"server_id": 223344,
"gtid": null,
"file": "mysql-bin.000003",
"pos": 805,
"row": 0,
"thread": 7,
"query": "DELETE FROM customers WHERE id=1004"
},
"op": "d", (4)
"ts_ms": 1465581902461 (5)
}
}
Item | Field name | Description |
---|---|---|
1 |
|
Optional field that specifies the state of the row before the event occurred. In a delete event value, the |
2 |
|
Optional field that specifies the state of the row after the event occurred. In a delete event value, the |
3 |
|
Mandatory field that describes the source metadata for the event. In a delete event value, the
If the |
4 |
|
Mandatory string that describes the type of operation. The |
5 |
|
Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
A delete change event record provides a consumer with the information it needs to process the removal of this row. The old values are included because some consumers might require them in order to properly handle the removal.
MySQL connector events are designed to work with Kafka log compaction. Log compaction enables removal of some older messages as long as at least the most recent message for every key is kept. This lets Kafka reclaim storage space while ensuring that the topic contains a complete data set and can be used for reloading key-based state.
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.
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 type | Literal type | Semantic type | ||
---|---|---|---|---|
|
|
n/a |
||
|
|
n/a |
||
|
|
example (where n is bits)
numBytes = n/8 + (n%8== 0 ? 0 : 1) |
||
|
|
n/a |
||
|
|
n/a |
||
|
|
n/a |
||
|
|
n/a |
||
|
|
n/a |
||
|
|
n/a |
||
|
|
n/a |
||
|
|
n/a |
||
|
|
n/a |
||
|
|
n/a |
||
|
|
n/a Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the binary handling mode setting |
||
|
|
n/a Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the binary handling mode setting |
||
|
|
n/a Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the binary handling mode setting |
||
|
|
n/a |
||
|
|
n/a Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the binary handling mode setting |
||
|
|
n/a |
||
|
|
n/a Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the binary handling mode setting |
||
|
|
n/a |
||
|
|
n/a Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the binary handling mode setting |
||
|
|
n/a |
||
|
|
|
||
|
|
|
||
|
|
|
||
|
|
|
||
|
|
|
Temporal values
Excluding the TIMESTAMP
data type, MySQL temporal types depend on the value of the time.precision.mode
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.
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 milli-seconds or micro-seconds based on the column’s precision by using UTC. The TIMESTAMP
type represents a timestamp without time zone information and is converted by MySQL from the server (or session’s) current time zone into UTC when writing and vice versa when reading back the value. For example:
-
DATETIME
with a value of2018-06-20 06:37:03
becomes1529476623000
. -
TIMESTAMP
with a value of2018-06-20 06:37:03
becomes2018-06-20T13:37:03Z
.
Such columns are converted into an equivalent io.debezium.time.ZonedTimestamp
in UTC based on the server (or session’s) current time zone. The time zone will be queried from the server by default. If this fails, it must be specified explicitly by the database.serverTimezone
connector configuration property. For example, if the database’s time zone (either globally or configured for the connector by means of the database.serverTimezone property
) 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".
Note that 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 of00:00:00.000000
to23:59:59.999999
can be captured correctly.MySQL type Literal type Semantic 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 time zone information. MySQL allows M
to be in the range of0-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 time zone information. DATETIME(4), DATETIME(5), DATETIME(6)
INT64
io.debezium.time.MicroTimestamp
Represents the number of microseconds past epoch and does not include time zone 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
. Only values in the range of00:00:00.000
to23:59:59.999
can be handled. Settime.precision.mode=connect
only if you can ensure that theTIME
values in your tables never exceed the supported ranges. Theconnect
setting is expected to be removed in a future version of Debezium.MySQL type Literal type Semantic 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 time zone information. DATETIME[(M)]
INT64
org.apache.kafka.connect.data.Timestamp
Represents the number of milliseconds since epoch, and does not include time zone 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 type Literal type Semantic 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 type Literal type Semantic type NUMERIC[(M[,D])]
FLOAT64
n/a
DECIMAL[(M[,D])]
FLOAT64
n/a
- decimal.handling.mode=string
-
MySQL type Literal type Semantic 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 TINYINT(1)
datatype.
When the table is created during streaming then it uses proper BOOLEAN
mapping as Debezium receives the original DDL.
During snapshot Debezium executes SHOW CREATE TABLE
to obtain table definition which returns TINYINT(1)
for both BOOLEAN
and TINYINT(1)
columns.
Debezium then has no way how to obtain the original type mapping and will map to TINYINT(1)
.
The operator can configure the out-of-the-box custom converter TinyIntOneToBooleanConverter
that would either map all TINYINT(1)
columns to BOOLEAN
or if selector
parameter is set then a subset of columns could be enumerated using comma-separated regular expressions.
An example configuration is
converters=boolean boolean.type=io.debezium.connector.mysql.converters.TinyIntOneToBooleanConverter boolean.selector=db1.table1.*, db1.table2.column1
Spatial data types
Currently, the Debezium MySQL connector supports the following spatial data types:
MySQL type | Literal type | Semantic type | ||
---|---|---|---|---|
|
|
|
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:
serverName.databaseName.tableName
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:
fulfillment.inventory.orders
fulfillment.inventory.customers
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
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.
-
You must have a MySQL server.
-
You must know basic SQL commands.
-
Create the MySQL user:
mysql> CREATE USER 'user'@'localhost' IDENTIFIED BY 'password';
-
Grant the required permissions to the user:
mysql> GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO 'user' IDENTIFIED BY 'password';
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.
|
-
Finalize the user’s permissions:
mysql> FLUSH PRIVILEGES;
Permissions explained
Permission/item | Description | ||
---|---|---|---|
|
enables the connector to select rows from tables in databases
|
||
|
enables the connector the use of the
|
||
|
enables the connector to see database names by issuing the
|
||
|
enables the connector to connect to and read the MySQL server binlog. |
||
|
enables the connector the use of following statements:
|
||
|
Identifies the database to which the permission apply. |
||
|
Specifies the user to which the permissions are granted. |
||
|
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.
-
You must have a MySQL server.
-
You should have appropriate MySQL user privileges.
-
Check if the
log-bin
option is already on or not.
mysql> SELECT variable_value as "BINARY LOGGING STATUS (log-bin) ::" FROM information_schema.global_variables WHERE variable_name='log_bin';
-
If
OFF
, configure your MySQL server configuration file with the following:
See Binlog config properties for notes on each property. |
server-id = 223344 (1)
log_bin = mysql-bin (2)
binlog_format = ROW (3)
binlog_row_image = FULL (4)
expire_logs_days = 10 (5)
-
Confirm your changes by checking the binlog status once more.
mysql> SELECT variable_value as "BINARY LOGGING STATUS (log-bin) ::" FROM information_schema.global_variables WHERE variable_name='log_bin';
Binlog configuration properties
Number | Property | Description | ||
---|---|---|---|---|
1 |
|
The value for the |
||
2 |
|
The value of |
||
3 |
|
The |
||
4 |
|
The |
||
5 |
|
This is the number of days for automatic binlog file removal. The default is
|
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. |
-
You must have a MySQL server.
-
You must know basic SQL commands.
-
You must have access to the MySQL configuration file.
-
Enable
gtid_mode
:
mysql> gtid_mode=ON
-
Enable
enforce_gtid_consistency
:
mysql> enforce_gtid_consistency=ON
-
Confirm the changes:
mysql> show global variables like '%GTID%';
+--------------------------+-------+
| Variable_name | Value |
+--------------------------+-------+
| enforce_gtid_consistency | ON |
| gtid_mode | ON |
+--------------------------+-------+
Options explained
Permission/item | Description |
---|---|
|
Boolean which specifies whether GTID mode of the MySQL server is enabled or not.
|
|
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.
|
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.
-
You must have a MySQL server.
-
You must know basic SQL commands.
-
You must have access to the MySQL configuration file.
-
Configure
interactive_timeout
:
mysql> interactive_timeout=<duration-in-seconds>
-
Configure
wait_timeout
:
mysql> wait_timeout= <duration-in-seconds>
Options explained
Permission/item | Description | ||
---|---|---|---|
|
The number of seconds the server waits for activity on an interactive connection before closing it.
|
||
|
The number of seconds the server waits for activity on a noninteractive connection before closing it.
|
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. |
-
You must have a MySQL server.
-
You must know basic SQL commands.
-
You must have access to the MySQL configuration file.
-
Enable
binlog_rows_query_log_events
:
mysql> binlog_rows_query_log_events=ON
Deploying the MySQL connector
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 plug-in’s parent directory is specified in your Kafka Connect environment.
-
You have Zookeeper, Kafka, and Kafka Connect installed.
-
You have MySQL Server installed and setup.
-
Download the Debezium MySQL connector plug-in.
-
Extract the files into your Kafka Connect environment.
-
Add the plug-in’s parent directory to your Kafka Connect
plugin.path
:
plugin.path=/kafka/connect
The above example assumes you have extracted the Debezium MySQL connector to the /kafka/connect/debezium-connector-mysql path.
|
-
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.
-
You should have completed the installation process for the connector.
-
Set the
"name"
of the connector in the.json
file. -
Set the configuration properties that you require for your Debezium MySQL connector.
For a complete list of configuration properties, see MySQL connector configuration properties. |
{
"name": "inventory-connector", (1)
"config": {
"connector.class": "io.debezium.connector.mysql.MySqlConnector", (2)
"database.hostname": "192.168.99.100", (3)
"database.port": "3306", (4)
"database.user": "debezium-user", (5)
"database.password": "debezium-user-pw", (6)
"database.server.id": "184054", (7)
"database.server.name": "fullfillment", (8)
"database.whitelist": "inventory", (9)
"database.history.kafka.bootstrap.servers": "kafka:9092", (10)
"database.history.kafka.topic": "dbhistory.fullfillment", (11)
"include.schema.changes": "true" (12)
}
}
Example configuration properties explained
-
The connector’s name when registered with the Kafka Connect service.
-
The connector’s class name.
-
The MySQL server address.
-
The MySQL server port number.
-
The MySQL user with the appropriate privileges.
-
The MySQL user’s password.
-
The unique ID of the connector.
-
The logical name of the MySQL server or cluster.
-
A list of all databases hosted by the specified server.
-
A list of Kafka brokers that the connector uses to write and recover DDL statements to the database history topic.
-
The name of the database history topic.
-
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.
Adding MySQL connector configuration to Kafka Connect
To start running a MySQL connector, configure a connector and add the configuration to your Kafka Connect cluster.
-
MySQL server is set up for a Debezium connector.
-
Debezium MySQL connector is installed.
-
Create a configuration for the MySQL connector.
-
Use the Kafka Connect REST API to add that connector configuration to your Kafka Connect cluster.
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.
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 information about pass-through properties at the end of this section, and also see the Kafka documentation for more details about pass-through properties. |
Property | Default | Description |
---|---|---|
Unique name for the connector. Attempting to register again with the same name will fail. (This property is required by all Kafka Connect connectors.) |
||
The name of the Java class for the connector. Always use a value of |
||
|
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 hostname of the MySQL database server. |
||
|
Integer port number of the MySQL database server. |
|
Name of the MySQL database 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 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. |
||
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. |
|
The full name of the Kafka topic where the connector will store the database schema history. |
||
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. |
||
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 |
|
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 |
|
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 |
|
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 |
|
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. |
|
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. |
|
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 ( |
|
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 message values with a field value consisting of the hashed value using the algorithm 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. Example: column.mask.hash.SHA-256.with.salt.CzQMA0cB5K = inventory.orders.customerName, inventory.shipment.customerName where Note: Depending on the |
|
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 |
|
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 messages.
The schema parameters |
|
|
Time, date, and timestamps can be represented with different kinds of precision, including: |
|
|
Specifies how the connector should handle values for |
|
|
Specifies how BIGINT UNSIGNED columns should be represented in change events, including: |
|
|
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 |
|
|
Boolean value that specifies whether the connector should include the original SQL query that generated the change event. |
|
|
Specifies how the connector should react to exceptions during deserialization of binlog events.
|
|
|
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)
|
|
|
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 |
|
|
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. |
|
|
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. |
|
|
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 matching one of these include patterns will be used. May not be used with |
||
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 |
||
|
|
When set to |
|
Controls whether a tombstone event should be generated after a delete event. |
|
empty string |
A semi-colon list of regular expressions that match fully-qualified tables and columns to map a primary key. |
|
bytes |
Specifies how binary ( |
Advanced MySQL connector properties
The following table describes advanced MySQL connector properties.
Property | Default | Description |
---|---|---|
|
A boolean value that specifies whether a separate thread should be used to ensure the connection to the MySQL server/cluster is kept alive. |
|
|
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. |
|
|
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. |
|
|
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 |
|
|
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 |
|
|
Boolean value that specifies if connector should should record all DDL statements or (when |
|
|
Specifies whether to use an encrypted connection. The default is The The The The |
|
0 |
The size of a look-ahead buffer used by the binlog reader. |
|
|
Specifies the criteria for running a snapshot upon startup of the connector. The default is
|
|
|
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
|
|
Controls which rows from tables will be included in snapshot. |
||
|
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. |
|
|
Controls how frequently the heartbeat messages are sent. |
|
|
Controls the naming of the topic to which heartbeat messages are sent. |
|
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. |
||
An interval in milli-seconds that the connector should wait before taking a snapshot after starting up; |
||
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. |
||
|
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. |
|
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. |
||
v2 |
Schema version for the |
|
|
Whether field names will be sanitized to adhere to Avro naming requirements. |
|
comma-separated list of oplog operations that will be skipped during streaming.
The operations include: |
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:
database.history.producer.security.protocol=SSL database.history.producer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks database.history.producer.ssl.keystore.password=test1234 database.history.producer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks database.history.producer.ssl.truststore.password=test1234 database.history.producer.ssl.key.password=test1234 database.history.consumer.security.protocol=SSL database.history.consumer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks database.history.consumer.ssl.keystore.password=test1234 database.history.consumer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks database.history.consumer.ssl.truststore.password=test1234 database.history.consumer.ssl.key.password=test1234
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.
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; for monitoring the connector when performing snapshots
-
binlog metrics; for monitoring the connector when reading CDC table data
-
schema history metrics; for monitoring the status of the connector’s schema history
Refer to the monitoring documentation for details of how to expose these metrics via JMX.
Snapshot metrics
The MBean is debezium.mysql:type=connector-metrics,context=snapshot,server=<database.server.name>
.
Attributes | Type | Description |
---|---|---|
|
|
The last snapshot event that the connector has read. |
|
|
The number of milliseconds since the connector has read and processed the most recent event. |
|
|
The total number of events that this connector has seen since last started or reset. |
|
|
The number of events that have been filtered by whitelist or blacklist filtering rules configured on the connector. |
|
|
The list of tables that are monitored by the connector. |
|
|
The length the queue used to pass events between the snapshotter and the main Kafka Connect loop. |
|
|
The free capacity of the queue used to pass events between the snapshotter and the main Kafka Connect loop. |
|
|
The total number of tables that are being included in the snapshot. |
|
|
The number of tables that the snapshot has yet to copy. |
|
|
Whether the snapshot was started. |
|
|
Whether the snapshot was aborted. |
|
|
Whether the snapshot completed. |
|
|
The total number of seconds that the snapshot has taken so far, even if not complete. |
|
|
Map containing the number of rows scanned for each table in the snapshot. Tables are incrementally added to the Map during processing. Updates every 10,000 rows scanned and upon completing a table. |
The Debezium MySQL connector also provides the following custom snapshot metrics:
Attribute | Type | Description |
---|---|---|
|
|
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>
.
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. |
Attributes | Type | Description |
---|---|---|
|
|
The last streaming event that the connector has read. |
|
|
The number of milliseconds since the connector has read and processed the most recent event. |
|
|
The total number of events that this connector has seen since last started or reset. |
|
|
The number of events that have been filtered by whitelist or blacklist filtering rules configured on the connector. |
|
|
The list of tables that are monitored by the connector. |
|
|
The length the queue used to pass events between the streamer and the main Kafka Connect loop. |
|
|
The free capacity of the queue used to pass events between the streamer and the main Kafka Connect loop. |
|
|
Flag that denotes whether the connector is currently connected to the database server. |
|
|
The number of milliseconds between the last change event’s timestamp and the connector processing it. The values will incoporate any differences between the clocks on the machines where the database server and the connector are running. |
|
|
The number of processed transactions that were committed. |
|
|
The coordinates of the last received event. |
|
|
Transaction identifier of the last processed transaction. |
The Debezium MySQL connector also provides the following custom binlog metrics:
Attribute | Type | Description |
---|---|---|
|
|
The name of the binlog filename that the connector has most recently read. |
|
|
The most recent position (in bytes) within the binlog that the connector has read. |
|
|
Flag that denotes whether the connector is currently tracking GTIDs from MySQL server. |
|
|
The string representation of the most recent GTID set seen by the connector when reading the binlog. |
|
|
The number of events that have been skipped by the MySQL connector. Typically events are skipped due to a malformed or unparseable event from MySQL’s binlog. |
|
|
The number of disconnects by the MySQL connector. |
|
|
The number of processed transactions that were rolled back and not streamed. |
|
|
The number of transactions that have not conformed to expected protocol |
|
|
The number of transactions that have not fitted into the look-ahead buffer. Should be significantly smaller than |
Schema history metrics
The MBean is debezium.mysql:type=connector-metrics,context=schema-history,server=<database.server.name>
.
Attributes | Type | Description |
---|---|---|
|
|
One of |
|
|
The time in epoch seconds at what recovery has started. |
|
|
The number of changes that were read during recovery phase. |
|
|
the total number of schema changes applied during recovery and runtime. |
|
|
The number of milliseconds that elapsed since the last change was recovered from the history store. |
|
|
The number of milliseconds that elapsed since the last change was applied. |
|
|
The string representation of the last change recovered from the history store. |
|
|
The string representation of the last applied change. |
MySQL connector common issues
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.
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)
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. |