docs/content/docs/connectors/flink-sources/vitess-cdc.md
The Vitess CDC connector allows for reading of incremental data from Vitess cluster. The connector does not support snapshot feature at the moment. This document describes how to setup the Vitess CDC connector to run SQL queries against Vitess databases. Vitess debezium documentation
In order to setup the Vitess CDC connector, the following table provides dependency information for both projects using a build automation tool (such as Maven or SBT) and SQL Client with SQL JAR bundles.
{{< artifact flink-connector-vitess-cdc >}}
Download [flink-sql-connector-vitess-cdc-{{< param Version >}}.jar](https://repo1.maven.org/maven2/org/apache/flink/flink-sql-connector-vitess-cdc/{{< param Version >}}/flink-sql-connector-vitess-cdc-{{< param Version >}}.jar) and put it under <FLINK_HOME>/lib/.
Note: Refer to flink-sql-connector-vitess-cdc, more released versions will be available in the Maven central warehouse.
You can follow the Local Install via Docker guide, or the Vitess Operator for Kubernetes guide to install Vitess. No special setup is needed to support Vitess connector.
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.
The Vitess CDC table can be defined as following:
-- checkpoint every 3000 milliseconds
Flink SQL> SET 'execution.checkpointing.interval' = '3s';
-- register a Vitess table 'orders' in Flink SQL
Flink SQL> CREATE TABLE orders (
order_id INT,
order_date TIMESTAMP(0),
customer_name STRING,
price DECIMAL(10, 5),
product_id INT,
order_status BOOLEAN,
PRIMARY KEY(order_id) NOT ENFORCED
) WITH (
'connector' = 'vitess-cdc',
'hostname' = 'localhost',
'port' = '3306',
'keyspace' = 'mydb',
'table-name' = 'orders');
-- read snapshot and binlogs from orders table
Flink SQL> SELECT * FROM orders;
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 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.
Incremental snapshot reading provides the ability to perform checkpoint in chunk level. It resolves the checkpoint timeout problem in previous version with old snapshot reading mechanism.
The Vitess CDC connector is a Flink Source connector which will read table snapshot chunks first and then continues to read binlog, both snapshot phase and binlog phase, Vitess CDC connector read with exactly-once processing even failures happen.
The Incremental Reading feature of Vitess CDC Source only exposes in SQL currently, if you're using DataStream, please use Vitess Source:
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.cdc.debezium.JsonDebeziumDeserializationSchema;
import org.apache.flink.cdc.connectors.vitess.VitessSource;
public class VitessSourceExample {
public static void main(String[] args) throws Exception {
SourceFunction<String> sourceFunction = VitessSource.<String>builder()
.hostname("localhost")
.port(15991)
.keyspace("inventory")
.username("flinkuser")
.password("flinkpw")
.deserializer(new JsonDebeziumDeserializationSchema()) // converts SourceRecord to JSON String
.build();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env
.addSource(sourceFunction)
.print().setParallelism(1); // use parallelism 1 for sink to keep message ordering
env.execute();
}
}
TINYINT UNSIGNED</td>
<td>SMALLINT</td>
</tr>
<tr>
<td>
INT
MEDIUMINT
SMALLINT UNSIGNED</td>
<td>INT</td>
</tr>
<tr>
<td>
BIGINT
INT UNSIGNED</td>
<td>BIGINT</td>
</tr>
DOUBLE PRECISION</td>
<td>DOUBLE</td>
</tr>
<tr>
<td>
NUMERIC(p, s)
DECIMAL(p, s)</td>
<td>DECIMAL(p, s)</td>
</tr>
<tr>
<td>
BOOLEAN
TINYINT(1)</td>
<td>BOOLEAN</td>
</tr>
<tr>
<td>
CHAR(n)
VARCHAR(n)
TEXT</td>
<td>STRING</td>
</tr>
</tbody>
{{< top >}}