MySQL has a binary log (binlog) that records all operations in the order in which they are committed to the database. This includes changes to table schemas as well as changes to the data in tables. MySQL uses the binlog for replication and recovery.
The {prodname} MySQL connector reads the binlog, produces change events for row-level `INSERT`, `UPDATE`, and `DELETE` operations, and emits the change events to Kafka topics. Client applications read those Kafka topics.
As MySQL is typically set up to purge binlogs after a specified period of time, the MySQL connector performs an initial _consistent snapshot_ of each of your databases. The MySQL connector reads the binlog from the point at which the snapshot was made.
For information about the MySQL Database versions that are compatible with this connector, see the link:https://debezium.io/releases/[{prodname} release overview].
For information about the MySQL Database versions that are compatible with this connector, see the link:{LinkDebeziumSupportedConfigurations}[{NameDebeziumSupportedConfigurations}].
An overview of the MySQL topologies that the connector supports is useful for planning your application. To optimally configure and run a {prodname} MySQL connector, it is helpful to understand how the connector tracks the structure of tables, exposes schema changes, performs snapshots, and determines Kafka topic names.
The {prodname} MySQL connector has yet to be tested with MariaDB, but multiple reports from the community indicate successful usage of the connector with this database.
Official support for MariaDB is planned for a future {prodname} version.
// Title: MySQL topologies supported by {prodname} connectors
[id="supported-mysql-topologies"]
=== Supported MySQL topologies
The {prodname} 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 {prodname} MySQL connector can monitor the server. This is often acceptable, since the binary log can also be used as an incremental link:https://dev.mysql.com/doc/refman/{mysql-version}/en/backup-methods.html[backup]. In this case, the MySQL connector always connects to and follows this standalone MySQL server instance.
Primary and replica::
The {prodname} MySQL connector can follow one of the primary servers or one of the replicas (_if that replica has its binlog enabled_), but the connector sees changes in only the cluster that is visible to that server. Generally, this is not a problem except for the multi-primary topologies.
+
The connector records its position in the server’s binlog, which is different on each server in the cluster. Therefore, the connector must follow just one MySQL server instance. If that server fails, that server must be restarted or recovered before the connector can continue.
High available clusters::
A variety of link:https://dev.mysql.com/doc/mysql-ha-scalability/en/[high availability] solutions exist for MySQL, and they make it significantly easier to tolerate and almost immediately recover from problems and failures. Most HA MySQL clusters use GTIDs so that replicas are able to keep track of all changes on any of the primary servers.
Multi-primary::
link:https://dev.mysql.com/doc/refman/{mysql-version}/en/mysql-cluster-replication-multi-source.html[Network Database (NDB) cluster replication] uses one or more MySQL replica nodes that each replicate from multiple primary servers. This is a powerful way to aggregate the replication of multiple MySQL clusters. This topology requires the use of GTIDs.
+
A {prodname} MySQL connector can use these multi-primary MySQL replicas as sources, and can fail over to different multi-primary MySQL replicas as long as the new replica is caught up to the old replica. That is, the new replica has all transactions that were seen on the first replica. This works even if the connector is using only a subset of databases and/or tables, as the connector can be configured to include or exclude specific GTID sources when attempting to reconnect to a new multi-primary MySQL replica and find the correct position in the binlog.
Hosted::
There is support for the {prodname} 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_.
When a database client queries a database, the client uses the database’s current schema.
However, the database schema can be changed at any time, which means that the connector must be able to identify what the schema was at the time each insert, update, or delete operation was recorded.
Also, a connector cannot just use the current schema because the connector might be processing events that are relatively old that were recorded before the tables' schemas were changed.
To ensure correct processing of changes that occur after a schema change, MySQL includes in the binlog not only the row-level changes to the data, but also the DDL statements that are applied to the database.
As the connector reads the binlog and comes across these DDL statements, it parses them and updates an in-memory representation of each table’s schema.
The connector uses this schema representation to identify the structure of the tables at the time of each insert, update, or delete operation and to produce the appropriate change event.
In a separate database history Kafka topic, the connector records all DDL statements along with the position in the binlog where each DDL statement appeared.
When the connector restarts after having crashed or been stopped gracefully, the connector starts reading the binlog from a specific position, that is, from a specific point in time.
The connector rebuilds the table structures that existed at this point in time by reading the database history Kafka topic and parsing all DDL statements up to the point in the binlog where the connector is starting.
The connector can optionally xref:mysql-schema-change-topic[emit schema change events to a different topic that is intended for consumer applications].
When the MySQL connector captures changes in a table to which a schema change tool such as `gh-ost` or `pt-online-schema-change` is applied, there are helper tables created during the migration process.
You can configure a {prodname} MySQL connector to produce schema change events that describe schema changes that are applied to captured tables in the database.
The connector writes schema change events to a Kafka topic named `_<serverName>_`, where `_serverName_` is the logical server name that is specified in the xref:mysql-property-database-server-name[`database.server.name`] connector configuration property.
Messages that the connector sends to the schema change topic contain a payload, and, optionally, also contain the schema of the change event message.
`pos`:: The position in the binlog where the statements appear.
`tableChanges`:: A structured representation of the entire table schema after the schema change.
The `tableChanges` field contains an array that includes entries for each column of the table.
Because the structured representation presents data in JSON or Avro format, consumers can easily read messages without first processing them through a DDL parser.
For a table that is in capture mode, the connector not only stores the history of schema changes in the schema change topic, but also in an internal database history topic.
The internal database history topic is for connector use only and it is not intended for direct use by consuming applications.
Ensure that applications that require notifications about schema changes consume that information only from the schema change topic.
For the database history topic to function correctly, it must maintain a consistent, global order of the event records that the connector emits to it.
To ensure that the topic is not split among partitions, set the partition count for the topic by using one of the following methods:
* If you create the database history topic manually, specify a partition count of `1`.
* If you use the Apache Kafka broker to create the database history topic automatically, the topic is created, set the value of the link:{link-kafka-docs}/#brokerconfigs_num.partitions[Kafka `num.partitions`] configuration option to `1`.
====
[WARNING]
====
The format of the messages that a connector emits to its schema change topic is in an incubating state and is subject to change without notice.
|Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task.
In the source object, ts_ms indicates the time that the change was made in the database. By comparing the value for payload.source.ts_ms with the value for payload.ts_ms, you can determine the lag between the source database update and Debezium.
|This field contains the DDL that is responsible for the schema change.
The `ddl` field can contain multiple DDL statements.
Each statement applies to the database in the `databaseName` field.
Multiple DDL statements appear in the order in which they were applied to the database. +
+
Clients can submit multiple DDL statements that apply to multiple databases.
If MySQL applies them atomically, the connector takes the DDL statements in order, groups them by database, and creates a schema change event for each group.
If MySQL applies them individually, the connector creates a separate schema change event for each statement.
When a {prodname} MySQL connector is first started, it performs an initial _consistent snapshot_ of your database. The following flow describes how the connector creates this snapshot. This flow is for the default snapshot mode, which is `initial`. For information about other snapshot modes, see the xref:{link-mysql-connector}#mysql-property-snapshot-mode[MySQL connector `snapshot.mode` configuration property].
The snapshot itself does not prevent other clients from applying DDL that might interfere with the connector's attempt to read the binlog position and table schemas. The connector keeps the global read lock while it reads the binlog position, and releases the lock as described in a later step.
|2
a|Starts a transaction with link:https://dev.mysql.com/doc/refman/{mysql-version}/en/innodb-consistent-read.html[repeatable read semantics] to ensure that all subsequent reads within the transaction are done against the _consistent snapshot_.
|3
a|Reads the current binlog position.
|4
a|Reads the schema of the databases and tables for which the connector is configured to capture changes.
|5
a|Releases the global read lock. Other database clients can now write to the database.
If the connector fails, stops, or is rebalanced while performing the _initial snapshot_, then after the connector restarts, it performs a new snapshot. After that _intial snapshot_ is completed, the {prodname} 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 {prodname} MySQL connector, see xref:{link-mysql-connector}#mysql-when-things-go-wrong[behavior when things go wrong].
Some environments do not allow global read locks. If the {prodname} MySQL connector detects that global read locks are not permitted, the connector uses table-level locks instead and performs a snapshot with this method. This requires the database user for the {prodname} connector to have `LOCK TABLES` privileges.
a|Starts a transaction with link:https://dev.mysql.com/doc/refman/{mysql-version}/en/innodb-consistent-read.html[repeatable read semantics] to ensure that all subsequent reads within the transaction are done against the _consistent snapshot_.
|3
|Reads and filters the names of the databases and tables.
The MySQL connector allows for running incremental snapshots with a read-only connection to the database.
To run an incremental snapshot with read-only access, the connector uses the executed global transaction IDs (GTID) set as high and low watermarks.
The state of a chunk's window is updated by comparing the GTIDs of binary log (binlog) events or the server's heartbeats against low and high watermarks.
When the MySQL connection is read-only, the xref:{link-signalling}[signaling table] mechanism can also run a snapshot by sending a message to the Kafka topic that is specified in
the xref:{link-mysql-connector}#mysql-property-signal-kafka-topic[signal.kafka.topic] property.
The MySQL connector emits snapshot events as `READ` operations `("op" : "r")`.
If you prefer that the connector emits snapshot events as `CREATE` (`c`) events, configure the {prodname} `ReadToInsertEvent` single message transform (SMT) to modify the event type.
By default, the MySQL connector writes change events for all of the `INSERT`, `UPDATE`, and `DELETE` operations that occur in a table to a single Apache Kafka topic that is specific to that table.
The connector uses the following convention to name change event topics:
Suppose that `fulfillment` is the server name, `inventory` is the database name, and the database contains tables named `orders`, `customers`, and `products`.
The {prodname} MySQL connector emits events to three Kafka topics, one for each table in the database:
The following list provides definitions for the components of the default name:
_serverName_:: The logical name of the server as specified by the xref:mysql-property-database-server-name[`database.server.name`] connector configuration property.
_schemaName_:: The name of the schema in which the operation occurred.
_tableName_:: The name of the table in which the operation occurred.
The connector applies similar naming conventions to label its internal database history topics, xref:mysql-schema-change-topic[schema change topics], and xref:mysql-transaction-metadata[transaction metadata topics].
When transaction metadata is enabled the data message `Envelope` is enriched with a new `transaction` field.
This field provides information about every event in the form of a composite of fields:
* `id` - string representation of unique transaction identifier
* `total_order` - absolute position of the event among all events generated by the transaction
* `data_collection_order` - the per-data collection position of the event among all events that were emitted by the transaction
Following is an example of a message:
[source,json,indent=0,subs="+attributes"]
----
{
"before": null,
"after": {
"pk": "2",
"aa": "1"
},
"source": {
...
},
"op": "c",
"ts_ms": "1580390884335",
"transaction": {
"id": "0e4d5dcd-a33b-11ea-80f1-02010a22a99e:10",
"total_order": "1",
"data_collection_order": "1"
}
}
----
For systems which don't have GTID enabled, the transaction identifier is constructed using the combination of binlog filename and binlog position. For example, if the binlog filename and position corresponding to the transaction BEGIN event are mysql-bin.000002 and 1913 respectively then the {prodname} constructed transaction identifier would be `file=mysql-bin.000002,pos=1913`.
The {prodname} 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.
{prodname} and Kafka Connect are designed around _continuous streams of event messages_. However, the structure of these events may change over time, which can be difficult for consumers to handle. To address this, each event contains the schema for its content or, if you are using a schema registry, a schema ID that a consumer can use to obtain the schema from the registry. This makes each event self-contained.
The following skeleton JSON shows the basic four parts of a change event. However, how you configure the Kafka Connect converter that you choose to use in your application determines the representation of these four parts in change events. A `schema` field is in a change event only when you configure the converter to produce it. Likewise, the event key and event payload are in a change event only if you configure a converter to produce it. If you use the JSON converter and you configure it to produce all four basic change event parts, change events have this structure:
|The first `schema` field is part of the event key. It specifies a Kafka Connect schema that describes what is in the event key's `payload` portion. In other words, the first `schema` field describes the structure of the primary key, or the unique key if the table does not have a primary key, for the table that was changed. +
It is possible to override the table's primary key by setting the xref:{link-mysql-connector}#mysql-property-message-key-columns[`message.key.columns` connector configuration property]. In this case, the first schema field describes the structure of the key identified by that property.
|The first `payload` field is part of the event key. It has the structure described by the previous `schema` field and it contains the key for the row that was changed.
|The second `schema` field is part of the event value. It specifies the Kafka Connect schema that describes what is in the event value's `payload` portion. In other words, the second `schema` describes the structure of the row that was changed. Typically, this schema contains nested schemas.
|The second `payload` field is part of the event value. It has the structure described by the previous `schema` field and it contains the actual data for the row that was changed.
By default, the connector streams change event records to topics with names that are the same as the event's originating table. See xref:{link-mysql-connector}#mysql-topic-names[topic names].
The MySQL connector ensures that all Kafka Connect schema names adhere to the link:http://avro.apache.org/docs/current/spec.html#names[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 pass:[_]. 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 pass:[_]. 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.
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.
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:
a|Name of the schema that defines the structure of the key's payload. This schema describes the structure of the primary key for the table that was changed. Key schema names have the format _connector-name_._database-name_._table-name_.`Key`. In this example: +
* `inventory` is the database that contains the table that was changed. +
* `customers` is the table that was updated.
|3
|`optional`
|Indicates whether the event key must contain a value in its `payload` field. In this example, a value in the key's payload is required. A value in the key's payload field is optional when a table does not have a primary key.
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.
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:
|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.
a|In the `schema` section, each `name` field specifies the schema for a field in the value's payload. +
+
`mysql-server-1.inventory.customers.Value` is the schema for the payload's `before` and `after` fields. This schema is specific to the `customers` table. +
Names of schemas for `before` and `after` fields are of the form `_logicalName_._tableName_.Value`, which ensures that the schema name is unique in the database.
This means that when using the xref:{link-avro-serialization}#avro-serialization[Avro converter], the resulting Avro schema for each table in each logical source has its own evolution and history.
|`io.debezium.connector.mysql.Source` is the schema for the payload's `source` field. This schema is specific to the MySQL connector. The connector uses it for all events that it generates.
|`mysql-server-1.inventory.customers.Envelope` is the schema for the overall structure of the payload, where `mysql-server-1` is the connector name, `inventory` is the database, and `customers` is the table.
|5
|`payload`
|The value's actual data. This is the information that the change event is providing. +
+
It may appear that the JSON representations of the events are much larger than the rows they describe. This is because the JSON representation must include the schema and the payload portions of the message.
However, by using the xref:{link-avro-serialization}#avro-serialization[Avro converter], you can significantly decrease the size of the messages that the connector streams to Kafka topics.
a| Mandatory string that describes the type of operation that caused the connector to generate the event. In this example, `c` indicates that the operation created a row. Valid values are:
a| Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. +
+
In the `source` object, `ts_ms` indicates the time that the change was made in the database. By comparing the value for `payload.source.ts_ms` with the value for `payload.ts_ms`, you can determine the lag between the source database update and {prodname}.
| An optional field that specifies the state of the row before the event occurred. When the `op` field is `c` for create, as it is in this example, the `before` field is `null` since this change event is for new content.
| An optional field that specifies the state of the row after the event occurred. In this example, the `after` field contains the values of the new row's `id`, `first_name`, `last_name`, and `email` columns.
a| 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 xref:{link-mysql-connector}#enable-query-log-events[`binlog_rows_query_log_events`] MySQL configuration option is enabled and the connector configuration `include.query` property is enabled, the `source` field also provides the `query` field, which contains the original SQL statement that caused the change event.
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:
|An optional field that specifies the state of the row before the event occurred. In an _update_ event value, the `before` field contains a field for each table column and the value that was in that column before the database commit. In this example, the `first_name` value is `Anne.`
| An optional field that specifies the state of the row after the event occurred. You can compare the `before` and `after` structures to determine what the update to this row was. In the example, the `first_name` value is now `Anne Marie`.
a|Mandatory field that describes the source metadata for the event. The `source` field structure has the same fields as in a _create_ event, but some values are different, for example, the sample _update_ event is from a different position in the binlog. The source metadata includes:
If the xref:{link-mysql-connector}#enable-query-log-events[`binlog_rows_query_log_events`] MySQL configuration option is enabled and the connector configuration `include.query` property is enabled, the `source` field also provides the `query` field, which contains the original SQL statement that caused the change event.
a|Mandatory string that describes the type of operation. In an _update_ event value, the `op` field value is `u`, signifying that this row changed because of an update.
a| Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. +
+
In the `source` object, `ts_ms` indicates the time that the change was made in the database. By comparing the value for `payload.source.ts_ms` with the value for `payload.ts_ms`, you can determine the lag between the source database update and {prodname}.
Updating the columns for a row's primary/unique key changes the value of the row's key. When a key changes, {prodname} outputs _three_ events: a `DELETE` event and a xref:{link-mysql-connector}#mysql-tombstone-events[tombstone event] with the old key for the row, followed by an event with the new key for the row. Details are in the next section.
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.
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:
|Optional field that specifies the state of the row before the event occurred. In a _delete_ event value, the `before` field contains the values that were in the row before it was deleted with the database commit.
|2
|`after`
| Optional field that specifies the state of the row after the event occurred. In a _delete_ event value, the `after` field is `null`, signifying that the row no longer exists.
a|Mandatory field that describes the source metadata for the event. In a _delete_ event value, the `source` field structure is the same as for _create_ and _update_ events for the same table. Many `source` field values are also the same. In a _delete_ event value, the `ts_ms` and `pos` field values, as well as other values, might have changed. But the `source` field in a _delete_ event value provides the same metadata:
If the xref:{link-mysql-connector}#enable-query-log-events[`binlog_rows_query_log_events`] MySQL configuration option is enabled and the connector configuration `include.query` property is enabled, the `source` field also provides the `query` field, which contains the original SQL statement that caused the change event.
a|Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. +
+
In the `source` object, `ts_ms` indicates the time that the change was made in the database. By comparing the value for `payload.source.ts_ms` with the value for `payload.ts_ms`, you can determine the lag between the source database update and {prodname}.
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 link:{link-kafka-docs}/#compaction[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 {prodname}’s MySQL connector emits a _delete_ event, the connector emits a special tombstone event that has the same key but a `null` value.
// Title: How {prodname} MySQL connectors map data types
[[mysql-data-types]]
== Data type mappings
The {prodname} 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 {prodname} represents the value in the event.
Columns that store strings are defined in MySQL with a character set and collation. The MySQL connector uses the column's character set when reading the binary representation of the column values in the binlog events.
* *Literal type*: how the value is represented using Kafka Connect schema types.
* *Semantic type*: how the Kafka Connect schema captures the meaning of the field (schema name).
If the default data type conversions do not meet your needs, you can {link-prefix}:{link-custom-converters}#custom-converters[create a custom converter] for the connector.
The `length` schema parameter contains an integer that represents the number of bits. The `byte[]` contains the bits in _little-endian_ form and is sized to contain the specified number of bits. For example, where `n` is bits: +
a|As of MySQL 8.0.17, the nonstandard FLOAT(M,D) and DOUBLE(M,D) syntax is deprecated, and should expect support for it be removed in a future version of MySQL, set `FLOAT64` as default.
Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the xref:{link-mysql-connector}#mysql-property-binary-handling-mode[`binary.handling.mode`] connector configuration property setting.
Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the xref:{link-mysql-connector}#mysql-property-binary-handling-mode[`binary.handling.mode`] connector configuration property setting.
Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the xref:{link-mysql-connector}#mysql-property-binary-handling-mode[`binary.handling.mode`] connector configuration property setting.
Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the xref:{link-mysql-connector}#mysql-property-binary-handling-mode[`binary.handling.mode`] connector configuration property setting. +
Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the xref:{link-mysql-connector}#mysql-property-binary-handling-mode[`binary.handling.mode`] connector configuration property setting.
Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the xref:{link-mysql-connector}#mysql-property-binary-handling-mode[`binary.handling.mode`] connector configuration property setting. +
Contains the string representation of a `JSON` document, array, or scalar.
|`ENUM`
|`STRING`
a|`io.debezium.data.Enum` +
The `allowed` schema parameter contains the comma-separated list of allowed values.
|`SET`
|`STRING`
a|`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`
a|`io.debezium.time.ZonedTimestamp` +
In link:https://www.iso.org/iso-8601-date-and-time-format.html[ISO 8601] format with microsecond precision. MySQL allows `M` to be in the range of `0-6`.
Excluding the `TIMESTAMP` data type, MySQL temporal types depend on the value of the `time.precision.mode` connector configuration property. For `TIMESTAMP` columns whose default value is specified as `CURRENT_TIMESTAMP` or `NOW`, the value `1970-01-01 00:00:00` is used as the default value in the Kafka Connect schema.
MySQL allows zero-values for `DATE`, `DATETIME`, and `TIMESTAMP` columns because zero-values are sometimes preferred over null values. The MySQL connector represents zero-values as null values when the column definition allows null values, or as the epoch day when the column does not allow null values.
The `DATETIME` type represents a local date and time such as "2018-01-13 09:48:27". As you can see, there is no time zone information. Such columns are converted into epoch milliseconds or microseconds based on the column’s precision by using UTC. The `TIMESTAMP` type represents a timestamp without time zone information. It is converted by MySQL from the server (or session’s) current time zone into UTC when writing and from UTC into the server (or session's) current time zone when reading back the value. For example:
* `DATETIME` with a value of `2018-06-20 06:37:03` becomes `1529476623000`.
* `TIMESTAMP` with a value of `2018-06-20 06:37:03` becomes `2018-06-20T13:37:03Z`.
Such columns are converted into an equivalent `io.debezium.time.ZonedTimestamp` in UTC based on the server (or session’s) current time zone. The time zone will be queried from the server by default. If this fails, it must be specified explicitly by the database `connectionTimeZone` MySQL configuration option. For example, if the database’s time zone (either globally or configured for the connector by means of the `connectionTimeZone` option) is "America/Los_Angeles", the TIMESTAMP value "2018-06-20 06:37:03" is represented by a `ZonedTimestamp` with the value "2018-06-20T13:37:03Z".
More details about properties related to temporal values are in the documentation for xref:{link-mysql-connector}#mysql-connector-properties[MySQL connector configuration properties].
The MySQL connector determines the literal type and semantic type based on the column's data type definition so that events represent exactly the values in the database. All time fields are in microseconds. Only positive `TIME` field values in the range of `00:00:00.000000` to `23:59:59.999999` can be captured correctly.
Represents the number of milliseconds past the epoch and does not include time zone information.
|`DATETIME(4), DATETIME(5), DATETIME(6)`
|`INT64`
a|`io.debezium.time.MicroTimestamp` +
Represents the number of microseconds past the epoch and does not include time zone information.
|===
time.precision.mode=connect::
The MySQL connector uses defined Kafka Connect logical types. This approach is less precise than the default approach and the events could be less precise if the database column has a _fractional second precision_ value of greater than `3`. Values in only the range of `00:00:00.000` to `23:59:59.999` can be handled. Set `time.precision.mode=connect` only if you can ensure that the `TIME` values in your tables never exceed the supported ranges. The `connect` setting is expected to be removed in a future version of {prodname}.
{prodname} connectors handle decimals according to the setting of the xref:{link-mysql-connector}#mysql-property-decimal-handling-mode[`decimal.handling.mode` connector configuration property].
During snapshots, {prodname} executes `SHOW CREATE TABLE` to obtain table definitions that return `TINYINT(1)` for both `BOOLEAN` and `TINYINT(1)` columns. {prodname} then has no way to obtain the original type mapping and so maps to `TINYINT(1)`.
To enable you to convert source columns to Boolean data types, {prodname} provides a `TinyIntOneToBooleanConverter` {link-prefix}:{link-custom-converters}#custom-converters[custom converter] that you can use in one of the following ways:
To use this type of conversion, you must set the xref:mysql-property-converters[`converters`] configuration property with the `selector` parameter, as shown in the following example:
* `srid (INT32`: spatial reference system ID that defines the type of geometry object stored in the structure
* `wkb (BYTES)`: binary representation of the geometry object encoded in the Well-Known-Binary (wkb) format. See the link:https://www.opengeospatial.org/standards/sfa[Open Geospatial Consortium] for more details.
A {prodname} MySQL connector requires a MySQL user account. This MySQL user must have appropriate permissions on all databases for which the {prodname} MySQL connector captures changes.
.Prerequisites
* A MySQL server.
* Basic knowledge of SQL commands.
.Procedure
. Create the MySQL user:
+
[source,SQL]
----
mysql> CREATE USER 'user'@'localhost' IDENTIFIED BY 'password';
----
. Grant the required permissions to the user:
+
[source,SQL]
----
mysql> GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO 'user' IDENTIFIED BY 'password';
IMPORTANT: If using a hosted option such as Amazon RDS or Amazon Aurora that does not allow a global read lock, table-level locks are used to create the _consistent snapshot_. In this case, you need to also grant `LOCK TABLES` permissions to the user that you create. See xref:{link-mysql-connector}#mysql-snapshots[snapshots] for more details.
|Enables the connector the use of the `FLUSH` statement to clear or reload internal caches, flush tables, or acquire locks. This is used only when performing a snapshot.
|The value for the `server-id` must be unique for each server and replication client in the MySQL cluster. During MySQL connector set up, {prodname} assigns a unique server ID to the connector.
|This is the number of days for automatic binlog file removal. The default is `0`, which means no automatic removal. Set the value to match the needs of your environment. See xref:{link-mysql-connector}#mysql-purges-binlog-files-used-by-debezium[MySQL purges binlog files].
Global transaction identifiers (GTIDs) uniquely identify transactions that occur on a server within a cluster. Though not required for a {prodname} MySQL connector, using GTIDs simplifies replication and enables you to more easily confirm if primary and replica servers are consistent.
GTIDs are available in MySQL 5.6.5 and later. See the link:https://dev.mysql.com/doc/refman/{mysql-version}/en/replication-options-gtids.html#option_mysqld_gtid-mode[MySQL documentation] for more details.
a|Boolean that specifies whether the server enforces GTID consistency by allowing the execution of statements that can be logged in a transactionally safe manner. Required when using GTIDs.
// Title: Configuring MySQL session timesouts for {prodname}
[[mysql-session-timeouts]]
=== Configuring session timeouts
When an initial consistent snapshot is made for large databases, your established connection could timeout while the tables are being read. You can prevent this behavior by configuring `interactive_timeout` and `wait_timeout` in your MySQL configuration file.
a|The number of seconds the server waits for activity on an interactive connection before closing it. See link:https://dev.mysql.com/doc/refman/{mysql-version}/en/server-system-variables.html#sysvar_interactive_timeout[MySQL's documentation] for more details.
|`wait_timeout`
a|The number of seconds the server waits for activity on a non-interactive connection before closing it. See link:https://dev.mysql.com/doc/refman/{mysql-version}/en/server-system-variables.html#sysvar_wait_timeout[MySQL's documentation] for more details.
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.
To deploy a {prodname} MySQL connector, you install the {prodname} MySQL connector archive, configure the connector, and start the connector by adding its configuration to Kafka Connect.
* link:https://zookeeper.apache.org/[Apache Zookeeper], link:http://kafka.apache.org/[Apache Kafka], and link:{link-kafka-docs}.html#connect[Kafka Connect] are installed.
. xref:{link-mysql-connector}#mysql-example-configuration[Configure the connector] and xref:{link-mysql-connector}#mysql-adding-configuration[add the configuration to your Kafka Connect cluster.]
. Restart your Kafka Connect process to pick up the new JAR files.
If you are working with immutable containers, see link:https://hub.docker.com/r/debezium/[{prodname}'s Container images] for Apache Zookeeper, Apache Kafka, MySQL, and Kafka Connect with the MySQL connector already installed and ready to run.
You can also xref:operations/openshift.adoc[run {prodname} on Kubernetes and OpenShift].
To deploy a {prodname} MySQL connector, you must build a custom Kafka Connect container image that contains the {prodname} connector archive, and then push this container image to a container registry.
You then need to create the following custom resources (CRs):
* A `KafkaConnect` CR that defines your Kafka Connect instance.
The `image` property in the CR specifies the name of the container image that you create to run your {prodname} connector.
You apply this CR to the OpenShift instance where link:https://access.redhat.com/products/red-hat-amq#streams[Red Hat {StreamsName}] is deployed.
{StreamsName} offers operators and images that bring Apache Kafka to OpenShift.
* A `KafkaConnector` CR that defines your {prodname} MySQL connector.
Apply this CR to the same OpenShift instance where you apply the `KafkaConnect` CR.
.Prerequisites
* MySQL is running and you completed the steps to {LinkDebeziumUserGuide}#setting-up-mysql-to-run-a-debezium-connector[set up MySQL to work with a {prodname} connector].
* {StreamsName} is deployed on OpenShift and is running Apache Kafka and Kafka Connect.
* You have an account and permissions to create and manage containers in the container registry (such as `quay.io` or `docker.io`) to which you plan to add the container that will run your Debezium connector.
.Procedure
. Create the {prodname} MySQL container for Kafka Connect:
RUN curl -O {red-hat-maven-repository}debezium/debezium-connector-{connector-file}/{debezium-version}-redhat-__<build_number>__/debezium-connector-{connector-file}-{debezium-version}-redhat-__<build_number>__-plugin.zip
<2> Specifies the path to your Kafka Connect plug-ins directory. If your Kafka Connect plug-ins directory is in a different location, replace this path with the actual path of your directory.
.. Create a new {prodname} MySQL `KafkaConnect` custom resource (CR).
For example, create a `KafkaConnect` CR with the name `dbz-connect.yaml` that specifies `annotations` and `image` properties as shown in the following example:
+
[source,yaml,subs="+attributes"]
----
apiVersion: {KafkaConnectApiVersion}
kind: KafkaConnect
metadata:
name: my-connect-cluster
annotations:
strimzi.io/use-connector-resources: "true" // <1>
spec:
#...
image: debezium-container-for-mysql // <2>
----
<1> `metadata.annotations` indicates to the Cluster Operator that KafkaConnector resources are used to configure connectors in this Kafka Connect cluster.
<2> `spec.image` specifies the name of the image that you created to run your Debezium connector.
This property overrides the `STRIMZI_DEFAULT_KAFKA_CONNECT_IMAGE` variable in the Cluster Operator.
. Create a `KafkaConnector` custom resource that configures your {prodname} MySQL connector instance.
+
You configure a {prodname} MySQL connector in a `.yaml` file that specifies the configuration properties for the connector.
The connector configuration might instruct {prodname} to produce events for a subset of the schemas and tables, or it might set properties so that {prodname} ignores, masks, or truncates values in specified columns that are sensitive, too large, or not needed.
+
The following example configures a {prodname} connector that connects to a MySQL host, `192.168.99.100`, on port `3306`,
Upon restart, the connector recovers the schemas of the database that existed at the point in time in the binlog when the connector should begin reading.
The preceding command registers `inventory-connector` and the connector starts to run against the `inventory` database as defined in the `KafkaConnector` CR.
Following is an example of the configuration for a connector instance that captures data from a MySQL server on port 3306 at 192.168.99.100, which we logically name `fullfillment`.
Typically, you configure the {prodname} MySQL connector in a JSON file by setting the configuration properties that are available for the connector.
<1> Connector's name when registered with the Kafka Connect service.
<2> Connector's class name.
<3> MySQL server address.
<4> MySQL server port number.
<5> MySQL user with the appropriate privileges.
<6> MySQL user's password.
<7> Unique ID of the connector.
<8> Logical name of the MySQL server or cluster.
<9> List of databases hosted by the specified server.
<10> List of Kafka brokers that the connector uses to write and recover DDL statements to the database history topic.
<11> Name of the database history topic. This topic is for internal use only and should not be used by consumers.
<12> Flag that specifies if the connector should generate events for DDL changes and emit them to the `fulfillment` schema change topic for use by consumers.
After the connector starts, it xref:{link-mysql-connector}#mysql-snapshots[performs a consistent snapshot] of the MySQL databases that the connector is configured for.
* xref:debezium-{context}-connector-database-history-configuration-properties[Database history connector configuration properties] that control how {prodname} processes events that it reads from the database history topic.
** xref:{context}-pass-through-database-history-properties-for-configuring-producer-and-consumer-clients[Pass-through database history properties]
* xref:debezium-{context}-connector-pass-through-database-driver-configuration-properties[Pass-through database driver properties] that control the behavior of the database driver.
|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.
|Logical name that identifies and provides a namespace for the particular MySQL database server/cluster in which {prodname} is capturing changes. The logical name should be unique across all other connectors, since it is used as a prefix for all Kafka topic names that receive events emitted by this connector.
If you change the name value, after a restart, instead of continuing to emit events to the original topics, the connector emits subsequent events to topics whose names are based on the new value.
The connector is also unable to recover its database history topic.
|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.
|An optional, comma-separated list of regular expressions that match the names of the databases for which to capture changes. The connector does not capture changes in any database whose name is not in `database.include.list`. By default, the connector captures changes in all databases.
|An optional, comma-separated list of regular expressions that match the names of databases for which you do not want to capture changes. The connector captures changes in any database whose name is not in the `database.exclude.list`.
|An optional, comma-separated list of regular expressions that match fully-qualified table identifiers of tables whose changes you want to capture. The connector does not capture changes in any table not included in `table.include.list`. Each identifier is of the form _databaseName_._tableName_. By default, the connector captures changes in every non-system table in each database whose changes are being captured.
Do not also specify the `table.exclude.list` connector configuration property.
|An optional, comma-separated list of regular expressions that match fully-qualified table identifiers for tables whose changes you do not want to capture. The connector captures changes in any table not included in `table.exclude.list`. Each identifier is of the form _databaseName_._tableName_.
Do not also specify the `table.include.list` connector configuration property.
|An optional, comma-separated list of regular expressions that match the fully-qualified names of columns to exclude from change event record values. Fully-qualified names for columns are of the form _databaseName_._tableName_._columnName_.
|An optional, comma-separated list of regular expressions that match the fully-qualified names of columns to include in change event record values. Fully-qualified names for columns are of the form _databaseName_._tableName_._columnName_.
|An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns whose values should be truncated in the change event record values if the field values are longer than the specified number of characters. You can configure multiple properties with different lengths in a single configuration. The length must be a positive integer. Fully-qualified names for columns are of the form _databaseName_._tableName_._columnName_.
|An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns whose values should be replaced in the change event message values with a field value consisting of the specified number of asterisk (`*`) characters. You can configure multiple properties with different lengths in a single configuration. Each length must be a positive integer or zero. Fully-qualified names for columns are of the form _databaseName_._tableName_._columnName_.
In the resulting change event record, the values for the specified columns are replaced with pseudonyms. +
A pseudonym consists of the hashed value that results from applying the specified _hashAlgorithm_ and _salt_.
Based on the hash function that is used, referential integrity is maintained, while column values are replaced with pseudonyms.
Supported hash functions are described in the {link-java7-standard-names}[MessageDigest section] of the Java Cryptography Architecture Standard Algorithm Name Documentation. +
a|An optional, comma-separated list of regular expressions that match the fully-qualified names of columns whose original type and length should be added as a parameter to the corresponding field schemas in the emitted change event records. These schema parameters:
are used to propagate the original type name and length for variable-width types, respectively. This is useful to properly size corresponding columns in sink databases. Fully-qualified names for columns are of one of these forms:
a|An optional, comma-separated list of regular expressions that match the database-specific data type name of columns whose original type and length should be added as a parameter to the corresponding field schemas in the emitted change event records. These schema parameters:
are used to propagate the original type name and length for variable-width types, respectively. This is useful to properly size corresponding columns in sink databases. Fully-qualified data type names are of one of these forms:
`adaptive_time_microseconds` (the default) captures the date, datetime and timestamp values exactly as in the database using either millisecond, microsecond, or nanosecond precision values based on the database column's type, with the exception of TIME type fields, which are always captured as microseconds. +
`adaptive` (deprecated) captures the time and timestamp values exactly as in the database using either millisecond, microsecond, or nanosecond precision values based on the database column's type. +
`connect` always represents time and timestamp values using Kafka Connect's built-in representations for Time, Date, and Timestamp, which use millisecond precision regardless of the database columns' precision.
|Specifies how BIGINT UNSIGNED columns should be represented in change events. Possible settings are: +
+
`long` represents values by using Java's `long`, which might not offer the precision but which is easy to use in consumers. `long` is usually the preferred setting. +
+
`precise` uses `java.math.BigDecimal` to represent values, which are encoded in the change events by using a binary representation and Kafka Connect's `org.apache.kafka.connect.data.Decimal` type. Use this setting when working with values larger than 2^63, because these values cannot be conveyed by using `long`.
|Boolean value that specifies whether the connector should publish changes in the database schema to a Kafka topic with the same name as the database server ID. Each schema change is recorded by using a key that contains the database name and whose value includes the DDL statement(s). This is independent of how the connector internally records database history.
|Boolean value that specifies whether the connector should parse and publish table and column comments on metadata objects. Enabling this option will bring the implications on memory usage. The number and size of logical schema objects is what largely impacts how much memory is consumed by the Debezium connectors, and adding potentially large string data to each of them can potentially be quite expensive.
If you set this option to `true` then you must also configure MySQL with the `binlog_rows_query_log_events` option set to `ON`. When `include.query` is `true`, the query is not present for events that the snapshot process generates. +
Setting `include.query` to `true` might expose tables or fields that are explicitly excluded or masked by including the original SQL statement in the change event. For this reason, the default setting is `false`.
|Specifies how the connector should react to binlog events that relate to tables that are not present in internal schema representation. That is, the internal representation is not consistent with the database. +
+
`fail` throws an exception that indicates the problematic event and its binlog offset, and causes the connector to stop. +
+
`warn` logs the problematic event and its binlog offset and skips the event. +
+
`skip` passes over the problematic event and does not log anything.
|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.
If xref:mysql-property-max-queue-size[`max.queue.size`] is also set, writing to the queue is blocked when the size of the queue reaches the limit specified by either property.
For example, if you set `max.queue.size=1000`, and `max.queue.size.in.bytes=5000`, writing to the queue is blocked after the queue contains 1000 records, or after the volume of the records in the queue reaches 5000 bytes.
|Positive integer value that specifies the number of milliseconds the connector should wait for new change events to appear before it starts processing a batch of events. Defaults to 1000 milliseconds, or 1 second.
|A positive integer value that specifies the maximum time in milliseconds this connector should wait after trying to connect to the MySQL database server before timing out. Defaults to 30 seconds.
|A comma-separated list of regular expressions that match source UUIDs in the GTID set used to find the binlog position in the MySQL server. Only the GTID ranges that have sources that match one of these include patterns are used.
Do not also specify a setting for `gtid.source.excludes`.
|A comma-separated list of regular expressions that match source UUIDs in the GTID set used to find the binlog position in the MySQL server. Only the GTID ranges that have sources that do not match any of these exclude patterns are used. Do not also specify a value for `gtid.source.includes`.
After a source record is deleted, emitting a tombstone event (the default behavior) allows Kafka to completely delete all events that pertain to the key of the deleted row in case {link-kafka-docs}/#compaction[log compaction] is enabled for the topic.
|A list of expressions that specify the columns that the connector uses to form custom message keys for change event records that it publishes to the Kafka topics for specified tables.
By default, {prodname} uses the primary key column of a table as the message key for records that it emits.
In place of the default, or to specify a key for tables that lack a primary key, you can configure custom message keys based on one or more columns. +
+
To establish a custom message key for a table, list the table, followed by the columns to use as the message key.
The following table describes xref:{link-mysql-connector}#mysql-advanced-connector-configuration-properties[advanced MySQL connector properties]. The default values for these properties rarely need to be changed. Therefore, you do not need to specify them in the connector configuration.
|Enumerates a comma-separated list of the symbolic names of the {link-prefix}:{link-custom-converters}#custom-converters[custom converter] instances that the connector can use. +
For each converter that you configure for a connector, you must also add a `.type` property, which specifies the fully-qualifed name of the class that implements the converter interface.
If you want to further control the behavior of a configured converter, you can add one or more configuration parameters to pass values to the converter.
To associate these additional configuration parameter with a converter, prefix the paraemeter name with the symbolic name of the converter. +
For example, to define a `selector` parameter that specifies the subset of columns that the `boolean` converter processes, add the following property: +
|A Boolean value that specifies whether built-in system tables should be ignored. This applies regardless of the table include and exclude lists. By default, system tables are excluded from having their changes captured, and no events are generated when changes are made to any system tables.
|Specifies whether to use an encrypted connection. Possible settings are: +
+
`disabled` specifies the use of an unencrypted connection. +
+
`preferred` establishes an encrypted connection if the server supports secure connections. If the server does not support secure connections, falls back to an unencrypted connection. +
+
`required` establishes an encrypted connection or fails if one cannot be made for any reason. +
+
`verify_ca` behaves like `required` but additionally it verifies the server TLS certificate against the configured Certificate Authority (CA) certificates and fails if the server TLS certificate does not match any valid CA certificates. +
+
`verify_identity` behaves like `verify_ca` but additionally verifies that the server certificate matches the host of the remote connection.
When a beginning of a transaction is detected then {prodname} tries to roll forward the binlog position and find either `COMMIT` or `ROLLBACK` so it can determine whether to stream the changes from the transaction.
The size of the binlog buffer defines the maximum number of changes in the transaction that {prodname} can buffer while searching for transaction boundaries.
If the size of the transaction is larger than the buffer then {prodname} must rewind and re-read the events that have not fit into the buffer while streaming. +
+
NOTE: This feature is incubating. Feedback is encouraged. It is expected that this feature is not completely polished.
`initial_only` - the connector runs a snapshot only when no offsets have been recorded for the logical server name and then stops; i.e. it will not read change events from the binlog. +
`when_needed` - the connector runs a snapshot upon startup whenever it deems it necessary. That is, when no offsets are available, or when a previously recorded offset specifies a binlog location or GTID that is not available in the server. +
+
`never` - the connector never uses snapshots. Upon first startup with a logical server name, the connector reads from the beginning of the binlog. Configure this behavior with care. It is valid only when the binlog is guaranteed to contain the entire history of the database. +
+
`schema_only` - the connector runs a snapshot of the schemas and not the data. This setting is useful when you do not need the topics to contain a consistent snapshot of the data but need them to have only the changes since the connector was started. +
+
`schema_only_recovery` - this is a recovery setting for a connector that has already been capturing changes. When you restart the connector, this setting enables recovery of a corrupted or lost database history topic. You might set it periodically to "clean up" a database history topic that has been growing unexpectedly. Database history topics require infinite retention.
a|Controls whether and how long the connector holds the global MySQL read lock, which prevents any updates to the database, while the connector is performing a snapshot. Possible settings are: +
+
`minimal` - the connector holds the global read lock for only the initial portion of the snapshot during which the connector reads the database schemas and other metadata. The remaining work in a snapshot involves selecting all rows from each table. The connector can do this in a consistent fashion by using a REPEATABLE READ transaction. This is the case even when the global read lock is no longer held and other MySQL clients are updating the database. +
+
`minimal_percona` - the connector holds link:https://www.percona.com/doc/percona-server/5.7/management/backup_locks.html[the global backup lock] for only the initial portion of the snapshot during which the connector reads the database schemas and other metadata. The remaining work in a snapshot involves selecting all rows from each table. The connector can do this in a consistent fashion by using a REPEATABLE READ transaction. This is the case even when the global backup lock is no longer held and other MySQL clients are updating the database. This mode does not flush tables to disk, is not blocked by long-running reads, and is available only in Percona Server. +
+
`extended` - blocks all writes for the duration of the snapshot. Use this setting if there are clients that are submitting operations that MySQL excludes from REPEATABLE READ semantics. +
+
`none` - prevents the connector from acquiring any table locks during the snapshot. While this setting is allowed with all snapshot modes, it is safe to use if and _only_ if no schema changes are happening while the snapshot is running. For tables defined with MyISAM engine, the tables would still be locked despite this property being set as MyISAM acquires a table lock. This behavior is unlike InnoDB engine, which acquires row level locks.
|An optional, comma-separated list of regular expressions that match the fully-qualified names (`_<databaseName>.<tableName>_`) of the tables to include in a snapshot.
The specified items must be named in the connector's xref:mysql-property-table-include-list[`table.include.list`] property.
This property takes effect only if the connector's `snapshot.mode` property is set to a value other than `never`. +
For each table in the list, add a further configuration property that specifies the `SELECT` statement for the connector to run on the table when it takes a snapshot.
The specified `SELECT` statement determines the subset of table rows to include in the snapshot.
Use the following format to specify the name of this `SELECT` statement property: +
From a `customers.orders` table that includes the soft-delete column, `delete_flag`, add the following properties if you want a snapshot to include only those records that are not soft-deleted:
|During a snapshot, the connector queries each table for which the connector is configured to capture changes. The connector uses each query result to produce a read event that contains data for all rows in that table. This property determines whether the MySQL connector puts results for a table into memory, which is fast but requires large amounts of memory, or streams the results, which can be slower but work for very large tables. The setting of this property specifies the minimum number of rows a table must contain before the connector streams results. +
+
To skip all table size checks and always stream all results during a snapshot, set this property to `0`.
|Controls how frequently the connector sends heartbeat messages to a Kafka topic. The default behavior is that the connector does not send heartbeat messages. +
Heartbeat messages are useful for monitoring whether the connector is receiving change events from the database. Heartbeat messages might help decrease the number of change events that need to be re-sent when a connector restarts. To send heartbeat messages, set this property to a positive integer, which indicates the number of milliseconds between heartbeat messages.
|A semicolon separated list of SQL statements to be executed when a JDBC connection, not the connection that is reading the transaction log, to the database is established.
The connector might establish JDBC connections at its own discretion, so this property is ony for configuring session parameters. It is not for executing DML statements.
|An interval in milliseconds that the connector should wait before performing a snapshot when the connector starts. If you are starting multiple connectors in a cluster, this property is useful for avoiding snapshot interruptions, which might cause re-balancing of connectors.
|Positive integer that specifies the maximum amount of time (in milliseconds) to wait to obtain table locks when performing a snapshot. If the connector cannot acquire table locks in this time interval, the snapshot fails. See xref:{link-mysql-connector}#mysql-snapshots[how MySQL connectors perform database snapshots].
|Boolean value that indicates whether the connector converts a 2-digit year specification to 4 digits. Set to `false` when conversion is fully delegated to the database. +
MySQL allows users to insert year values with either 2-digits or 4-digits. For 2-digit values, the value gets mapped to a year in the range 1970 - 2069. The default behavior is that the connector does the conversion.
|Schema version for the `source` block in {prodname} events. {prodname} 0.10 introduced a few breaking changes to the structure of the `source` block in order to unify the exposed structure across all the connectors. +
+
By setting this option to `v1`, the structure used in earlier versions can be produced. However, this setting is not recommended and is planned for removal in a future {prodname} version.
|Comma-separated list of operation types to skip during streaming. The following values are possible: `c` for inserts/create, `u` for updates, `d` for deletes. By default, no operations are skipped.
|Fully-qualified name of the data collection that is used to send xref:{link-signalling}#debezium-signaling-enabling-signaling[signals] to the connector. +
| Allow schema changes during an incremental snapshot. When enabled the connector will detect schema change during an incremental snapshot and re-select a current chunk to avoid locking DDLs. +
+
Note that changes to a primary key are not supported and can cause incorrect results if performed during an incremental snapshot. Another limitation is that if a schema change affects only columns' default values, then the change won't be detected until the DDL is processed from the binlog stream. This doesn't affect the snapshot events' values, but the schema of snapshot events may have outdated defaults.
|Determines whether the connector generates events with transaction boundaries and enriches change event envelopes with transaction metadata. Specify `true` if you want the connector to do this. See xref:{link-mysql-connector}#mysql-transaction-metadata[Transaction metadata] for details.
|Controls the name of the topic to which the connector sends transaction metadata messages. The placeholder `${database.server.name}` can be used for referring to the connector's logical name; defaults to `${database.server.name}.transaction`, for example `dbserver1.transaction`.
|A list of host/port pairs that the connector uses for establishing an initial connection to the Kafka cluster. Each pair should point to the same Kafka cluster used by the Kafka Connect process.
The {prodname} connector provides for pass-through configuration of the signals Kafka consumer.
Pass-through signals properties begin with the prefix `signals.consumer.*`.
For example, the connector passes properties such as `signal.consumer.security.protocol=SSL` to the Kafka consumer.
As is the case with the xref:{context}-pass-through-database-history-properties-for-configuring-producer-and-consumer-clients[pass-through properties for database history clients], {prodname} strips the prefixes from the properties before it passes them to the Kafka signals consumer.
// Title: Monitoring {prodname} MySQL connector performance
[[mysql-monitoring]]
== Monitoring
The {prodname} MySQL connector provides three types of metrics that are in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect provide.
* xref:{link-mysql-connector}#mysql-snapshot-metrics[Snapshot metrics] provide information about connector operation while performing a snapshot.
* xref:{link-mysql-connector}#mysql-streaming-metrics[Streaming metrics] provide information about connector operation when the connector is reading the binlog.
* xref:{link-mysql-connector}#mysql-schema-history-metrics[Schema history metrics] provide information about the status of the connector's schema history.
{link-prefix}:{link-debezium-monitoring}#monitoring-debezium[{prodname} monitoring documentation] provides details for how to expose these metrics by using JMX.
The {prodname} MySQL connector also provides the `HoldingGlobalLock` custom snapshot metric. This metric is set to a Boolean value that indicates whether the connector currently holds a global or table write lock.
Transaction-related attributes are available only if binlog event buffering is enabled. See xref:{link-mysql-connector}#mysql-property-binlog-buffer-size[`binlog.buffer.size`] in the advanced connector configuration properties for more details.
|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 transactions that have not conformed to the expected protocol of `BEGIN` + `COMMIT`/`ROLLBACK`. This value should be `0` under normal conditions.
|The number of transactions that have not fit into the look-ahead buffer. For optimal performance, this value should be significantly smaller than `NumberOfCommittedTransactions` and `NumberOfRolledBackTransactions`.
{prodname} is a distributed system that captures all changes in multiple upstream databases; it never misses or loses an event. When the system is operating normally or being managed carefully then {prodname} provides _exactly once_ delivery of every change event record.
If a fault does happen then the system does not lose any events. However, while it is recovering from the fault, it might repeat some change events. In these abnormal situations, {prodname}, like Kafka, provides _at least once_ delivery of change events.
ifdef::community[]
The rest of this section describes how {prodname} handles various kinds of faults and problems.
In these cases, the error message has details about the problem and possibly a suggested workaround. After you correct the configuration or address the MySQL problem, restart the connector.
If your MySQL server becomes unavailable, the {prodname} MySQL connector fails with an error and the connector stops. When the server is available again, restart the connector.
However, if GTIDs are enabled for a highly available MySQL cluster, you can restart the connector immediately. It will connect to a different MySQL server in the cluster, find the location in the server's binlog that represents the last transaction, and begin reading the new server's binlog from that specific location.
If GTIDs are not enabled, the connector records the binlog position of only the MySQL server to which it was connected. To restart from the correct binlog position, you must reconnect to that specific server.
When Kafka Connect stops gracefully, there is a short delay while the {prodname} MySQL connector tasks are stopped and restarted on new Kafka Connect processes.
If Kafka Connect crashes, the process stops and any {prodname} MySQL connector tasks terminate without their most recently-processed offsets being recorded. In distributed mode, Kafka Connect restarts the connector tasks on other processes. However, the MySQL connector resumes from the last offset recorded by the earlier processes. This means that the replacement tasks might generate some of the same events processed prior to the crash, creating duplicate events.
The Kafka Connect framework records {prodname} change events in Kafka by using the Kafka producer API. If the Kafka brokers become unavailable, the {prodname} MySQL connector pauses until the connection is reestablished and the connector resumes where it left off.
If the {prodname} 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.