Hongshun Wang created FLINK-35849:
-------------------------------------

             Summary: [flink-cdc] Use expose_snapshot to read snapshot of 
postgres cdc connector.
                 Key: FLINK-35849
                 URL: https://issues.apache.org/jira/browse/FLINK-35849
             Project: Flink
          Issue Type: New Feature
          Components: Flink CDC
    Affects Versions: cdc-3.1.1
            Reporter: Hongshun Wang
             Fix For: cdc-3.3.0


In current postgres cdc connector, we use incremental framework to read 
data[1], which include the following step:
 # create a global slot in case that the wal log be recycle.
 # Enumerator split the table into multiple chunks(named "snapshot split" in 
cdc), than assigned this snapshot splits to the readers.
 # The read read the snapshot data of the snapshot split and backfill log. Each 
reader need a temporary slot to read log.
 # when all snapshot snapshots are finished, enumerator will send a stream 
split to reader. The one reader will read log.

 

However, read backfill log will also increase burden in source database. For 
example, the Postgres cdc connector will establish many logical replication 
connections to the Postgres database, which can easily reach the max_sender_num 
or max_slot_number limit. Assuming there are 10 Postgres cdc sources and each 
runs 4 parallel processes, a total of 10*(4+1) = 50 replication connections 
will be created.In many situations, the sink databases provides idempotence. 
Therefore, We can also support at-least-once semantics by skipping the backfill 
period, which will reduce budget on the source databases. Users can choose 
between at-least-once or exactly-once based on their demands.[2]

 
The two methods make a tradeoff between semantics and performance. Is there any 
other method to do well in both?

It seems expose_snapshot[3] can do both. When creating global slot, we can save 
the the snapshot name, and search it in snapshot split reading(thus no need to 
read backfill log). Then we just read the wal-log based on global slot. It can 
also provide exactly-once semantics. 
And expose_snapshot is also a default behavior when create a new replication 
slot, thus will not occur other side effects .
 
 
 
 

 

[1] [https://github.com/apache/flink-cdc/pull/2216]

 [2][https://github.com/apache/flink-cdc/issues/2553]

 [3] [https://www.postgresql.org/docs/14/protocol-replication.html]

 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to