Debezium Format
is a CDC (Changelog Data Capture) tool that can stream changes in real-time from MySQL, PostgreSQL, Oracle, Microsoft SQL Server and many other databases into Kafka. Debezium provides a unified format schema for changelog and supports to serialize messages using JSON and Apache Avro.
Flink supports to interpret Debezium JSON and Avro messages as INSERT/UPDATE/DELETE messages into Flink SQL system. This is useful in many cases to leverage this feature, such as
- synchronizing incremental data from databases to other systems
- auditing logs
- real-time materialized views on databases
- temporal join changing history of a database table and so on.
Flink also supports to encode the INSERT/UPDATE/DELETE messages in Flink SQL as Debezium JSON or Avro messages, and emit to external systems like Kafka. However, currently Flink can’t combine UPDATE_BEFORE and UPDATE_AFTER into a single UPDATE message. Therefore, Flink encodes UPDATE_BEFORE and UDPATE_AFTER as DELETE and INSERT Debezium messages.
Debezium Avro
In order to use the Debezium format the following dependencies are required for both projects using a build automation tool (such as Maven or SBT) and SQL Client with SQL JAR bundles.
Debezium Json
In order to use the Debezium format the following dependencies are required for both projects using a build automation tool (such as Maven or SBT) and SQL Client with SQL JAR bundles.
Debezium provides a unified format for changelog, here is a simple example for an update operation captured from a MySQL products
table in JSON format:
Note: please refer to about the meaning of each fields.
The MySQL products
table has 4 columns (id
, name
, description
and weight
). The above JSON message is an update change event on the products
table where the weight
value of the row with id = 111
is changed from 5.18
to 5.15
. Assuming this messages is synchronized to Kafka topic products_binlog
, then we can use the following DDL to consume this topic and interpret the change events.
CREATE TABLE topic_products (
-- schema is totally the same to the MySQL "products" table
id BIGINT,
name STRING,
description STRING,
weight DECIMAL(10, 2)
) WITH (
'connector' = 'kafka',
'topic' = 'products_binlog',
'properties.bootstrap.servers' = 'localhost:9092',
'properties.group.id' = 'testGroup',
-- using 'debezium-json' as the format to interpret Debezium JSON messages
'format' = 'debezium-json'
)
In some cases, users may setup the Debezium Kafka Connect with the Kafka configuration 'value.converter.schemas.enable'
enabled to include schema in the message. Then the Debezium JSON message may look like this:
In order to interpret such messages, you need to add the option 'debezium-json.schema-include' = 'true'
into above DDL WITH clause (false
by default). Usually, this is not recommended to include schema because this makes the messages very verbose and reduces parsing performance.
-- a real-time materialized view on the MySQL "products"
-- which calculate the latest average of weight for the same products
SELECT name, AVG(weight) FROM topic_products GROUP BY name;
-- synchronize all the data and incremental changes of MySQL "products" table to
-- Elasticsearch "products" index for future searching
INSERT INTO elasticsearch_products
SELECT * FROM topic_products;
The following format metadata can be exposed as read-only (VIRTUAL
) columns in a table definition.
Attention Format metadata fields are only available if the corresponding connector forwards format metadata. Currently, only the Kafka connector is able to expose metadata fields for its value format.
The following example shows how to access Debezium metadata fields in Kafka:
Flink provides debezium-avro-confluent
and debezium-json
formats to interpret Avro or Json messages produced by Debezium. Use format debezium-avro-confluent
to interpret Debezium Avro messages and format debezium-json
to interpret Debezium Json messages.
Debezium Avro