Debezium Format

Changelog-Data-Capture Format Format: Deserialization Schema

Debezium 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 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 Debezium Avro messages and emitting Debezium messages is on the roadmap.

Dependencies

In order to setup the Debezium 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 dependencySQL Client JAR
flink-jsonBuilt-in

Note: please refer to Debezium documentation about how to setup a Debezium Kafka Connect to synchronize changelog to Kafka topics.

How to use Debezium format

Debezium provides a unified format for changelog, here is a simple example for an update operation captured from a MySQL products table:

  1. {
  2. "before": {
  3. "id": 111,
  4. "name": "scooter",
  5. "description": "Big 2-wheel scooter",
  6. "weight": 5.18
  7. },
  8. "after": {
  9. "id": 111,
  10. "name": "scooter",
  11. "description": "Big 2-wheel scooter",
  12. "weight": 5.15
  13. },
  14. "source": {...},
  15. "op": "u",
  16. "ts_ms": 1589362330904,
  17. "transaction": null
  18. }

Note: please refer to Debezium 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 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.

  1. CREATE TABLE topic_products (
  2. -- schema is totally the same to the MySQL "products" table
  3. id BIGINT,
  4. name STRING,
  5. description STRING,
  6. weight DECIMAL(10, 2)
  7. ) WITH (
  8. 'connector' = 'kafka',
  9. 'topic' = 'products_binlog',
  10. 'properties.bootstrap.servers' = 'localhost:9092',
  11. 'properties.group.id' = 'testGroup',
  12. 'format' = 'debezium-json' -- using debezium-json as the format
  13. )

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:

  1. {
  2. "schema": {...},
  3. "payload": {
  4. "before": {
  5. "id": 111,
  6. "name": "scooter",
  7. "description": "Big 2-wheel scooter",
  8. "weight": 5.18
  9. },
  10. "after": {
  11. "id": 111,
  12. "name": "scooter",
  13. "description": "Big 2-wheel scooter",
  14. "weight": 5.15
  15. },
  16. "source": {...},
  17. "op": "u",
  18. "ts_ms": 1589362330904,
  19. "transaction": null
  20. }
  21. }

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.

After registering the topic as a Flink table, then you can consume the Debezium messages as a changelog source.

  1. -- a real-time materialized view on the MySQL "products"
  2. -- which calculate the latest average of weight for the same products
  3. SELECT name, AVG(weight) FROM topic_products GROUP BY name;
  4. -- synchronize all the data and incremental changes of MySQL "products" table to
  5. -- Elasticsearch "products" index for future searching
  6. INSERT INTO elasticsearch_products
  7. SELECT * FROM topic_products;

Format Options

OptionRequiredDefaultTypeDescription
format
required(none)StringSpecify what format to use, here should be ‘debezium-json’.
debezium-json.schema-include
optionalfalseBooleanWhen setting up a Debezium Kafka Connect, users may enable a Kafka configuration ‘value.converter.schemas.enable’ to include schema in the message. This option indicates whether the Debezium JSON message includes the schema or not.
debezium-json.ignore-parse-errors
optionalfalseBooleanSkip fields and rows with parse errors instead of failing. Fields are set to null in case of errors.
debezium-json.timestamp-format.standard
optional‘SQL’StringSpecify the input and output timestamp format. Currently supported values are ‘SQL’ and ‘ISO-8601’:
  • Option ‘SQL’ will parse input timestamp in “yyyy-MM-dd HH:mm:ss.s{precision}” format, e.g ‘2020-12-30 12:13:14.123’ and output timestamp in the same format.
  • Option ‘ISO-8601’will parse input timestamp in “yyyy-MM-ddTHH:mm:ss.s{precision}” format, e.g ‘2020-12-30T12:13:14.123’ and output timestamp in the same format.

Data Type Mapping

Currently, the Debezium format uses JSON format for deserialization. Please refer to JSON format documentation for more details about the data type mapping.