Debezium connector for MySQL
MySQL has a binary log (binlog) that records all operations in the order in which they are committed to the database. This includes changes to table schemas as well as changes to the data in tables. MySQL uses the binlog for replication and recovery.
的Debezium MySQL connector reads the binlog, produces change events for row-levelINSERT
,UPDATE
, andDELETE
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 initialconsistent snapshotof 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 theDebezium release overview.
How the connector works
An overview of the MySQL topologies that the connector supports is useful for planning your application. To optimally configure and run a Debezium MySQL connector, it is helpful to understand how the connector tracks the structure of tables, exposes schema changes, performs snapshots, and determines Kafka topic names.
的Debezium 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 Debezium version. |
Supported MySQL topologies
的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 incrementalbackup.In this case, the MySQL connector always connects to and follows this standalone MySQL server instance.
- Primary and replica
-
的Debezium MySQL connector can follow one of the primary servers or one of the replicas (if that replica has its binlog enabled), but the connector sees changes in only the cluster that is visible to that server. Generally, this is not a problem except for the multi-primary topologies.
的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 ofhigh availability解决方案为MySQL,他们使它significantly easier to tolerate and almost immediately recover from problems and failures. Most HA MySQL clusters use GTIDs so that replicas are able to keep track of all changes on any of the primary servers.
- Multi-primary
-
Network Database (NDB) cluster replicationuses one or more MySQL replica nodes that each replicate from multiple primary servers. This is a powerful way to aggregate the replication of multiple MySQL clusters. This topology requires the use of GTIDs.
A Debezium MySQL connector can use these multi-primary MySQL replicas as sources, and can fail over to different multi-primary MySQL replicas as long as the new replica is caught up to the old replica. That is, the new replica has all transactions that were seen on the first replica. This works even if the connector is using only a subset of databases and/or tables, as the connector can be configured to include or exclude specific GTID sources when attempting to reconnect to a new multi-primary MySQL replica and find the correct position in the binlog.
- Hosted
-
的re 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 theconsistent snapshot.
Schema history topic
When a database client queries a database, the client uses the database’s current schema. However, the database schema can be changed at any time, which means that the connector must be able to identify what the schema was at the time each insert, update, or delete operation was recorded. Also, a connector cannot just use the current schema because the connector might be processing events that are relatively old 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 schema history Kafka topic, the connector records all DDL statements along with the position in the binlog where each DDL statement appeared.
When the connector restarts after 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 schema history Kafka topic and parsing all DDL statements up to the point in the binlog where the connector is starting.
This database schema history topic is for connector use only. The connector can optionallyemit 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 asgh-ost
orpt-online-schema-change
is applied, there are helper tables created during the migration process. The connector needs to be configured to capture change to these helper tables. If consumers do not need the records generated for helper tables, then a single message transform can be applied to filter them out.
Seedefault names for topicsthat receive Debezium event records.
Schema change topic
You can configure a Debezium 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
, wheretopicPrefix
is the namespace specified in thetopic.prefix
连接器配置属性。Messages that the connector sends to the schema change topic contain a payload, and, optionally, also contain the schema of the change event message.
的payload of a schema change event message includes the following elements:
-
ddl
-
Provides the SQL
CREATE
,ALTER
, orDROP
statement that results in the schema change. -
databaseName
-
的名称database to which the DDL statements are applied. The value of
databaseName
serves as the message key. -
pos
-
的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 schema history topic. The internal database schema history topic is for connector use only and it is not intended for direct use by consuming applications. Ensure that applications that require notifications about schema changes consume that information only from the schema change topic. |
Never partition the database schema history topic. For the database schema history topic to function correctly, it must maintain a consistent, global order of the event records that the connector emits to it. 确保主题不是partit之间的分裂ions, set the partition count for the topic by using one of the following methods:
|
的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. |
的following example shows a typical schema change message in JSON format. The message contains a logical representation of the table schema.
{ "schema": { }, "payload": { "source": {(1)"version": "2.2.0.Alpha2", "connector": "mysql", "name": "mysql", "ts_ms": 1651535750218,(2)"snapshot": "false", "db": "inventory", "sequence": null, "table": "customers", "server_id": 223344, "gtid": null, "file": "mysql-bin.000003", "pos": 570, "row": 0, "thread": null, "query": null }, "databaseName": "inventory",(3)“schemaName”:空,“ddl”:“ALTER TABLE的客户ADD middle_name varchar(255) AFTER first_name",(4)"tableChanges": [(5){ "type": "ALTER",(6)"id": "\"inventory\".\"customers\"",(7)"table": {(8)"defaultCharsetName": "utf8mb4", "primaryKeyColumnNames": [(9)"id" ], "columns": [(10){ "name": "id", "jdbcType": 4, "nativeType": null, "typeName": "INT", "typeExpression": "INT", "charsetName": null, "length": null, "scale": null, "position": 1, "optional": false, "autoIncremented": true, "generated": true }, { "name": "first_name", "jdbcType": 12, "nativeType": null, "typeName": "VARCHAR", "typeExpression": "VARCHAR", "charsetName": "utf8mb4", "length": 255, "scale": null, "position": 2, "optional": false, "autoIncremented": false, "generated": false }, { "name": "middle_name", "jdbcType": 12, "nativeType": null, "typeName": "VARCHAR", "typeExpression": "VARCHAR", "charsetName": "utf8mb4", "length": 255, "scale": null, "position": 3, "optional": true, "autoIncremented": false, "generated": false }, { "name": "last_name", "jdbcType": 12, "nativeType": null, "typeName": "VARCHAR", "typeExpression": "VARCHAR", "charsetName": "utf8mb4", "length": 255, "scale": null, "position": 4, "optional": false, "autoIncremented": false, "generated": false }, { "name": "email", "jdbcType": 12, "nativeType": null, "typeName": "VARCHAR", "typeExpression": "VARCHAR", "charsetName": "utf8mb4", "length": 255, "scale": null, "position": 5, "optional": false, "autoIncremented": false, "generated": false } ], "attributes": [(11){ "customAttribute": "attributeValue" } ] } } ] } }
Item | Field name | Description |
---|---|---|
1 |
|
的 |
2 |
|
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. |
3 |
|
Identifies the database and the schema that contains the change. The value of the |
4 |
|
This field contains the DDL that is responsible for the schema change. The |
5 |
|
An array of one or more items that contain the schema changes generated by a DDL command. |
6 |
|
Describes the kind of change. The value is one of the following:
|
7 |
|
Full identifier of the table that was created, altered, or dropped. In the case of a table rename, this identifier is a concatenation of |
8 |
|
代表resents table metadata after the applied change. |
9 |
|
List of columns that compose the table’s primary key. |
10 |
|
Metadata for each column in the changed table. |
11 |
|
Custom attribute metadata for each table change. |
See also:schema history topic.
Snapshots
When a Debezium MySQL connector is first started, it performs an initialconsistent snapshotof your database. The following flow describes how the connector creates this snapshot. This flow is for the default snapshot mode, which isinitial
.For information about other snapshot modes, see theMySQL connectorsnapshot.mode
configuration property.
Step | Action |
---|---|
1 |
Grabs a global read lock that blockswritesby other database clients. |
2 |
Starts a transaction withrepeatable read semanticsto ensure that all subsequent reads within the transaction are done against theconsistent snapshot. |
3 |
Reads the current binlog position. |
4 |
Reads the schema of the databases and tables for which the connector is configured to capture changes. |
5 |
Releases the global read lock. Other database clients can now write to the database. |
6 |
If applicable, writes the DDL changes to the schema change topic, including all necessary |
7 |
Scans the database tables. For each row, the connector emits |
8 |
Commits the transaction. |
9 |
Records the completed snapshot in the connector offsets. |
- Connector restarts
-
If the connector fails, stops, or is rebalanced while performing theinitial snapshot, then after the connector restarts, it performs a new snapshot. After thatintial snapshotis 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 theinitial snapshot它的起始位置。troubl的更多提示eshooting the Debezium MySQL connector, seebehavior when things go wrong.
- Global read locks not allowed
-
Some environments do not allow global read locks. If the Debezium MySQL connector detects that global read locks are not permitted, the connector uses table-level locks instead and performs a snapshot with this method. This requires the database user for the Debezium connector to have
LOCK TABLES
特权。Table 3. Workflow for performing an initial snapshot with table-level locks Step Action 1
Obtains table-level locks.
2
Starts a transaction withrepeatable read semanticsto ensure that all subsequent reads within the transaction are done against theconsistent snapshot.
3
Reads and filters the names of the databases and tables.
4
Reads the current binlog position.
5
Reads the schema of the databases and tables for which the connector is configured to capture changes.
6
If applicable, writes the DDL changes to the schema change topic, including all necessary
DROP…
andCREATE…
DDL statements.7
Scans the database tables. For each row, the connector emits
CREATE
events to the relevant table-specific Kafka topics.8
Commits the transaction.
9
Releases the table-level locks.
10
Records the completed snapshot in the connector offsets.
Ad hoc snapshots
By default, a connector runs an initial snapshot operation only after it starts for the first time. Following this initial snapshot, under normal circumstances, the connector does not repeat the snapshot process. Any future change event data that the connector captures comes in through the streaming process only.
然而,在某些情况下的数据ector obtained during the initial snapshot might become stale, lost, or incomplete. To provide a mechanism for recapturing table data, Debezium includes an option to perform ad hoc snapshots. The following changes in a database might be cause for performing an ad hoc snapshot:
的connector configuration is modified to capture a different set of tables.
Kafka topics are deleted and must be rebuilt.
Data corruption occurs due to a configuration error or some other problem.
You can re-run a snapshot for a table for which you previously captured a snapshot by initiating a so-calledad-hoc snapshot.Ad hoc snapshots require the use ofsignaling tables.You initiate an ad hoc snapshot by sending a signal request to the Debezium signaling table.
When you initiate an ad hoc snapshot of an existing table, the connector appends content to the topic that already exists for the table. If a previously existing topic was removed, Debezium can create a topic automatically ifautomatic topic creationis enabled.
Ad hoc snapshot signals specify the tables to include in the snapshot. The snapshot can capture the entire contents of the database, or capture only a subset of the tables in the database. Also, the snapshot can capture a subset of the contents of the table(s) in the database.
You specify the tables to capture by sending anexecute-snapshot
message to the signaling table. Set the type of theexecute-snapshot
signal toincremental
, and provide the names of the tables to include in the snapshot, as described in the following table:
Field | Default | Value |
---|---|---|
|
|
Specifies the type of snapshot that you want to run. |
|
N/A |
An array that contains regular expressions matching the fully-qualified names of the table to be snapshotted. |
|
N/A |
An optional string, which specifies a condition based on the column(s) of the table(s), to capture a subset of the contents of the table(s). |
You initiate an ad hoc snapshot by adding an entry with theexecute-snapshot
signal type to the signaling table. After the connector processes the message, it begins the snapshot operation. The snapshot process reads the first and last primary key values and uses those values as the start and end point for each table. Based on the number of entries in the table, and the configured chunk size, Debezium divides the table into chunks, and proceeds to snapshot each chunk, in succession, one at a time.
Currently, theexecute-snapshot
action type triggersincremental snapshotsonly. For more information, seeIncremental snapshots.
Incremental snapshots
To provide flexibility in managing snapshots, Debezium includes a supplementary snapshot mechanism, known asincremental snapshotting.Incremental snapshots rely on the Debezium mechanism forsending signals to a Debezium connector.Incremental snapshots are based on theDDD-3design document.
In an incremental snapshot, instead of capturing the full state of a database all at once, as in an initial snapshot, Debezium captures each table in phases, in a series of configurable chunks. You can specify the tables that you want the snapshot to capture and thesize of each chunk.的chunk size determines the number of rows that the snapshot collects during each fetch operation on the database. The default chunk size for incremental snapshots is 1 KB.
As an incremental snapshot proceeds, Debezium uses watermarks to track its progress, maintaining a record of each table row that it captures. This phased approach to capturing data provides the following advantages over the standard initial snapshot process:
You can run incremental snapshots in parallel with streamed data capture, instead of postponing streaming until the snapshot completes. The connector continues to capture near real-time events from the change log throughout the snapshot process, and neither operation blocks the other.
If the progress of an incremental snapshot is interrupted, you can resume it without losing any data. After the process resumes, the snapshot begins at the point where it stopped, rather than recapturing the table from the beginning.
You can run an incremental snapshot on demand at any time, and repeat the process as needed to adapt to database updates. For example, you might re-run a snapshot after you modify the connector configuration to add a table to its
table.include.list
property.
When you run an incremental snapshot, Debezium sorts each table by primary key and then splits the table into chunks based on theconfigured chunk size.Working chunk by chunk, it then captures each table row in a chunk. For each row that it captures, the snapshot emits aREAD
event. That event represents the value of the row when the snapshot for the chunk began.
As a snapshot proceeds, it’s likely that other processes continue to access the database, potentially modifying table records. To reflect such changes,INSERT
,UPDATE
, orDELETE
operations are committed to the transaction log as per usual. Similarly, the ongoing Debezium streaming process continues to detect these change events and emits corresponding change event records to Kafka.
In some cases, theUPDATE
orDELETE
events that the streaming process emits are received out of sequence. That is, the streaming process might emit an event that modifies a table row before the snapshot captures the chunk that contains theREAD
event for that row. When the snapshot eventually emits the correspondingREAD
event for the row, its value is already superseded. To ensure that incremental snapshot events that arrive out of sequence are processed in the correct logical order, Debezium employs a buffering scheme for resolving collisions. Only after collisions between the snapshot events and the streamed events are resolved does Debezium emit an event record to Kafka.
To assist in resolving collisions between late-arrivingREAD
events and streamed events that modify the same table row, Debezium employs a so-calledsnapshot window.的snapshot windows demarcates the interval during which an incremental snapshot captures data for a specified table chunk. Before the snapshot window for a chunk opens, Debezium follows its usual behavior and emits events from the transaction log directly downstream to the target Kafka topic. But from the moment that the snapshot for a particular chunk opens, until it closes, Debezium performs a de-duplication step to resolve collisions between events that have the same primary key..
For each data collection, the Debezium emits two types of events, and stores the records for them both in a single destination Kafka topic. The snapshot records that it captures directly from a table are emitted asREAD
operations. Meanwhile, as users continue to update records in the data collection, and the transaction log is updated to reflect each commit, Debezium emitsUPDATE
orDELETE
operations for each change.
As the snapshot window opens, and Debezium begins processing a snapshot chunk, it delivers snapshot records to a memory buffer. During the snapshot windows, the primary keys of theREAD
events in the buffer are compared to the primary keys of the incoming streamed events. If no match is found, the streamed event record is sent directly to Kafka. If Debezium detects a match, it discards the bufferedREAD
event, and writes the streamed record to the destination topic, because the streamed event logically supersede the static snapshot event. After the snapshot window for the chunk closes, the buffer contains onlyREAD
events for which no related transaction log events exist. Debezium emits these remainingREAD
events to the table’s Kafka topic.
的connector repeats the process for each snapshot chunk.
Triggering an incremental snapshot
Currently, the only way to initiate an incremental snapshot is to send anad hoc snapshot signalto the signaling table on the source database. You submit a signal to the signaling table as SQLINSERT
queries.
After Debezium detects the change in the signaling table, it reads the signal, and runs the requested snapshot operation.
的query that you submit specifies the tables to include in the snapshot, and, optionally, specifies the kind of snapshot operation. Currently, the only valid option for snapshots operations is the default value,incremental
.
To specify the tables to include in the snapshot, provide adata-collections
数组列出常规的表或数组expressions used to match tables, for example,{"data-collections": ["public.MyFirstTable", "public.MySecondTable"]}
的data-collections
array for an incremental snapshot signal has no default value. If thedata-collections
array is empty, Debezium detects that no action is required and does not perform a snapshot.
If the name of a table that you want to include in a snapshot contains a dot ( |
-
A signaling data collection exists on the source database.
的signaling data collection is specified in the
signal.data.collection
property.
Send a SQL query to add the ad hoc incremental snapshot request to the signaling table:
INSERT INTO
(id, type, data) VALUES (' ' ,'' , '{"data-collections": [""," "],"type":" ","additional-condition":" "}'); For example,
INSERT INTO myschema.debezium_signal (id, type, data)(1)values ('ad-hoc-1',(2)'execute-snapshot',(3)'{"data-collections": ["schema1.table1", "schema2.table2"],(4)"type":"incremental"},(5)"additional-condition":"color=blue"}');(6)
的values of the
id
,type
, anddata
parameters in the command correspond to thefields of the signaling table.的following table describes the parameters in the example:
Table 5. Descriptions of fields in a SQL command for sending an incremental snapshot signal to the signaling table Item Value Description 1
myschema.debezium_signal
Specifies the fully-qualified name of the signaling table on the source database.
2
ad-hoc-1
的
id
parameter specifies an arbitrary string that is assigned as theid
identifier for the signal request.
Use this string to identify logging messages to entries in the signaling table. Debezium does not use this string. Rather, during the snapshot, Debezium generates its ownid
string as a watermarking signal.3
execute-snapshot
Specifies
type
parameter specifies the operation that the signal is intended to trigger.4
data-collections
A required component of the
data
field of a signal that specifies an array of table names or regular expressions to match table names to include in the snapshot.
的array lists regular expressions which match tables by their fully-qualified names, using the same format as you use to specify the name of the connector’s signaling table in thesignal.data.collection
configuration property.5
incremental
An optional
type
component of thedata
field of a signal that specifies the kind of snapshot operation to run.
Currently, the only valid option is the default value,incremental
.
If you do not specify a value, the connector runs an incremental snapshot.6
additional-condition
An optional string, which specifies a condition based on the column(s) of the table(s), to capture a subset of the contents of the tables. For more information about the
additional-condition
parameter, seeAd hoc incremental snapshots withadditional-condition
.
additional-condition
If you want a snapshot to include only a subset of the content in a table, you can modify the signal request by appending anadditional-condition
parameter to the snapshot signal.
的SQL query for a typical snapshot takes the following form:
SELECT * FROM ....
By adding anadditional-condition
parameter, you append aWHERE
condition to the SQL query, as in the following example:
SELECT * FROM WHERE ....
的following example shows a SQL query to send an ad hoc incremental snapshot request with an additional condition to the signaling table:
INSERT INTO (id, type, data) VALUES ('' ,'' , '{"data-collections": [" "," "],"type":" ","additional-condition":" "}');
For example, suppose you have aproducts
table that contains the following columns:
id
(primary key)color
quantity
If you want an incremental snapshot of theproducts
table to include only the data items wherecolor=blue
, you can use the following SQL statement to trigger the snapshot:
INSERT INTO myschema.debezium_signal (id, type, data) VALUES('ad-hoc-1', 'execute-snapshot', '{"data-collections": ["schema1.products"],"type":"incremental", "additional-condition":"color=blue"}');
的additional-condition
parameter also enables you to pass conditions that are based on more than on column. For example, using theproducts
table from the previous example, you can submit a query that triggers an incremental snapshot that includes the data of only those items for whichcolor=blue
andquantity>10
:
INSERT INTO myschema.debezium_signal (id, type, data) VALUES('ad-hoc-1', 'execute-snapshot', '{"data-collections": ["schema1.products"],"type":"incremental", "additional-condition":"color=blue AND quantity>10"}');
的following example, shows the JSON for an incremental snapshot event that is captured by a connector.
{ "before":null, "after": { "pk":"1", "value":"New data" }, "source": { ... "snapshot":"incremental"(1)}, "op":"r",(2)"ts_ms":"1620393591654", "transaction":null }
Item | Field name | Description |
---|---|---|
1 |
|
Specifies the type of snapshot operation to run. |
2 |
|
Specifies the event type. |
Stopping an incremental snapshot
You can also stop an incremental snapshot by sending a signal to the table on the source database. You submit a stop snapshot signal to the table by sending a SQLINSERT
query. After Debezium detects the change in the signaling table, it reads the signal, and stops the incremental snapshot operation if it’s in progress.
的query that you submit specifies the snapshot operation ofincremental
, and, optionally, the tables of the current running snapshot to be removed.
-
A signaling data collection exists on the source database.
的signaling data collection is specified in the
signal.data.collection
property.
Send a SQL query to stop the ad hoc incremental snapshot to the signaling table:
INSERT INTO
(id, type, data) values (' ' , 'stop-snapshot', '{"data-collections": [""," "],"type":"incremental"}'); For example,
INSERT INTO myschema.debezium_signal (id, type, data)(1)values ('ad-hoc-1',(2)'stop-snapshot',(3)'{"data-collections": ["schema1.table1", "schema2.table2"],(4)"type":"incremental"}');(5)
的values of the
id
,type
, anddata
parameters in the signal command correspond to thefields of the signaling table.的following table describes the parameters in the example:
Table 6. Descriptions of fields in a SQL command for sending a stop incremental snapshot signal to the signaling table Item Value Description 1
myschema.debezium_signal
Specifies the fully-qualified name of the signaling table on the source database.
2
ad-hoc-1
的
id
parameter specifies an arbitrary string that is assigned as theid
identifier for the signal request.
Use this string to identify logging messages to entries in the signaling table. Debezium does not use this string.3
stop-snapshot
Specifies
type
parameter specifies the operation that the signal is intended to trigger.4
data-collections
An optional component of the
data
field of a signal that specifies an array of table names or regular expressions to match table names to remove from the snapshot.
的array lists regular expressions which match tables by their fully-qualified names, using the same format as you use to specify the name of the connector’s signaling table in thesignal.data.collection
configuration property. If this component of thedata
field is omitted, the signal stops the entire incremental snapshot that is in progress.5
incremental
A required component of the
data
field of a signal that specifies the kind of snapshot operation that is to be stopped.
Currently, the only valid option isincremental
.
If you do not specify atype
value, the signal fails to stop the incremental snapshot.
Read-only incremental snapshots
MySQL连接器允许运行的增量snapshots with a read-only connection to the database. To run an incremental snapshot with read-only access, the connector uses the executed global transaction IDs (GTID) set as high and low watermarks. The state of a chunk’s window is updated by comparing the GTIDs of binary log (binlog) events or the server’s heartbeats against low and high watermarks.
To switch to a read-only implementation, set the value of theread.only
property totrue
.
If the connector reads from a multi-threaded replica (that is, a replica for which the value of
replica_parallel_workers
is greater than0
) you must set one of the following options:replica_preserve_commit_order=ON
slave_preserve_commit_order=ON
Ad hoc read-only incremental snapshots
When the MySQL connection is read-only, thesignaling tablemechanism can also run a snapshot by sending a message to the Kafka topic that is specified in thesignal.kafka.topicproperty.
的key of the Kafka message must match the value of thetopic.prefix
connector configuration option.
的value is a JSON object withtype
anddata
fields.
的signal type isexecute-snapshot
and thedata
field must have the following fields:
Field | Default | Value |
---|---|---|
|
|
的type of the snapshot to be executed. Currently only |
|
N/A |
An array of comma-separated regular expressions that match fully-qualified names of tables to be snapshotted. |
|
N/A |
An optional string, which specifies a condition based on the column(s) of the table(s), to capture a subset of the contents of the table(s). |
An example of the execute-snapshot Kafka message:
Key = `test_connector` Value = `{"type":"execute-snapshot","data": {"data-collections": ["schema1.table1", "schema1.table2"], "type": "INCREMENTAL"}}`
Ad hoc read-only incremental snapshots with additional-condition
additional-condition
is used to select a subset of a table’s content.To give an analogy how
additional-condition
is used:For a snapshot, the SQL query executed behind the scenes is something like:
SELECT * FROM
…. For a snapshot with a
additional-condition
, theadditional-condition
is appended to the SQL query, something like:SELECT * FROM
WHERE ….
Suppose there is a
products
table with columnsid
(primary key),color
andbrand
.To snapshot just the content of the
products
table wherecolor=blue
Key = `test_connector` Value = `{"type":"execute-snapshot","data": {"data-collections": ["schema1.products"], "type": "INCREMENTAL", "additional-condition":"color=blue"}}`
additional-condition
can be used to pass condition based on multiple columns. Using the sameproducts
table, to snapshot content of theproducts
table wherecolor=blue
andbrand=foo
Key = `test_connector` Value = `{"type":"execute-snapshot","data": {"data-collections": ["schema1.products"], "type": "INCREMENTAL", "additional-condition":"color=blue AND brand=foo"}}`
Stopping an Ad hoc read-only incremental snapshot
When the MySQL connection is read-only, thesignaling tablemechanism can also stop a snapshot by sending a message to the Kafka topic that is specified in thesignal.kafka.topicproperty.
的key of the Kafka message must match the value of thetopic.prefix
connector configuration option.
的value is a JSON object withtype
anddata
fields.
的signal type isstop-snapshot
and thedata
field must have the following fields:
Field | Default | Value |
---|---|---|
|
|
的type of the snapshot to be executed. Currently only |
|
N/A |
An optional array of comma-separated regular expressions that match fully-qualified names of tables to be snapshotted. |
的一个例子stop-snapshot卡夫卡的信息:
Key = `test_connector` Value = `{"type":"stop-snapshot","data": {"data-collections": ["schema1.table1", "schema1.table2"], "type": "INCREMENTAL"}}`
快照事件的操作类型
MySQL连接器放出快照事件READ
operations("op" : "r")
.If you prefer that the connector emits snapshot events asCREATE
(c
) events, configure the DebeziumReadToInsertEvent
single message transform (SMT) to modify the event type.
的following example shows how to configure the SMT:
ReadToInsertEvent
SMT to change the type of snapshot eventstransforms=snapshotasinsert,... transforms.snapshotasinsert.type=io.debezium.connector.mysql.transforms.ReadToInsertEvent
Topic names
By default, the MySQL connector writes change events for all of theINSERT
,UPDATE
, andDELETE
operations that occur in a table to a single Apache Kafka topic that is specific to that table.
的connector uses the following convention to name change event topics:
topicPrefix.databaseName.tableName
Suppose thatfulfillment
is the topic prefix,inventory
is the database name, and the database contains tables namedorders
,customers
, andproducts
.的Debezium MySQL connector emits events to three Kafka topics, one for each table in the database:
fulfillment.inventory.orders fulfillment.inventory.customers fulfillment.inventory.products
的following list provides definitions for the components of the default name:
- topicPrefix
-
的topic prefix as specified by the
topic.prefix
连接器配置属性。 - schemaName
-
的名称schema in which the operation occurred.
- tableName
-
的名称table in which the operation occurred.
的connector applies similar naming conventions to label its internal database schema history topics,schema change topics, and事务metadata topics.
If the default topic name do not meet your requirements, you can configure custom topic names. To configure custom topic names, you specify regular expressions in the logical topic routing SMT. For more information about using the logical topic routing SMT to customize topic naming, seeTopic routing.
Transaction metadata
Debezium can generate events that represent transaction boundaries and that enrich data change event messages.
Limits on when Debezium receives transaction metadata
Debezium registers and receives metadata only for transactions that occur after you deploy the connector. Metadata for transactions that occur before you deploy the connector is not available. |
Debezium generates transaction boundary events for theBEGIN
andEND
delimiters in every transaction. Transaction boundary events contain the following fields:
-
status
-
BEGIN
orEND
. -
id
-
String representation of the unique transaction identifier.
-
ts_ms
-
办理的时间ion boundary event (
BEGIN
orEND
event) at the data source. If the data source does not provide Debezium with the event time, then the field instead represents the time at which Debezium processes the event. -
event_count
(forEND
events) -
Total number of events emitted by the transaction.
-
data_collections
(forEND
events) -
An array of pairs of
data_collection
andevent_count
elements that indicates the number of events that the connector emits for changes that originate from a data collection.
{ "status": "BEGIN", "id": "0e4d5dcd-a33b-11ea-80f1-02010a22a99e:10", "ts_ms": 1486500577125, "event_count": null, "data_collections": null } { "status": "END", "id": "0e4d5dcd-a33b-11ea-80f1-02010a22a99e:10", "ts_ms": 1486500577691, "event_count": 2, "data_collections": [ { "data_collection": "s1.a", "event_count": 1 }, { "data_collection": "s2.a", "event_count": 1 } ] }
Unless overridden via thetopic.transaction
option, the connector emits transaction events to the.事务
topic.
When transaction metadata is enabled the data messageEnvelope
is enriched with a new事务
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
-
的per-data collection position of the event among all events that were emitted by the transaction.
Following is an example of a message:
{ "before": null, "after": { "pk": "2", "aa": "1" }, "source": { ... }, "op": "c", "ts_ms": "1580390884335", "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 Debezium constructed transaction identifier would befile=mysql-bin.000002,pos=1913
.
Data change events
的Debezium MySQL connector generates a data change event for each row-levelINSERT
,UPDATE
, andDELETE
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 aroundcontinuous streams of event messages.然而,这些事件青稞酒的结构e 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.
下面的骨架JSON显示了基本的四个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. Aschema
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:
{ "schema": {(1)... }, "payload": {(2)... }, "schema": {(3)... }, "payload": {(4)... }, }
Item | Field name | Description |
---|---|---|
1 |
|
的first |
2 |
|
的first |
3 |
|
的second |
4 |
|
的second |
By default, the connector streams change event records to topics with names that are the same as the event’s originating table. Seetopic names.
的MySQL connector ensures that all Kafka Connect schema names adhere to theAvro 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’sPRIMARY KEY
(or unique constraint) at the time the connector created the event.
Consider the followingcustomers
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 thecustomers
table has the same event key schema. For as long as thecustomers
table has the previous definition, every change event that captures a change to thecustomers
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 |
|
的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 formatconnector-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
的value in a change event is a bit more complicated than the key. Like the key, the value has aschema
section and apayload
section. Theschema
section contains the schema that describes theEnvelope
structure of thepayload
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;
的value portion of a change event for a change to this table is described for:
createevents
的following example shows the value portion of a change event that the connector generates for an operation that creates data in thecustomers
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)“字段”:“之前”},{“类型”:“结构”、“字段”:[ { "type": "int32", "optional": false, "field": "id" }, { "type": "string", "optional": false, "field": "first_name" }, { "type": "string", "optional": false, "field": "last_name" }, { "type": "string", "optional": false, "field": "email" } ], "optional": true, "name": "mysql-server-1.inventory.customers.Value", "field": "after" }, { "type": "struct", "fields": [ { "type": "string", "optional": false, "field": "version" }, { "type": "string", "optional": false, "field": "connector" }, { "type": "string", "optional": false, "field": "name" }, { "type": "int64", "optional": false, "field": "ts_ms" }, { "type": "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": "2.2.0.Alpha2", "connector": "mysql", "name": "mysql-server-1", "ts_ms": 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 |
|
的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 |
|
的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 |
updateevents
的value of a change event for an update in the samplecustomers
table has the same schema as acreateevent for that table. Likewise, the event value’s payload has the same structure. However, the event value payload contains different values in anupdateevent. Here is an example of a change event value in an event that the connector generates for an update in thecustomers
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": "2.2.0.Alpha2", "name": "mysql-server-1", "connector": "mysql", "name": "mysql-server-1", "ts_ms": 1465581029100, "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(5)} }
Item | Field name | Description |
---|---|---|
1 |
|
An optional field that specifies the state of the row before the event occurred. In anupdateevent 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 anupdateevent value, 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. |
Updating the columns for a row’s primary/unique key changes the value of the row’s key. When a key changes, Debezium outputsthreeevents: a |
Primary key updates
AnUPDATE
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 anUPDATE
event record, the connector emits aDELETE
event record for the old key and aCREATE
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:
的
DELETE
event record has__debezium.newkey
as a message header. The value of this header is the new primary key for the updated row.的
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.
deleteevents
的value in adeletechange event has the sameschema
portion ascreateandupdateevents for the same table. Thepayload
portion in adeleteevent for the samplecustomers
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": "2.2.0.Alpha2", "connector": "mysql", "name": "mysql-server-1", "ts_ms": 1465581902300, "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 adeleteevent value, the |
2 |
|
Optional field that specifies the state of the row after the event occurred. In adeleteevent value, the |
3 |
|
Mandatory field that describes the source metadata for the event. In adeleteevent 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. |
Adeletechange 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 withKafka log compaction.Log compaction enables removal of some older messages as long as at least the most recent message for every key is kept. This lets Kafka reclaim storage space while ensuring that the topic contains a complete data set and can be used for reloading key-based state.
Tombstone events
When a row is deleted, thedeleteevent 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 benull
.To make this possible, after Debezium’s MySQL connector emits adeleteevent, the connector emits a special tombstone event that has the same key but anull
value.
truncateevents
Atruncatechange event signals that a table has been truncated. The message key isnull
in this case, the message value looks like this:
{ "schema": { ... }, "payload": { "source": {(1)"version": "2.2.0.Alpha2", "name": "mysql-server-1", "connector": "mysql", "name": "mysql-server-1", "ts_ms": 1465581029100, "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": "t",(2)"ts_ms": 1465581029523(3)} }
Item | Field name | Description |
---|---|---|
1 |
|
Mandatory field that describes the source metadata for the event. In atruncateevent value, the
|
2 |
|
Mandatory string that describes the type of operation. The |
3 |
|
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 |
In case a singleTRUNCATE
statement applies to multiple tables, onetruncatechange event record for each truncated table will be emitted.
Note that sincetruncateevents represent a change made to an entire table and don’t have a message key, unless you’re working with topics with a single partition, there are no ordering guarantees for the change events pertaining to a table (create,update, etc.) andtruncateevents for that table. For instance a consumer may receive anupdateevent only after atruncateevent for that table, when those events are read from different partitions.
Data type mappings
的Debezium MySQL connector represents changes to rows with events that are structured like the table in which the row exists. The event contains a field for each column value. The MySQL data type of that column dictates how Debezium represents the value in the event.
Columns that store strings are defined in MySQL with a character set and collation. The MySQL connector uses the column’s character set when reading the binary representation of the column values in the binlog events.
的connector can map MySQL data types to bothliteralandsemantictypes.
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 cancreate a custom converterfor the connector.
Basic types
的following table shows how the connector maps basic MySQL data types.
MySQL type | Literal type | Semantic type |
---|---|---|
|
|
n/a |
|
|
n/a |
|
|
|
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
的precision is used only to determine storage size. A precision |
|
|
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 |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
Temporal types
Excluding theTIMESTAMP
data type, MySQL temporal types depend on the value of thetime.precision.mode
连接器配置属性。ForTIMESTAMP
columns whose default value is specified asCURRENT_TIMESTAMP
orNOW
, the value1970-01-01 00:00:00
is used as the default value in the Kafka Connect schema.
MySQL allows zero-values forDATE
,DATETIME
, andTIMESTAMP
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.
的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. TheTIMESTAMP
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 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 equivalentio.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 databaseconnectionTimeZone
MySQL configuration option. For example, if the database’s time zone (either globally or configured for the connector by means of theconnectionTimeZone
option) is "America/Los_Angeles", the TIMESTAMP value "2018-06-20 06:37:03" is represented by aZonedTimestamp
with the value "2018-06-20T13:37:03Z".
的time zone of the JVM running Kafka Connect and Debezium does not affect these conversions.
More details about properties related to temporal values are in the documentation forMySQL connector configuration properties.
- time.precision.mode=adaptive_time_microseconds(default)
-
的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.Table 16. Mappings when time.precision.mode=adaptive_time_microseconds
MySQL type Literal type Semantic type DATE
INT32
io.debezium.time.Date
代表resents the number of days since the epoch.TIME[(M)]
INT64
io.debezium.time.MicroTime
代表resents the time value in microseconds and does not include time zone information. MySQL allowsM
to be in the range of0-6
.DATETIME, DATETIME(0), DATETIME(1), DATETIME(2), DATETIME(3)
INT64
io.debezium.time.Timestamp
代表resents the number of milliseconds past the epoch and does not include time zone information.DATETIME(4), DATETIME(5), DATETIME(6)
INT64
io.debezium.time.MicroTimestamp
代表resents the number of microseconds past the epoch and does not include time zone information. - time.precision.mode =连接
-
卡夫卡连接定义的MySQL连接器使用日志ical types. This approach is less precise than the default approach and the events could be less precise if the database column has afractional second precisionvalue of greater than
3
.Values in only the range of00:00:00.000
to23:59:59.999
can be handled. Settime.precision.mode =连接
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.Table 17. Mappings when time.precision.mode =连接
MySQL type Literal type Semantic type DATE
INT32
org.apache.kafka.connect.data.Date
代表resents the number of days since the epoch.TIME[(M)]
INT64
org.apache.kafka.connect.data.Time
代表resents the time value in microseconds since midnight and does not include time zone information.DATETIME[(M)]
INT64
org.apache.kafka.connect.data.Timestamp
代表resents the number of milliseconds since the epoch, and does not include time zone information.
Decimal types
Debezium connectors handle decimals according to the setting of thedecimal.handling.mode
connector configuration property.
- decimal.handling.mode=precise
-
Table 18. Mappings when decimal.handling.mode=precise
MySQL type Literal type Semantic type NUMERIC[(M[,D])]
BYTES
org.apache.kafka.connect.data.Decimal
的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
的scale
schema parameter contains an integer that represents how many digits the decimal point shifted. - decimal.handling.mode=double
-
Table 19. Mappings when 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
-
Table 20. Mappings when 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 theBOOLEAN
value internally in a specific way. TheBOOLEAN
column is internally mapped to theTINYINT(1)
data type. When the table is created during streaming then it uses properBOOLEAN
mapping as Debezium receives the original DDL. During snapshots, Debezium executesSHOW CREATE TABLE
to obtain table definitions that returnTINYINT(1)
for bothBOOLEAN
andTINYINT(1)
columns. Debezium then has no way to obtain the original type mapping and so maps toTINYINT(1)
.
To enable you to convert source columns to Boolean data types, Debezium provides aTinyIntOneToBooleanConverter
custom converterthat you can use in one of the following ways:
Map all
TINYINT(1)
orTINYINT(1) UNSIGNED
columns toBOOLEAN
types.Enumerate a subset of columns by using a comma-separated list of regular expressions.
To use this type of conversion, you must set theconverters
configuration property with theselector
parameter, as shown in the following example:converters=boolean boolean.type=io.debezium.connector.mysql.converters.TinyIntOneToBooleanConverter boolean.selector=db1.table1.*, db1.table2.column1
NOTE: MySQL8 not showing the length of
tinyint unsigned
type when snapshot executesSHOW CREATE TABLE
, which means this converter doesn’t work. The new optionlength.checker
can solve this issue, the default value istrue
.Disable thelength.checker
and specify the columns that need to be converted toselector
property instead of converting all columns based on type, as shown in the following example:converters=boolean boolean.type=io.debezium.connector.mysql.converters.TinyIntOneToBooleanConverter boolean.length.checker=false boolean.selector=db1.table1.*, db1.table2.column1
Spatial types
Currently, the Debezium MySQL connector supports the following spatial data types.
MySQL type | Literal type | Semantic type |
---|---|---|
|
|
|
Setting up MySQL
Some MySQL setup tasks are required before you can install and run a Debezium connector.
Creating a user
A Debezium MySQL connector requires a MySQL user account. This MySQL user must have appropriate permissions on all databases for which the Debezium MySQL connector captures changes.
A MySQL server.
Basic knowledge of 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';
的table below describes the permissions.
If using a hosted option such as Amazon RDS or Amazon Aurora that does not allow a global read lock, table-level locks are used to create theconsistent snapshot.In this case, you need to also grant LOCK TABLES
permissions to the user that you create. Seesnapshotsfor more details.Finalize the user’s permissions:
mysql> FLUSH PRIVILEGES;
Keyword | Description |
---|---|
|
Enables the connector to select rows from tables in databases. This is used only when performing a snapshot. |
|
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 the following statements:
的connector always requires this. |
|
Identifies the database to which the permissions apply. |
|
Specifies the user to grant the permissions to. |
|
Specifies the user’s MySQL password. |
Enabling the binlog
你必须enable binary logging for MySQL replication. The binary logs record transaction updates for replication tools to propagate changes.
A MySQL server.
Appropriate MySQL user privileges.
Check whether the
log-bin
option is already on:// for MySql 5.x mysql> SELECT variable_value as "BINARY LOGGING STATUS (log-bin) ::" FROM information_schema.global_variables WHERE variable_name='log_bin'; // for MySql 8.x mysql> SELECT variable_value as "BINARY LOGGING STATUS (log-bin) ::" FROM performance_schema.global_variables WHERE variable_name='log_bin';
If it is
OFF
, configure your MySQL server configuration file with the following properties, which are described in the table below:server-id = 223344 # Querying variable is called server_id, e.g. SELECT variable_value FROM information_schema.global_variables WHERE variable_name='server_id'; log_bin = mysql-bin binlog_format = ROW binlog_row_image = FULL expire_logs_days = 10
Confirm your changes by checking the binlog status once more:
// for MySql 5.x mysql> SELECT variable_value as "BINARY LOGGING STATUS (log-bin) ::" FROM information_schema.global_variables WHERE variable_name='log_bin'; // for MySql 8.x mysql> SELECT variable_value as "BINARY LOGGING STATUS (log-bin) ::" FROM performance_schema.global_variables WHERE variable_name='log_bin';
Property | Description |
---|---|
|
的value for the |
|
的value of |
|
的 |
|
的 |
|
This is the number of days for automatic binlog file removal. The default is |
Enabling GTIDs
Global transaction identifiers (GTIDs) uniquely identify transactions that occur on a server within a cluster. Though not required for a Debezium MySQL connector, using GTIDs simplifies replication and enables you to more easily confirm if primary and replica servers are consistent.
GTIDs are available in MySQL 5.6.5 and later. See theMySQL documentationfor more details.
A MySQL server.
Basic knowledge of SQL commands.
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 | +--------------------------+-------+
Option | Description |
---|---|
|
Boolean that specifies whether GTID mode of the MySQL server is enabled or not.
|
|
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.
|
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 configuringinteractive_timeout
andwait_timeout
in your MySQL configuration file.
A MySQL server.
Basic knowledge of SQL commands.
Access to the MySQL configuration file.
Configure
interactive_timeout
:mysql> interactive_timeout=
Configure
wait_timeout
:mysql> wait_timeout=
Option | Description |
---|---|
|
的number of seconds the server waits for activity on an interactive connection before closing it. SeeMySQL’s documentationfor more details. |
|
的number of seconds the server waits for activity on a non-interactive connection before closing it. SeeMySQL’s documentationfor more details. |
Enabling query log events
You might want to see the originalSQL
statement for each binlog event. Enabling thebinlog_rows_query_log_events
option in the MySQL configuration file allows you to do this.
This option is available in MySQL 5.6 and later.
A MySQL server.
Basic knowledge of SQL commands.
Access to the MySQL configuration file.
Enable
binlog_rows_query_log_events
:mysql> binlog_rows_query_log_events=ON
binlog_rows_query_log_events
is set to a value that enables/disables support for including the originalSQL
statement in the binlog entry.ON
= enabledOFF
= disabled
Validating binlog row value options
Checkbinlog_row_value_options
variable, and make sure that value isnotset toPARTIAL_JSON
, since in such case connector might fail to consumeUPDATEevents.
A MySQL server.
Basic knowledge of SQL commands.
Access to the MySQL configuration file.
Check current variable value
mysql> show global variables where variable_name = 'binlog_row_value_options';
Result
+--------------------------+-------+ | Variable_name | Value | +--------------------------+-------+ | binlog_row_value_options | | +--------------------------+-------+
In case value is
PARTIAL_JSON
, unset this variable by:mysql> set @@global.binlog_row_value_options="" ;
Deployment
To deploy a Debezium MySQL connector, you install the Debezium MySQL connector archive, configure the connector, and start the connector by adding its configuration to Kafka Connect.
Apache Zookeeper,Apache Kafka, andKafka Connectare installed.
MySQL Server is installed and isset up to work with the Debezium connector.
Download the DebeziumMySQL connector plug-in.
Extract the files into your Kafka Connect environment.
Add the directory with the JAR files toKafka Connect’s
plugin.path
.Configure the connectorandadd 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, seeDebezium’s Container imagesfor Apache Zookeeper, Apache Kafka, MySQL, and Kafka Connect with the MySQL connector already installed and ready to run.
You can alsorun Debezium on Kubernetes and OpenShift.
MySQL connector configuration example
Following is an example of the configuration for a connector instance that captures data from a MySQL server on port 3306 at 192.168.99.100, which we logically namefullfillment
.Typically, you configure the Debezium MySQL connector in a JSON file by setting the configuration properties that are available for the connector.
You can choose to produce events for a subset of the schemas and tables in a database. Optionally, you can ignore, mask, or truncate columns that contain sensitive data, that are larger than a specified size, or that you do not need.
{ "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)"topic.prefix": "fullfillment",(8)"database.include.list": "inventory",(9)"schema.history.internal.kafka.bootstrap.servers": "kafka:9092",(10)"schema.history.internal.kafka.topic": "schemahistory.fullfillment",(11)"include.schema.changes": "true"(12)} }
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 | Topic prefix for 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 schema history topic. |
11 | Name of the database schema 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 thefulfillment schema change topic for use by consumers. |
For the complete list of the configuration properties that you can set for the Debezium MySQL connector, seeMySQL connector configuration properties.
You can send this configuration with aPOST
command to a running Kafka Connect service. The service records the configuration and starts one connector task that performs the following actions:
Connects to the MySQL database.
Reads change-data tables for tables in capture mode.
Streams change event records to Kafka topics.
Adding connector configuration
To start running a MySQL connector, configure a connector configuration, and add the configuration to your Kafka Connect cluster.
的Debezium MySQL connector is installed.
Create a configuration for the MySQL connector.
Use theKafka Connect REST APIto add that connector configuration to your Kafka Connect cluster.
After the connector starts, itperforms a consistent snapshotof the MySQL databases that the connector is configured for. The connector then starts generating data change events for row-level operations and streaming change event records to Kafka topics.
Connector properties
的Debezium MySQL connector has numerous configuration properties that you can use to achieve the right connector behavior for your application. Many properties have default values. Information about the properties is organized as follows:
Database schema history connector configuration propertiesthat control how Debezium processes events that it reads from the database schema history topic.
Pass-through database driver propertiesthat control the behavior of the database driver.
的following configuration properties arerequiredunless a default value is available.
Required Debezium MySQL connector configuration properties
Property | Default | Description | ||
---|---|---|---|---|
No default |
Unique name for the connector. Attempting to register again with the same name fails. This property is required by all Kafka Connect connectors. |
|||
No default |
的名称Java class for the connector. Always specify |
|||
|
的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. |
|||
No default |
IP address or host name of the MySQL database server. |
|||
|
Integer port number of the MySQL database server. |
|||
No default |
Name of the MySQL user to use when connecting to the MySQL database server. |
|||
No default |
Password to use when connecting to the MySQL database server. |
|||
No default |
Topic prefix that provides a namespace for the particular MySQL database server/cluster in which Debezium is capturing changes. The topic prefix should be unique across all other connectors, since it is used as a prefix for all Kafka topic names that receive events emitted by this connector. Only alphanumeric characters, hyphens, dots and underscores must be used in the database server logical name.
|
|||
No default |
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. |
|||
empty string |
一个可选,以逗号分隔的正则表达ssions 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 To match the name of a database, Debezium applies the regular expression that you specify as ananchoredregular expression. That is, the specified expression is matched against the entire name string of the database; it does not match substrings that might be present in a database name. |
|||
empty string |
一个可选,以逗号分隔的正则表达ssions 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 To match the name of a database, Debezium applies the regular expression that you specify as ananchoredregular expression. That is, the specified expression is matched against the entire name string of the database; it does not match substrings that might be present in a database name. |
|||
empty string |
一个可选,以逗号分隔的正则表达ssions that match fully-qualified table identifiers of tables whose changes you want to capture. The connector does not capture changes in any table that is not included in To match the name of a table, Debezium applies the regular expression that you specify as ananchoredregular expression. That is, the specified expression is matched against the entire name string of the table; it does not match substrings that might be present in a table name. |
|||
empty string |
一个可选,以逗号分隔的正则表达ssions that match fully-qualified table identifiers for tables whose changes you do not want to capture. The connector captures changes in any table that is not included in To match the name of a column, Debezium applies the regular expression that you specify as ananchoredregular expression. That is, the specified expression is matched against the entire name string of the table; it does not match substrings that might be present in a table name. |
|||
empty string |
一个可选,以逗号分隔的正则表达ssions that match the fully-qualified names of columns to exclude from change event record values. Fully-qualified names for columns are of the formdatabaseName.tableName.columnName. To match the name of a column, Debezium applies the regular expression that you specify as ananchoredregular expression. That is, the specified expression is matched against the entire name string of the column; it does not match substrings that might be present in a column name. If you include this property in the configuration, do not also set the |
|||
empty string |
一个可选,以逗号分隔的正则表达ssions that match the fully-qualified names of columns to include in change event record values. Fully-qualified names for columns are of the formdatabaseName.tableName.columnName. To match the name of a column, Debezium applies the regular expression that you specify as ananchoredregular expression. That is, the specified expression is matched against the entire name string of the column; it does not match substrings that might be present in a column name. |
|||
n/a |
一个可选,以逗号分隔的正则表达ssions that match the fully-qualified names of character-based columns. Set this property if you want to truncate the data in a set of columns when it exceeds the number of characters specified by thelengthin the property name. Set 的fully-qualified name of a column observes the following format:databaseName.tableName.columnName.To match the name of a column, Debezium applies the regular expression that you specify as ananchoredregular expression. That is, the specified expression is matched against the entire name string of the column; the expression does not match substrings that might be present in a column name. You can specify multiple properties with different lengths in a single configuration. |
|||
n/a |
一个可选,以逗号分隔的正则表达ssions that match the fully-qualified names of character-based columns. Set this property if you want the connector to mask the values for a set of columns, for example, if they contain sensitive data. Set 的fully-qualified name of a column observes the following format:databaseName.tableName.columnName.To match the name of a column, Debezium applies the regular expression that you specify as ananchoredregular expression. That is, the specified expression is matched against the entire name string of the column; the expression does not match substrings that might be present in a column name. You can specify multiple properties with different lengths in a single configuration. |
|||
n/a |
一个可选,以逗号分隔的正则表达ssions that match the fully-qualified names of character-based columns. Fully-qualified names for columns are of the form A pseudonym consists of the hashed value that results from applying the specifiedhashAlgorithmandsalt.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 theMessageDigest sectionof the Java Cryptography Architecture Standard Algorithm Name Documentation. column.mask.hash.sha with.salt.czq——256.MA0cB5K = inventory.orders.customerName, inventory.shipment.customerName If necessary, the pseudonym is automatically shortened to the length of the column. The connector configuration can include multiple properties that specify different hash algorithms and salts. |
|||
n/a |
一个可选,以逗号分隔的正则表达ssions that match the fully-qualified names of columns for which you want the connector to emit extra parameters that represent column metadata. When this property is set, the connector adds the following fields to the schema of event records:
的se parameters propagate a column’s original type name and length (for variable-width types), respectively. 的fully-qualified name of a column observes one of the following formats:databaseName.tableName.columnName, ordatabaseName.schemaName.tableName.columnName. |
|||
n/a |
一个可选,以逗号分隔的正则表达ssions that specify the fully-qualified names of data types that are defined for columns in a database. When this property is set, for columns with matching data types, the connector emits event records that include the following extra fields in their schema:
的se parameters propagate a column’s original type name and length (for variable-width types), respectively. 的fully-qualified name of a column observes one of the following formats:databaseName.tableName.typeName, ordatabaseName.schemaName.tableName.typeName. For the list of MySQL-specific data type names, see theMySQL data type mappings. |
|||
|
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. Possible settings are: |
|||
|
布尔值,用于指定是否连接器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 schema history. |
|||
|
布尔值,用于指定是否连接器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. |
|||
|
布尔值,用于指定是否连接器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. That is, the internal representation is not consistent with the database. |
|||
|
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 maximum number of records that the blocking queue can hold. When Debezium reads events streamed from the database, it places the events in the blocking queue before it writes them to Kafka. The blocking queue can provide backpressure for reading change events from the database in cases where the connector ingests messages faster than it can write them to Kafka, or when Kafka becomes unavailable. Events that are held in the queue are disregarded when the connector periodically records offsets. Always set the value of |
|||
|
A long integer value that specifies the maximum volume of the blocking queue in bytes. By default, volume limits are not specified for the blocking queue. To specify the number of bytes that the queue can consume, set this property to a positive long value. |
|||
|
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 500 milliseconds, or 0.5 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. |
|||
No default |
一个以逗号分隔的正则表达式match source UUIDs in the GTID set used that the connector uses to find the binlog position on the MySQL server. When this property is set, the connector uses only the GTID ranges that have source UUIDs that match one of the specified To match the value of a GTID, Debezium applies the regular expression that you specify as ananchoredregular expression. That is, the specified expression is matched against the entire UUID string; it does not match substrings that might be present in the UUID. |
|||
No default |
一个以逗号分隔的正则表达式match source UUIDs in the GTID set that the connector uses to find the binlog position on the MySQL server. When this property is set, the connector uses only the GTID ranges that have source UUIDs that do not match any of the specified To match the value of a GTID, Debezium applies the regular expression that you specify as ananchoredregular expression. That is, the specified expression is matched against the entire UUID string; it does not match substrings that might be present in the UUID. |
|||
|
Controls whether adeleteevent is followed by a tombstone event. |
|||
n/a |
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, Debezium 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. Each fully-qualified table name is a regular expression in the following format: 的re is no limit to the number of columns that you use to create custom message keys. However, it’s best to use the minimum number that are required to specify a unique key. |
|||
bytes |
Specifies how binary columns, for example, |
|||
none |
Specifies how schema names should be adjusted for compatibility with the message converter used by the connector. Possible settings:
|
|||
none |
Specifies how field names should be adjusted for compatibility with the message converter used by the connector. Possible settings:
SeeAvro namingfor more details. |
Advanced MySQL connector configuration properties
的following table describesadvanced MySQL connector properties.的default values for these properties rarely need to be changed. Therefore, you do not need to specify them in the connector configuration.
Property | Default | Description |
---|---|---|
|
A Boolean value that specifies whether a separate thread should be used to ensure that the connection to the MySQL server/cluster is kept alive. |
|
No default |
Enumerates a comma-separated list of the symbolic names of thecustom converterinstances that the connector can use. For each converter that you configure for a connector, you must also add a For example, boolean.type: io.debezium.connector.mysql.converters.TinyIntOneToBooleanConverter 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. boolean.selector=db1.table1.*, db1.table2.column1 |
|
|
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: |
|
0 |
的size of a look-ahead buffer used by the binlog reader. The default setting of |
|
|
Specifies the criteria for running a snapshot when the connector starts. Possible settings are: |
|
|
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: |
|
All tables specified in |
一个可选,以逗号分隔的正则表达ssions that match the fully-qualified names ( To match the name of a table, Debezium applies the regular expression that you specify as ananchoredregular expression. That is, the specified expression is matched against the entire name string of the table; it does not match substrings that might be present in a table name. |
|
No default |
Specifies the table rows to include in a snapshot. Use the property if you want a snapshot to include only a subset of the rows in a table. This property affects snapshots only. It does not apply to events that the connector reads from the log. 的property contains a comma-separated list of fully-qualified table names in the form From a "snapshot.select.statement.overrides": "customer.orders", "snapshot.select.statement.overrides.customer.orders": "SELECT * FROM [customers].[orders] WHERE delete_flag = 0 ORDER BY id DESC" In the resulting snapshot, the connector includes only the records for which |
|
|
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. |
|
|
Controls how frequently the connector sends heartbeat messages to a Kafka topic. The default behavior is that the connector does not send heartbeat messages. |
|
No default |
Specifies a query that the connector executes on the source database when the connector sends a heartbeat message. |
|
No default |
A semicolon separated list of SQL statements to be executed when a JDBC connection, not the connection that is reading the transaction log, to the database is established. To specify a semicolon as a character in a SQL statement and not as a delimiter, use two semicolons, ( |
|
No default |
一个在terval 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. |
|
No default |
During a snapshot, the connector reads table content in batches of rows. This property specifies the maximum number of rows in a batch. |
|
|
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. Seehow MySQL connectors perform database snapshots. |
|
|
Boolean value that indicates whether the connector converts a 2-digit year specification to 4 digits. Set to |
|
|
Schema version for the |
|
|
A comma-separated list of operation types that will be skipped during streaming. The operations include: |
|
No default value |
Fully-qualified name of the data collection that is used to sendsignalsto 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. |
|
|
的maximum number of rows that the connector fetches and reads into memory during an incremental snapshot chunk. Increasing the chunk size provides greater efficiency, because the snapshot runs fewer snapshot queries of a greater size. However, larger chunk sizes also require more memory to buffer the snapshot data. Adjust the chunk size to a value that provides the best performance in your environment. |
|
|
Switch to alternative incremental snapshot watermarks implementation to avoid writes to signal data collection |
|
|
Determines whether the connector generates events with transaction boundaries and enriches change event envelopes with transaction metadata. Specify |
|
|
Specify how failures during processing of events (i.e. when encountering a corrupted event) should be handled. By default, |
|
|
的名称TopicNamingStrategy class that should be used to determine the topic name for data change, schema change, transaction, heartbeat event etc., defaults to |
|
|
Specify the delimiter for topic name, defaults to |
|
|
的size used for holding the topic names in bounded concurrent hash map. This cache will help to determine the topic name corresponding to a given data collection. |
|
|
Controls the name of the topic to which the connector sends heartbeat messages. The topic name has this pattern: |
|
|
Controls the name of the topic to which the connector sends transaction metadata messages. The topic name has this pattern: |
Debezium connector database schema history configuration properties
Debezium provides a set ofschema.history.internal.*
properties that control how the connector interacts with the schema history topic.
的following table describes theschema.history.internal
properties for configuring the Debezium connector.
Property | Default | Description |
---|---|---|
No default |
的full name of the Kafka topic where the connector stores the database schema history. |
|
No default |
A list of host/port pairs that the connector uses for establishing an initial connection to the Kafka cluster. This connection is used for retrieving the database schema history previously stored by the connector, and for writing each DDL statement read from the source database. Each pair should point to the same Kafka cluster used by the Kafka Connect process. |
|
|
一个整数值,它指定的最大数量of milliseconds the connector should wait during startup/recovery while polling for persisted data. The default is 100ms. |
|
|
一个整数值,它指定的最大数量of milliseconds the connector should wait while fetching cluster information using Kafka admin client. |
|
|
一个整数值,它指定的最大数量of milliseconds the connector should wait while create kafka history topic using Kafka admin client. |
|
|
的maximum number of times that the connector should try to read persisted history data before the connector recovery fails with an error. The maximum amount of time to wait after receiving no data is |
|
|
A Boolean value that specifies whether the connector should ignore malformed or unknown database statements or stop processing so a human can fix the issue. The safe default is |
|
|
A Boolean value that specifies whether the connector should record all DDL statements
的safe default is |
Debezium relies on a Kafka producer to write schema changes to database schema history topics. Similarly, it relies on a Kafka consumer to read from database schema history topics when a connector starts. You define the configuration for the Kafka producer and consumer clients by assigning values to a set of pass-through configuration properties that begin with theschema.history.internal.producer.*
andschema.history.internal.consumer.*
prefixes. The pass-through producer and consumer database schema history properties control a range of behaviors, such as how these clients secure connections with the Kafka broker, as shown in the following example:
schema.history.internal.producer.security.protocol=SSL schema.history.internal.producer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks schema.history.internal.producer.ssl.keystore.password=test1234 schema.history.internal.producer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks schema.history.internal.producer.ssl.truststore.password=test1234 schema.history.internal.producer.ssl.key.password=test1234 schema.history.internal.consumer.security.protocol=SSL schema.history.internal.consumer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks schema.history.internal.consumer.ssl.keystore.password=test1234 schema.history.internal.consumer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks schema.history.internal.consumer.ssl.truststore.password=test1234 schema.history.internal.consumer.ssl.key.password=test1234
Debezium strips the prefix from the property name before it passes the property to the Kafka client.
See the Kafka documentation for more details aboutKafka producer configuration propertiesandKafka consumer configuration properties.
Debezium connector Kafka signals configuration properties
When the MySQL connector is configured as read-only, the alternative for the signaling table is the signals Kafka topic.
Debezium provides a set ofsignal.*
properties that control how the connector interacts with the Kafka signals topic.
的following table describes thesignal
properties.
Property | Default | Description |
---|---|---|
No default |
的名称Kafka topic that the connector monitors for ad hoc signals. |
|
No default |
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. |
|
|
一个整数值,它指定的最大数量of milliseconds the connector should wait when polling signals. The default is 100ms. |
Debezium connector pass-through signals Kafka consumer client configuration properties
的Debezium connector provides for pass-through configuration of the signals Kafka consumer. Pass-through signals properties begin with the prefixsignals.consumer.*
.For example, the connector passes properties such assignal.consumer.security.protocol=SSL
to the Kafka consumer.
As is the case with thepass-through properties for database schema history clients, Debezium strips the prefixes from the properties before it passes them to the Kafka signals consumer.
Debezium connector pass-through database driver configuration properties
的Debezium connector provides for pass-through configuration of the database driver. Pass-through database properties begin with the prefixdriver.*
.For example, the connector passes properties such asdriver.foobar=false
to the JDBC URL.
As is the case with thepass-through properties for database schema history clients, Debezium strips the prefixes from the properties before it passes them to the database driver.
Monitoring
的Debezium MySQL connector provides three types of metrics that are in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect provide.
Snapshot metricsprovide information about connector operation while performing a snapshot.
Streaming metricsprovide information about connector operation when the connector is reading the binlog.
Schema history metricsprovide information about the status of the connector’s schema history.
Debezium monitoring documentationprovides details for how to expose these metrics by using JMX.
Snapshot metrics
的MBeanisdebezium.mysql:type=connector-metrics,context=snapshot,server=
.
Snapshot metrics are not exposed unless a snapshot operation is active, or if a snapshot has occurred since the last connector start.
的following table lists the shapshot metrics that are available.
Attributes | Type | Description |
---|---|---|
|
的last snapshot event that the connector has read. |
|
|
的number of milliseconds since the connector has read and processed the most recent event. |
|
|
的total number of events that this connector has seen since last started or reset. |
|
|
的number of events that have been filtered by include/exclude list filtering rules configured on the connector. |
|
|
的list of tables that are captured by the connector. |
|
|
的length the queue used to pass events between the snapshotter and the main Kafka Connect loop. |
|
|
的free capacity of the queue used to pass events between the snapshotter and the main Kafka Connect loop. |
|
|
的total number of tables that are being included in the snapshot. |
|
|
的number of tables that the snapshot has yet to copy. |
|
|
Whether the snapshot was started. |
|
|
Whether the snapshot was paused. |
|
|
Whether the snapshot was aborted. |
|
|
Whether the snapshot completed. |
|
|
的total number of seconds that the snapshot has taken so far, even if not complete. Includes also time when snapshot was paused. |
|
|
的total number of seconds that the snapshot was paused. If the snapshot was paused several times, the paused time adds up. |
|
|
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. |
|
|
的maximum buffer of the queue in bytes. This metric is available if |
|
|
的current volume, in bytes, of records in the queue. |
的connector also provides the following additional snapshot metrics when an incremental snapshot is executed:
Attributes | Type | Description |
---|---|---|
|
的identifier of the current snapshot chunk. |
|
|
的lower bound of the primary key set defining the current chunk. |
|
|
的upper bound of the primary key set defining the current chunk. |
|
|
的lower bound of the primary key set of the currently snapshotted table. |
|
|
的upper bound of the primary key set of the currently snapshotted table. |
的Debezium MySQL connector also provides theHoldingGlobalLock
custom snapshot metric. This metric is set to a Boolean value that indicates whether the connector currently holds a global or table write lock.
Streaming metrics
Transaction-related attributes are available only if binlog event buffering is enabled. Seebinlog.buffer.size
in the advanced connector configuration properties for more details.
的MBeanisdebezium.mysql:type=connector-metrics,context=streaming,server=
.
的following table lists the streaming metrics that are available.
Attributes | Type | Description |
---|---|---|
|
的last streaming event that the connector has read. |
|
|
的number of milliseconds since the connector has read and processed the most recent event. |
|
|
的total number of events that this connector has seen since the last start or metrics reset. |
|
|
的total number of create events that this connector has seen since the last start or metrics reset. |
|
|
的total number of update events that this connector has seen since the last start or metrics reset. |
|
|
的total number of delete events that this connector has seen since the last start or metrics reset. |
|
|
的number of events that have been filtered by include/exclude list filtering rules configured on the connector. |
|
|
的list of tables that are captured by the connector. |
|
|
的length the queue used to pass events between the streamer and the main Kafka Connect loop. |
|
|
的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. |
|
|
的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. |
|
|
的number of processed transactions that were committed. |
|
|
的coordinates of the last received event. |
|
|
Transaction identifier of the last processed transaction. |
|
|
的maximum buffer of the queue in bytes. This metric is available if |
|
|
的current volume, in bytes, of records in the queue. |
的Debezium MySQL connector also provides the following additional streaming metrics:
Attribute | Type | Description |
---|---|---|
|
的名称binlog file that the connector has most recently read. |
|
|
的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. |
|
|
的string representation of the most recent GTID set processed by the connector when reading the binlog. |
|
|
的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. |
|
|
的number of disconnects by the MySQL connector. |
|
|
的number of processed transactions that were rolled back and not streamed. |
|
|
的number of transactions that have not conformed to the expected protocol of |
|
|
的number of transactions that have not fit into the look-ahead buffer. For optimal performance, this value should be significantly smaller than |
Schema history metrics
的MBeanisdebezium.mysql:type=connector-metrics,context=schema-history,server=
.
的following table lists the schema history metrics that are available.
Attributes | Type | Description |
---|---|---|
|
One of |
|
|
的time in epoch seconds at what recovery has started. |
|
|
的number of changes that were read during recovery phase. |
|
|
the total number of schema changes applied during recovery and runtime. |
|
|
的number of milliseconds that elapsed since the last change was recovered from the history store. |
|
|
的number of milliseconds that elapsed since the last change was applied. |
|
|
的string representation of the last change recovered from the history store. |
|
|
的string representation of the last applied change. |
Behavior when things go wrong
Debezium is a distributed system that captures all changes in multiple upstream databases; it never misses or loses an event. When the system is operating normally or being managed carefully then Debezium providesexactly oncedelivery of every change event record.
If a fault does happen then the system does not lose any events. However, while it is recovering from the fault, it might repeat some change events. In these abnormal situations, Debezium, like Kafka, providesat least oncedelivery of change events.
的rest of this section describes how Debezium handles various kinds of faults and problems.
Configuration and startup errors
In the following situations, the connector fails when trying to start, reports an error or exception in the log, and stops running:
的connector’s configuration is invalid.
的connector cannot successfully connect to the MySQL server by using the specified connection parameters.
的connector is attempting to restart at a position in the binlog for which MySQL no longer has the history available.
In these cases, the error message has details about the problem and possibly a suggested workaround. After you correct the configuration or address the MySQL problem, restart the connector.
MySQL becomes unavailable
If your MySQL server becomes unavailable, the Debezium MySQL connector fails with an error and the connector stops. When the server is available again, restart the connector.
However, if GTIDs are enabled for a highly available MySQL cluster, you can restart the connector immediately. It will connect to a different MySQL server in the cluster, find the location in the server’s binlog that represents the last transaction, and begin reading the new server’s binlog from that specific location.
If GTIDs are not enabled, the connector records the binlog position of only the MySQL server to which it was connected. To restart from the correct binlog position, you must reconnect to that specific server.
Kafka Connect stops gracefully
When Kafka Connect stops gracefully, there is a short delay while the Debezium MySQL connector tasks are stopped and restarted on new Kafka Connect processes.
Kafka Connect process crashes
If Kafka Connect crashes, the process stops and any Debezium MySQL connector tasks terminate without their most recently-processed offsets being recorded. In distributed mode, Kafka Connect restarts the connector tasks on other processes. However, the MySQL connector resumes from the last offset recorded by the earlier processes. This means that the replacement tasks might generate some of the same events processed prior to the crash, creating duplicate events.
Each change event message includes source-specific information that you can use to identify duplicate events, for example:
Event origin
MySQL server’s event time
的binlog file name and position
GTIDs (if used)
Kafka becomes unavailable
的Kafka Connect framework records Debezium change events in Kafka by using the Kafka producer API. If the Kafka brokers become unavailable, the Debezium MySQL connector pauses until the connection is reestablished and the connector resumes where it left off.
MySQL purges binlog files
If the Debezium MySQL connector stops for too long, the MySQL server purges older binlog files and the connector’s last position may be lost. When the connector is restarted, the MySQL server no longer has the starting point and the connector performs another initial snapshot. If the snapshot is disabled, the connector fails with an error.
Seesnapshotsfor details about how MySQL connectors perform initial snapshots.