Debezium connector for Vitess
- Overview
- How the connector works
- Data change events
- Data type mappings
- Seting up Vitess
- Deployment
- Behavior when things go wrong
- Configuration and startup errors
- Vitess becomes unavailable
- Invalid column name error
- Kafka Connect process stops gracefully
- Kafka Connect process crashes
- Kafka becomes unavailable
- Connector is stopped for a duration
- Connector fails before finishing the snapshot
- Limitations with earlier Vitess versions
Debezium’s Vitess connector captures row-level changes in the shards of a Vitess keyspace. For information about the Vitess versions that are compatible with this connector, see the Debezium release overview.
When the connector first connects to a Vitess cluster, it takes a consistent snapshot of the keyspace. After that snapshot is complete, the connector continuously captures row-level changes that are committed to a Vitess keyspace to insert, update, or delete database content. The connector generates data change event records and streams them to Kafka topics. For each table, the default behavior is that the connector streams all generated events to a separate Kafka topic for that table. Applications and services can then consume data change event records from the resulting topic.
Overview
Vitess’s VStream feature was introduced in version 4.0. It is a change event subscription service that provides equivalent information to the MySQL binary logs from the underlying MySQL shards of the Vitess cluster. An user can subscribe to multiple shards in a keyspace, making it a convenient tool to feed downstream CDC processes.
To read and process database changes, the Vitess connector subscribes to VTGate's VStream gRPC service. VTGate is a lightweight, stateless gRPC server, which is part of the Vitess cluster setup.
The connector gives you the flexibility to choose to subscribe to the MASTER
nodes, or to the REPLICA
nodes for change events.
The connector produces a change event for every row-level insert, update, and delete operation that was captured and sends change event records for each table in a separate Kafka topic. Client applications read the Kafka topics that correspond to the database tables of interest, and can react to every row-level event they receive from those topics.
The underlying MySQL implementation in Vitess purges binary logs based on some configurable period of time. Because the contents of the binlog might not be complete, the connector requires another mechanism to ensure that it captures the complete content of a particular database. Thus, when the connector connects to a database for the first time, it performs a consistent snapshot of the database. After the connector completes the snapshot, it continues streaming changes from the exact point at which the snapshot was made. In this way, the connector starts with a consistent view of all of the data, and does not omit any changes that were made while the snapshot was being taken.
The connector is tolerant of failures. As the connector reads changes and produces events, it records the VGTID position for each event. If the connector stops for any reason (including communication failures, network problems, or crashes), after the connectors restarts, it continues reading the WAL from the point at which it stopped. This behavior does not apply to snapshots. If the connector stops during a snapshot, after a restart, the connector does not continue performing snapshots where it last left off. We’ll talk later about how the connector behaves when things go wrong.
How the connector works
To optimally configure and run a Debezium Vitess connector, it is helpful to understand how the connector performs snapshots, streams change events, determines Kafka topic names, and uses metadata.
Snapshot
Typically, a MySQL server is not configured to retain the complete history of the database in the binary logs.
As a result, the connector is unable to read the entire history of the database from the binary logs.
For this reason, the first time that the connector starts, it performs an initial consistent snapshot of the database.
You can change this behavior by setting the snapshot.mode
connector configuration property to a value other than initial
.
This snapshot feature is built on VStream Copy introduced in version 7.0.
Automatic retry of a failed snapshot is expected to be available in a future release. |
Streaming changes
The Vitess connector spends all its time streaming changes from the VTGate’s VStream gRPC service to which it is subscribed. The client receives changes from VStream as they are committed in the underlying MySQL server’s binlog at certain positions, which are referred to as VGTID.
The VGTID in Vitess is the equivalent of GTID in MySQL, it describes the position in the VStream in which a change event happens. Typically, A VGTID has multiple shard GTIDs, each shard GTID is a tuple of (Keyspace, Shard, GTID)
, which describes the GTID position of a given shard.
When subscribing to a VStream service, the connector needs to provide a VGTID and a Tablet Type (e.g. MASTER
, REPLICA
). The VGTID describes the position from which VStream should starts sending change events; the Tablet type describes which underlying MySQL instance (master or replica) in each shard do we read change events from.
The first time the connector connects to a Vitess cluster, it gets the current VGTID from a Vitess component called VTCtld and provides the current VGTID to VStream.
The Debezium Vitess connector acts as a gRPC client of VStream. When the connector receives changes it transforms the events into Debezium create, update, or delete events that include the VGTID of the event. The Vitess connector forwards these change events in records to the Kafka Connect framework, which is running in the same process. The Kafka Connect process asynchronously writes the change event records in the same order in which they were generated to the appropriate Kafka topic.
Periodically, Kafka Connect records the most recent offset in another Kafka topic. The offset indicates source-specific position information that Debezium includes with each event. For the Vitess connector, the VGTID recorded in each change event is the offset.
When Kafka Connect gracefully shuts down, it stops the connectors, flushes all event records to Kafka, and records the last offset received from each connector. When Kafka Connect restarts, it reads the last recorded offset for each connector, and starts each connector at its last recorded offset. When the connector restarts, it sends a request to VStream to send the events starting just after that position.
Topics names
The Vitess connector writes events for all insert, update, and delete operations on a single table to a single Kafka topic. By default, the Kafka topic name is topicPrefix.keyspaceName.tableName where:
-
topicPrefix is the topic prefix as specified by the
topic.prefix
connector configuration property. -
keyspaceName is the name of the keyspace (a.k.a. database) where the operation occurred.
-
tableName is the name of the database table in which the operation occurred.
For example, suppose that fulfillment
is the logical server name in the configuration for a connector that is capturing changes in a Vitess installation that has an commerce
keyspace that contains four tables: products
, products_on_hand
, customers
, and orders
. Regardless of how many shards the keyspace has, the connector would stream records to these four Kafka topics:
-
fulfillment.commerce.products
-
fulfillment.commerce.products_on_hand
-
fulfillment.commerce.customers
-
fulfillment.commerce.orders
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 the BEGIN
and END
delimiters in every transaction.
Transaction boundary events contain the following fields:
status
-
BEGIN
orEND
. id
-
String representation of the unique transaction identifier.
ts_ms
-
The time of a transaction 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": "[{\"keyspace\":\"test_unsharded_keyspace\",\"shard\":\"0\",\"gtid\":\"MySQL56/e03ece6c-4c04-11ec-8e20-0242ac110004:1-37\"}]",
"ts_ms": 1486500577125,
"event_count": null,
"data_collections": null
}
{
"status": "END",
"id": "[{\"keyspace\":\"test_unsharded_keyspace\",\"shard\":\"0\",\"gtid\":\"MySQL56/e03ece6c-4c04-11ec-8e20-0242ac110004:1-37\"}]",
"ts_ms": 1486500577691,
"event_count": 1,
"data_collections": [
{
"data_collection": "test_unsharded_keyspace.my_seq",
"event_count": 1
}
]
}
Unless overridden via the topic.transaction
option,
the connector emits transaction events to the <topic.prefix>
.transaction
topic.
When transaction metadata is enabled the data message Envelope
is enriched with a new transaction
field.
This field provides information about every event in the form of a composite of fields:
-
id
- string representation of unique transaction identifier -
total_order
- absolute position of the event among all events generated by the transaction -
data_collection_order
- the per-data collection position of the event among all events that were emitted by the transaction
Following is an example of a message:
{
"before": null,
"after": {
"pk": "2",
"aa": "1"
},
"source": {
...
},
"op": "c",
"ts_ms": 1637988245467,
"transaction": {
"id": "[{\"keyspace\":\"test_unsharded_keyspace\",\"shard\":\"0\",\"gtid\":\"MySQL56/e03ece6c-4c04-11ec-8e20-0242ac110004:1-68\"}]",
"total_order": 1,
"data_collection_order": 1
}
}
Data change events
The Debezium Vitess connector generates a data change event for each row-level INSERT
, UPDATE
, and DELETE
operation. Each event contains a key and a value. The structure of the key and the value depends on the table that was changed.
Debezium and Kafka Connect are designed around continuous streams of event messages. However, the structure of these events may change over time, which can be difficult for consumers to handle. To address this, each event contains the schema for its content or, if you are using a schema registry, a schema ID that a consumer can use to obtain the schema from the registry. This makes each event self-contained.
The following skeleton JSON shows the basic four parts of a change event. However, how you configure the Kafka Connect converter that you choose to use in your application determines the representation of these four parts in change events. A schema
field is in a change event only when you configure the converter to produce it. Likewise, the event key and event payload are in a change event only if you configure a converter to produce it. If you use the JSON converter and you configure it to produce all four basic change event parts, change events have this structure:
{
"schema": { (1)
...
},
"payload": { (2)
...
},
"schema": { (3)
...
},
"payload": { (4)
...
},
}
Item | Field name | Description |
---|---|---|
1 |
|
The first |
2 |
|
The first |
3 |
|
The second |
4 |
|
The second |
By default behavior is that the connector streams change event records to topics with names that are the same as the event’s originating table.
Starting with Kafka 0.10, Kafka can optionally record the event key and value with the timestamp at which the message was created (recorded by the producer) or written to the log by Kafka. |
The Vitess connector ensures that all Kafka Connect schema names adhere to the Avro schema name format. This means that the logical server name must start with a Latin letter or an underscore, that is, a-z, A-Z, or _. Each remaining character in the logical server name and each character in the schema 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 schema 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. |
The connector doesn’t allow to name columns with the |
Change event keys
For a given table, the change event’s key has a structure that contains a field for each column in the primary key of the table at the time the event was created.
Consider a customers
table defined in the commerce
keyspace and the example of a change event key for that table.
CREATE TABLE customers (
id INT NOT NULL,
first_name VARCHAR(255) NOT NULL,
last_name VARCHAR(255) NOT NULL,
email VARCHAR(255) NOT NULL,
PRIMARY KEY(id)
);
If the topic.prefix
connector configuration property has the value Vitess_server
, every change event for the customers
table while it has this definition has the same key structure, which in JSON looks like this:
{
"schema": { (1)
"type": "struct",
"name": "Vitess_server.commerce.customers.Key", (2)
"optional": false, (3)
"fields": [ (4)
{
"name": "id",
"index": "0",
"schema": {
"type": "INT32",
"optional": "false"
}
}
]
},
"payload": { (5)
"id": "1"
},
}
Item | Field name | Description |
---|---|---|
1 |
|
The schema portion of the key specifies a Kafka Connect schema that describes what is in the key’s |
2 |
|
Name of the schema that defines the structure of the key’s payload. This schema describes the structure of the primary key for the table that was changed. Key schema names have the format connector-name.keyspace-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 |
Although the |
If the table does not have a primary, then the change event’s key is null. The rows in a table without a primary key constraint cannot be uniquely identified. |
Change event values
The value in a change event is a bit more complicated than the key. Like the key, the value has a schema
section and a payload
section. The schema
section contains the schema that describes the Envelope
structure of the payload
section, including its nested fields. Change events for operations that create, update or delete data all have a value payload with an envelope structure.
Consider the same sample table that was used to show an example of a change event key:
CREATE TABLE customers (
id INT NOT NULL,
first_name VARCHAR(255) NOT NULL,
last_name VARCHAR(255) NOT NULL,
email VARCHAR(255) NOT NULL,
PRIMARY KEY(id)
);
The emitted events for UPDATE
and DELETE
operations contain the previous values of all columns in the table.
create events
The following example shows the value portion of a change event that the connector generates for an operation that creates data in the customers
table:
{
"schema": { (1)
"type": "struct",
"fields": [
{
"type": "struct",
"fields": [
{
"type": "int32",
"optional": false,
"field": "id"
},
{
"type": "string",
"optional": false,
"field": "first_name"
},
{
"type": "string",
"optional": false,
"field": "last_name"
},
{
"type": "string",
"optional": false,
"field": "email"
}
],
"optional": true,
"name": "Vitess_server.commerce.customers.Value", (2)
"field": "before"
},
{
"type": "struct",
"fields": [
{
"type": "int32",
"optional": false,
"field": "id"
},
{
"type": "string",
"optional": false,
"field": "first_name"
},
{
"type": "string",
"optional": false,
"field": "last_name"
},
{
"type": "string",
"optional": false,
"field": "email"
}
],
"optional": true,
"name": "Vitess_server.commerce.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": false,
"field": "schema"
},
{
"type": "string",
"optional": false,
"field": "table"
},
{
"type": "int64",
"optional": true,
"field": "vgtid"
}
],
"optional": false,
"name": "io.debezium.connector.vitess.Source", (3)
"field": "source"
},
{
"type": "string",
"optional": false,
"field": "op"
},
{
"type": "int64",
"optional": true,
"field": "ts_ms"
}
],
"optional": false,
"name": "Vitess_server.commerce.customers.Envelope" (4)
},
"payload": { (5)
"before": null, (6)
"after": { (7)
"id": 1,
"first_name": "Anne",
"last_name": "Kretchmar",
"email": "annek@noanswer.org"
},
"source": { (8)
"version": "2.1.4.Final",
"connector": "vitess",
"name": "my_sharded_connector",
"ts_ms": 1559033904863,
"snapshot": false,
"db": "",
"keyspace": "commerce",
"table": "customers",
"vgtid": "[{\"keyspace\":\"commerce\",\"shard\":\"80-\",\"gtid\":\"MariaDB/0-54610504-47\"},{\"keyspace\":\"commerce\",\"shard\":\"-80\",\"gtid\":\"MariaDB/0-1592148-45\"}]"
},
"op": "c", (9)
"ts_ms": 1559033904863 (10)
}
}
Item | Field name | Description |
---|---|---|
1 |
|
The value’s schema, which describes the structure of the value’s payload. A change event’s value schema is the same in every change event that the connector generates for a particular table. |
2 |
|
In the |
3 |
|
|
4 |
|
|
5 |
|
The value’s actual data. This is the information that the change event is providing. |
6 |
|
An optional field that specifies the state of the row before the event occurred. When the |
7 |
|
An optional field that specifies the state of the row after the event occurred. In this example, the |
8 |
|
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:
|
9 |
|
Mandatory string that describes the type of operation that caused the connector to generate the event. In this example,
|
10 |
|
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. |
update events
The value of a change event for an update in the sample customers
table has the same schema as a create event for that table. Likewise, the event value’s payload has the same structure. However, the event value payload contains different values in an update event. Here is an example of a change event value in an event that the connector generates for an update in the customers
table:
{
"schema": { ... },
"payload": {
"before": { (1)
"id": 1,
"first_name": "Anne",
"last_name": "Kretchmar",
"email": "annek@noanswer.org"
},
"after": { (2)
"id": 1,
"first_name": "Anne Marie",
"last_name": "Kretchmar",
"email": "annek@noanswer.org"
},
"source": { (3)
"version": "2.1.4.Final",
"connector": "vitess",
"name": "my_sharded_connector",
"ts_ms": 1559033904863,
"snapshot": false,
"db": "",
"keyspace": "commerce",
"table": "customers",
"vgtid": "[{\"keyspace\":\"commerce\",\"shard\":\"80-\",\"gtid\":\"MariaDB/0-54610504-47\"},{\"keyspace\":\"commerce\",\"shard\":\"-80\",\"gtid\":\"MariaDB/0-1592148-46\"}]"
},
"op": "u", (4)
"ts_ms": 1465584025523 (5)
}
}
Item | Field name | Description |
---|---|---|
1 |
|
An optional field that contains all values of all columns that were in the row before the database commit. |
2 |
|
An optional field that specifies the state of the row after the event occurred. In this example, the |
3 |
|
Mandatory field that describes the source metadata for the event. The
|
4 |
|
Mandatory string that describes the type of operation. In an update event 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 key changes the value of the row’s key. When a key changes, Debezium outputs three events: a |
delete events
The value in a delete change event has the same schema
portion as create and update events for the same table. The payload
portion in a delete event for the sample customers
table looks like this:
{
"schema": { ... },
"payload": {
"before": { (1)
"id": 1,
"first_name": "Anne Marie",
"last_name": "Kretchmar",
"email": "annek@noanswer.org"
},
"after": null, (2)
"source": { (3)
"version": "2.1.4.Final",
"connector": "vitess",
"name": "my_sharded_connector",
"ts_ms": 1559033904863,
"snapshot": false,
"db": "",
"keyspace": "commerce",
"table": "customers",
"vgtid": "[{\"keyspace\":\"commerce\",\"shard\":\"80-\",\"gtid\":\"MariaDB/0-54610504-47\"},{\"keyspace\":\"commerce\",\"shard\":\"-80\",\"gtid\":\"MariaDB/0-1592148-47\"}]"
},
"op": "d", (4)
"ts_ms": 1465581902461 (5)
}
}
Item | Field name | Description |
---|---|---|
1 |
|
Optional field that specifies the state of the row before the event occurred. In a delete event value, the |
2 |
|
Optional field that specifies the state of the row after the event occurred. In a delete event value, the |
3 |
|
Mandatory field that describes the source metadata for the event. In a delete event value, the
|
4 |
|
Mandatory string that describes the type of operation. The |
5 |
|
Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
A delete change event record provides a consumer with the information it needs to process the removal of this row.
Vitess connector events are designed to work with Kafka log compaction. Log compaction enables removal of some older messages as long as at least the most recent message for every key is kept. This lets Kafka reclaim storage space while ensuring that the topic contains a complete data set and can be used for reloading key-based state.
When a row is deleted, the delete event value still works with log compaction, because Kafka can remove all earlier messages that have that same key. However, for Kafka to remove all messages that have that same key, the message value must be null
. To make this possible, the Vitess connector follows a delete event with a special tombstone event that has the same key but a null
value.
Data type mappings
The Vitess 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. How that value is represented in the event depends on the Vitess data type of the column. This section describes these mappings.
If the default data type conversions do not meet your needs, you can create a custom converter for the connector.
Basic types
The following table describes how the connector maps basic Vitess data types to a literal type and a semantic type in event fields.
-
literal type describes how the value is literally represented using Kafka Connect schema types:
INT8
,INT16
,INT32
,INT64
,FLOAT32
,FLOAT64
,BOOLEAN
,STRING
,BYTES
,ARRAY
,MAP
, andSTRUCT
. -
semantic type describes how the Kafka Connect schema captures the meaning of the field using the name of the Kafka Connect schema for the field.
Vitess data type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
|
n/a |
|
Unsupported yet |
n/a |
|
Unsupported yet |
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 |
|
|
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 |
|
Unsupported yet |
n/a |
Seting up Vitess
Debezium does not require any specific configuration for use with Vitess. Install Vitess according to the standard instructions in the Local Install via Docker guide, or the Vitess Operator for Kubernetes guide.
-
Make sure that the VTGate host and its gRPC port (default is 15991) is accessible from the machine where the Vitess connector is installed
-
Make sure that the VTCtld host and its gRPC port (default is 15999) is accessible from the machine where the Vitess connector is installed
gRPC authentication
Because Vitess connector reads change events from the VTGate VStream gRPC server, it does not need to connect directly to MySQL instances. Therefore, no special database user and permissions are needed. At the moment, Vitess connector only supports unauthenticated access to the VTGate gRPC server.
Deployment
With Zookeeper, Kafka, and Kafka Connect installed, the remaining tasks to deploy a Debezium Vitess connector are to download the connector’s plug-in archive, extract the JAR files into your Kafka Connect environment, and add the directory with the JAR files to Kafka Connect’s plugin.path
. You then need to restart your Kafka Connect process to pick up the new JAR files.
If you are working with immutable containers, see Debezium’s Container images for Zookeeper, Kafka and Kafka Connect with the Vitess connector already installed and ready to run. You can also run Debezium on Kubernetes and OpenShift.
Connector configuration example
Following is an example of the configuration for a Vitess connector that connects to a Vitess (VTGate’s VStream) server on port 15991 at 192.168.99.100, whose logical name is fullfillment
. It also connects to a VTCtld server on port 15999 at 192.168.99.101 to get the initial VGTID. Typically, you configure the Debezium Vitess connector in a .json
file using the configuration properties available for the connector.
You can choose to produce events for a subset of the schemas and tables. Optionally, ignore, mask, or truncate columns that are sensitive, too large, or not needed.
{
"name": "inventory-connector", (1)
"config": {
"connector.class": "io.debezium.connector.vitess.VitessConnector", (2)
"database.hostname": "192.168.99.100", (3)
"database.port": "15991", (4)
"database.user": "vitess", (5)
"database.password": "vitess_password", (6)
"vitess.keyspace": "commerce", (7)
"vitess.tablet.type": "MASTER", (8)
"vitess.vtctld.host": "192.168.99.101", (9)
"vitess.vtctld.port": "15999", (10)
"vitess.vtctld.user": "vitess", (11)
"vitess.vtctld.password": "vitess_password", (12)
"topic.prefix": "fullfillment", (13)
"tasks.max": 1 (14)
}
}
1 | The name of the connector when registered with a Kafka Connect service. |
2 | The name of this Vitess connector class. |
3 | The address of the Vitess (VTGate’s VStream) server. |
4 | The port number of the Vitess (VTGate’s VStream) server. |
5 | The username of the Vitess database server (VTGate gRPC). |
6 | The password of the Vitess database server (VTGate gRPC). |
7 | The name of the keyspce (a.k.a database). Because no shard is specified, it reads change events from all shards in the keyspace. |
8 | The type of MySQL instance (MASTER OR REPLICA) to read change events from. |
9 | The address of the VTCtld server. |
10 | The port of the VTCtld server. |
11 | The username of the VTCtld server (VTCtld gRPC). |
12 | The password of the VTCtld database server (VTCtld gRPC). |
13 | The topic prefix for the Vitess cluster, which forms a namespace and is used in all the names of the Kafka topics to which the connector writes, the Kafka Connect schema names, and the namespaces of the corresponding Avro schema when the Avro converter is used. |
14 | Only one task should operate at any one time. |
See the complete list of Vitess connector properties that can be specified in these configurations.
You can send this configuration with a POST
command to a running Kafka Connect service. The service records the configuration and starts the connector task that connects to the Vitess database and streams change event records to Kafka topics.
Connector configuration example for offset-storage-per-task mode
When you have a big Vitess installation which requires more than one connector task to process the change logs, you can use offset-storage-per-task feature to launch multiple connector tasks and have each task work with a subset of vitess shards. Each task will persist its offsets (the vgtids for the shards it’s tracking) in Kafka’s offset topic in its own partition space.
Following is the same example for a Vitess connector that connects to a Vitess (VTGate’s VStream) server but with three additional parameteres to invoke the offset-storage-per-task mode.
{
"name": "inventory-connector",
"config": {
"connector.class": "io.debezium.connector.vitess.VitessConnector",
"database.hostname": "192.168.99.100",
"database.port": "15991",
"database.user": "vitess",
"database.password": "vitess_password",
"vitess.keyspace": "commerce",
"vitess.tablet.type": "MASTER",
"vitess.vtctld.host": "192.168.99.101",
"vitess.vtctld.port": "15999",
"vitess.vtctld.user": "vitess",
"vitess.vtctld.password": "vitess_password",
"database.server.name": "fullfillment",
"vitess.offset.storage.per.task": true, (1)
"vitess.offset.storage.task.key.gen": 1, (2)
"vitess.prev.num.tasks": 1, (3)
"tasks.max": 2 (4)
}
}
1 | Specify that we want to turn on offset-storage-per-task feature |
2 | Specify that the generation number for the current task parallelism is 1 |
3 | Specify that the number of tasks in the previous generation of task parallelism is 1 |
4 | Specify that we want to launch two tasks for the current task parallelism |
The task to vitess shards distribution is based on a simple round robin algorithm. In this example of launching two connector tasks and assume we have 4 vitess shards (-40,40-80,80-c0,c0-), task0 will be working on shards (-40,80-c0) and task1 will be working on shards (40-80,c0-).
The reason that we need three config params is to make sure the offsets saved by each connector task don’t collide with each other and to handle the migration of offsets by the previous task paralleism automatically. In order to make sure that we don’t collide on the partition keys in Kafka offset topic, we are using this partition name scheme for each connector task: taskId_numTasks_gen. So for the current example of launching two tasks with generation number 1, task0 will be writing its offsets in Kafka’s offset topic in partition key: task0_2_1 and task1 will be using partition key: (task1_2_1). The gen config param is used to distinguish the partition keys generated from different generations (generation corresponds to each change of task parallelism)
When the task paralleism changes (e.g. you want to launch 4 connector tasks instead of 2 to handle the bigger volume of traffic from vitess), you will specify tasks.max=4, vitess.offset.storage.task.key.gen=2, vitess.prev.num.tasks=2, the offset partition for this task paralleism generation will be: task0_4_2, task1_4_2, task2_4_2, task3_4_2. Once the connector restarts, the connector will detect there is no previous offsets saved for the current 4 partition keys and it will invoke an automatic offset migration from the offsets saved in the previous generation keys: task0_2_1 and task1_2_1. For the current example of 4 vitess shards (-40,40-80,80-c0,c0-), task0 will be working on shard:(-40), task1:(40-80), task2:(80-c0), task3:(c0-). The offsets for those 4 shards from the previous generation of parallelism (using 2 tasks with each task working with 2 shards) will be auto-migrated to this generation of using 4 tasks (one task working with one shard each).
Note that the task parallelism gen number is defaulted to be 0 for the offsets saved in Kafka offset topic before offset-storage-per-task feature is enabled, there is a special offset lookup during offset migration. So if you have the vitess connector running for a while without the offset-storage-per-task feature on and now you want to turn on this feature, please specify vitess.offset.storage.task.key.gen=1, vitess.prev.num.tasks=1 to help the offset auto migration.
Note that vitess.prev.num.tasks needs to match the actual number of tasks launched in the previous task parallelism generation. The number of connector tasks is usually the same as the tasks.max config params you specified, but in the rare case that tasks.max > number of vitess shards, the connector will only launch the_number_of_tasks = the_number_of_vitess_shards. This rare case is probably a mis-configuration in the first place.
See the complete list of Vitess connector properties that can be specified in these configurations.
You can send this configuration with a POST
command to a running Kafka Connect service. The service records the configuration and starts the connector task that connects to the Vitess database and streams change event records to Kafka topics.
Adding connector configuration
To start running a Vitess connector, create a connector configuration and add the configuration to your Kafka Connect cluster.
-
The VTGate host and its gRPC port (default is 15991) is accessible from the machine where the Vitess connector is installed
-
The VTCtld host and its gRPC port (default is 15999) is accessible from the machine where the Vitess connector is installed
-
The Vitess connector is installed.
-
Create a configuration for the Vitess connector.
-
Use the Kafka Connect REST API to add that connector configuration to your Kafka Connect cluster.
When the connector starts, it starts generating data change events for row-level operations and streaming change event records to Kafka topics.
Monitoring
The Debezium Vitess connector provides only one type of metrics that are in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect provide.
-
Streaming metrics provide information about connector operation when the connector is capturing changes and streaming change event records.
Debezium monitoring documentation provides details for how to expose these metrics by using JMX.
Streaming metrics
The MBean is debezium.vitess:type=connector-metrics,context=streaming,server=<vitess.server.name>
.
Attributes | Type | Description |
---|---|---|
|
The number of milliseconds since the connector has read and processed the most recent event. |
|
|
The total number of events that this connector has seen since last started or reset. |
|
|
The number of events that have been filtered by include/exclude list filtering rules configured on the connector. |
|
|
The length the queue used to pass events between the streamer and the main Kafka Connect loop. |
|
|
The free capacity of the queue used to pass events between the streamer and the main Kafka Connect loop. |
|
|
Flag that denotes whether the connector is currently connected to the database server. |
|
|
The number of milliseconds between the last change event’s timestamp and the connector processing it. The values will incoporate any differences between the clocks on the machines where the database server and the connector are running. |
|
|
The number of processed transactions that were committed. |
|
|
The maximum buffer of the queue in bytes used to pass events between the streamer and the main Kafka Connect loop. |
|
|
The current buffer of the queue in bytes used to pass events between the streamer and the main Kafka Connect loop. |
Connector configuration properties
The Debezium Vitess connector has many configuration properties that you can use to achieve the right connector behavior for your application. Many properties have default values. Information about the properties is organized as follows:
The following configuration properties are required unless a default value is available.
Property | Default | Description | ||
---|---|---|---|---|
No default |
Unique name for the connector. Attempting to register again with the same name will fail. This property is required by all Kafka Connect connectors. |
|||
No default |
The name of the Java class for the connector. Always use a value of |
|||
|
The maximum number of tasks that should be created for this connector. The Vitess connector can use more than 1 tasks if you enable offset.storage.per.task mode. |
|||
No default |
IP address or hostname of the Vitess database server (VTGate). |
|||
|
Integer port number of the Vitess database server (VTGate). |
|||
The name of the keyspace from which to stream the changes. |
||||
n/a |
An optional name of the shard from which to stream the changes. If not configured, in case of unsharded keyspace, the connector streams changes from the only shard, in case of sharded keyspace, the connector streams changes from all shards in the keyspace. We recommend not configuring it in order to stream from all shards in the keyspace because it has better support for reshard operation. If configured, for example, |
|||
|
An optional GTID position for a shard to stream from. This has to be set together with |
|||
|
Controls Vitess flag stop_on_reshard. |
|||
n/a |
An optional username of the Vitess database server (VTGate). If not configured, unauthenticated VTGate gRPC is used. |
|||
n/a |
An optional password of the Vitess database server (VTGate). If not configured, unauthenticated VTGate gRPC is used. |
|||
|
The type of Tablet (hence MySQL) from which to stream the changes: |
|||
No default |
Topic prefix that provides a namespace for the particular Vitess database server or cluster in which Debezium is capturing changes. Only alphanumeric characters, hyphens, dots and underscores must be used in the database server logical name. The prefix should be unique across all other connectors, since it is used as a topic name prefix for all Kafka topics that receive records from this connector. +
|
|||
No default |
An optional, comma-separated list of regular expressions that match fully-qualified table identifiers for tables whose changes you want to capture. Any table not included in |
|||
No default |
An optional, comma-separated list of regular expressions that match fully-qualified table identifiers for tables whose changes you do not want to capture. Any table not included in |
|||
No default |
An optional, comma-separated list of regular expressions that match the fully-qualified names of columns that should be included in change event record values. Fully-qualified names for columns are of the form keyspace.tableName.columnName. Do not also set the |
|||
No default |
An optional, comma-separated list of regular expressions that match the fully-qualified names of columns that should be excluded from change event record values. Fully-qualified names for columns are of the form keyspace.tableName.columnName. Do not also set the |
|||
|
Controls whether a delete event is followed by a tombstone event. |
|||
|
Specify whether to turn on offset-storage-per-task mode launch multiple connector tasks and persist offsets partitioned by task. |
|||
|
Specify the task paralleism generation number when vitess.offset.storage.per.task is turned on. You should increase the generation number when you decide to change the connector task parallelism (either launch more connector tasks or less) |
|||
|
Specify the number of connector tasks used in the previous generation of task paralleism when vitess.offset.storage.per.task is turned on. |
|||
empty string |
A semicolon separated list of tables with regular expressions that match table column names. The connector maps values in matching columns to key fields in change event records that it sends to Kafka topics. This is useful when a table does not have a primary key, or when you want to order change event records in a Kafka topic according to a field that is not a primary key. |
|||
none |
Specifies how schema names should be adjusted for compatibility with the message converter used by the connector. Possible settings:
|
|||
initial |
Specifies the criteria for performing a snapshot when the connector starts. Set the property to one of the following values:
|
The following advanced configuration properties have defaults that work in most situations and therefore rarely need to be specified in the connector’s configuration.
Property | Default | Description |
---|---|---|
No default |
Enumerates a comma-separated list of the symbolic names of the custom converter instances that the connector can use.
For example,
You must set the For each converter that you configure for a connector, you must also add a
For example, isbn.type: io.debezium.test.IsbnConverter 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 any additional configuration parameter with a converter, prefix the parameter names with the symbolic name of the converter.
For example, isbn.schema.name: io.debezium.vitess.type.Isbn |
|
|
Specifies how the connector should react to exceptions during processing of events: |
|
|
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 |
|
|
Positive integer value that specifies the maximum size of each batch of events that the connector processes. |
|
|
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. |
|
|
Indicates whether field names are sanitized to adhere to Avro naming requirements. |
|
|
A comma-separated list of operation types that will be skipped during streaming.
The operations include: |
|
|
Determines whether the connector generates events with transaction boundaries and enriches change event envelopes with transaction metadata. Specify |
|
|
Control the interval between periodic gPRC keepalive pings for VStream. Defaults to |
|
No default |
Specify a comma-separated list of gRPC headers. Defaults to empty. The format is: |
|
No default |
Specify the maximum message size in bytes allowed to be received on the channel. |
|
n/a |
An optional, comma-separated list of regular expressions that match the fully-qualified names of columns whose original type and length should be added as a parameter to the corresponding field schemas in the emitted change event records. These schema parameters:
are used to propagate the original type name and length for variable-width types, respectively. This is useful to properly size corresponding columns in sink databases. Fully-qualified names for columns are of the following form: keyspaceName.tableName.columnName |
|
n/a |
An optional, comma-separated list of regular expressions that match the database-specific data type name of columns whose original type and length should be added as a parameter to the corresponding field schemas in the emitted change event records. These schema parameters:
are used to propagate the original type name and length for variable-width types, respectively. This is useful to properly size corresponding columns in sink databases. Fully-qualified names for columns are of the following form: keyspaceName.tableName.columnName See how Vitess connectors map data types for the list of Vitess-specific data type names. |
|
|
The name of the 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 |
|
|
The 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 transaction metadata messages. The topic name has this pattern: |
The connector also supports pass-through configuration properties that are used when creating the Kafka producer and consumer.
Be sure to consult the Kafka documentation for all of the configuration properties for Kafka producers and consumers. The Vitess connector does use the new consumer configuration properties.
Behavior when things go wrong
Debezium is a distributed system that captures all changes in multiple upstream databases; it never misses or loses an event. When the system is operating normally or being managed carefully then Debezium provides exactly once delivery of every change event record.
If a fault does happen then the system does not lose any events. However, while it is recovering from the fault, it might repeat some change events. In these abnormal situations, Debezium, like Kafka, provides at least once delivery of change events.
The rest of this section describes how Debezium handles various kinds of faults and problems.
Configuration and startup errors
In the following situations, the connector fails when trying to start, reports an error/exception in the log, and stops running:
-
The connector’s configuration is invalid.
-
The connector cannot successfully connect to Vitess by using the specified connection parameters.
In these cases, the error message has details about the problem and possibly a suggested workaround. After you correct the configuration or address the Vitess problem, restart the connector.
Vitess becomes unavailable
When the connector is running, the Vitses server (VTGate) that it is connected to could become unavailable for any number of reasons. If this happens, the connector fails with an error and stops. When the server is available again, restart the connector.
The Vitess connector externally stores the last processed offset in the form of a Vitess VGTID. After a connector restarts and connects to a server instance, the connector communicates with the server to continue streaming from that particular offset.
Invalid column name error
This error happens very rarely. If you receive an error with the message Illegal prefix '@' for column: x, from schema: y, table: z
, and your table doesn’t have such a column, it is a Vitess vstream bug that is caused by column renaming or column type change. It is a transient error. You can restart the connector after a small backoff and it should resolve automatically.
Kafka Connect process stops gracefully
Suppose that Kafka Connect is being run in distributed mode and a Kafka Connect process is stopped gracefully. Prior to shutting down that process, Kafka Connect migrates the process’s connector tasks to another Kafka Connect process in that group. The new connector tasks start processing exactly where the prior tasks stopped. There is a short delay in processing while the connector tasks are stopped gracefully and restarted on the new processes.
Kafka Connect process crashes
If the Kafka Connector process stops unexpectedly, any connector tasks it was running terminate without recording their most recently processed offsets. When Kafka Connect is being run in distributed mode, Kafka Connect restarts those connector tasks on other processes. However, Vitess connectors resume from the last offset that was recorded by the earlier processes. This means that the new replacement tasks might generate some of the same change events that were processed just prior to the crash. The number of duplicate events depends on the offset flush period and the volume of data changes just before the crash.
Because there is a chance that some events might be duplicated during a recovery from failure, consumers should always anticipate some duplicate events. Debezium changes are idempotent, so a sequence of events always results in the same state.
In each change event record, Debezium connectors insert source-specific information about the origin of the event, including the Vitess server’s time of the event, the position in the binlog where the transaction changes were written. Consumers can keep track of this information, especially the VGTID, to determine whether an event is a duplicate.
Kafka becomes unavailable
As the connector generates change events, the Kafka Connect framework records those events in Kafka by using the Kafka producer API. Periodically, at a frequency that you specify in the Kafka Connect configuration, Kafka Connect records the latest offset that appears in those change events. If the Kafka brokers become unavailable, the Kafka Connect process that is running the connectors repeatedly tries to reconnect to the Kafka brokers. In other words, the connector tasks pause until a connection can be re-established, at which point the connectors resume exactly where they left off.
Connector is stopped for a duration
If the connector is gracefully stopped, the database can continue to be used. Any changes are recorded in the Vitess binlog. When the connector restarts, it resumes streaming changes where it left off. That is, it generates change event records for all database changes that were made while the connector was stopped.
A properly configured Kafka cluster is able to handle massive throughput. Kafka Connect is written according to Kafka best practices, and given enough resources a Kafka Connect connector can also handle very large numbers of database change events. Because of this, after being stopped for a while, when a Debezium connector restarts, it is very likely to catch up with the database changes that were made while it was stopped. How quickly this happens depends on the capabilities and performance of Kafka and the volume of changes being made to the data in Vitess.
Connector fails before finishing the snapshot
If a snapshot fails to complete, the connector does not automatically reattempt the snapshot. If the connector is restarted with the previous offset, the connector skips the snapshot process and immediately begins to stream change events.
Consequently, to recover from the failure, remove the connector offsets manually and start the connector.
Limitations with earlier Vitess versions
-
Due to a minor Vitess padding issue (which is fixed in Vitess 9.0.0), decimal values with a precision that is greater than or equal to 13 will cause extra whitespaces in front of the number. E.g. if the column type is
decimal(13,4)
in the table definition, the value-1.2300
becomes"- 1.2300"
, and the value1.2300
becomes" 1.2300"
. -
Does not support the
JSON
column type. -
VStream 8.0.0 doesn’t provide additional metadata of permitted values for
ENUM
columns. Therefore, the Connector does not support theENUM
column type. The index number (1-based) will be emitted instead of the enumeration value. E.g."3"
will be emitted as the value instead of"L"
if theENUM
definition isenum('S','M','L')
.