Canal Format
Changelog-Data-Capture Format Format: Deserialization Schema
Canal is a CDC (Changelog Data Capture) tool that can stream changes in real-time from MySQL into other systems. Canal provides a unified format schema for changelog and supports to serialize messages using JSON and protobuf (protobuf is the default format for Canal).
Flink supports to interpret Canal JSON 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.
Note: Support for interpreting Canal protobuf messages and emitting Canal messages is on the roadmap.
Dependencies
In order to setup the Canal format, 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.
Maven dependency | SQL Client JAR |
---|---|
flink-json | Built-in |
Note: please refer to Canal documentation about how to deploy Canal to synchronize changelog to message queues.
How to use Canal format
Canal provides a unified format for changelog, here is a simple example for an update operation captured from a MySQL products
table:
{
"data": [
{
"id": "111",
"name": "scooter",
"description": "Big 2-wheel scooter",
"weight": "5.18"
}
],
"database": "inventory",
"es": 1589373560000,
"id": 9,
"isDdl": false,
"mysqlType": {
"id": "INTEGER",
"name": "VARCHAR(255)",
"description": "VARCHAR(512)",
"weight": "FLOAT"
},
"old": [
{
"weight": "5.15"
}
],
"pkNames": [
"id"
],
"sql": "",
"sqlType": {
"id": 4,
"name": 12,
"description": 12,
"weight": 7
},
"table": "products",
"ts": 1589373560798,
"type": "UPDATE"
}
Note: please refer to Canal documentation 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 the messages have been 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',
'format' = 'canal-json' -- using canal-json as the format
)
After registering the topic as a Flink table, you can consume the Canal messages as a changelog source.
-- a real-time materialized view on the MySQL "products"
-- which calculates 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;
Format Options
Option | Required | Default | Type | Description |
---|---|---|---|---|
format | required | (none) | String | Specify what format to use, here should be ‘canal-json’ . |
canal-json.ignore-parse-errors | optional | false | Boolean | Skip fields and rows with parse errors instead of failing. Fields are set to null in case of errors. |
canal-json.timestamp-format.standard | optional | ‘SQL’ | String | Specify the input and output timestamp format. Currently supported values are ‘SQL’ and ‘ISO-8601’ :
|
Data Type Mapping
Currently, the Canal format uses JSON format for deserialization. Please refer to JSON format documentation for more details about the data type mapping.