About Flink CDC¶
Flink CDC Connectors is a set of source connectors for Apache Flink, ingesting changes from different databases using change data capture (CDC). The Flink CDC Connectors integrates Debezium as the engine to capture data changes. So it can fully leverage the ability of Debezium. See more about what is Debezium.
Supported Connectors¶
Database |
Version |
---|---|
MySQL |
Database: 5.7, 8.0.x |
PostgreSQL |
Database: 9.6, 10, 11, 12 |
Supported Formats¶
Format |
Supported Connector |
Flink Version |
---|---|---|
1.11+ |
Supported Flink Versions¶
The version mapping between Flink CDC Connectors and Flink.
Flink CDC Connector Version |
Flink Version |
---|---|
1.0.0 |
1.11.* |
1.1.0 |
1.11.* |
1.2.0 |
1.12.* |
1.3.0 |
1.12.* |
1.4.0 |
1.13.* |
2.0.* |
1.13.* |
Features¶
Supports reading database snapshot and continues to read binlogs with exactly-once processing even failures happen.
CDC connectors for DataStream API, users can consume changes on multiple databases and tables in a single job without Debezium and Kafka deployed.
CDC connectors for Table/SQL API, users can use SQL DDL to create a CDC source to monitor changes on a single table.
Usage for Table/SQL API¶
We need several steps to setup a Flink cluster with the provided connector.
Setup a Flink cluster with version 1.12+ and Java 8+ installed.
Download the connector SQL jars from the Downloads page (or build yourself).
Put the downloaded jars under
FLINK_HOME/lib/
.Restart the Flink cluster.
The example shows how to create a MySQL CDC source in Flink SQL Client and execute queries on it.
-- creates a mysql cdc table source
CREATE TABLE mysql_binlog (
id INT NOT NULL,
name STRING,
description STRING,
weight DECIMAL(10,3),
PRIMARY KEY(id) NOT ENFORCED
) WITH (
'connector' = 'mysql-cdc',
'hostname' = 'localhost',
'port' = '3306',
'username' = 'flinkuser',
'password' = 'flinkpw',
'database-name' = 'inventory',
'table-name' = 'products'
);
-- read snapshot and binlog data from mysql, and do some transformation, and show on the client
SELECT id, UPPER(name), description, weight FROM mysql_binlog;
Usage for DataStream API¶
Include following Maven dependency (available through Maven Central):
<dependency>
<groupId>com.ververica</groupId>
<!-- add the dependency matching your database -->
<artifactId>flink-connector-mysql-cdc</artifactId>
<version>2.0.2</version>
</dependency>
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import com.ververica.cdc.debezium.StringDebeziumDeserializationSchema;
import com.ververica.cdc.connectors.mysql.MySqlSource;
public class MySqlBinlogSourceExample {
public static void main(String[] args) throws Exception {
Properties debeziumProperties = new Properties();
debeziumProperties.put("snapshot.locking.mode", "none");// do not use lock
SourceFunction<String> sourceFunction = MySqlSource.<String>builder()
.hostname("yourHostname")
.port(yourPort)
.databaseList("yourDatabaseName") // set captured database
.tableList("yourDatabaseName.yourTableName") // set captured table
.username("yourUsername")
.password("yourPassword")
.deserializer(new StringDebeziumDeserializationSchema()) // converts SourceRecord to String
.debeziumProperties(debeziumProperties)
.build();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env
.addSource(sourceFunction)
.print().setParallelism(1); // use parallelism 1 for sink to keep message ordering
env.execute();
}
}
Building from source¶
Prerequisites:
git
Maven
At least Java 8
git clone https://github.com/ververica/flink-cdc-connectors.git
cd flink-cdc-connectors
mvn clean install -DskipTests
Flink CDC Connectors is now available at your local .m2
repository.
License¶
The code in this repository is licensed under the Apache Software License 2.