@ -530,7 +530,7 @@ Incremental snapshot reading is a new mechanism to read snapshot of a table. Com
If you would like the source run in parallel, each parallel reader should have an unique server id, so the 'server-id' must be a range like '5400-6400',
and the range must be larger than the parallelism.
During the incremental snapshot reading, the MySQL CDC Source firstly splits snapshot chunks (splits) by primary key of table,
During the incremental snapshot reading, the MySQL CDC Source firstly splits snapshot chunks (splits) by user specified chunk key of table,
and then MySQL CDC Source assigns the chunks to multiple readers to read the data of snapshot chunk.
#### Controlling Parallelism
@ -580,7 +580,7 @@ The CDC job may restart fails in this case. So the heartbeat event will help upd
When the MySQL CDC source is started, it reads snapshot of table parallelly and then reads binlog of table with single parallelism.
In snapshot phase, the snapshot is cut into multiple snapshot chunks according to primary key of table and the size of table rows.
In snapshot phase, the snapshot is cut into multiple snapshot chunks according to chunk key of table and the size of table rows.
Snapshot chunks is assigned to multiple snapshot readers. Each snapshot reader reads its received chunks with [chunk reading algorithm](#snapshot-chunk-reading) and send the read data to downstream.
The source manages the process status (finished or not) of chunks, thus the source of snapshot phase can support checkpoint in chunk level.
If a failure happens, the source can be restored and continue to read chunks from last finished chunks.
@ -596,7 +596,9 @@ Flink performs checkpoints for the source periodically, in case of failover, the
When performing incremental snapshot reading, MySQL CDC source need a criterion which used to split the table.
MySQL CDC Source use a splitting column to split the table to multiple splits (chunks). By default, MySQL CDC source will identify the primary key column of the table and use the first column in primary key as the splitting column.
If there is no primary key in the table, incremental snapshot reading will fail and you can disable `scan.incremental.snapshot.enabled` to fallback to old snapshot reading mechanism.
If there is no primary key in the table, user must specify `scan.incremental.snapshot.chunk.key-column`,
otherwise incremental snapshot reading will fail and you can disable `scan.incremental.snapshot.enabled` to fallback to old snapshot reading mechanism.
Please note that using a column not in primary key as a chunk key can result in slower table query performance.
For numeric and auto incremental splitting column, MySQL CDC Source efficiently splits chunks by fixed step length.
For example, if you had a table with a primary key column of `id` which is auto-incremental BIGINT type, the minimum value was `0` and maximum value was `100`,