title: Debezium weight: 5 type: docs aliases:

  • /dev/table/connectors/formats/debezium.html

Debezium Format

{{< label “Changelog-Data-Capture Format” >}} {{< label “Format: Serialization Schema” >}} {{< label “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 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.

Dependencies

Debezium Avro

{{< sql_download_table “debezium-avro-confluent” >}}

Debezium Json

{{< sql_download_table “debezium-json” >}}

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 in JSON format:

{
  "before": {
    "id": 111,
    "name": "scooter",
    "description": "Big 2-wheel scooter",
    "weight": 5.18
  },
  "after": {
    "id": 111,
    "name": "scooter",
    "description": "Big 2-wheel scooter",
    "weight": 5.15
  },
  "source": {...},
  "op": "u",
  "ts_ms": 1589362330904,
  "transaction": null
}

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.

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
 -- please use 'debezium-avro-confluent' if Debezium encodes messages in Avro format
 '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:

{
  "schema": {...},
  "payload": {
    "before": {
      "id": 111,
      "name": "scooter",
      "description": "Big 2-wheel scooter",
      "weight": 5.18
    },
    "after": {
      "id": 111,
      "name": "scooter",
      "description": "Big 2-wheel scooter",
      "weight": 5.15
    },
    "source": {...},
    "op": "u",
    "ts_ms": 1589362330904,
    "transaction": null
  }
}

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.

-- 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;

Available Metadata

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:

CREATE TABLE KafkaTable (
  origin_ts TIMESTAMP(3) METADATA FROM 'value.ingestion-timestamp' VIRTUAL,
  event_time TIMESTAMP(3) METADATA FROM 'value.source.timestamp' VIRTUAL,
  origin_database STRING METADATA FROM 'value.source.database' VIRTUAL,
  origin_schema STRING METADATA FROM 'value.source.schema' VIRTUAL,
  origin_table STRING METADATA FROM 'value.source.table' VIRTUAL,
  origin_properties MAP<STRING, STRING> METADATA FROM 'value.source.properties' VIRTUAL,
  user_id BIGINT,
  item_id BIGINT,
  behavior STRING
) WITH (
  'connector' = 'kafka',
  'topic' = 'user_behavior',
  'properties.bootstrap.servers' = 'localhost:9092',
  'properties.group.id' = 'testGroup',
  'scan.startup.mode' = 'earliest-offset',
  'value.format' = 'debezium-json'
);

Format Options

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.

{{< tabs “a8edce02-58d5-4e0b-bc4b-75d05a98a0f9” >}} {{< tab “Debezium Avro” >}}

{{< /tab >}} {{< tab “Debezium Json” >}}

{{< /tab >}} {{< /tabs >}}

Caveats

Duplicate change events

Under normal operating scenarios, the Debezium application delivers every change event exactly-once. Flink works pretty well when consuming Debezium produced events in this situation. However, Debezium application works in at-least-once delivery if any failover happens. See more details about delivery guarantee from Debezium documentation. That means, in the abnormal situations, Debezium may deliver duplicate change events to Kafka and Flink will get the duplicate events. This may cause Flink query to get wrong results or unexpected exceptions. Thus, it is recommended to set job configuration [table.exec.source.cdc-events-duplicate]({{< ref “docs/dev/table/config” >}}#table-exec-source-cdc-events-duplicate) to true and define PRIMARY KEY on the source in this situation. Framework will generate an additional stateful operator, and use the primary key to deduplicate the change events and produce a normalized changelog stream.

Consuming data produced by Debezium Postgres Connector

If you are using Debezium Connector for PostgreSQL to capture the changes to Kafka, please make sure the REPLICA IDENTITY configuration of the monitored PostgreSQL table has been set to FULL which is by default DEFAULT. Otherwise, Flink SQL currently will fail to interpret the Debezium data.

In FULL strategy, the UPDATE and DELETE events will contain the previous values of all the table’s columns. In other strategies, the “before” field of UPDATE and DELETE events will only contain primary key columns or null if no primary key. You can change the REPLICA IDENTITY by running ALTER TABLE <your-table-name> REPLICA IDENTITY FULL. See more details in Debezium Documentation for PostgreSQL REPLICA IDENTITY.

Data Type Mapping

Currently, the Debezium format uses JSON and Avro format for serialization and deserialization. Please refer to [JSON Format documentation]({{< ref “docs/connectors/table/formats/json” >}}#data-type-mapping) and [Confluent Avro Format documentation]({< ref “docs/connectors/table/formats/avro-confluent” >}}#data-type-mapping) for more details about the data type mapping.