Maxwell
This documentation is for an out-of-date version of Apache Flink. We recommend you use the latest stable version.

Maxwell Format #

Changelog-Data-Capture Format Format: Serialization Schema Format: Deserialization Schema

Maxwell is a CDC (Changelog Data Capture) tool that can stream changes in real-time from MySQL into Kafka, Kinesis and other streaming connectors. Maxwell provides a unified format schema for changelog and supports to serialize messages using JSON.

Flink supports to interpret Maxwell 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.

Flink also supports to encode the INSERT/UPDATE/DELETE messages in Flink SQL as Maxwell JSON 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 Maxwell messages.

Dependencies #

In order to use the Maxwell 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.

Maven dependency SQL Client
Built-in

Note: please refer to Maxwell documentation about how to synchronize changelog to Kafka topics with Maxwell JSON.

How to use Maxwell format #

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

{
   "database":"test",
   "table":"e",
   "type":"insert",
   "ts":1477053217,
   "xid":23396,
   "commit":true,
   "position":"master.000006:800911",
   "server_id":23042,
   "thread_id":108,
   "primary_key": [1, "2016-10-21 05:33:37.523000"],
   "primary_key_columns": ["id", "c"],
   "data":{
     "id":111,
     "name":"scooter",
     "description":"Big 2-wheel scooter",
     "weight":5.15
   },
   "old":{
     "weight":5.18,
   }
}

Note: please refer to Maxwell 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',
 'format' = 'maxwell-json'
)

After registering the topic as a Flink table, then you can consume the Maxwell 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;

Format Options #

Option Required Default Type Description
format
required (none) String Specify what format to use, here should be 'maxwell-json'.
maxwell-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.
maxwell-json.timestamp-format.standard
optional 'SQL' String Specify 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.
maxwell-json.map-null-key.mode
optional 'FAIL' String Specify the handling mode when serializing null keys for map data. Currently supported values are 'FAIL', 'DROP' and 'LITERAL':
  • Option 'FAIL' will throw exception when encountering map with null key.
  • Option 'DROP' will drop null key entries for map data.
  • Option 'LITERAL' will replace null key with string literal. The string literal is defined by maxwell-json.map-null-key.literal option.
maxwell-json.map-null-key.literal
optional 'null' String Specify string literal to replace null key when 'maxwell-json.map-null-key.mode' is LITERAL.
maxwell-json.encode.decimal-as-plain-number
optional false Boolean Encode all decimals as plain numbers instead of possible scientific notations. By default, decimals may be written using scientific notation. For example, 0.000000027 is encoded as 2.7E-8 by default, and will be written as 0.000000027 if set this option to true.

Caveats #

Duplicate change events #

The Maxwell application allows to deliver every change event exactly-once. Flink works pretty well when consuming Maxwell produced events in this situation. If Maxwell application works in at-least-once delivery, it 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 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.

Data Type Mapping #

Currently, the Maxwell format uses JSON for serialization and deserialization. Please refer to JSON Format documentation for more details about the data type mapping.