You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by he...@apache.org on 2022/04/27 09:08:24 UTC

[incubator-inlong] branch master updated: [INLONG-3956][Sort] Add Hive connector to support CDC (#3963)

This is an automated email from the ASF dual-hosted git repository.

healchow pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-inlong.git


The following commit(s) were added to refs/heads/master by this push:
     new 7069f25fe [INLONG-3956][Sort] Add Hive connector to support CDC (#3963)
7069f25fe is described below

commit 7069f25fec170b06ee25568b67fde5b649f8143a
Author: ganfengtan <Ga...@users.noreply.github.com>
AuthorDate: Wed Apr 27 17:08:18 2022 +0800

    [INLONG-3956][Sort] Add Hive connector to support CDC (#3963)
    
    * [INLONG-3961][sort] add mysql cdc append
    
    * [INLONG-3956][sort]add hive connector to support CDC to hive
    
    * [INLONG-3956][Sort] add hive load UT and fix checkstyle
    
    * [INLONG-3956][sort] fix UT
    
    * [INLONG-3956][Sort] add hive load UT and fix checkstyle
    
    * [INLONG-3956][Sort] delete mysql cdc executor UT
    
    * [INLONG-3956][Sort] remove sort pom dependency flink-table-common
---
 .../service/sort/util/ExtractNodeUtils.java        |   6 +-
 .../apache/inlong/sort/protocol/node/LoadNode.java |  13 +-
 .../org/apache/inlong/sort/protocol/node/Node.java |  11 +-
 .../protocol/node/extract/MySqlExtractNode.java    |  13 +-
 .../sort/protocol/node/load/HiveLoadNode.java      | 139 ++++++
 .../protocol/transformation/OrderDirection.java    |   3 -
 .../inlong/sort/protocol/StreamInfoTest.java       | 136 +++++-
 .../sort/protocol/node/HiveLoadNodeTest.java       |  58 +++
 inlong-sort/sort-core/pom.xml                      |  10 +
 inlong-sort/sort-single-tenant/pom.xml             | 128 ++++-
 .../debezium/DebeziumDeserializationSchema.java    |  24 +-
 .../flink/cdc/debezium/DebeziumSourceFunction.java |  33 +-
 .../JsonDebeziumDeserializationSchema.java         |  98 ++++
 ...va => StringDebeziumDeserializationSchema.java} |  29 +-
 .../history/FlinkJsonTableChangeSerializer.java    | 207 ++++++++
 .../debezium/internal/DebeziumChangeConsumer.java  | 103 ++++
 .../debezium/internal/DebeziumChangeFetcher.java   | 309 ++++++++++++
 .../cdc/debezium/internal/DebeziumOffset.java      |  64 +++
 .../internal/DebeziumOffsetSerializer.java         |  41 ++
 .../debezium/internal/FlinkDatabaseHistory.java    | 116 +++++
 .../internal/FlinkDatabaseSchemaHistory.java       | 199 ++++++++
 .../debezium/internal/FlinkOffsetBackingStore.java | 201 ++++++++
 .../flink/cdc/debezium/internal/Handover.java      | 194 ++++++++
 .../flink/cdc/debezium/internal/SchemaRecord.java  |  95 ++++
 .../debezium/table/AppendMetadataCollector.java    |   7 +-
 .../cdc/debezium/table/MetadataConverter.java      |  10 +-
 .../table/RowDataDebeziumDeserializeSchema.java    | 330 +++++++------
 .../cdc/debezium/utils/DatabaseHistoryUtil.java    |   4 +-
 .../flink/cdc/io/BufferedSocketInputStream.java    |  80 ++++
 .../singletenant/flink/cdc/mysql/MySqlSource.java  |  41 +-
 .../flink/cdc/mysql/MySqlValidator.java            |  13 +-
 .../cdc/mysql/SeekBinlogToTimestampFilter.java     |   3 +-
 .../cdc/mysql/{ => debezium}/DebeziumUtils.java    |  48 +-
 .../debezium/EmbeddedFlinkDatabaseHistory.java     | 153 ++++++
 .../debezium/dispatcher/EventDispatcherImpl.java   | 228 +++++++++
 .../debezium/dispatcher/SignalEventDispatcher.java | 125 +++++
 .../mysql/debezium/reader/BinlogSplitReader.java   | 275 +++++++++++
 .../cdc/mysql/debezium/reader/DebeziumReader.java  |  46 ++
 .../mysql/debezium/reader/SnapshotSplitReader.java | 321 +++++++++++++
 .../debezium/task/MySqlBinlogSplitReadTask.java    | 120 +++++
 .../debezium/task/MySqlSnapshotSplitReadTask.java  | 362 ++++++++++++++
 .../debezium/task/context/MySqlErrorHandler.java   |  73 +++
 .../task/context/MySqlTaskContextImpl.java         |  43 ++
 .../debezium/task/context/StatefulTaskContext.java | 322 +++++++++++++
 .../cdc/mysql/schema/MySqlFieldDefinition.java     |  98 ++++
 .../flink/cdc/mysql/schema/MySqlSchema.java        | 162 +++++++
 .../cdc/mysql/schema/MySqlTableDefinition.java     |  71 +++
 .../flink/cdc/mysql/schema/MySqlTypeUtils.java     | 133 ++++++
 .../flink/cdc/mysql/source/MySqlSource.java        | 233 +++++++++
 .../flink/cdc/mysql/source/MySqlSourceBuilder.java | 238 ++++++++++
 .../cdc/mysql/source/assigners/AssignerStatus.java | 205 ++++++++
 .../cdc/mysql/source/assigners/ChunkRange.java     |  86 ++++
 .../cdc/mysql/source/assigners/ChunkSplitter.java  | 325 +++++++++++++
 .../source/assigners/MySqlBinlogSplitAssigner.java | 189 ++++++++
 .../source/assigners/MySqlHybridSplitAssigner.java | 228 +++++++++
 .../assigners/MySqlSnapshotSplitAssigner.java      | 404 ++++++++++++++++
 .../mysql/source/assigners/MySqlSplitAssigner.java | 119 +++++
 .../assigners/state/BinlogPendingSplitsState.java} |  30 +-
 .../assigners/state/HybridPendingSplitsState.java  |  69 +++
 .../assigners/state/PendingSplitsState.java}       |  25 +-
 .../state/PendingSplitsStateSerializer.java        | 350 ++++++++++++++
 .../state/SnapshotPendingSplitsState.java          | 171 +++++++
 .../{ => source}/config/MySqlSourceConfig.java     |   4 +-
 .../source/config/MySqlSourceConfigFactory.java    | 334 +++++++++++++
 .../{ => source}/config/MySqlSourceOptions.java    |   6 +-
 .../mysql/{ => source}/config/ServerIdRange.java   |   2 +-
 .../{ => source}/connection/ConnectionPoolId.java  |   2 +-
 .../{ => source}/connection/ConnectionPools.java   |   4 +-
 .../connection/JdbcConnectionFactory.java          |   4 +-
 .../connection/JdbcConnectionPools.java            |   6 +-
 .../connection/PooledDataSourceFactory.java        |   4 +-
 .../source/enumerator/MySqlSourceEnumerator.java   | 315 +++++++++++++
 .../mysql/source/events/BinlogSplitMetaEvent.java  |  63 +++
 .../source/events/BinlogSplitMetaRequestEvent.java |  48 ++
 .../events/FinishedSnapshotSplitsAckEvent.java     |  49 ++
 .../events/FinishedSnapshotSplitsReportEvent.java  |  49 ++
 .../FinishedSnapshotSplitsRequestEvent.java}       |  27 +-
 .../events/LatestFinishedSplitsSizeEvent.java      |  49 ++
 .../LatestFinishedSplitsSizeRequestEvent.java}     |  26 +-
 .../events/SuspendBinlogReaderAckEvent.java}       |  26 +-
 .../source/events/SuspendBinlogReaderEvent.java}   |  26 +-
 .../cdc/mysql/source/events/WakeupReaderEvent.java |  47 ++
 .../source/metrics/MySqlSourceReaderMetrics.java   |  86 ++++
 .../cdc/mysql/source/offset/BinlogOffset.java      | 264 +++++++++++
 .../source/offset/BinlogOffsetSerializer.java      |  43 ++
 .../mysql/source/reader/MySqlRecordEmitter.java    | 264 +++++++++++
 .../cdc/mysql/source/reader/MySqlSourceReader.java | 342 ++++++++++++++
 .../source/reader/MySqlSourceReaderContext.java    |  52 +++
 .../cdc/mysql/source/reader/MySqlSplitReader.java  | 174 +++++++
 .../source/split/FinishedSnapshotSplitInfo.java    | 156 +++++++
 .../cdc/mysql/source/split/MySqlBinlogSplit.java   | 202 ++++++++
 .../mysql/source/split/MySqlBinlogSplitState.java  |  90 ++++
 .../flink/cdc/mysql/source/split/MySqlRecords.java |  87 ++++
 .../cdc/mysql/source/split/MySqlSnapshotSplit.java | 144 ++++++
 .../source/split/MySqlSnapshotSplitState.java      |  64 +++
 .../flink/cdc/mysql/source/split/MySqlSplit.java   |  79 ++++
 .../mysql/source/split/MySqlSplitSerializer.java   | 256 ++++++++++
 .../cdc/mysql/source/split/MySqlSplitState.java    |  61 +++
 .../flink/cdc/mysql/source/utils/ChunkUtils.java   |  79 ++++
 .../source}/utils/ObjectUtils.java                 |   2 +-
 .../flink/cdc/mysql/source/utils/RecordUtils.java  | 433 +++++++++++++++++
 .../cdc/mysql/source/utils/SerializerUtils.java    |  98 ++++
 .../cdc/mysql/source/utils/StatementUtils.java     | 326 +++++++++++++
 .../mysql/source/utils/TableDiscoveryUtils.java    | 121 +++++
 .../MySqlDeserializationConverterFactory.java      |  22 +-
 .../cdc/mysql/table/MySqlReadableMetadata.java     | 454 +++++++++---------
 ...ory.java => MySqlTableInlongSourceFactory.java} | 169 +++----
 .../flink/cdc/mysql/table/MySqlTableSource.java    | 106 +++--
 .../cdc/mysql/table/OldFieldMetadataConverter.java | 411 ++++++++++++++++
 .../flink/cdc/mysql/table/StartupOptions.java      |   4 +-
 .../connectors/hive/HadoopFileSystemFactory.java   |  46 ++
 .../flink/connectors/hive/HiveOptions.java         |  65 +++
 .../hive/HiveRowDataPartitionComputer.java         |  79 ++++
 .../connectors/hive/HiveRowPartitionComputer.java  |  73 +++
 .../connectors/hive/HiveTableMetaStoreFactory.java | 149 ++++++
 .../flink/connectors/hive/HiveTableSink.java       | 520 +++++++++++++++++++++
 .../flink/connectors/hive/HiveValidator.java       |  46 ++
 .../catalog/factories/HiveTableInlongFactory.java  | 173 +++++++
 .../flink/hive/filesystems/CHDFSFsFactory.java     | 111 -----
 .../org.apache.flink.table.factories.Factory       |  34 ++
 .../hive/HiveSinkWithoutPartitionTestCase.java     |  18 +-
 .../flink/parser/FlinkSqlParserTest.java           |  61 ++-
 pom.xml                                            |  34 +-
 123 files changed, 14242 insertions(+), 817 deletions(-)

diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sort/util/ExtractNodeUtils.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sort/util/ExtractNodeUtils.java
index 9198c5754..572777847 100644
--- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sort/util/ExtractNodeUtils.java
+++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/sort/util/ExtractNodeUtils.java
@@ -20,6 +20,8 @@ package org.apache.inlong.manager.service.sort.util;
 import com.google.common.base.Splitter;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import java.util.List;
+import java.util.stream.Collectors;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.inlong.common.enums.DataTypeEnum;
@@ -41,9 +43,6 @@ import org.apache.inlong.sort.protocol.node.format.DebeziumJsonFormat;
 import org.apache.inlong.sort.protocol.node.format.Format;
 import org.apache.inlong.sort.protocol.node.format.JsonFormat;
 
-import java.util.List;
-import java.util.stream.Collectors;
-
 /**
  * Parse SourceResponse to ExtractNode which sort needed
  */
@@ -73,6 +72,7 @@ public class ExtractNodeUtils {
 
     /**
      * Create MySqlExtractNode based on BinlogSourceResponse
+     *
      * @param binlogSourceResponse
      * @return
      */
diff --git a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/LoadNode.java b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/LoadNode.java
index 76e5799c5..94cc5202f 100644
--- a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/LoadNode.java
+++ b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/LoadNode.java
@@ -18,6 +18,10 @@
 package org.apache.inlong.sort.protocol.node;
 
 import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
 import lombok.Data;
 import lombok.NoArgsConstructor;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
@@ -27,21 +31,18 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonPro
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeInfo;
 import org.apache.inlong.sort.protocol.FieldInfo;
+import org.apache.inlong.sort.protocol.node.load.HiveLoadNode;
 import org.apache.inlong.sort.protocol.node.load.KafkaLoadNode;
 import org.apache.inlong.sort.protocol.transformation.FieldRelationShip;
 import org.apache.inlong.sort.protocol.transformation.FilterFunction;
 
-import javax.annotation.Nullable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
 @JsonTypeInfo(
         use = JsonTypeInfo.Id.NAME,
         include = JsonTypeInfo.As.PROPERTY,
         property = "type")
 @JsonSubTypes({
-        @JsonSubTypes.Type(value = KafkaLoadNode.class, name = "kafkaLoad")
+        @JsonSubTypes.Type(value = KafkaLoadNode.class, name = "kafkaLoad"),
+        @JsonSubTypes.Type(value = HiveLoadNode.class, name = "hiveLoad")
 })
 @NoArgsConstructor
 @Data
diff --git a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/Node.java b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/Node.java
index 674347fab..210994ba8 100644
--- a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/Node.java
+++ b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/Node.java
@@ -17,6 +17,9 @@
 
 package org.apache.inlong.sort.protocol.node;
 
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude.Include;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes;
@@ -24,14 +27,11 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTyp
 import org.apache.inlong.sort.protocol.FieldInfo;
 import org.apache.inlong.sort.protocol.node.extract.KafkaExtractNode;
 import org.apache.inlong.sort.protocol.node.extract.MySqlExtractNode;
+import org.apache.inlong.sort.protocol.node.load.HiveLoadNode;
 import org.apache.inlong.sort.protocol.node.load.KafkaLoadNode;
 import org.apache.inlong.sort.protocol.node.transform.DistinctNode;
 import org.apache.inlong.sort.protocol.node.transform.TransformNode;
 
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
 @JsonTypeInfo(
         use = JsonTypeInfo.Id.NAME,
         include = JsonTypeInfo.As.PROPERTY,
@@ -41,7 +41,8 @@ import java.util.TreeMap;
         @JsonSubTypes.Type(value = KafkaExtractNode.class, name = "kafkaExtract"),
         @JsonSubTypes.Type(value = TransformNode.class, name = "baseTransform"),
         @JsonSubTypes.Type(value = KafkaLoadNode.class, name = "kafkaLoad"),
-        @JsonSubTypes.Type(value = DistinctNode.class, name = "distinct")
+        @JsonSubTypes.Type(value = DistinctNode.class, name = "distinct"),
+        @JsonSubTypes.Type(value = HiveLoadNode.class, name = "hiveLoad")
 })
 public interface Node {
 
diff --git a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/extract/MySqlExtractNode.java b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/extract/MySqlExtractNode.java
index b631b3c90..ae5d4818e 100644
--- a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/extract/MySqlExtractNode.java
+++ b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/extract/MySqlExtractNode.java
@@ -18,6 +18,11 @@
 package org.apache.inlong.sort.protocol.node.extract;
 
 import com.google.common.base.Preconditions;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import lombok.Data;
 import lombok.EqualsAndHashCode;
 import org.apache.commons.lang3.StringUtils;
@@ -30,12 +35,6 @@ import org.apache.inlong.sort.protocol.FieldInfo;
 import org.apache.inlong.sort.protocol.node.ExtractNode;
 import org.apache.inlong.sort.protocol.transformation.WatermarkField;
 
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-
 @EqualsAndHashCode(callSuper = true)
 @JsonTypeName("mysqlExtract")
 @Data
@@ -113,7 +112,7 @@ public class MySqlExtractNode extends ExtractNode implements Serializable {
     @Override
     public Map<String, String> tableOptions() {
         Map<String, String> options = super.tableOptions();
-        options.put("connector", "mysql-cdc");
+        options.put("connector", "mysql-cdc-inlong");
         options.put("hostname", hostname);
         options.put("username", username);
         options.put("password", password);
diff --git a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/load/HiveLoadNode.java b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/load/HiveLoadNode.java
new file mode 100644
index 000000000..e5a2c6be9
--- /dev/null
+++ b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/load/HiveLoadNode.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.protocol.node.load;
+
+import com.google.common.base.Preconditions;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import lombok.NoArgsConstructor;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.inlong.sort.protocol.FieldInfo;
+import org.apache.inlong.sort.protocol.node.LoadNode;
+import org.apache.inlong.sort.protocol.transformation.FieldRelationShip;
+import org.apache.inlong.sort.protocol.transformation.FilterFunction;
+
+@EqualsAndHashCode(callSuper = true)
+@JsonTypeName("hiveLoad")
+@Data
+@NoArgsConstructor
+public class HiveLoadNode extends LoadNode implements Serializable {
+
+    private static final long serialVersionUID = -4547768154621816459L;
+
+    private static final String trigger = "sink.partition-commit.trigger";
+    private static final String timestampPattern = "partition.time-extractor.timestamp-pattern";
+    private static final String delay = "sink.partition-commit.delay";
+    private static final String policyKind = "sink.partition-commit.policy.kind";
+
+    @JsonProperty("tableName")
+    @Nonnull
+    private String tableName;
+
+    @JsonProperty("catalogName")
+    @Nonnull
+    private String catalogName;
+
+    @JsonProperty("database")
+    @Nonnull
+    private String database;
+
+    @JsonProperty("hiveConfDir")
+    @Nonnull
+    private String hiveConfDir;
+
+    @JsonProperty("hiveVersion")
+    @Nonnull
+    private String hiveVersion;
+
+    @JsonProperty("hadoopConfDir")
+    private String hadoopConfDir;
+
+    @JsonProperty("partitionFields")
+    private List<FieldInfo> partitionFields;
+
+    @JsonCreator
+    public HiveLoadNode(@JsonProperty("id") String id,
+            @JsonProperty("name") String name,
+            @JsonProperty("fields") List<FieldInfo> fields,
+            @JsonProperty("fieldRelationShips") List<FieldRelationShip> fieldRelationShips,
+            @JsonProperty("filters") List<FilterFunction> filters,
+            @Nullable @JsonProperty("sinkParallelism") Integer sinkParallelism,
+            @JsonProperty("properties") Map<String, String> properties,
+            @Nullable @JsonProperty("catalogName") String catalogName,
+            @Nullable @JsonProperty("database") String database,
+            @Nullable @JsonProperty("tableName") String tableName,
+            @Nullable @JsonProperty("hiveConfDir") String hiveConfDir,
+            @Nullable @JsonProperty("hiveVersion") String hiveVersion,
+            @JsonProperty("hadoopConfDir") String hadoopConfDir,
+            @JsonProperty("parFields") List<FieldInfo> partitionFields) {
+        super(id, name, fields, fieldRelationShips, filters, sinkParallelism, properties);
+        this.catalogName = Preconditions.checkNotNull(catalogName, "catalog of hive is null");
+        this.database = Preconditions.checkNotNull(database, "database of hive is null");
+        this.tableName = Preconditions.checkNotNull(tableName, "table of hive is null");
+        this.hiveConfDir = Preconditions.checkNotNull(hiveConfDir, "hive config path is null");
+        this.hiveVersion = Preconditions.checkNotNull(hiveVersion, "version of hive is null");
+        this.hadoopConfDir = hadoopConfDir;
+        this.partitionFields = partitionFields;
+    }
+
+    @Override
+    public Map<String, String> tableOptions() {
+        Map<String, String> map = super.tableOptions();
+        map.put("connector", "hive");
+        map.put("default-database", database);
+        map.put("hive-conf-dir", hiveConfDir);
+        map.put("hive-version", hiveVersion);
+        if (null != hadoopConfDir) {
+            map.put("hadoop-conf-dir", hadoopConfDir);
+        }
+        if (null != partitionFields) {
+            Map<String, String> properties = super.getProperties();
+            if (null == properties || !properties.containsKey(trigger)) {
+                map.put(trigger, "process-time");
+            }
+            if (null == properties || !properties.containsKey(timestampPattern)) {
+                map.put(timestampPattern, "yyyy-MM-dd");
+            }
+            if (null == properties || !properties.containsKey(delay)) {
+                map.put(delay, "10s");
+            }
+            if (null == properties || !properties.containsKey(policyKind)) {
+                map.put(policyKind, "metastore,success-file");
+            }
+        }
+        return map;
+    }
+
+    @Override
+    public List<FieldInfo> getPartitionFields() {
+        return partitionFields;
+    }
+
+    @Override
+    public String genTableName() {
+        return tableName;
+    }
+
+}
diff --git a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/OrderDirection.java b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/OrderDirection.java
index ddfe73385..73b69e8bf 100644
--- a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/OrderDirection.java
+++ b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/OrderDirection.java
@@ -17,9 +17,6 @@
 
 package org.apache.inlong.sort.protocol.transformation;
 
-/**
- * OrderDirection class defines whether the sorting direction is ascending or descending
- */
 public enum OrderDirection {
     /**
      * Asc for order direction
diff --git a/inlong-sort/sort-common/src/test/java/org/apache/inlong/sort/protocol/StreamInfoTest.java b/inlong-sort/sort-common/src/test/java/org/apache/inlong/sort/protocol/StreamInfoTest.java
index f6b2ffab2..0063b7b1e 100644
--- a/inlong-sort/sort-common/src/test/java/org/apache/inlong/sort/protocol/StreamInfoTest.java
+++ b/inlong-sort/sort-common/src/test/java/org/apache/inlong/sort/protocol/StreamInfoTest.java
@@ -17,6 +17,12 @@
 
 package org.apache.inlong.sort.protocol;
 
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.inlong.sort.formats.common.FloatFormatInfo;
@@ -27,6 +33,7 @@ import org.apache.inlong.sort.formats.common.TimestampFormatInfo;
 import org.apache.inlong.sort.protocol.node.Node;
 import org.apache.inlong.sort.protocol.node.extract.MySqlExtractNode;
 import org.apache.inlong.sort.protocol.node.format.JsonFormat;
+import org.apache.inlong.sort.protocol.node.load.HiveLoadNode;
 import org.apache.inlong.sort.protocol.node.load.KafkaLoadNode;
 import org.apache.inlong.sort.protocol.transformation.FieldRelationShip;
 import org.apache.inlong.sort.protocol.transformation.StringConstantParam;
@@ -36,12 +43,6 @@ import org.apache.inlong.sort.protocol.transformation.WatermarkField;
 import org.apache.inlong.sort.protocol.transformation.relation.NodeRelationShip;
 import org.junit.Test;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.stream.Collectors;
-import static org.junit.Assert.assertEquals;
-
 /**
  * StreamInfo unit test class
  */
@@ -83,12 +84,135 @@ public class StreamInfoTest {
                 1, null, "id");
     }
 
+    private HiveLoadNode buildHiveNode() {
+        List<FieldInfo> fields = Arrays.asList(new FieldInfo("id", new LongFormatInfo()),
+                new FieldInfo("name", new StringFormatInfo()),
+                new FieldInfo("age", new IntFormatInfo()),
+                new FieldInfo("salary", new FloatFormatInfo()),
+                new FieldInfo("ts", new TimestampFormatInfo()));
+        List<FieldRelationShip> relations = Arrays
+                .asList(new FieldRelationShip(new FieldInfo("id", new LongFormatInfo()),
+                                new FieldInfo("id", new LongFormatInfo())),
+                        new FieldRelationShip(new FieldInfo("name", new StringFormatInfo()),
+                                new FieldInfo("name", new StringFormatInfo())),
+                        new FieldRelationShip(new FieldInfo("age", new IntFormatInfo()),
+                                new FieldInfo("age", new IntFormatInfo())),
+                        new FieldRelationShip(new FieldInfo("ts", new TimestampFormatInfo()),
+                                new FieldInfo("ts", new TimestampFormatInfo()))
+                );
+        return new HiveLoadNode("2", "hive_output", fields, relations, null,
+                1, null, "myHive", "default", "test", "/opt/hive-conf", "3.1.2",
+                null, Arrays.asList(new FieldInfo("day", new LongFormatInfo())));
+    }
+
     private NodeRelationShip buildNodeRelation(List<Node> inputs, List<Node> outputs) {
         List<String> inputIds = inputs.stream().map(Node::getId).collect(Collectors.toList());
         List<String> outputIds = outputs.stream().map(Node::getId).collect(Collectors.toList());
         return new NodeRelationShip(inputIds, outputIds);
     }
 
+    /**
+     * Test serialize for streamInfo,when data from mysql to hive.
+     *
+     * @throws JsonProcessingException
+     */
+    @Test
+    public void testMysqlToHiveSerialize() throws JsonProcessingException {
+        Node input = buildMySqlExtractNode();
+        Node output = buildHiveNode();
+        StreamInfo streamInfo = new StreamInfo("1", Arrays.asList(input, output), Collections.singletonList(
+                buildNodeRelation(Collections.singletonList(input), Collections.singletonList(output))));
+        ObjectMapper objectMapper = new ObjectMapper();
+        String expected = "{\"streamId\":\"1\",\"nodes\":[{\"type\":\"mysqlExtract\","
+                + "\"id\":\"1\",\"name\":\"mysql_input\",\"fields\":[{\"type\":\"base\","
+                + "\"name\":\"id\",\"formatInfo\":{\"type\":\"long\"}},{\"type\":\"base\","
+                + "\"name\":\"name\",\"formatInfo\":{\"type\":\"string\"}},{\"type\":\""
+                + "base\",\"name\":\"age\",\"formatInfo\":{\"type\":\"int\"}},{\"type\":"
+                + "\"base\",\"name\":\"salary\",\"formatInfo\":{\"type\":\"float\"}},"
+                + "{\"type\":\"base\",\"name\":\"ts\",\"formatInfo\":{\"type\":\"timestamp\","
+                + "\"format\":\"yyyy-MM-dd HH:mm:ss\"}}],\"watermarkField\":{\"type\":\"watermark\","
+                + "\"timeAttr\":{\"type\":\"base\",\"name\":\"ts\",\"formatInfo\":{\"type\":"
+                + "\"timestamp\",\"format\":\"yyyy-MM-dd HH:mm:ss\"}},\"interval\":{\"type\":"
+                + "\"stringConstant\",\"value\":\"1\"},\"timeUnit\":{\"type\":\"timeUnitConstant\","
+                + "\"timeUnit\":\"MINUTE\",\"value\":\"MINUTE\"}},\"primaryKey\":\"id\","
+                + "\"tableNames\":[\"table\"],\"hostname\":\"localhost\",\"username\":\"username\","
+                + "\"password\":\"username\",\"database\":\"test_database\",\"port\":3306,\"serverId\":123,"
+                + "\"incrementalSnapshotEnabled\":true},{\"type\":\"hiveLoad\",\"id\":\"2\","
+                + "\"name\":\"hive_output\",\"fields\":[{\"type\":\"base\",\"name\":\"id\","
+                + "\"formatInfo\":{\"type\":\"long\"}},{\"type\":\"base\",\"name\":\"name\","
+                + "\"formatInfo\":{\"type\":\"string\"}},{\"type\":\"base\",\"name\":\"age\","
+                + "\"formatInfo\":{\"type\":\"int\"}},{\"type\":\"base\",\"name\":\"salary\","
+                + "\"formatInfo\":{\"type\":\"float\"}},{\"type\":\"base\",\"name\":\"ts\","
+                + "\"formatInfo\":{\"type\":\"timestamp\",\"format\":\"yyyy-MM-dd HH:mm:ss\"}}],"
+                + "\"fieldRelationShips\":[{\"type\":\"fieldRelationShip\",\"inputField\":{"
+                + "\"type\":\"base\",\"name\":\"id\",\"formatInfo\":{\"type\":\"long\"}},"
+                + "\"outputField\":{\"type\":\"base\",\"name\":\"id\",\"formatInfo\":{\"type\":"
+                + "\"long\"}}},{\"type\":\"fieldRelationShip\",\"inputField\":{\"type\":\"base\","
+                + "\"name\":\"name\",\"formatInfo\":{\"type\":\"string\"}},\"outputField\":{\"type\":"
+                + "\"base\",\"name\":\"name\",\"formatInfo\":{\"type\":\"string\"}}},{\"type\":"
+                + "\"fieldRelationShip\",\"inputField\":{\"type\":\"base\",\"name\":\"age\","
+                + "\"formatInfo\":{\"type\":\"int\"}},\"outputField\":{\"type\":\"base\","
+                + "\"name\":\"age\",\"formatInfo\":{\"type\":\"int\"}}},{\"type\":\"fieldRelationShip\","
+                + "\"inputField\":{\"type\":\"base\",\"name\":\"ts\",\"formatInfo\":{\"type\":"
+                + "\"timestamp\",\"format\":\"yyyy-MM-dd HH:mm:ss\"}},\"outputField\":{\"type\":"
+                + "\"base\",\"name\":\"ts\",\"formatInfo\":{\"type\":\"timestamp\",\"format\":"
+                + "\"yyyy-MM-dd HH:mm:ss\"}}}],\"sinkParallelism\":1,\"catalogName\":\"myHive\","
+                + "\"database\":\"default\",\"tableName\":\"test\",\"hiveConfDir\":\"/opt/hive-conf\","
+                + "\"hiveVersion\":\"3.1.2\",\"hadoopConfDir\":null,\"partitionFields\":[{\"type\":"
+                + "\"base\",\"name\":\"day\",\"formatInfo\":{\"type\":\"long\"}}]}],\"relations\":"
+                + "[{\"type\":\"baseRelation\",\"inputs\":[\"1\"],\"outputs\":[\"2\"]}]}";
+        assertEquals(expected, objectMapper.writeValueAsString(streamInfo));
+    }
+
+    @Test
+    public void testMysqlToHiveDeserialize() throws JsonProcessingException {
+        Node input = buildMySqlExtractNode();
+        Node output = buildHiveNode();
+        StreamInfo streamInfo = new StreamInfo("1", Arrays.asList(input, output), Collections.singletonList(
+                buildNodeRelation(Collections.singletonList(input), Collections.singletonList(output))));
+        ObjectMapper objectMapper = new ObjectMapper();
+        String streamInfoStr = "{\"streamId\":\"1\",\"nodes\":[{\"type\":\"mysqlExtract\","
+                + "\"id\":\"1\",\"name\":\"mysql_input\",\"fields\":[{\"type\":\"base\","
+                + "\"name\":\"id\",\"formatInfo\":{\"type\":\"long\"}},{\"type\":\"base\","
+                + "\"name\":\"name\",\"formatInfo\":{\"type\":\"string\"}},{\"type\":\""
+                + "base\",\"name\":\"age\",\"formatInfo\":{\"type\":\"int\"}},{\"type\":"
+                + "\"base\",\"name\":\"salary\",\"formatInfo\":{\"type\":\"float\"}},"
+                + "{\"type\":\"base\",\"name\":\"ts\",\"formatInfo\":{\"type\":\"timestamp\","
+                + "\"format\":\"yyyy-MM-dd HH:mm:ss\"}}],\"watermarkField\":{\"type\":\"watermark\","
+                + "\"timeAttr\":{\"type\":\"base\",\"name\":\"ts\",\"formatInfo\":{\"type\":"
+                + "\"timestamp\",\"format\":\"yyyy-MM-dd HH:mm:ss\"}},\"interval\":{\"type\":"
+                + "\"stringConstant\",\"value\":\"1\"},\"timeUnit\":{\"type\":\"timeUnitConstant\","
+                + "\"timeUnit\":\"MINUTE\",\"value\":\"MINUTE\"}},\"primaryKey\":\"id\","
+                + "\"tableNames\":[\"table\"],\"hostname\":\"localhost\",\"username\":\"username\","
+                + "\"password\":\"username\",\"database\":\"test_database\",\"port\":3306,\"serverId\":123,"
+                + "\"incrementalSnapshotEnabled\":true},{\"type\":\"hiveLoad\",\"id\":\"2\","
+                + "\"name\":\"hive_output\",\"fields\":[{\"type\":\"base\",\"name\":\"id\","
+                + "\"formatInfo\":{\"type\":\"long\"}},{\"type\":\"base\",\"name\":\"name\","
+                + "\"formatInfo\":{\"type\":\"string\"}},{\"type\":\"base\",\"name\":\"age\","
+                + "\"formatInfo\":{\"type\":\"int\"}},{\"type\":\"base\",\"name\":\"salary\","
+                + "\"formatInfo\":{\"type\":\"float\"}},{\"type\":\"base\",\"name\":\"ts\","
+                + "\"formatInfo\":{\"type\":\"timestamp\",\"format\":\"yyyy-MM-dd HH:mm:ss\"}}],"
+                + "\"fieldRelationShips\":[{\"type\":\"fieldRelationShip\",\"inputField\":{"
+                + "\"type\":\"base\",\"name\":\"id\",\"formatInfo\":{\"type\":\"long\"}},"
+                + "\"outputField\":{\"type\":\"base\",\"name\":\"id\",\"formatInfo\":{\"type\":"
+                + "\"long\"}}},{\"type\":\"fieldRelationShip\",\"inputField\":{\"type\":\"base\","
+                + "\"name\":\"name\",\"formatInfo\":{\"type\":\"string\"}},\"outputField\":{\"type\":"
+                + "\"base\",\"name\":\"name\",\"formatInfo\":{\"type\":\"string\"}}},{\"type\":"
+                + "\"fieldRelationShip\",\"inputField\":{\"type\":\"base\",\"name\":\"age\","
+                + "\"formatInfo\":{\"type\":\"int\"}},\"outputField\":{\"type\":\"base\","
+                + "\"name\":\"age\",\"formatInfo\":{\"type\":\"int\"}}},{\"type\":\"fieldRelationShip\","
+                + "\"inputField\":{\"type\":\"base\",\"name\":\"ts\",\"formatInfo\":{\"type\":"
+                + "\"timestamp\",\"format\":\"yyyy-MM-dd HH:mm:ss\"}},\"outputField\":{\"type\":"
+                + "\"base\",\"name\":\"ts\",\"formatInfo\":{\"type\":\"timestamp\",\"format\":"
+                + "\"yyyy-MM-dd HH:mm:ss\"}}}],\"sinkParallelism\":1,\"catalogName\":\"myHive\","
+                + "\"database\":\"default\",\"tableName\":\"test\",\"hiveConfDir\":\"/opt/hive-conf\","
+                + "\"hiveVersion\":\"3.1.2\",\"hadoopConfDir\":null,\"partitionFields\":[{\"type\":"
+                + "\"base\",\"name\":\"day\",\"formatInfo\":{\"type\":\"long\"}}]}],\"relations\":"
+                + "[{\"type\":\"baseRelation\",\"inputs\":[\"1\"],\"outputs\":[\"2\"]}]}";
+        StreamInfo expected = objectMapper.readValue(streamInfoStr, StreamInfo.class);
+        assertEquals(expected, streamInfo);
+    }
+
     /**
      * Test serialize for StreamInfo
      *
diff --git a/inlong-sort/sort-common/src/test/java/org/apache/inlong/sort/protocol/node/HiveLoadNodeTest.java b/inlong-sort/sort-common/src/test/java/org/apache/inlong/sort/protocol/node/HiveLoadNodeTest.java
new file mode 100644
index 000000000..a17014574
--- /dev/null
+++ b/inlong-sort/sort-common/src/test/java/org/apache/inlong/sort/protocol/node/HiveLoadNodeTest.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.protocol.node;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import org.apache.inlong.sort.formats.common.LongFormatInfo;
+import org.apache.inlong.sort.formats.common.StringFormatInfo;
+import org.apache.inlong.sort.protocol.FieldInfo;
+import org.apache.inlong.sort.protocol.node.load.HiveLoadNode;
+import org.apache.inlong.sort.protocol.transformation.FieldRelationShip;
+
+public class HiveLoadNodeTest extends NodeBaseTest {
+
+    @Override
+    public Node getNode() {
+        return new HiveLoadNode("1", "test_hive_node",
+                Arrays.asList(new FieldInfo("field", new StringFormatInfo())),
+                Arrays.asList(new FieldRelationShip(new FieldInfo("field", new StringFormatInfo()),
+                        new FieldInfo("field", new StringFormatInfo()))), null,
+                1, new HashMap<>(), "myHive", "default", "test", "/opt/hive-conf", "3.1.2",
+                null, Arrays.asList(new FieldInfo("day", new LongFormatInfo())));
+    }
+
+    @Override
+    public String getExpectSerializeStr() {
+        return "{\"type\":\"hiveLoad\",\"id\":\"1\","
+                + "\"name\":\"test_hive_node\",\"fields\":"
+                + "[{\"type\":\"base\",\"name\":\"field\","
+                + "\"formatInfo\":{\"type\":\"string\"}}],"
+                + "\"fieldRelationShips\":[{\"type\":"
+                + "\"fieldRelationShip\",\"inputField\":"
+                + "{\"type\":\"base\",\"name\":\"field\","
+                + "\"formatInfo\":{\"type\":\"string\"}},"
+                + "\"outputField\":{\"type\":\"base\",\"name\":"
+                + "\"field\",\"formatInfo\":{\"type\":\"string\"}}}],"
+                + "\"sinkParallelism\":1,\"properties\":{},\"catalogName\":"
+                + "\"myHive\",\"database\":\"default\",\"tableName\":\"test\","
+                + "\"hiveConfDir\":\"/opt/hive-conf\",\"hiveVersion\":\"3.1.2\","
+                + "\"hadoopConfDir\":null,\"partitionFields\":[{\"type\":\"base\","
+                + "\"name\":\"day\",\"formatInfo\":{\"type\":\"long\"}}]}";
+    }
+}
\ No newline at end of file
diff --git a/inlong-sort/sort-core/pom.xml b/inlong-sort/sort-core/pom.xml
index ac6a92a0e..951cd701b 100644
--- a/inlong-sort/sort-core/pom.xml
+++ b/inlong-sort/sort-core/pom.xml
@@ -168,6 +168,16 @@
         <dependency>
             <groupId>org.apache.hive</groupId>
             <artifactId>hive-exec</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.avro</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.calcite</groupId>
+                    <artifactId>calcite-core</artifactId>
+                </exclusion>
+            </exclusions>
         </dependency>
 
         <dependency>
diff --git a/inlong-sort/sort-single-tenant/pom.xml b/inlong-sort/sort-single-tenant/pom.xml
index a4e5ad628..8fa956e7b 100644
--- a/inlong-sort/sort-single-tenant/pom.xml
+++ b/inlong-sort/sort-single-tenant/pom.xml
@@ -18,8 +18,8 @@
     under the License.
 -->
 
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-        xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+        xmlns="http://maven.apache.org/POM/4.0.0"
         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <parent>
         <groupId>org.apache.inlong</groupId>
@@ -58,6 +58,10 @@
                     <groupId>org.apache.avro</groupId>
                     <artifactId>avro</artifactId>
                 </exclusion>
+                <exclusion>
+                    <artifactId>hive-storage-api</artifactId>
+                    <groupId>org.apache.hive</groupId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -84,10 +88,12 @@
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-streaming-java_${flink.scala.binary.version}</artifactId>
             <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-table-common</artifactId>
+            <exclusions>
+                <exclusion>
+                    <artifactId>scala-parser-combinators_2.11</artifactId>
+                    <groupId>org.scala-lang.modules</groupId>
+                </exclusion>
+            </exclusions>
         </dependency>
         <dependency>
             <groupId>org.apache.iceberg</groupId>
@@ -97,6 +103,10 @@
                     <groupId>org.apache.avro</groupId>
                     <artifactId>avro</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>org.apache.iceberg</groupId>
+                    <artifactId>iceberg-hive-metastore</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
 
@@ -112,6 +122,13 @@
             </exclusions>
         </dependency>
 
+        <dependency>
+            <groupId>xml-apis</groupId>
+            <artifactId>xml-apis</artifactId>
+            <version>1.4.01</version>
+        </dependency>
+
+        <!--flink-->
         <dependency>
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-connector-kafka_${flink.scala.binary.version}</artifactId>
@@ -119,14 +136,41 @@
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-avro</artifactId>
+            <artifactId>flink-connector-hive_${flink.scala.binary.version}</artifactId>
         </dependency>
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-csv</artifactId>
+            <artifactId>flink-table-api-java-bridge_${flink.scala.binary.version}</artifactId>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-json</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-avro</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-exec</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.calcite</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.calcite</groupId>
+                    <artifactId>calcite-druid</artifactId>
+                </exclusion>
+            </exclusions>
+            <scope>provided</scope>
+        </dependency>
+
+
         <dependency>
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-table-runtime-blink_${flink.scala.binary.version}</artifactId>
@@ -185,6 +229,64 @@
         <dependency>
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-table-planner-blink_${flink.scala.binary.version}</artifactId>
+            <exclusions>
+                <exclusion>
+                    <artifactId>scala-parser-combinators_2.11</artifactId>
+                    <groupId>org.scala-lang.modules</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.ververica</groupId>
+            <artifactId>flink-connector-debezium</artifactId>
+            <version>2.0.1</version>
+            <exclusions>
+                <exclusion>
+                    <artifactId>kafka-log4j-appender</artifactId>
+                    <groupId>org.apache.kafka</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.zaxxer</groupId>
+            <artifactId>HikariCP</artifactId>
+            <version>4.0.3</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>io.debezium</groupId>
+            <artifactId>debezium-connector-mysql</artifactId>
+            <version>1.5.4.Final</version>
+        </dependency>
+        <dependency>
+            <groupId>io.debezium</groupId>
+            <artifactId>debezium-core</artifactId>
+            <version>1.5.4.Final</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.esri.geometry</groupId>
+            <artifactId>esri-geometry-api</artifactId>
+            <version>2.0.0</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-core</artifactId>
+            <version>2.10.1</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-core</artifactId>
+            <version>2.10.1</version>
+            <scope>compile</scope>
         </dependency>
 
         <dependency>
@@ -210,10 +312,6 @@
             <artifactId>debezium-core</artifactId>
             <version>${debezium-core.version}</version>
         </dependency>
-        <dependency>
-            <groupId>com.zaxxer</groupId>
-            <artifactId>HikariCP</artifactId>
-        </dependency>
 
     </dependencies>
 
@@ -249,11 +347,13 @@
                             <relocations>
                                 <relocation>
                                     <pattern>org.apache.flink.formats.avro</pattern>
-                                    <shadedPattern>org.apache.inlong.shaded.flink.formats.avro</shadedPattern>
+                                    <shadedPattern>org.apache.inlong.shaded.flink.formats.avro
+                                    </shadedPattern>
                                 </relocation>
                             </relocations>
                             <transformers>
-                                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
+                                <transformer
+                                        implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
                             </transformers>
                         </configuration>
                     </execution>
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/DebeziumDeserializationSchema.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/DebeziumDeserializationSchema.java
index c47d2220a..93f9562f2 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/DebeziumDeserializationSchema.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/DebeziumDeserializationSchema.java
@@ -18,10 +18,9 @@
 
 package org.apache.inlong.sort.singletenant.flink.cdc.debezium;
 
-import io.debezium.relational.history.TableChanges.TableChange;
+import io.debezium.relational.history.TableChanges;
 import java.io.Serializable;
 import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.util.Collector;
 import org.apache.kafka.connect.source.SourceRecord;
@@ -33,14 +32,21 @@ import org.apache.kafka.connect.source.SourceRecord;
  * @param <T> The type created by the deserialization schema.
  */
 @PublicEvolving
-public interface DebeziumDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T>,
-    com.ververica.cdc.debezium.DebeziumDeserializationSchema<T> {
+public interface DebeziumDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
 
-    void deserialize(SourceRecord record, Collector<T> out, TableChange tableChange) throws Exception;
+    /**
+     * Deserialize the Debezium record, it is represented in Kafka {@link SourceRecord}.
+     */
+    void deserialize(SourceRecord record, Collector<T> out) throws Exception;
 
-    @Override
-    void deserialize(SourceRecord sourceRecord, Collector<T> collector) throws Exception;
+    /**
+     * Deserialize the Debezium record with tableSchema, it is represented in Kafka {@link
+     * SourceRecord}.
+     */
+    default void deserialize(
+            SourceRecord record, Collector<T> out, TableChanges.TableChange tableSchema)
+            throws Exception {
+        deserialize(record, out);
+    }
 
-    @Override
-    TypeInformation<T> getProducedType();
 }
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/DebeziumSourceFunction.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/DebeziumSourceFunction.java
index 60ef2bc4f..82d5f5871 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/DebeziumSourceFunction.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/DebeziumSourceFunction.java
@@ -21,15 +21,6 @@ package org.apache.inlong.sort.singletenant.flink.cdc.debezium;
 import static org.apache.inlong.sort.singletenant.flink.cdc.debezium.utils.DatabaseHistoryUtil.registerHistory;
 import static org.apache.inlong.sort.singletenant.flink.cdc.debezium.utils.DatabaseHistoryUtil.retrieveHistory;
 
-import com.ververica.cdc.debezium.internal.DebeziumChangeConsumer;
-import com.ververica.cdc.debezium.internal.DebeziumChangeFetcher;
-import com.ververica.cdc.debezium.internal.DebeziumOffset;
-import com.ververica.cdc.debezium.internal.DebeziumOffsetSerializer;
-import com.ververica.cdc.debezium.internal.FlinkDatabaseHistory;
-import com.ververica.cdc.debezium.internal.FlinkDatabaseSchemaHistory;
-import com.ververica.cdc.debezium.internal.FlinkOffsetBackingStore;
-import com.ververica.cdc.debezium.internal.Handover;
-import com.ververica.cdc.debezium.internal.SchemaRecord;
 import io.debezium.document.DocumentReader;
 import io.debezium.document.DocumentWriter;
 import io.debezium.embedded.Connect;
@@ -69,6 +60,15 @@ import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal.DebeziumChangeConsumer;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal.DebeziumChangeFetcher;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal.DebeziumOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal.DebeziumOffsetSerializer;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal.FlinkDatabaseHistory;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal.FlinkDatabaseSchemaHistory;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal.FlinkOffsetBackingStore;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal.Handover;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal.SchemaRecord;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -165,6 +165,12 @@ public class DebeziumSourceFunction<T> extends RichSourceFunction<T>
     /** Accessor for state in the operator state backend. */
     private transient ListState<byte[]> offsetState;
 
+    /**
+     * State to store the history records, i.e. schema changes.
+     *
+     * @see FlinkDatabaseHistory
+     * @see FlinkDatabaseSchemaHistory
+     */
     private transient ListState<String> schemaRecordsState;
 
     // ---------------------------------------------------------------------------------------
@@ -173,7 +179,10 @@ public class DebeziumSourceFunction<T> extends RichSourceFunction<T>
 
     private transient ExecutorService executor;
     private transient DebeziumEngine<?> engine;
-
+    /**
+     * Unique name of this Debezium Engine instance across all the jobs. Currently we randomly
+     * generate a UUID for it. This is used for {@link FlinkDatabaseHistory}.
+     */
     private transient String engineInstanceName;
 
     /** Consume the events from the engine and commit the offset to the engine. */
@@ -384,9 +393,9 @@ public class DebeziumSourceFunction<T> extends RichSourceFunction<T>
                         Heartbeat.HEARTBEAT_TOPICS_PREFIX.name(),
                         Heartbeat.HEARTBEAT_TOPICS_PREFIX.defaultValueAsString());
         this.debeziumChangeFetcher =
-                new DebeziumChangeFetcher<T>(
+                new DebeziumChangeFetcher<>(
                         sourceContext,
-                    deserializer,
+                        deserializer,
                         restoredOffsetState == null, // DB snapshot phase if restore state is null
                         dbzHeartbeatPrefix,
                         handover);
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/JsonDebeziumDeserializationSchema.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/JsonDebeziumDeserializationSchema.java
new file mode 100644
index 000000000..1539a3570
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/JsonDebeziumDeserializationSchema.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.debezium;
+
+import io.debezium.relational.history.TableChanges.TableChange;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.util.Collector;
+import org.apache.kafka.connect.json.JsonConverter;
+import org.apache.kafka.connect.json.JsonConverterConfig;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.apache.kafka.connect.storage.ConverterConfig;
+import org.apache.kafka.connect.storage.ConverterType;
+
+/**
+ * A JSON format implementation of {@link DebeziumDeserializationSchema} which deserializes the
+ * received {@link SourceRecord} to JSON String.
+ */
+public class JsonDebeziumDeserializationSchema implements DebeziumDeserializationSchema<String> {
+
+    private static final long serialVersionUID = 1L;
+
+    private transient JsonConverter jsonConverter;
+
+    /**
+     * Configuration whether to enable {@link JsonConverterConfig#SCHEMAS_ENABLE_CONFIG} to include
+     * schema in messages.
+     */
+    private final Boolean includeSchema;
+
+    /** The custom configurations for {@link JsonConverter}. */
+    private Map<String, Object> customConverterConfigs;
+
+    public JsonDebeziumDeserializationSchema() {
+        this(false);
+    }
+
+    public JsonDebeziumDeserializationSchema(Boolean includeSchema) {
+        this.includeSchema = includeSchema;
+    }
+
+    public JsonDebeziumDeserializationSchema(
+            Boolean includeSchema, Map<String, Object> customConverterConfigs) {
+        this.includeSchema = includeSchema;
+        this.customConverterConfigs = customConverterConfigs;
+    }
+
+    @Override
+    public void deserialize(SourceRecord record, Collector<String> out) throws Exception {
+        if (jsonConverter == null) {
+            initializeJsonConverter();
+        }
+        byte[] bytes =
+                jsonConverter.fromConnectData(record.topic(), record.valueSchema(), record.value());
+        out.collect(new String(bytes));
+    }
+
+    @Override
+    public void deserialize(SourceRecord record, Collector<String> out, TableChange tableChange)
+        throws Exception {
+
+    }
+
+    /** Initialize {@link JsonConverter} with given configs. */
+    private void initializeJsonConverter() {
+        jsonConverter = new JsonConverter();
+        final HashMap<String, Object> configs = new HashMap<>(2);
+        configs.put(ConverterConfig.TYPE_CONFIG, ConverterType.VALUE.getName());
+        configs.put(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, includeSchema);
+        if (customConverterConfigs != null) {
+            configs.putAll(customConverterConfigs);
+        }
+        jsonConverter.configure(configs);
+    }
+
+    @Override
+    public TypeInformation<String> getProducedType() {
+        return BasicTypeInfo.STRING_TYPE_INFO;
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/DebeziumDeserializationSchema.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/StringDebeziumDeserializationSchema.java
similarity index 59%
copy from inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/DebeziumDeserializationSchema.java
copy to inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/StringDebeziumDeserializationSchema.java
index c47d2220a..1b7931abd 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/DebeziumDeserializationSchema.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/StringDebeziumDeserializationSchema.java
@@ -19,28 +19,31 @@
 package org.apache.inlong.sort.singletenant.flink.cdc.debezium;
 
 import io.debezium.relational.history.TableChanges.TableChange;
-import java.io.Serializable;
-import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.util.Collector;
 import org.apache.kafka.connect.source.SourceRecord;
 
 /**
- * The deserialization schema describes how to turn the Debezium SourceRecord into data types
- * (Java/Scala objects) that are processed by Flink.
- *
- * @param <T> The type created by the deserialization schema.
+ * A simple implementation of {@link DebeziumDeserializationSchema} which converts the received
+ * {@link SourceRecord} into String.
  */
-@PublicEvolving
-public interface DebeziumDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T>,
-    com.ververica.cdc.debezium.DebeziumDeserializationSchema<T> {
+public class StringDebeziumDeserializationSchema implements DebeziumDeserializationSchema<String> {
+    private static final long serialVersionUID = -3168848963265670603L;
 
-    void deserialize(SourceRecord record, Collector<T> out, TableChange tableChange) throws Exception;
+    @Override
+    public void deserialize(SourceRecord record, Collector<String> out) throws Exception {
+        out.collect(record.toString());
+    }
 
     @Override
-    void deserialize(SourceRecord sourceRecord, Collector<T> collector) throws Exception;
+    public void deserialize(SourceRecord record, Collector<String> out, TableChange tableChange)
+        throws Exception {
+
+    }
 
     @Override
-    TypeInformation<T> getProducedType();
+    public TypeInformation<String> getProducedType() {
+        return BasicTypeInfo.STRING_TYPE_INFO;
+    }
 }
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/history/FlinkJsonTableChangeSerializer.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/history/FlinkJsonTableChangeSerializer.java
new file mode 100644
index 000000000..856d7fa83
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/history/FlinkJsonTableChangeSerializer.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.debezium.history;
+
+import io.debezium.document.Array;
+import io.debezium.document.Array.Entry;
+import io.debezium.document.Document;
+import io.debezium.document.Value;
+import io.debezium.relational.Column;
+import io.debezium.relational.ColumnEditor;
+import io.debezium.relational.Table;
+import io.debezium.relational.TableEditor;
+import io.debezium.relational.TableId;
+import io.debezium.relational.history.TableChanges;
+import io.debezium.relational.history.TableChanges.TableChange;
+import io.debezium.relational.history.TableChanges.TableChangeType;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/**
+ * The serializer responsible for converting of {@link TableChanges} into a JSON format. Copied from
+ * io.debezium.relational.history.JsonTableChangeSerializer, but add serialization/deserialization
+ * for column's enumValues
+ */
+public class FlinkJsonTableChangeSerializer implements TableChanges.TableChangesSerializer<Array> {
+
+    @Override
+    public Array serialize(TableChanges tableChanges) {
+        List<Value> values =
+                StreamSupport.stream(tableChanges.spliterator(), false)
+                        .map(this::toDocument)
+                        .map(Value::create)
+                        .collect(Collectors.toList());
+
+        return Array.create(values);
+    }
+
+    public Document toDocument(TableChange tableChange) {
+        Document document = Document.create();
+
+        document.setString("type", tableChange.getType().name());
+        document.setString("id", tableChange.getId().toDoubleQuotedString());
+        document.setDocument("table", toDocument(tableChange.getTable()));
+        return document;
+    }
+
+    private Document toDocument(Table table) {
+        Document document = Document.create();
+
+        document.set("defaultCharsetName", table.defaultCharsetName());
+        document.set("primaryKeyColumnNames", Array.create(table.primaryKeyColumnNames()));
+
+        List<Document> columns =
+                table.columns().stream().map(this::toDocument).collect(Collectors.toList());
+
+        document.setArray("columns", Array.create(columns));
+
+        return document;
+    }
+
+    private Document toDocument(Column column) {
+        Document document = Document.create();
+
+        document.setString("name", column.name());
+        document.setNumber("jdbcType", column.jdbcType());
+
+        if (column.nativeType() != Column.UNSET_INT_VALUE) {
+            document.setNumber("nativeType", column.nativeType());
+        }
+
+        document.setString("typeName", column.typeName());
+        document.setString("typeExpression", column.typeExpression());
+        document.setString("charsetName", column.charsetName());
+
+        if (column.length() != Column.UNSET_INT_VALUE) {
+            document.setNumber("length", column.length());
+        }
+
+        column.scale().ifPresent(s -> document.setNumber("scale", s));
+
+        document.setNumber("position", column.position());
+        document.setBoolean("optional", column.isOptional());
+        document.setBoolean("autoIncremented", column.isAutoIncremented());
+        document.setBoolean("generated", column.isGenerated());
+
+        // BEGIN FLINK MODIFICATION
+        document.setArray("enumValues", column.enumValues().toArray());
+        // END FLINK MODIFICATION
+
+        return document;
+    }
+
+    @Override
+    public TableChanges deserialize(Array array, boolean useCatalogBeforeSchema) {
+        TableChanges tableChanges = new TableChanges();
+
+        for (Entry entry : array) {
+            TableChange change =
+                    fromDocument(entry.getValue().asDocument(), useCatalogBeforeSchema);
+
+            if (change.getType() == TableChangeType.CREATE) {
+                tableChanges.create(change.getTable());
+            } else if (change.getType() == TableChangeType.ALTER) {
+                tableChanges.alter(change.getTable());
+            } else if (change.getType() == TableChangeType.DROP) {
+                tableChanges.drop(change.getTable());
+            }
+        }
+
+        return tableChanges;
+    }
+
+    private static Table fromDocument(TableId id, Document document) {
+        TableEditor editor =
+                Table.editor()
+                        .tableId(id)
+                        .setDefaultCharsetName(document.getString("defaultCharsetName"));
+
+        document.getArray("columns")
+                .streamValues()
+                .map(Value::asDocument)
+                .map(
+                        v -> {
+                            ColumnEditor columnEditor =
+                                    Column.editor()
+                                            .name(v.getString("name"))
+                                            .jdbcType(v.getInteger("jdbcType"));
+
+                            Integer nativeType = v.getInteger("nativeType");
+                            if (nativeType != null) {
+                                columnEditor.nativeType(nativeType);
+                            }
+
+                            columnEditor
+                                    .type(v.getString("typeName"), v.getString("typeExpression"))
+                                    .charsetName(v.getString("charsetName"));
+
+                            Integer length = v.getInteger("length");
+                            if (length != null) {
+                                columnEditor.length(length);
+                            }
+
+                            Integer scale = v.getInteger("scale");
+                            if (scale != null) {
+                                columnEditor.scale(scale);
+                            }
+
+                            columnEditor
+                                    .position(v.getInteger("position"))
+                                    .optional(v.getBoolean("optional"))
+                                    .autoIncremented(v.getBoolean("autoIncremented"))
+                                    .generated(v.getBoolean("generated"));
+
+                            // BEGIN FLINK MODIFICATION
+                            Array enumValues = v.getArray("enumValues");
+                            if (enumValues != null && !enumValues.isEmpty()) {
+                                columnEditor.enumValues(
+                                        enumValues
+                                                .streamValues()
+                                                .map(Value::asString)
+                                                .collect(Collectors.toList()));
+                            }
+                            // END FLINK MODIFICATION
+
+                            return columnEditor.create();
+                        })
+                .forEach(editor::addColumn);
+
+        editor.setPrimaryKeyNames(
+                document.getArray("primaryKeyColumnNames")
+                        .streamValues()
+                        .map(Value::asString)
+                        .collect(Collectors.toList()));
+
+        return editor.create();
+    }
+
+    public static TableChange fromDocument(Document document, boolean useCatalogBeforeSchema) {
+        TableChangeType type = TableChangeType.valueOf(document.getString("type"));
+        TableId id = TableId.parse(document.getString("id"), useCatalogBeforeSchema);
+        Table table = null;
+
+        if (type == TableChangeType.CREATE || type == TableChangeType.ALTER) {
+            table = fromDocument(id, document.getDocument("table"));
+        } else {
+            table = Table.editor().tableId(id).create();
+        }
+        return new TableChange(type, table);
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/DebeziumChangeConsumer.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/DebeziumChangeConsumer.java
new file mode 100644
index 000000000..bb31eadec
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/DebeziumChangeConsumer.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal;
+
+import io.debezium.embedded.EmbeddedEngineChangeEvent;
+import io.debezium.engine.ChangeEvent;
+import io.debezium.engine.DebeziumEngine;
+import io.debezium.engine.DebeziumEngine.RecordCommitter;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Consume debezium change events. */
+@Internal
+public class DebeziumChangeConsumer
+        implements DebeziumEngine.ChangeConsumer<ChangeEvent<SourceRecord, SourceRecord>> {
+    public static final String LAST_COMPLETELY_PROCESSED_LSN_KEY = "lsn_proc";
+    public static final String LAST_COMMIT_LSN_KEY = "lsn_commit";
+    private static final Logger LOG = LoggerFactory.getLogger(DebeziumChangeConsumer.class);
+
+    private final Handover handover;
+    // keep the modification is visible to the source function
+    private volatile RecordCommitter<ChangeEvent<SourceRecord, SourceRecord>> currentCommitter;
+
+    public DebeziumChangeConsumer(Handover handover) {
+        this.handover = handover;
+    }
+
+    @Override
+    public void handleBatch(
+            List<ChangeEvent<SourceRecord, SourceRecord>> events,
+            RecordCommitter<ChangeEvent<SourceRecord, SourceRecord>> recordCommitter) {
+        try {
+            currentCommitter = recordCommitter;
+            handover.produce(events);
+        } catch (Throwable e) {
+            // Hold this exception in handover and trigger the fetcher to exit
+            handover.reportError(e);
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public void commitOffset(DebeziumOffset offset) throws InterruptedException {
+        // Although the committer is read/write by multi-thread, the committer will be not changed
+        // frequently.
+        if (currentCommitter == null) {
+            LOG.info(
+                    "commitOffset() called on Debezium change consumer which doesn't receive records yet.");
+            return;
+        }
+
+        // only the offset is used
+        SourceRecord recordWrapper =
+                new SourceRecord(
+                        offset.sourcePartition,
+                        adjustSourceOffset((Map<String, Object>) offset.sourceOffset),
+                        "DUMMY",
+                        Schema.BOOLEAN_SCHEMA,
+                        true);
+        EmbeddedEngineChangeEvent<SourceRecord, SourceRecord> changeEvent =
+                new EmbeddedEngineChangeEvent<>(null, recordWrapper, recordWrapper);
+        currentCommitter.markProcessed(changeEvent);
+        currentCommitter.markBatchFinished();
+    }
+
+    /**
+     * We have to adjust type of LSN values to Long, because it might be Integer after
+     * deserialization, however {@code
+     * io.debezium.connector.postgresql.PostgresStreamingChangeEventSource#commitOffset(java.util.Map)}
+     * requires Long.
+     */
+    private Map<String, Object> adjustSourceOffset(Map<String, Object> sourceOffset) {
+        if (sourceOffset.containsKey(LAST_COMPLETELY_PROCESSED_LSN_KEY)) {
+            String value = sourceOffset.get(LAST_COMPLETELY_PROCESSED_LSN_KEY).toString();
+            sourceOffset.put(LAST_COMPLETELY_PROCESSED_LSN_KEY, Long.parseLong(value));
+        }
+        if (sourceOffset.containsKey(LAST_COMMIT_LSN_KEY)) {
+            String value = sourceOffset.get(LAST_COMMIT_LSN_KEY).toString();
+            sourceOffset.put(LAST_COMMIT_LSN_KEY, Long.parseLong(value));
+        }
+        return sourceOffset;
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/DebeziumChangeFetcher.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/DebeziumChangeFetcher.java
new file mode 100644
index 000000000..cde0fbe38
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/DebeziumChangeFetcher.java
@@ -0,0 +1,309 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal;
+
+import io.debezium.connector.SnapshotRecord;
+import io.debezium.data.Envelope;
+import io.debezium.engine.ChangeEvent;
+import io.debezium.engine.DebeziumEngine;
+import java.util.ArrayDeque;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.util.Collector;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumDeserializationSchema;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.data.Struct;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A Handler that convert change messages from {@link DebeziumEngine} to data in Flink. Considering
+ * Debezium in different mode has different strategies to hold the lock, e.g. snapshot, the handler
+ * also needs different strategy. In snapshot phase, the handler needs to hold the lock until the
+ * snapshot finishes. But in non-snapshot phase, the handler only needs to hold the lock when
+ * emitting the records.
+ *
+ * @param <T> The type of elements produced by the handler.
+ */
+@Internal
+public class DebeziumChangeFetcher<T> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(DebeziumChangeFetcher.class);
+
+    private final SourceFunction.SourceContext<T> sourceContext;
+
+    /**
+     * The lock that guarantees that record emission and state updates are atomic, from the view of
+     * taking a checkpoint.
+     */
+    private final Object checkpointLock;
+
+    /** The schema to convert from Debezium's messages into Flink's objects. */
+    private final DebeziumDeserializationSchema<T> deserialization;
+
+    /** A collector to emit records in batch (bundle). */
+    private final DebeziumCollector debeziumCollector;
+
+    private final DebeziumOffset debeziumOffset;
+
+    private final DebeziumOffsetSerializer stateSerializer;
+
+    private final String heartbeatTopicPrefix;
+
+    private boolean isInDbSnapshotPhase;
+
+    private final Handover handover;
+
+    private volatile boolean isRunning = true;
+
+    // ---------------------------------------------------------------------------------------
+    // Metrics
+    // ---------------------------------------------------------------------------------------
+
+    /** Timestamp of change event. If the event is a snapshot event, the timestamp is 0L. */
+    private volatile long messageTimestamp = 0L;
+
+    /** The last record processing time. */
+    private volatile long processTime = 0L;
+
+    /**
+     * currentFetchEventTimeLag = FetchTime - messageTimestamp, where the FetchTime is the time the
+     * record fetched into the source operator.
+     */
+    private volatile long fetchDelay = 0L;
+
+    /**
+     * emitDelay = EmitTime - messageTimestamp, where the EmitTime is the time the record leaves the
+     * source operator.
+     */
+    private volatile long emitDelay = 0L;
+
+    // ------------------------------------------------------------------------
+
+    public DebeziumChangeFetcher(
+            SourceFunction.SourceContext<T> sourceContext,
+            DebeziumDeserializationSchema<T> deserialization,
+            boolean isInDbSnapshotPhase,
+            String heartbeatTopicPrefix,
+            Handover handover) {
+        this.sourceContext = sourceContext;
+        this.checkpointLock = sourceContext.getCheckpointLock();
+        this.deserialization = deserialization;
+        this.isInDbSnapshotPhase = isInDbSnapshotPhase;
+        this.heartbeatTopicPrefix = heartbeatTopicPrefix;
+        this.debeziumCollector = new DebeziumCollector();
+        this.debeziumOffset = new DebeziumOffset();
+        this.stateSerializer = DebeziumOffsetSerializer.INSTANCE;
+        this.handover = handover;
+    }
+
+    /**
+     * Take a snapshot of the Debezium handler state.
+     *
+     * <p>Important: This method must be called under the checkpoint lock.
+     */
+    public byte[] snapshotCurrentState() throws Exception {
+        // this method assumes that the checkpoint lock is held
+        assert Thread.holdsLock(checkpointLock);
+        if (debeziumOffset.sourceOffset == null || debeziumOffset.sourcePartition == null) {
+            return null;
+        }
+
+        return stateSerializer.serialize(debeziumOffset);
+    }
+
+    /**
+     * Process change messages from the {@link Handover} and collect the processed messages by
+     * {@link Collector}.
+     */
+    public void runFetchLoop() throws Exception {
+        try {
+            // begin snapshot database phase
+            if (isInDbSnapshotPhase) {
+                List<ChangeEvent<SourceRecord, SourceRecord>> events = handover.pollNext();
+
+                synchronized (checkpointLock) {
+                    LOG.info(
+                            "Database snapshot phase can't perform checkpoint, acquired Checkpoint lock.");
+                    handleBatch(events);
+                    while (isRunning && isInDbSnapshotPhase) {
+                        handleBatch(handover.pollNext());
+                    }
+                }
+                LOG.info("Received record from streaming binlog phase, released checkpoint lock.");
+            }
+
+            // begin streaming binlog phase
+            while (isRunning) {
+                // If the handover is closed or has errors, exit.
+                // If there is no streaming phase, the handover will be closed by the engine.
+                handleBatch(handover.pollNext());
+            }
+        } catch (Handover.ClosedException e) {
+            // ignore
+        }
+    }
+
+    public void close() {
+        isRunning = false;
+        handover.close();
+    }
+
+    // ---------------------------------------------------------------------------------------
+    // Metric getter
+    // ---------------------------------------------------------------------------------------
+
+    /**
+     * The metric indicates delay from data generation to entry into the system.
+     *
+     * <p>Note: the metric is available during the binlog phase. Use 0 to indicate the metric is
+     * unavailable.
+     */
+    public long getFetchDelay() {
+        return fetchDelay;
+    }
+
+    /**
+     * The metric indicates delay from data generation to leaving the source operator.
+     *
+     * <p>Note: the metric is available during the binlog phase. Use 0 to indicate the metric is
+     * unavailable.
+     */
+    public long getEmitDelay() {
+        return emitDelay;
+    }
+
+    public long getIdleTime() {
+        return System.currentTimeMillis() - processTime;
+    }
+
+    // ---------------------------------------------------------------------------------------
+    // Helper
+    // ---------------------------------------------------------------------------------------
+
+    private void handleBatch(List<ChangeEvent<SourceRecord, SourceRecord>> changeEvents)
+            throws Exception {
+        if (CollectionUtils.isEmpty(changeEvents)) {
+            return;
+        }
+        this.processTime = System.currentTimeMillis();
+
+        for (ChangeEvent<SourceRecord, SourceRecord> event : changeEvents) {
+            SourceRecord record = event.value();
+            updateMessageTimestamp(record);
+            fetchDelay = isInDbSnapshotPhase ? 0L : processTime - messageTimestamp;
+
+            if (isHeartbeatEvent(record)) {
+                // keep offset update
+                synchronized (checkpointLock) {
+                    debeziumOffset.setSourcePartition(record.sourcePartition());
+                    debeziumOffset.setSourceOffset(record.sourceOffset());
+                }
+                // drop heartbeat events
+                continue;
+            }
+
+            deserialization.deserialize(record, debeziumCollector);
+
+            if (!isSnapshotRecord(record)) {
+                LOG.debug("Snapshot phase finishes.");
+                isInDbSnapshotPhase = false;
+            }
+
+            // emit the actual records. this also updates offset state atomically
+            emitRecordsUnderCheckpointLock(
+                    debeziumCollector.records, record.sourcePartition(), record.sourceOffset());
+        }
+    }
+
+    private void emitRecordsUnderCheckpointLock(
+            Queue<T> records, Map<String, ?> sourcePartition, Map<String, ?> sourceOffset) {
+        // Emit the records. Use the checkpoint lock to guarantee
+        // atomicity of record emission and offset state update.
+        // The synchronized checkpointLock is reentrant. It's safe to sync again in snapshot mode.
+        synchronized (checkpointLock) {
+            T record;
+            while ((record = records.poll()) != null) {
+                emitDelay =
+                        isInDbSnapshotPhase ? 0L : System.currentTimeMillis() - messageTimestamp;
+                sourceContext.collect(record);
+            }
+            // update offset to state
+            debeziumOffset.setSourcePartition(sourcePartition);
+            debeziumOffset.setSourceOffset(sourceOffset);
+        }
+    }
+
+    private void updateMessageTimestamp(SourceRecord record) {
+        Schema schema = record.valueSchema();
+        Struct value = (Struct) record.value();
+        if (schema.field(Envelope.FieldName.SOURCE) == null) {
+            return;
+        }
+
+        Struct source = value.getStruct(Envelope.FieldName.SOURCE);
+        if (source.schema().field(Envelope.FieldName.TIMESTAMP) == null) {
+            return;
+        }
+
+        Long tsMs = source.getInt64(Envelope.FieldName.TIMESTAMP);
+        if (tsMs != null) {
+            this.messageTimestamp = tsMs;
+        }
+    }
+
+    private boolean isHeartbeatEvent(SourceRecord record) {
+        String topic = record.topic();
+        return topic != null && topic.startsWith(heartbeatTopicPrefix);
+    }
+
+    private boolean isSnapshotRecord(SourceRecord record) {
+        Struct value = (Struct) record.value();
+        if (value != null) {
+            Struct source = value.getStruct(Envelope.FieldName.SOURCE);
+            SnapshotRecord snapshotRecord = SnapshotRecord.fromSource(source);
+            // even if it is the last record of snapshot, i.e. SnapshotRecord.LAST
+            // we can still recover from checkpoint and continue to read the binlog,
+            // because the checkpoint contains binlog position
+            return SnapshotRecord.TRUE == snapshotRecord;
+        }
+        return false;
+    }
+
+    // ---------------------------------------------------------------------------------------
+
+    private class DebeziumCollector implements Collector<T> {
+
+        private final Queue<T> records = new ArrayDeque<>();
+
+        @Override
+        public void collect(T record) {
+            records.add(record);
+        }
+
+        @Override
+        public void close() {
+        }
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/DebeziumOffset.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/DebeziumOffset.java
new file mode 100644
index 000000000..a51aad8d4
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/DebeziumOffset.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal;
+
+import java.io.Serializable;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+
+/**
+ * The state that the Flink Debezium Consumer holds for each instance.
+ *
+ * <p>This class describes the most basic state that Debezium used for recovering based on Kafka
+ * Connect mechanism. It includes a sourcePartition and sourceOffset.
+ *
+ * <p>The sourcePartition represents a single input sourcePartition that the record came from (e.g.
+ * a filename, table name, or topic-partition). The sourceOffset represents a position in that
+ * sourcePartition which can be used to resume consumption of data.
+ *
+ * <p>These values can have arbitrary structure and should be represented using
+ * org.apache.kafka.connect.data objects (or primitive values). For example, a database connector
+ * might specify the sourcePartition as a record containing { "db": "database_name", "table":
+ * "table_name"} and the sourceOffset as a Long containing the timestamp of the row.
+ */
+@Internal
+public class DebeziumOffset implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    public Map<String, ?> sourcePartition;
+    public Map<String, ?> sourceOffset;
+
+    public void setSourcePartition(Map<String, ?> sourcePartition) {
+        this.sourcePartition = sourcePartition;
+    }
+
+    public void setSourceOffset(Map<String, ?> sourceOffset) {
+        this.sourceOffset = sourceOffset;
+    }
+
+    @Override
+    public String toString() {
+        return "DebeziumOffset{"
+                + "sourcePartition="
+                + sourcePartition
+                + ", sourceOffset="
+                + sourceOffset
+                + '}';
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/DebeziumOffsetSerializer.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/DebeziumOffsetSerializer.java
new file mode 100644
index 000000000..9f193a8d9
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/DebeziumOffsetSerializer.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal;
+
+import java.io.IOException;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+/** Serializer implementation for a {@link DebeziumOffset}. */
+@Internal
+public class DebeziumOffsetSerializer {
+    public static final DebeziumOffsetSerializer INSTANCE = new DebeziumOffsetSerializer();
+
+    public byte[] serialize(DebeziumOffset debeziumOffset) throws IOException {
+        // we currently use JSON serialization for simplification, as the state is very small.
+        // we can improve this in the future if needed
+        ObjectMapper objectMapper = new ObjectMapper();
+        return objectMapper.writeValueAsBytes(debeziumOffset);
+    }
+
+    public DebeziumOffset deserialize(byte[] bytes) throws IOException {
+        ObjectMapper objectMapper = new ObjectMapper();
+        return objectMapper.readValue(bytes, DebeziumOffset.class);
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/FlinkDatabaseHistory.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/FlinkDatabaseHistory.java
new file mode 100644
index 000000000..77315db48
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/FlinkDatabaseHistory.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.debezium.utils.DatabaseHistoryUtil.registerHistory;
+import static org.apache.inlong.sort.singletenant.flink.cdc.debezium.utils.DatabaseHistoryUtil.removeHistory;
+import static org.apache.inlong.sort.singletenant.flink.cdc.debezium.utils.DatabaseHistoryUtil.retrieveHistory;
+
+import io.debezium.config.Configuration;
+import io.debezium.relational.history.AbstractDatabaseHistory;
+import io.debezium.relational.history.DatabaseHistoryException;
+import io.debezium.relational.history.DatabaseHistoryListener;
+import io.debezium.relational.history.HistoryRecord;
+import io.debezium.relational.history.HistoryRecordComparator;
+import java.util.Collection;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.function.Consumer;
+
+/**
+ * Inspired from {@link io.debezium.relational.history.MemoryDatabaseHistory} but we will store the
+ * HistoryRecords in Flink's state for persistence.
+ *
+ * <p>Note: This is not a clean solution because we depends on a global variable and all the history
+ * records will be stored in state (grow infinitely). We may need to come up with a
+ * FileSystemDatabaseHistory in the future to store history in HDFS.
+ */
+public class FlinkDatabaseHistory extends AbstractDatabaseHistory {
+
+    public static final String DATABASE_HISTORY_INSTANCE_NAME = "database.history.instance.name";
+
+    private ConcurrentLinkedQueue<SchemaRecord> schemaRecords;
+    private String instanceName;
+
+    /** Gets the registered HistoryRecords under the given instance name. */
+    private ConcurrentLinkedQueue<SchemaRecord> getRegisteredHistoryRecord(String instanceName) {
+        Collection<SchemaRecord> historyRecords = retrieveHistory(instanceName);
+        return new ConcurrentLinkedQueue<>(historyRecords);
+    }
+
+    @Override
+    public void configure(
+            Configuration config,
+            HistoryRecordComparator comparator,
+            DatabaseHistoryListener listener,
+            boolean useCatalogBeforeSchema) {
+        super.configure(config, comparator, listener, useCatalogBeforeSchema);
+        this.instanceName = config.getString(DATABASE_HISTORY_INSTANCE_NAME);
+        this.schemaRecords = getRegisteredHistoryRecord(instanceName);
+
+        // register the schema changes into state
+        // every change should be visible to the source function
+        registerHistory(instanceName, schemaRecords);
+    }
+
+    @Override
+    public void stop() {
+        super.stop();
+        removeHistory(instanceName);
+    }
+
+    @Override
+    protected void storeRecord(HistoryRecord record) throws DatabaseHistoryException {
+        this.schemaRecords.add(new SchemaRecord(record));
+    }
+
+    @Override
+    protected void recoverRecords(Consumer<HistoryRecord> records) {
+        this.schemaRecords.stream().map(SchemaRecord::getHistoryRecord).forEach(records);
+    }
+
+    @Override
+    public boolean exists() {
+        return !schemaRecords.isEmpty();
+    }
+
+    @Override
+    public boolean storageExists() {
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "Flink Database History";
+    }
+
+    /**
+     * Determine whether the {@link FlinkDatabaseHistory} is compatible with the specified state.
+     */
+    public static boolean isCompatible(Collection<SchemaRecord> records) {
+        for (SchemaRecord record : records) {
+            // check the source/position/ddl is not null
+            if (!record.isHistoryRecord()) {
+                return false;
+            } else {
+                break;
+            }
+        }
+        return true;
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/FlinkDatabaseSchemaHistory.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/FlinkDatabaseSchemaHistory.java
new file mode 100644
index 000000000..46b981fdc
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/FlinkDatabaseSchemaHistory.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal;
+
+import static io.debezium.relational.history.TableChanges.TableChange;
+import static org.apache.inlong.sort.singletenant.flink.cdc.debezium.utils.DatabaseHistoryUtil.registerHistory;
+import static org.apache.inlong.sort.singletenant.flink.cdc.debezium.utils.DatabaseHistoryUtil.removeHistory;
+import static org.apache.inlong.sort.singletenant.flink.cdc.debezium.utils.DatabaseHistoryUtil.retrieveHistory;
+
+import io.debezium.config.Configuration;
+import io.debezium.relational.TableId;
+import io.debezium.relational.Tables;
+import io.debezium.relational.ddl.DdlParser;
+import io.debezium.relational.history.DatabaseHistory;
+import io.debezium.relational.history.DatabaseHistoryException;
+import io.debezium.relational.history.DatabaseHistoryListener;
+import io.debezium.relational.history.HistoryRecord;
+import io.debezium.relational.history.HistoryRecordComparator;
+import io.debezium.relational.history.TableChanges;
+import io.debezium.schema.DatabaseSchema;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.history.FlinkJsonTableChangeSerializer;
+
+/**
+ * The {@link FlinkDatabaseSchemaHistory} only stores the latest schema of the monitored tables.
+ * When recovering from the checkpoint, it should apply all the tables to the {@link
+ * DatabaseSchema}, which doesn't need to replay the history anymore.
+ *
+ * <p>Considering the data structure maintained in the {@link FlinkDatabaseSchemaHistory} is much
+ * different from the {@link FlinkDatabaseHistory}, it's not compatible with the {@link
+ * FlinkDatabaseHistory}. Because it only maintains the latest schema of the table rather than all
+ * history DDLs, it's useful to prevent OOM when meet massive history DDLs.
+ */
+public class FlinkDatabaseSchemaHistory implements DatabaseHistory {
+
+    public static final String DATABASE_HISTORY_INSTANCE_NAME = "database.history.instance.name";
+
+    private final FlinkJsonTableChangeSerializer tableChangesSerializer =
+            new FlinkJsonTableChangeSerializer();
+
+    private ConcurrentMap<TableId, SchemaRecord> latestTables;
+    private String instanceName;
+    private DatabaseHistoryListener listener;
+    private boolean storeOnlyMonitoredTablesDdl;
+    private boolean skipUnparseableDDL;
+    private boolean useCatalogBeforeSchema;
+
+    @Override
+    public void configure(
+            Configuration config,
+            HistoryRecordComparator comparator,
+            DatabaseHistoryListener listener,
+            boolean useCatalogBeforeSchema) {
+        this.instanceName = config.getString(DATABASE_HISTORY_INSTANCE_NAME);
+        this.listener = listener;
+        this.storeOnlyMonitoredTablesDdl = config.getBoolean(STORE_ONLY_MONITORED_TABLES_DDL);
+        this.skipUnparseableDDL = config.getBoolean(SKIP_UNPARSEABLE_DDL_STATEMENTS);
+        this.useCatalogBeforeSchema = useCatalogBeforeSchema;
+
+        // recover
+        this.latestTables = new ConcurrentHashMap<>();
+        for (SchemaRecord schemaRecord : retrieveHistory(instanceName)) {
+            // validate here
+            TableChange tableChange =
+                    FlinkJsonTableChangeSerializer.fromDocument(
+                            schemaRecord.toDocument(), useCatalogBeforeSchema);
+            latestTables.put(tableChange.getId(), schemaRecord);
+        }
+        // register
+        registerHistory(instanceName, latestTables.values());
+    }
+
+    @Override
+    public void start() {
+        listener.started();
+    }
+
+    @Override
+    public void record(
+            Map<String, ?> source, Map<String, ?> position, String databaseName, String ddl)
+            throws DatabaseHistoryException {
+        throw new UnsupportedOperationException(
+                String.format(
+                        "The %s cannot work with 'debezium.internal.implementation' = 'legacy',"
+                                + "please use %s",
+                        FlinkDatabaseSchemaHistory.class.getCanonicalName(),
+                        FlinkDatabaseHistory.class.getCanonicalName()));
+    }
+
+    @Override
+    public void record(
+            Map<String, ?> source,
+            Map<String, ?> position,
+            String databaseName,
+            String schemaName,
+            String ddl,
+            TableChanges changes)
+            throws DatabaseHistoryException {
+        for (TableChange change : changes) {
+            switch (change.getType()) {
+                case CREATE:
+                case ALTER:
+                    latestTables.put(
+                            change.getId(),
+                            new SchemaRecord(tableChangesSerializer.toDocument(change)));
+                    break;
+                case DROP:
+                    latestTables.remove(change.getId());
+                    break;
+                default:
+                    // impossible
+                    throw new RuntimeException(
+                            String.format("Unknown change type: %s.", change.getType()));
+            }
+        }
+        listener.onChangeApplied(
+                new HistoryRecord(source, position, databaseName, schemaName, ddl, changes));
+    }
+
+    @Override
+    public void recover(
+            Map<String, ?> source, Map<String, ?> position, Tables schema, DdlParser ddlParser) {
+        listener.recoveryStarted();
+        for (SchemaRecord record : latestTables.values()) {
+            TableChange tableChange =
+                    FlinkJsonTableChangeSerializer.fromDocument(
+                            record.getTableChangeDoc(), useCatalogBeforeSchema);
+            schema.overwriteTable(tableChange.getTable());
+        }
+        listener.recoveryStopped();
+    }
+
+    @Override
+    public void stop() {
+        if (instanceName != null) {
+            removeHistory(instanceName);
+        }
+        listener.stopped();
+    }
+
+    @Override
+    public boolean exists() {
+        return latestTables != null && !latestTables.isEmpty();
+    }
+
+    @Override
+    public boolean storageExists() {
+        return true;
+    }
+
+    @Override
+    public void initializeStorage() {
+        // do nothing
+    }
+
+    @Override
+    public boolean storeOnlyMonitoredTables() {
+        return storeOnlyMonitoredTablesDdl;
+    }
+
+    @Override
+    public boolean skipUnparseableDdlStatements() {
+        return skipUnparseableDDL;
+    }
+
+    /**
+     * Determine whether the {@link FlinkDatabaseSchemaHistory} is compatible with the specified
+     * state.
+     */
+    public static boolean isCompatible(Collection<SchemaRecord> records) {
+        for (SchemaRecord record : records) {
+            if (!record.isTableChangeRecord()) {
+                return false;
+            } else {
+                break;
+            }
+        }
+        return true;
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/FlinkOffsetBackingStore.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/FlinkOffsetBackingStore.java
new file mode 100644
index 000000000..daec6b46b
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/FlinkOffsetBackingStore.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal;
+
+import io.debezium.embedded.EmbeddedEngine;
+import io.debezium.engine.DebeziumEngine;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumSourceFunction;
+import org.apache.kafka.common.utils.ThreadUtils;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.apache.kafka.connect.json.JsonConverter;
+import org.apache.kafka.connect.runtime.WorkerConfig;
+import org.apache.kafka.connect.storage.Converter;
+import org.apache.kafka.connect.storage.OffsetBackingStore;
+import org.apache.kafka.connect.storage.OffsetStorageWriter;
+import org.apache.kafka.connect.util.Callback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A implementation of {@link OffsetBackingStore} backed on Flink's state mechanism.
+ *
+ * <p>The {@link #OFFSET_STATE_VALUE} in the {@link WorkerConfig} is the raw position and offset
+ * data in JSON format. It is set into the config when recovery from failover by {@link
+ * DebeziumSourceFunction} before startup the {@link DebeziumEngine}. If it is not a restoration,
+ * the {@link #OFFSET_STATE_VALUE} is empty. {@link DebeziumEngine} relies on the {@link
+ * OffsetBackingStore} for failover recovery.
+ *
+ * @see DebeziumSourceFunction
+ */
+public class FlinkOffsetBackingStore implements OffsetBackingStore {
+    private static final Logger LOG = LoggerFactory.getLogger(FlinkOffsetBackingStore.class);
+
+    public static final String OFFSET_STATE_VALUE = "offset.storage.flink.state.value";
+    public static final int FLUSH_TIMEOUT_SECONDS = 10;
+
+    protected Map<ByteBuffer, ByteBuffer> data = new HashMap<>();
+    protected ExecutorService executor;
+
+    @Override
+    public void configure(WorkerConfig config) {
+        // eagerly initialize the executor, because OffsetStorageWriter will use it later
+        start();
+
+        Map<String, ?> conf = config.originals();
+        if (!conf.containsKey(OFFSET_STATE_VALUE)) {
+            // a normal startup from clean state, not need to initialize the offset
+            return;
+        }
+
+        String stateJson = (String) conf.get(OFFSET_STATE_VALUE);
+        DebeziumOffsetSerializer serializer = new DebeziumOffsetSerializer();
+        DebeziumOffset debeziumOffset;
+        try {
+            debeziumOffset = serializer.deserialize(stateJson.getBytes(StandardCharsets.UTF_8));
+        } catch (IOException e) {
+            LOG.error("Can't deserialize debezium offset state from JSON: " + stateJson, e);
+            throw new RuntimeException(e);
+        }
+
+        final String engineName = (String) conf.get(EmbeddedEngine.ENGINE_NAME.name());
+        Converter keyConverter = new JsonConverter();
+        Converter valueConverter = new JsonConverter();
+        keyConverter.configure(config.originals(), true);
+        Map<String, Object> valueConfigs = new HashMap<>(conf);
+        valueConfigs.put("schemas.enable", false);
+        valueConverter.configure(valueConfigs, true);
+        OffsetStorageWriter offsetWriter =
+                new OffsetStorageWriter(
+                        this,
+                        // must use engineName as namespace to align with Debezium Engine
+                        // implementation
+                        engineName,
+                        keyConverter,
+                        valueConverter);
+
+        offsetWriter.offset(debeziumOffset.sourcePartition, debeziumOffset.sourceOffset);
+
+        // flush immediately
+        if (!offsetWriter.beginFlush()) {
+            // if nothing is needed to be flushed, there must be something wrong with the
+            // initialization
+            LOG.warn(
+                    "Initialize FlinkOffsetBackingStore from empty offset state, this shouldn't happen.");
+            return;
+        }
+
+        // trigger flushing
+        Future<Void> flushFuture =
+                offsetWriter.doFlush(
+                        (error, result) -> {
+                            if (error != null) {
+                                LOG.error("Failed to flush initial offset.", error);
+                            } else {
+                                LOG.debug("Successfully flush initial offset.");
+                            }
+                        });
+
+        // wait until flushing finished
+        try {
+            flushFuture.get(FLUSH_TIMEOUT_SECONDS, TimeUnit.SECONDS);
+            LOG.info(
+                    "Flush offsets successfully, partition: {}, offsets: {}",
+                    debeziumOffset.sourcePartition,
+                    debeziumOffset.sourceOffset);
+        } catch (InterruptedException e) {
+            LOG.warn("Flush offsets interrupted, cancelling.", e);
+            offsetWriter.cancelFlush();
+        } catch (ExecutionException e) {
+            LOG.error("Flush offsets threw an unexpected exception.", e);
+            offsetWriter.cancelFlush();
+        } catch (TimeoutException e) {
+            LOG.error("Timed out waiting to flush offsets to storage.", e);
+            offsetWriter.cancelFlush();
+        }
+    }
+
+    @Override
+    public void start() {
+        if (executor == null) {
+            executor =
+                    Executors.newFixedThreadPool(
+                            1,
+                            ThreadUtils.createThreadFactory(
+                                    this.getClass().getSimpleName() + "-%d", false));
+        }
+    }
+
+    @Override
+    public void stop() {
+        if (executor != null) {
+            executor.shutdown();
+            // Best effort wait for any get() and set() tasks (and caller's callbacks) to complete.
+            try {
+                executor.awaitTermination(30, TimeUnit.SECONDS);
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+            if (!executor.shutdownNow().isEmpty()) {
+                throw new ConnectException(
+                        "Failed to stop FlinkOffsetBackingStore. Exiting without cleanly "
+                                + "shutting down pending tasks and/or callbacks.");
+            }
+            executor = null;
+        }
+    }
+
+    @Override
+    public Future<Map<ByteBuffer, ByteBuffer>> get(final Collection<ByteBuffer> keys) {
+        return executor.submit(
+                () -> {
+                    Map<ByteBuffer, ByteBuffer> result = new HashMap<>();
+                    for (ByteBuffer key : keys) {
+                        result.put(key, data.get(key));
+                    }
+                    return result;
+                });
+    }
+
+    @Override
+    public Future<Void> set(
+            final Map<ByteBuffer, ByteBuffer> values, final Callback<Void> callback) {
+        return executor.submit(
+                () -> {
+                    for (Map.Entry<ByteBuffer, ByteBuffer> entry : values.entrySet()) {
+                        data.put(entry.getKey(), entry.getValue());
+                    }
+                    if (callback != null) {
+                        callback.onCompletion(null, null);
+                    }
+                    return null;
+                });
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/Handover.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/Handover.java
new file mode 100644
index 000000000..9af2ff246
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/Handover.java
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+import io.debezium.engine.ChangeEvent;
+import java.io.Closeable;
+import java.util.Collections;
+import java.util.List;
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The Handover is a utility to hand over data (a buffer of records) and exception from a
+ * <i>producer</i> thread to a <i>consumer</i> thread. It effectively behaves like a "size one
+ * blocking queue", with some extras around exception reporting, closing, and waking up thread
+ * without {@link Thread#interrupt() interrupting} threads.
+ *
+ * <p>This class is used in the Flink Debezium Engine Consumer to hand over data and exceptions
+ * between the thread that runs the DebeziumEngine class and the main thread.
+ *
+ * <p>The Handover can also be "closed", signalling from one thread to the other that it the thread
+ * has terminated.
+ */
+@ThreadSafe
+@Internal
+public class Handover implements Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(Handover.class);
+    private final Object lock = new Object();
+
+    @GuardedBy("lock")
+    private List<ChangeEvent<SourceRecord, SourceRecord>> next;
+
+    @GuardedBy("lock")
+    private Throwable error;
+
+    private boolean wakeupProducer;
+
+    /**
+     * Polls the next element from the Handover, possibly blocking until the next element is
+     * available. This method behaves similar to polling from a blocking queue.
+     *
+     * <p>If an exception was handed in by the producer ({@link #reportError(Throwable)}), then that
+     * exception is thrown rather than an element being returned.
+     *
+     * @return The next element (buffer of records, never null).
+     * @throws ClosedException Thrown if the Handover was {@link #close() closed}.
+     * @throws Exception Rethrows exceptions from the {@link #reportError(Throwable)} method.
+     */
+    public List<ChangeEvent<SourceRecord, SourceRecord>> pollNext() throws Exception {
+        synchronized (lock) {
+            while (next == null && error == null) {
+                lock.wait();
+            }
+            List<ChangeEvent<SourceRecord, SourceRecord>> n = next;
+            if (n != null) {
+                next = null;
+                lock.notifyAll();
+                return n;
+            } else {
+                ExceptionUtils.rethrowException(error, error.getMessage());
+
+                // this statement cannot be reached since the above method always throws an
+                // exception this is only here to silence the compiler and any warnings
+                return Collections.emptyList();
+            }
+        }
+    }
+
+    /**
+     * Hands over an element from the producer. If the Handover already has an element that was not
+     * yet picked up by the consumer thread, this call blocks until the consumer picks up that
+     * previous element.
+     *
+     * <p>This behavior is similar to a "size one" blocking queue.
+     *
+     * @param element The next element to hand over.
+     * @throws InterruptedException Thrown, if the thread is interrupted while blocking for the
+     *     Handover to be empty.
+     */
+    public void produce(final List<ChangeEvent<SourceRecord, SourceRecord>> element)
+            throws InterruptedException {
+
+        checkNotNull(element);
+
+        synchronized (lock) {
+            while (next != null && !wakeupProducer) {
+                lock.wait();
+            }
+
+            wakeupProducer = false;
+
+            // an error marks this as closed for the producer
+            if (error != null) {
+                ExceptionUtils.rethrow(error, error.getMessage());
+            } else {
+                // if there is no error, then this is open and can accept this element
+                next = element;
+                lock.notifyAll();
+            }
+        }
+    }
+
+    /**
+     * Reports an exception. The consumer will throw the given exception immediately, if it is
+     * currently blocked in the {@link #pollNext()} method, or the next time it calls that method.
+     *
+     * <p>After this method has been called, no call to either {@link #produce( List)} or {@link
+     * #pollNext()} will ever return regularly any more, but will always return exceptionally.
+     *
+     * <p>If another exception was already reported, this method does nothing.
+     *
+     * <p>For the producer, the Handover will appear as if it was {@link #close() closed}.
+     *
+     * @param t The exception to report.
+     */
+    public void reportError(Throwable t) {
+        checkNotNull(t);
+
+        synchronized (lock) {
+            LOG.error("Reporting error:", t);
+            // do not override the initial exception
+            if (error == null) {
+                error = t;
+            }
+            next = null;
+            lock.notifyAll();
+        }
+    }
+
+    /**
+     * Return whether there is an error.
+     *
+     * @return whether there is an error
+     */
+    public boolean hasError() {
+        return error != null;
+    }
+
+    /**
+     * Closes the handover. Both the {@link #produce(List)} method and the {@link #pollNext()} will
+     * throw a {@link ClosedException} on any currently blocking and future invocations.
+     *
+     * <p>If an exception was previously reported via the {@link #reportError(Throwable)} method,
+     * that exception will not be overridden. The consumer thread will throw that exception upon
+     * calling {@link #pollNext()}, rather than the {@code ClosedException}.
+     */
+    @Override
+    public void close() {
+        synchronized (lock) {
+            next = null;
+            wakeupProducer = false;
+
+            if (error == null) {
+                error = new ClosedException();
+            }
+            lock.notifyAll();
+        }
+    }
+
+    // ------------------------------------------------------------------------
+
+    /**
+     * An exception thrown by the Handover in the {@link #pollNext()} or {@link #produce(List)}
+     * method, after the Handover was closed via {@link #close()}.
+     */
+    public static final class ClosedException extends Exception {
+
+        private static final long serialVersionUID = 1L;
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/SchemaRecord.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/SchemaRecord.java
new file mode 100644
index 000000000..bcc48ce52
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/internal/SchemaRecord.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal;
+
+import io.debezium.document.Document;
+import io.debezium.document.DocumentWriter;
+import io.debezium.relational.history.HistoryRecord;
+import io.debezium.relational.history.TableChanges.TableChange;
+import java.io.IOException;
+import javax.annotation.Nullable;
+
+/**
+ * The Record represents a schema change event, it contains either one {@link HistoryRecord} or
+ * {@link TableChange}.
+ *
+ * <p>The {@link HistoryRecord} will be used by {@link FlinkDatabaseHistory} which keeps full
+ * history of table change events for all tables, the {@link TableChange} will be used by {@link
+ * FlinkDatabaseSchemaHistory} which keeps the latest table change for each table.
+ */
+public class SchemaRecord {
+
+    @Nullable private final HistoryRecord historyRecord;
+
+    @Nullable private final Document tableChangeDoc;
+
+    public SchemaRecord(HistoryRecord historyRecord) {
+        this.historyRecord = historyRecord;
+        this.tableChangeDoc = null;
+    }
+
+    public SchemaRecord(Document document) {
+        if (isHistoryRecordDocument(document)) {
+            this.historyRecord = new HistoryRecord(document);
+            this.tableChangeDoc = null;
+        } else {
+            this.tableChangeDoc = document;
+            this.historyRecord = null;
+        }
+    }
+
+    @Nullable
+    public HistoryRecord getHistoryRecord() {
+        return historyRecord;
+    }
+
+    @Nullable
+    public Document getTableChangeDoc() {
+        return tableChangeDoc;
+    }
+
+    public boolean isHistoryRecord() {
+        return historyRecord != null;
+    }
+
+    public boolean isTableChangeRecord() {
+        return tableChangeDoc != null;
+    }
+
+    public Document toDocument() {
+        if (historyRecord != null) {
+            return historyRecord.document();
+        } else {
+            return tableChangeDoc;
+        }
+    }
+
+    @Override
+    public String toString() {
+        try {
+            return DocumentWriter.defaultWriter().write(toDocument());
+        } catch (IOException e) {
+            return super.toString();
+        }
+    }
+
+    private boolean isHistoryRecordDocument(Document document) {
+        return new HistoryRecord(document).isValid();
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/AppendMetadataCollector.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/AppendMetadataCollector.java
index 50430463c..6e83a8358 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/AppendMetadataCollector.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/AppendMetadataCollector.java
@@ -27,9 +27,12 @@ import org.apache.flink.table.data.utils.JoinedRowData;
 import org.apache.flink.util.Collector;
 import org.apache.kafka.connect.source.SourceRecord;
 
-/** Emits a row with physical fields and metadata fields. */
+/**
+ * Emits a row with physical fields and metadata fields.
+ */
 @Internal
 public final class AppendMetadataCollector implements Collector<RowData>, Serializable {
+
     private static final long serialVersionUID = 1L;
 
     private final MetadataConverter[] metadataConverters;
@@ -53,7 +56,7 @@ public final class AppendMetadataCollector implements Collector<RowData>, Serial
 
     @Override
     public void collect(RowData record) {
-
+        collect(record, null);
     }
 
     @Override
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/MetadataConverter.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/MetadataConverter.java
index 8c3470841..a63c963ac 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/MetadataConverter.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/MetadataConverter.java
@@ -22,15 +22,23 @@ import io.debezium.relational.history.TableChanges;
 import java.io.Serializable;
 import javax.annotation.Nullable;
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.RowData;
 import org.apache.kafka.connect.source.SourceRecord;
 
-/** A converter converts {@link SourceRecord} metadata into Flink internal data structures. */
+/**
+ * A converter converts {@link SourceRecord} metadata into Flink internal data structures.
+ */
 @FunctionalInterface
 @Internal
 public interface MetadataConverter extends Serializable {
+
     Object read(SourceRecord record);
 
     default Object read(SourceRecord record, @Nullable TableChanges.TableChange tableSchema) {
         return read(record);
     }
+
+    default Object read(SourceRecord record, @Nullable TableChanges.TableChange tableSchema, RowData rowData) {
+        return read(record);
+    }
 }
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/RowDataDebeziumDeserializeSchema.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/RowDataDebeziumDeserializeSchema.java
index b5cea2aef..217b4264c 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/RowDataDebeziumDeserializeSchema.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/RowDataDebeziumDeserializeSchema.java
@@ -20,7 +20,6 @@ package org.apache.inlong.sort.singletenant.flink.cdc.debezium.table;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
-import org.apache.inlong.sort.singletenant.flink.cdc.debezium.utils.TemporalConversions;
 import io.debezium.data.Envelope;
 import io.debezium.data.SpecialValueDecimal;
 import io.debezium.data.VariableScaleDecimal;
@@ -49,6 +48,7 @@ import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.types.RowKind;
 import org.apache.flink.util.Collector;
 import org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumDeserializationSchema;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.utils.TemporalConversions;
 import org.apache.kafka.connect.data.Decimal;
 import org.apache.kafka.connect.data.Field;
 import org.apache.kafka.connect.data.Schema;
@@ -61,78 +61,41 @@ import org.apache.kafka.connect.source.SourceRecord;
  */
 public final class RowDataDebeziumDeserializeSchema
         implements DebeziumDeserializationSchema<RowData> {
-    private static final long serialVersionUID = 2L;
 
-    @Override
-    public void deserialize(SourceRecord record, Collector<RowData> out) throws Exception {
-        deserialize(record, out, null);
-    }
-
-    @Override
-    public void deserialize(SourceRecord record, Collector<RowData> out, TableChange tableChange)
-        throws Exception {
-        Envelope.Operation op = Envelope.operationFor(record);
-        Struct value = (Struct) record.value();
-        Schema valueSchema = record.valueSchema();
-        if (op == Envelope.Operation.CREATE || op == Envelope.Operation.READ) {
-            GenericRowData insert = extractAfterRow(value, valueSchema);
-            validator.validate(insert, RowKind.INSERT);
-            insert.setRowKind(RowKind.INSERT);
-            emit(record, insert, out, tableChange);
-        } else if (op == Envelope.Operation.DELETE) {
-            GenericRowData delete = extractBeforeRow(value, valueSchema);
-            validator.validate(delete, RowKind.DELETE);
-            delete.setRowKind(RowKind.DELETE);
-            emit(record, delete, out, tableChange);
-        } else {
-            GenericRowData before = extractBeforeRow(value, valueSchema);
-            validator.validate(before, RowKind.UPDATE_BEFORE);
-            before.setRowKind(RowKind.UPDATE_BEFORE);
-            emit(record, before, out, tableChange);
-
-            GenericRowData after = extractAfterRow(value, valueSchema);
-            validator.validate(after, RowKind.UPDATE_AFTER);
-            after.setRowKind(RowKind.UPDATE_AFTER);
-            emit(record, after, out, tableChange);
-        }
-    }
-
-    /** Custom validator to validate the row value. */
-    public interface ValueValidator extends Serializable {
-        void validate(RowData rowData, RowKind rowKind) throws Exception;
-    }
-
-    /** TypeInformation of the produced {@link RowData}. * */
+    private static final long serialVersionUID = 2L;
+    /**
+     * TypeInformation of the produced {@link RowData}. *
+     */
     private final TypeInformation<RowData> resultTypeInfo;
-
     /**
      * Runtime converter that converts Kafka {@link SourceRecord}s into {@link RowData} consisted of
      * physical column values.
      */
     private final DeserializationRuntimeConverter physicalConverter;
-
-    /** Whether the deserializer needs to handle metadata columns. */
+    /**
+     * Whether the deserializer needs to handle metadata columns.
+     */
     private final boolean hasMetadata;
-
+    /**
+     * Whether works append source.
+     */
+    private final boolean appendSource;
     /**
      * A wrapped output collector which is used to append metadata columns after physical columns.
      */
     private final AppendMetadataCollector appendMetadataCollector;
-
-    /** Validator to validate the row value. */
+    /**
+     * Validator to validate the row value.
+     */
     private final ValueValidator validator;
 
-    /** Returns a builder to build {@link RowDataDebeziumDeserializeSchema}. */
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
     RowDataDebeziumDeserializeSchema(
             RowType physicalDataType,
             MetadataConverter[] metadataConverters,
             TypeInformation<RowData> resultTypeInfo,
             ValueValidator validator,
             ZoneId serverTimeZone,
+            boolean appendSource,
             DeserializationRuntimeConverterFactory userDefinedConverterFactory) {
         this.hasMetadata = checkNotNull(metadataConverters).length > 0;
         this.appendMetadataCollector = new AppendMetadataCollector(metadataConverters);
@@ -143,98 +106,19 @@ public final class RowDataDebeziumDeserializeSchema
                         userDefinedConverterFactory);
         this.resultTypeInfo = checkNotNull(resultTypeInfo);
         this.validator = checkNotNull(validator);
+        this.appendSource = checkNotNull(appendSource);
     }
 
-    private GenericRowData extractAfterRow(Struct value, Schema valueSchema) throws Exception {
-        Schema afterSchema = valueSchema.field(Envelope.FieldName.AFTER).schema();
-        Struct after = value.getStruct(Envelope.FieldName.AFTER);
-        return (GenericRowData) physicalConverter.convert(after, afterSchema);
-    }
-
-    private GenericRowData extractBeforeRow(Struct value, Schema valueSchema) throws Exception {
-        Schema beforeSchema = valueSchema.field(Envelope.FieldName.BEFORE).schema();
-        Struct before = value.getStruct(Envelope.FieldName.BEFORE);
-        return (GenericRowData) physicalConverter.convert(before, beforeSchema);
-    }
-
-    private void emit(SourceRecord inRecord, RowData physicalRow, Collector<RowData> collector,
-        TableChange tableChange) {
-        if (!hasMetadata) {
-            collector.collect(physicalRow);
-            return;
-        }
-
-        appendMetadataCollector.inputRecord = inRecord;
-        appendMetadataCollector.outputCollector = collector;
-        appendMetadataCollector.collect(physicalRow, tableChange);
-    }
-
-    @Override
-    public TypeInformation<RowData> getProducedType() {
-        return resultTypeInfo;
-    }
-
-    // -------------------------------------------------------------------------------------
-    // Builder
-    // -------------------------------------------------------------------------------------
-
-    /** Builder of {@link RowDataDebeziumDeserializeSchema}. */
-    public static class Builder {
-        private RowType physicalRowType;
-        private TypeInformation<RowData> resultTypeInfo;
-        private MetadataConverter[] metadataConverters = new MetadataConverter[0];
-        private ValueValidator validator = (rowData, rowKind) -> {};
-        private ZoneId serverTimeZone = ZoneId.of("UTC");
-        private DeserializationRuntimeConverterFactory userDefinedConverterFactory =
-                DeserializationRuntimeConverterFactory.DEFAULT;
-
-        public Builder setPhysicalRowType(RowType physicalRowType) {
-            this.physicalRowType = physicalRowType;
-            return this;
-        }
-
-        public Builder setMetadataConverters(MetadataConverter[] metadataConverters) {
-            this.metadataConverters = metadataConverters;
-            return this;
-        }
-
-        public Builder setResultTypeInfo(TypeInformation<RowData> resultTypeInfo) {
-            this.resultTypeInfo = resultTypeInfo;
-            return this;
-        }
-
-        public Builder setValueValidator(ValueValidator validator) {
-            this.validator = validator;
-            return this;
-        }
-
-        public Builder setServerTimeZone(ZoneId serverTimeZone) {
-            this.serverTimeZone = serverTimeZone;
-            return this;
-        }
-
-        public Builder setUserDefinedConverterFactory(
-                DeserializationRuntimeConverterFactory userDefinedConverterFactory) {
-            this.userDefinedConverterFactory = userDefinedConverterFactory;
-            return this;
-        }
-
-        public RowDataDebeziumDeserializeSchema build() {
-            return new RowDataDebeziumDeserializeSchema(
-                    physicalRowType,
-                    metadataConverters,
-                    resultTypeInfo,
-                    validator,
-                    serverTimeZone,
-                    userDefinedConverterFactory);
-        }
+    /**
+     * Returns a builder to build {@link RowDataDebeziumDeserializeSchema}.
+     */
+    public static Builder newBuilder() {
+        return new Builder();
     }
 
-    // -------------------------------------------------------------------------------------
-    // Runtime Converters
-    // -------------------------------------------------------------------------------------
-
-    /** Creates a runtime converter which is null safe. */
+    /**
+     * Creates a runtime converter which is null safe.
+     */
     private static DeserializationRuntimeConverter createConverter(
             LogicalType type,
             ZoneId serverTimeZone,
@@ -243,13 +127,9 @@ public final class RowDataDebeziumDeserializeSchema
                 createNotNullConverter(type, serverTimeZone, userDefinedConverterFactory));
     }
 
-    // --------------------------------------------------------------------------------
-    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
-    // necessary because the maven shade plugin cannot relocate classes in
-    // SerializedLambdas (MSHADE-260).
-    // --------------------------------------------------------------------------------
-
-    /** Creates a runtime converter which assuming input object is not null. */
+    /**
+     * Creates a runtime converter which assuming input object is not null.
+     */
     public static DeserializationRuntimeConverter createNotNullConverter(
             LogicalType type,
             ZoneId serverTimeZone,
@@ -425,6 +305,10 @@ public final class RowDataDebeziumDeserializeSchema
         };
     }
 
+    // -------------------------------------------------------------------------------------
+    // Builder
+    // -------------------------------------------------------------------------------------
+
     private static DeserializationRuntimeConverter convertToDate() {
         return new DeserializationRuntimeConverter() {
 
@@ -437,6 +321,10 @@ public final class RowDataDebeziumDeserializeSchema
         };
     }
 
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
     private static DeserializationRuntimeConverter convertToTime() {
         return new DeserializationRuntimeConverter() {
 
@@ -460,6 +348,12 @@ public final class RowDataDebeziumDeserializeSchema
         };
     }
 
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260).
+    // --------------------------------------------------------------------------------
+
     private static DeserializationRuntimeConverter convertToTimestamp(ZoneId serverTimeZone) {
         return new DeserializationRuntimeConverter() {
 
@@ -648,4 +542,144 @@ public final class RowDataDebeziumDeserializeSchema
             }
         };
     }
+
+    @Override
+    public void deserialize(SourceRecord record, Collector<RowData> out) throws Exception {
+        deserialize(record, out, null);
+    }
+
+    public void deserialize(SourceRecord record, Collector<RowData> out,
+            TableChange tableSchema)
+            throws Exception {
+        Envelope.Operation op = Envelope.operationFor(record);
+        Struct value = (Struct) record.value();
+        Schema valueSchema = record.valueSchema();
+        if (op == Envelope.Operation.CREATE || op == Envelope.Operation.READ) {
+            GenericRowData insert = extractAfterRow(value, valueSchema);
+            validator.validate(insert, RowKind.INSERT);
+            insert.setRowKind(RowKind.INSERT);
+            emit(record, insert, tableSchema, out);
+        } else if (op == Envelope.Operation.DELETE) {
+            GenericRowData delete = extractBeforeRow(value, valueSchema);
+            validator.validate(delete, RowKind.DELETE);
+            delete.setRowKind(RowKind.DELETE);
+            emit(record, delete, tableSchema, out);
+        } else {
+            if (!appendSource) {
+                GenericRowData before = extractBeforeRow(value, valueSchema);
+                validator.validate(before, RowKind.UPDATE_BEFORE);
+                before.setRowKind(RowKind.UPDATE_BEFORE);
+                emit(record, before, tableSchema, out);
+            }
+
+            GenericRowData after = extractAfterRow(value, valueSchema);
+            validator.validate(after, RowKind.UPDATE_AFTER);
+            after.setRowKind(RowKind.UPDATE_AFTER);
+            emit(record, after, tableSchema, out);
+        }
+    }
+
+    private GenericRowData extractAfterRow(Struct value, Schema valueSchema) throws Exception {
+        Schema afterSchema = valueSchema.field(Envelope.FieldName.AFTER).schema();
+        Struct after = value.getStruct(Envelope.FieldName.AFTER);
+        return (GenericRowData) physicalConverter.convert(after, afterSchema);
+    }
+
+    private GenericRowData extractBeforeRow(Struct value, Schema valueSchema) throws Exception {
+        Schema beforeSchema = valueSchema.field(Envelope.FieldName.BEFORE).schema();
+        Struct before = value.getStruct(Envelope.FieldName.BEFORE);
+        return (GenericRowData) physicalConverter.convert(before, beforeSchema);
+    }
+
+    private void emit(SourceRecord inRecord, RowData physicalRow,
+            TableChange tableChange, Collector<RowData> collector
+    ) {
+        if (appendSource) {
+            physicalRow.setRowKind(RowKind.INSERT);
+        }
+        if (!hasMetadata) {
+            collector.collect(physicalRow);
+            return;
+        }
+
+        appendMetadataCollector.inputRecord = inRecord;
+        appendMetadataCollector.outputCollector = collector;
+        appendMetadataCollector.collect(physicalRow, tableChange);
+    }
+
+    @Override
+    public TypeInformation<RowData> getProducedType() {
+        return resultTypeInfo;
+    }
+
+    /**
+     * Custom validator to validate the row value.
+     */
+    public interface ValueValidator extends Serializable {
+
+        void validate(RowData rowData, RowKind rowKind) throws Exception;
+    }
+
+    /**
+     * Builder of {@link RowDataDebeziumDeserializeSchema}.
+     */
+    public static class Builder {
+
+        private RowType physicalRowType;
+        private TypeInformation<RowData> resultTypeInfo;
+        private MetadataConverter[] metadataConverters = new MetadataConverter[0];
+        private ValueValidator validator = (rowData, rowKind) -> {
+        };
+        private ZoneId serverTimeZone = ZoneId.of("UTC");
+        private boolean appendSource = false;
+        private DeserializationRuntimeConverterFactory userDefinedConverterFactory =
+                DeserializationRuntimeConverterFactory.DEFAULT;
+
+        public Builder setPhysicalRowType(RowType physicalRowType) {
+            this.physicalRowType = physicalRowType;
+            return this;
+        }
+
+        public Builder setMetadataConverters(MetadataConverter[] metadataConverters) {
+            this.metadataConverters = metadataConverters;
+            return this;
+        }
+
+        public Builder setResultTypeInfo(TypeInformation<RowData> resultTypeInfo) {
+            this.resultTypeInfo = resultTypeInfo;
+            return this;
+        }
+
+        public Builder setValueValidator(ValueValidator validator) {
+            this.validator = validator;
+            return this;
+        }
+
+        public Builder setServerTimeZone(ZoneId serverTimeZone) {
+            this.serverTimeZone = serverTimeZone;
+            return this;
+        }
+
+        public Builder setAppendSource(boolean appendSource) {
+            this.appendSource = appendSource;
+            return this;
+        }
+
+        public Builder setUserDefinedConverterFactory(
+                DeserializationRuntimeConverterFactory userDefinedConverterFactory) {
+            this.userDefinedConverterFactory = userDefinedConverterFactory;
+            return this;
+        }
+
+        public RowDataDebeziumDeserializeSchema build() {
+            return new RowDataDebeziumDeserializeSchema(
+                    physicalRowType,
+                    metadataConverters,
+                    resultTypeInfo,
+                    validator,
+                    serverTimeZone,
+                    appendSource,
+                    userDefinedConverterFactory);
+        }
+    }
 }
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/utils/DatabaseHistoryUtil.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/utils/DatabaseHistoryUtil.java
index 958638234..cbadb1c5f 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/utils/DatabaseHistoryUtil.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/utils/DatabaseHistoryUtil.java
@@ -18,13 +18,13 @@
 
 package org.apache.inlong.sort.singletenant.flink.cdc.debezium.utils;
 
-import com.ververica.cdc.debezium.internal.SchemaRecord;
-import org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumSourceFunction;
 import io.debezium.relational.history.DatabaseHistory;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumSourceFunction;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal.SchemaRecord;
 
 /**
  * Util to safely visit schema history between {@link DatabaseHistory} and {@link
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/io/BufferedSocketInputStream.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/io/BufferedSocketInputStream.java
new file mode 100644
index 000000000..c03ed53c4
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/io/BufferedSocketInputStream.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.io;
+
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Copied from https://github.com/osheroff/mysql-binlog-connector-java project to fix
+ * https://github.com/ververica/flink-cdc-connectors/issues/460.
+ *
+ * <p>Line 50 and Line 70 ~ 72: Returns -1 means reach the end of InputStream. We should remove this
+ * class after we bumped a higher mysql-binlog-connector-java version where the
+ * https://github.com/osheroff/mysql-binlog-connector-java/issues/66 has been fixed.
+ */
+public class BufferedSocketInputStream extends FilterInputStream {
+
+    private byte[] buffer;
+    private int offset;
+    private int limit;
+
+    public BufferedSocketInputStream(InputStream in) {
+        this(in, 512 * 1024);
+    }
+
+    public BufferedSocketInputStream(InputStream in, int bufferSize) {
+        super(in);
+        this.buffer = new byte[bufferSize];
+    }
+
+    @Override
+    public int available() throws IOException {
+        return limit == -1 ? in.available() : limit - offset + in.available();
+    }
+
+    @Override
+    public int read() throws IOException {
+        if (offset < limit) {
+            return buffer[offset++] & 0xff;
+        }
+        offset = 0;
+        limit = in.read(buffer, 0, buffer.length);
+        return limit != -1 ? buffer[offset++] & 0xff : -1;
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+        if (offset >= limit) {
+            if (len >= buffer.length) {
+                return in.read(b, off, len);
+            }
+            offset = 0;
+            limit = in.read(buffer, 0, buffer.length);
+            if (limit == -1) {
+                return limit;
+            }
+        }
+        int bytesRemainingInBuffer = Math.min(len, limit - offset);
+        System.arraycopy(buffer, offset, b, off, bytesRemainingInBuffer);
+        offset += bytesRemainingInBuffer;
+        return bytesRemainingInBuffer;
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/MySqlSource.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/MySqlSource.java
index adbdb0132..d05bb2546 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/MySqlSource.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/MySqlSource.java
@@ -18,30 +18,43 @@
 
 package org.apache.inlong.sort.singletenant.flink.cdc.mysql;
 
-import com.ververica.cdc.debezium.internal.DebeziumOffset;
-import org.apache.flink.table.data.RowData;
-import org.apache.inlong.sort.singletenant.flink.cdc.mysql.table.StartupOptions;
-import org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumDeserializationSchema;
-import org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumSourceFunction;
-import io.debezium.connector.mysql.MySqlConnector;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumSourceFunction.LEGACY_IMPLEMENTATION_KEY;
+import static org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumSourceFunction.LEGACY_IMPLEMENTATION_VALUE;
 
+import io.debezium.connector.mysql.MySqlConnector;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumDeserializationSchema;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumSourceFunction;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.internal.DebeziumOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.table.StartupOptions;
 
-import static org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumSourceFunction.LEGACY_IMPLEMENTATION_KEY;
-import static org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumSourceFunction.LEGACY_IMPLEMENTATION_VALUE;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
+/**
+ * A builder to build a SourceFunction which can read snapshot and continue to consume binlog.
+ *
+ * @deprecated please use {@link org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.MySqlSource} instead
+ *     which supports more rich features, e.g. parallel reading from historical data. The {@link
+ *     MySqlSource} will be dropped in the future version.
+ */
 @Deprecated
 public class MySqlSource {
 
     private static final String DATABASE_SERVER_NAME = "mysql_binlog_source";
 
     public static <T> Builder<T> builder() {
-        return new Builder<T>();
+        return new Builder<>();
     }
 
+    /**
+     * Builder class of {@link MySqlSource}.
+     *
+     * @deprecated please use {@link
+     *     org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.MySqlSource#builder()} instead which supports
+     *     more rich features, e.g. parallel reading from historical data. The {@link
+     *     Builder} will be dropped in the future version.
+     */
     @Deprecated
     public static class Builder<T> {
 
@@ -55,7 +68,7 @@ public class MySqlSource {
         private String[] tableList;
         private Properties dbzProperties;
         private StartupOptions startupOptions = StartupOptions.initial();
-        private DebeziumDeserializationSchema<RowData> deserializer;
+        private DebeziumDeserializationSchema<T> deserializer;
 
         public Builder<T> hostname(String hostname) {
             this.hostname = hostname;
@@ -132,7 +145,7 @@ public class MySqlSource {
          * The deserializer used to convert from consumed {@link
          * org.apache.kafka.connect.source.SourceRecord}.
          */
-        public Builder<T> deserializer(DebeziumDeserializationSchema<RowData> deserializer) {
+        public Builder<T> deserializer(DebeziumDeserializationSchema<T> deserializer) {
             this.deserializer = deserializer;
             return this;
         }
@@ -143,7 +156,7 @@ public class MySqlSource {
             return this;
         }
 
-        public DebeziumSourceFunction<RowData> build() {
+        public DebeziumSourceFunction<T> build() {
             Properties props = new Properties();
             props.setProperty("connector.class", MySqlConnector.class.getCanonicalName());
             // hard code server name, because we don't need to distinguish it, docs:
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/MySqlValidator.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/MySqlValidator.java
index eb7cea3ba..690522526 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/MySqlValidator.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/MySqlValidator.java
@@ -19,20 +19,19 @@
 package org.apache.inlong.sort.singletenant.flink.cdc.mysql;
 
 import io.debezium.config.Configuration;
+import io.debezium.jdbc.JdbcConnection;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Properties;
 import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.util.FlinkRuntimeException;
-
-import org.apache.inlong.sort.singletenant.flink.cdc.mysql.config.MySqlSourceConfig;
 import org.apache.inlong.sort.singletenant.flink.cdc.debezium.Validator;
-import io.debezium.jdbc.JdbcConnection;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.DebeziumUtils;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.sql.SQLException;
-import java.util.Arrays;
-import java.util.Properties;
-
 /**
  * The validator for MySql: it only cares about the version of the database is larger than or equal
  * to 5.7. It also requires the binlog format in the database is ROW and row image is FULL.
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/SeekBinlogToTimestampFilter.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/SeekBinlogToTimestampFilter.java
index a2613510b..4a25c6763 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/SeekBinlogToTimestampFilter.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/SeekBinlogToTimestampFilter.java
@@ -18,12 +18,11 @@
 
 package org.apache.inlong.sort.singletenant.flink.cdc.mysql;
 
+import io.debezium.data.Envelope;
 import io.debezium.relational.history.TableChanges.TableChange;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.util.Collector;
-
 import org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumDeserializationSchema;
-import io.debezium.data.Envelope;
 import org.apache.kafka.connect.data.Struct;
 import org.apache.kafka.connect.source.SourceRecord;
 import org.slf4j.Logger;
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/DebeziumUtils.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/DebeziumUtils.java
similarity index 78%
rename from inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/DebeziumUtils.java
rename to inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/DebeziumUtils.java
index 008bbdd51..fe1e91314 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/DebeziumUtils.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/DebeziumUtils.java
@@ -16,28 +16,33 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.sort.singletenant.flink.cdc.mysql;
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.TableDiscoveryUtils.listTables;
 
 import com.github.shyiko.mysql.binlog.BinaryLogClient;
-import com.ververica.cdc.connectors.mysql.source.offset.BinlogOffset;
 import io.debezium.config.Configuration;
 import io.debezium.connector.mysql.MySqlConnection;
 import io.debezium.connector.mysql.MySqlConnectorConfig;
 import io.debezium.connector.mysql.MySqlDatabaseSchema;
+import io.debezium.connector.mysql.MySqlSystemVariables;
 import io.debezium.connector.mysql.MySqlTopicSelector;
 import io.debezium.connector.mysql.MySqlValueConverters;
 import io.debezium.jdbc.JdbcConnection;
 import io.debezium.jdbc.JdbcValueConverters;
 import io.debezium.jdbc.TemporalPrecisionMode;
+import io.debezium.relational.RelationalTableFilters;
 import io.debezium.relational.TableId;
 import io.debezium.schema.TopicSelector;
 import io.debezium.util.SchemaNameAdjuster;
 import java.sql.SQLException;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import org.apache.flink.util.FlinkRuntimeException;
-import org.apache.inlong.sort.singletenant.flink.cdc.mysql.config.MySqlSourceConfig;
-import org.apache.inlong.sort.singletenant.flink.cdc.mysql.connection.JdbcConnectionFactory;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfig;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.connection.JdbcConnectionFactory;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -167,6 +172,32 @@ public class DebeziumUtils {
                 MySqlValueConverters::defaultParsingErrorHandler);
     }
 
+    public static List<TableId> discoverCapturedTables(
+            JdbcConnection jdbc, MySqlSourceConfig sourceConfig) {
+
+        final List<TableId> capturedTableIds;
+        try {
+            capturedTableIds = listTables(jdbc, sourceConfig.getTableFilters());
+        } catch (SQLException e) {
+            throw new FlinkRuntimeException("Failed to discover captured tables", e);
+        }
+        if (capturedTableIds.isEmpty()) {
+            throw new IllegalArgumentException(
+                    String.format(
+                            "Can't find any matched tables, please check your"
+                                    + " configured database-name: %s and table-name: %s",
+                            sourceConfig.getDatabaseList(), sourceConfig.getTableList()));
+        }
+        return capturedTableIds;
+    }
+
+    public static boolean isTableIdCaseSensitive(JdbcConnection connection) {
+        return !"0"
+                .equals(
+                        readMySqlSystemVariables(connection)
+                                .get(MySqlSystemVariables.LOWER_CASE_TABLE_NAMES));
+    }
+
     public static Map<String, String> readMySqlSystemVariables(JdbcConnection connection) {
         // Read the system variables from the MySQL instance and get the current database name ...
         return querySystemVariables(connection, "SHOW VARIABLES");
@@ -194,4 +225,13 @@ public class DebeziumUtils {
         return variables;
     }
 
+    /**
+     * Creates {@link RelationalTableFilters} from configuration. The {@link RelationalTableFilters}
+     * can be used to filter tables according to "table.whitelist" and "database.whitelist" options.
+     */
+    public static RelationalTableFilters createTableFilters(MySqlSourceConfig configuration) {
+        Configuration debeziumConfig = configuration.getDbzConfiguration();
+        final MySqlConnectorConfig mySqlConnectorConfig = new MySqlConnectorConfig(debeziumConfig);
+        return mySqlConnectorConfig.getTableFilters();
+    }
 }
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/EmbeddedFlinkDatabaseHistory.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/EmbeddedFlinkDatabaseHistory.java
new file mode 100644
index 000000000..3dbbdfb50
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/EmbeddedFlinkDatabaseHistory.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium;
+
+import io.debezium.config.Configuration;
+import io.debezium.relational.TableId;
+import io.debezium.relational.Tables;
+import io.debezium.relational.ddl.DdlParser;
+import io.debezium.relational.history.DatabaseHistory;
+import io.debezium.relational.history.DatabaseHistoryException;
+import io.debezium.relational.history.DatabaseHistoryListener;
+import io.debezium.relational.history.HistoryRecord;
+import io.debezium.relational.history.HistoryRecordComparator;
+import io.debezium.relational.history.TableChanges;
+import io.debezium.relational.history.TableChanges.TableChange;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSplitState;
+
+/**
+ * A {@link DatabaseHistory} implementation which store the latest table schema in Flink state.
+ *
+ * <p>It stores/recovers history using data offered by {@link MySqlSplitState}.
+ */
+public class EmbeddedFlinkDatabaseHistory implements DatabaseHistory {
+
+    public static final String DATABASE_HISTORY_INSTANCE_NAME = "database.history.instance.name";
+
+    public static final ConcurrentMap<String, Collection<TableChange>> TABLE_SCHEMAS =
+            new ConcurrentHashMap<>();
+
+    private Map<TableId, TableChange> tableSchemas;
+    private DatabaseHistoryListener listener;
+    private boolean storeOnlyMonitoredTablesDdl;
+    private boolean skipUnparseableDDL;
+
+    @Override
+    public void configure(
+            Configuration config,
+            HistoryRecordComparator comparator,
+            DatabaseHistoryListener listener,
+            boolean useCatalogBeforeSchema) {
+        this.listener = listener;
+        this.storeOnlyMonitoredTablesDdl = config.getBoolean(
+            DatabaseHistory.STORE_ONLY_MONITORED_TABLES_DDL);
+        this.skipUnparseableDDL = config.getBoolean(DatabaseHistory.SKIP_UNPARSEABLE_DDL_STATEMENTS);
+
+        // recover
+        String instanceName = config.getString(DATABASE_HISTORY_INSTANCE_NAME);
+        this.tableSchemas = new HashMap<>();
+        for (TableChange tableChange : removeHistory(instanceName)) {
+            tableSchemas.put(tableChange.getId(), tableChange);
+        }
+    }
+
+    @Override
+    public void start() {
+        listener.started();
+    }
+
+    @Override
+    public void record(
+            Map<String, ?> source, Map<String, ?> position, String databaseName, String ddl)
+            throws DatabaseHistoryException {
+        throw new UnsupportedOperationException("should not call here, error");
+    }
+
+    @Override
+    public void record(
+            Map<String, ?> source,
+            Map<String, ?> position,
+            String databaseName,
+            String schemaName,
+            String ddl,
+            TableChanges changes)
+            throws DatabaseHistoryException {
+        final HistoryRecord record =
+                new HistoryRecord(source, position, databaseName, schemaName, ddl, changes);
+        listener.onChangeApplied(record);
+    }
+
+    @Override
+    public void recover(
+            Map<String, ?> source, Map<String, ?> position, Tables schema, DdlParser ddlParser) {
+        listener.recoveryStarted();
+        for (TableChange tableChange : tableSchemas.values()) {
+            schema.overwriteTable(tableChange.getTable());
+        }
+        listener.recoveryStopped();
+    }
+
+    @Override
+    public void stop() {
+        listener.stopped();
+    }
+
+    @Override
+    public boolean exists() {
+        return tableSchemas != null && !tableSchemas.isEmpty();
+    }
+
+    @Override
+    public boolean storageExists() {
+        return true;
+    }
+
+    @Override
+    public void initializeStorage() {
+        // do nothing
+    }
+
+    @Override
+    public boolean storeOnlyMonitoredTables() {
+        return storeOnlyMonitoredTablesDdl;
+    }
+
+    @Override
+    public boolean skipUnparseableDdlStatements() {
+        return skipUnparseableDDL;
+    }
+
+    public static void registerHistory(String engineName, Collection<TableChange> engineHistory) {
+        TABLE_SCHEMAS.put(engineName, engineHistory);
+    }
+
+    public static Collection<TableChange> removeHistory(String engineName) {
+        if (engineName == null) {
+            return Collections.emptyList();
+        }
+        Collection<TableChange> tableChanges = TABLE_SCHEMAS.remove(engineName);
+        return tableChanges != null ? tableChanges : Collections.emptyList();
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/dispatcher/EventDispatcherImpl.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/dispatcher/EventDispatcherImpl.java
new file mode 100644
index 000000000..057577244
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/dispatcher/EventDispatcherImpl.java
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.dispatcher;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.dispatcher.SignalEventDispatcher.BINLOG_FILENAME_OFFSET_KEY;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.dispatcher.SignalEventDispatcher.BINLOG_POSITION_OFFSET_KEY;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.task.context.StatefulTaskContext.MySqlEventMetadataProvider.SERVER_ID_KEY;
+
+import io.debezium.config.CommonConnectorConfig;
+import io.debezium.connector.base.ChangeEventQueue;
+import io.debezium.document.DocumentWriter;
+import io.debezium.pipeline.DataChangeEvent;
+import io.debezium.pipeline.EventDispatcher;
+import io.debezium.pipeline.source.spi.EventMetadataProvider;
+import io.debezium.pipeline.spi.ChangeEventCreator;
+import io.debezium.pipeline.spi.SchemaChangeEventEmitter;
+import io.debezium.relational.history.HistoryRecord;
+import io.debezium.schema.DataCollectionFilters;
+import io.debezium.schema.DataCollectionId;
+import io.debezium.schema.DatabaseSchema;
+import io.debezium.schema.HistorizedDatabaseSchema;
+import io.debezium.schema.SchemaChangeEvent;
+import io.debezium.schema.TopicSelector;
+import io.debezium.util.SchemaNameAdjuster;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.data.SchemaBuilder;
+import org.apache.kafka.connect.data.Struct;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A subclass implementation of {@link EventDispatcher}.
+ *
+ * <pre>
+ *  1. This class shares one {@link ChangeEventQueue} between multiple readers.
+ *  2. This class override some methods for dispatching {@link HistoryRecord} directly,
+ *     this is useful for downstream to deserialize the {@link HistoryRecord} back.
+ * </pre>
+ */
+public class EventDispatcherImpl<T extends DataCollectionId> extends EventDispatcher<T> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(EventDispatcherImpl.class);
+
+    public static final String HISTORY_RECORD_FIELD = "historyRecord";
+    private static final DocumentWriter DOCUMENT_WRITER = DocumentWriter.defaultWriter();
+
+    private final ChangeEventQueue<DataChangeEvent> queue;
+    private final HistorizedDatabaseSchema historizedSchema;
+    private final DataCollectionFilters.DataCollectionFilter<T> filter;
+    private final CommonConnectorConfig connectorConfig;
+    private final TopicSelector<T> topicSelector;
+    private final Schema schemaChangeKeySchema;
+    private final Schema schemaChangeValueSchema;
+
+    public EventDispatcherImpl(
+            CommonConnectorConfig connectorConfig,
+            TopicSelector<T> topicSelector,
+            DatabaseSchema<T> schema,
+            ChangeEventQueue<DataChangeEvent> queue,
+            DataCollectionFilters.DataCollectionFilter<T> filter,
+            ChangeEventCreator changeEventCreator,
+            EventMetadataProvider metadataProvider,
+            SchemaNameAdjuster schemaNameAdjuster) {
+        super(
+                connectorConfig,
+                topicSelector,
+                schema,
+                queue,
+                filter,
+                changeEventCreator,
+                metadataProvider,
+                schemaNameAdjuster);
+        this.historizedSchema =
+                schema instanceof HistorizedDatabaseSchema
+                        ? (HistorizedDatabaseSchema<T>) schema
+                        : null;
+        this.filter = filter;
+        this.queue = queue;
+        this.connectorConfig = connectorConfig;
+        this.topicSelector = topicSelector;
+        this.schemaChangeKeySchema =
+                SchemaBuilder.struct()
+                        .name(
+                                schemaNameAdjuster.adjust(
+                                        "io.debezium.connector."
+                                                + connectorConfig.getConnectorName()
+                                                + ".SchemaChangeKey"))
+                        .field(HistoryRecord.Fields.DATABASE_NAME, Schema.STRING_SCHEMA)
+                        .build();
+        this.schemaChangeValueSchema =
+                SchemaBuilder.struct()
+                        .name(
+                                schemaNameAdjuster.adjust(
+                                        "io.debezium.connector."
+                                                + connectorConfig.getConnectorName()
+                                                + ".SchemaChangeValue"))
+                        .field(
+                                HistoryRecord.Fields.SOURCE,
+                                connectorConfig.getSourceInfoStructMaker().schema())
+                        .field(HISTORY_RECORD_FIELD, Schema.OPTIONAL_STRING_SCHEMA)
+                        .build();
+    }
+
+    public ChangeEventQueue<DataChangeEvent> getQueue() {
+        return queue;
+    }
+
+    @Override
+    public void dispatchSchemaChangeEvent(
+            T dataCollectionId, SchemaChangeEventEmitter schemaChangeEventEmitter)
+            throws InterruptedException {
+        if (dataCollectionId != null && !filter.isIncluded(dataCollectionId)) {
+            if (historizedSchema == null || historizedSchema.storeOnlyMonitoredTables()) {
+                LOG.trace("Filtering schema change event for {}", dataCollectionId);
+                return;
+            }
+        }
+        schemaChangeEventEmitter.emitSchemaChangeEvent(new SchemaChangeEventReceiver());
+    }
+
+    @Override
+    public void dispatchSchemaChangeEvent(
+            Collection<T> dataCollectionIds, SchemaChangeEventEmitter schemaChangeEventEmitter)
+            throws InterruptedException {
+        boolean anyNonfilteredEvent = false;
+        if (dataCollectionIds == null || dataCollectionIds.isEmpty()) {
+            anyNonfilteredEvent = true;
+        } else {
+            for (T dataCollectionId : dataCollectionIds) {
+                if (filter.isIncluded(dataCollectionId)) {
+                    anyNonfilteredEvent = true;
+                    break;
+                }
+            }
+        }
+        if (!anyNonfilteredEvent) {
+            if (historizedSchema == null || historizedSchema.storeOnlyMonitoredTables()) {
+                LOG.trace("Filtering schema change event for {}", dataCollectionIds);
+                return;
+            }
+        }
+
+        schemaChangeEventEmitter.emitSchemaChangeEvent(new SchemaChangeEventReceiver());
+    }
+
+    /** A {@link SchemaChangeEventEmitter.Receiver} implementation for {@link SchemaChangeEvent}. */
+    private final class SchemaChangeEventReceiver implements SchemaChangeEventEmitter.Receiver {
+
+        private Struct schemaChangeRecordKey(SchemaChangeEvent event) {
+            Struct result = new Struct(schemaChangeKeySchema);
+            result.put(HistoryRecord.Fields.DATABASE_NAME, event.getDatabase());
+            return result;
+        }
+
+        private Struct schemaChangeRecordValue(SchemaChangeEvent event) throws IOException {
+            Struct sourceInfo = event.getSource();
+            Map<String, Object> source = new HashMap<>();
+            String fileName = sourceInfo.getString(BINLOG_FILENAME_OFFSET_KEY);
+            Long pos = sourceInfo.getInt64(BINLOG_POSITION_OFFSET_KEY);
+            Long serverId = sourceInfo.getInt64(SERVER_ID_KEY);
+            source.put(SERVER_ID_KEY, serverId);
+            source.put(BINLOG_FILENAME_OFFSET_KEY, fileName);
+            source.put(BINLOG_POSITION_OFFSET_KEY, pos);
+            HistoryRecord historyRecord =
+                    new HistoryRecord(
+                            source,
+                            event.getOffset(),
+                            event.getDatabase(),
+                            null,
+                            event.getDdl(),
+                            event.getTableChanges());
+            String historyStr = DOCUMENT_WRITER.write(historyRecord.document());
+
+            Struct value = new Struct(schemaChangeValueSchema);
+            value.put(HistoryRecord.Fields.SOURCE, event.getSource());
+            value.put(HISTORY_RECORD_FIELD, historyStr);
+            return value;
+        }
+
+        @Override
+        public void schemaChangeEvent(SchemaChangeEvent event) throws InterruptedException {
+            historizedSchema.applySchemaChange(event);
+            if (connectorConfig.isSchemaChangesHistoryEnabled()) {
+                try {
+                    final String topicName = topicSelector.getPrimaryTopic();
+                    final Integer partition = 0;
+                    final Struct key = schemaChangeRecordKey(event);
+                    final Struct value = schemaChangeRecordValue(event);
+                    final SourceRecord record =
+                            new SourceRecord(
+                                    event.getPartition(),
+                                    event.getOffset(),
+                                    topicName,
+                                    partition,
+                                    schemaChangeKeySchema,
+                                    key,
+                                    schemaChangeValueSchema,
+                                    value);
+                    queue.enqueue(new DataChangeEvent(record));
+                } catch (IOException e) {
+                    throw new IllegalStateException(
+                            String.format("dispatch schema change event %s error ", event), e);
+                }
+            }
+        }
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/dispatcher/SignalEventDispatcher.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/dispatcher/SignalEventDispatcher.java
new file mode 100644
index 000000000..7ae066ba4
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/dispatcher/SignalEventDispatcher.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.dispatcher;
+
+import io.debezium.connector.base.ChangeEventQueue;
+import io.debezium.pipeline.DataChangeEvent;
+import io.debezium.util.SchemaNameAdjuster;
+import java.util.Map;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSplit;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.data.SchemaBuilder;
+import org.apache.kafka.connect.data.Struct;
+import org.apache.kafka.connect.source.SourceRecord;
+
+/**
+ * A dispatcher to dispatch watermark signal events.
+ *
+ * <p>The watermark signal event is used to describe the start point and end point of a split scan.
+ * The Watermark Signal Algorithm is inspired by https://arxiv.org/pdf/2010.12597v1.pdf.
+ */
+public class SignalEventDispatcher {
+
+    private static final SchemaNameAdjuster SCHEMA_NAME_ADJUSTER = SchemaNameAdjuster.create();
+
+    public static final String DATABASE_NAME = "db";
+    public static final String TABLE_NAME = "table";
+    public static final String WATERMARK_SIGNAL = "_split_watermark_signal_";
+    public static final String SPLIT_ID_KEY = "split_id";
+    public static final String BINLOG_FILENAME_OFFSET_KEY = "file";
+    public static final String BINLOG_POSITION_OFFSET_KEY = "pos";
+    public static final String WATERMARK_KIND = "watermark_kind";
+    public static final String SIGNAL_EVENT_KEY_SCHEMA_NAME =
+            "io.debezium.connector.flink.cdc.embedded.watermark.key";
+    public static final String SIGNAL_EVENT_VALUE_SCHEMA_NAME =
+            "io.debezium.connector.flink.cdc.embedded.watermark.value";
+
+    private final Schema signalEventKeySchema;
+    private final Schema signalEventValueSchema;
+    private final Map<String, ?> sourcePartition;
+    private final String topic;
+    private final ChangeEventQueue<DataChangeEvent> queue;
+
+    public SignalEventDispatcher(
+            Map<String, ?> sourcePartition, String topic, ChangeEventQueue<DataChangeEvent> queue) {
+        this.sourcePartition = sourcePartition;
+        this.topic = topic;
+        this.queue = queue;
+        this.signalEventKeySchema =
+                SchemaBuilder.struct()
+                        .name(SCHEMA_NAME_ADJUSTER.adjust(SIGNAL_EVENT_KEY_SCHEMA_NAME))
+                        .field(SPLIT_ID_KEY, Schema.STRING_SCHEMA)
+                        .field(WATERMARK_SIGNAL, Schema.BOOLEAN_SCHEMA)
+                        .build();
+        this.signalEventValueSchema =
+                SchemaBuilder.struct()
+                        .name(SCHEMA_NAME_ADJUSTER.adjust(SIGNAL_EVENT_VALUE_SCHEMA_NAME))
+                        .field(SPLIT_ID_KEY, Schema.STRING_SCHEMA)
+                        .field(WATERMARK_KIND, Schema.STRING_SCHEMA)
+                        .build();
+    }
+
+    public void dispatchWatermarkEvent(
+            MySqlSplit mySqlSplit, BinlogOffset watermark, WatermarkKind watermarkKind)
+            throws InterruptedException {
+
+        SourceRecord sourceRecord =
+                new SourceRecord(
+                        sourcePartition,
+                        watermark.getOffset(),
+                        topic,
+                        signalEventKeySchema,
+                        signalRecordKey(mySqlSplit.splitId()),
+                        signalEventValueSchema,
+                        signalRecordValue(mySqlSplit.splitId(), watermarkKind));
+        queue.enqueue(new DataChangeEvent(sourceRecord));
+    }
+
+    private Struct signalRecordKey(String splitId) {
+        Struct result = new Struct(signalEventKeySchema);
+        result.put(SPLIT_ID_KEY, splitId);
+        result.put(WATERMARK_SIGNAL, true);
+        return result;
+    }
+
+    private Struct signalRecordValue(String splitId, WatermarkKind watermarkKind) {
+        Struct result = new Struct(signalEventValueSchema);
+        result.put(SPLIT_ID_KEY, splitId);
+        result.put(WATERMARK_KIND, watermarkKind.toString());
+        return result;
+    }
+
+    /** The watermark kind. */
+    public enum WatermarkKind {
+        LOW,
+        HIGH,
+        BINLOG_END;
+
+        public WatermarkKind fromString(String kindString) {
+            if (LOW.name().equalsIgnoreCase(kindString)) {
+                return LOW;
+            } else if (HIGH.name().equalsIgnoreCase(kindString)) {
+                return HIGH;
+            } else {
+                return BINLOG_END;
+            }
+        }
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/reader/BinlogSplitReader.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/reader/BinlogSplitReader.java
new file mode 100644
index 000000000..e684ca4c8
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/reader/BinlogSplitReader.java
@@ -0,0 +1,275 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.reader;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.RecordUtils.getBinlogPosition;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.RecordUtils.getSplitKey;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.RecordUtils.getTableId;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.RecordUtils.isDataChangeRecord;
+
+import io.debezium.connector.base.ChangeEventQueue;
+import io.debezium.connector.mysql.MySqlOffsetContext;
+import io.debezium.connector.mysql.MySqlStreamingChangeEventSourceMetrics;
+import io.debezium.pipeline.DataChangeEvent;
+import io.debezium.pipeline.source.spi.ChangeEventSource;
+import io.debezium.relational.TableId;
+import io.debezium.relational.Tables;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import javax.annotation.Nullable;
+import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.task.MySqlBinlogSplitReadTask;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.task.context.StatefulTaskContext;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.FinishedSnapshotSplitInfo;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlBinlogSplit;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSplit;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.ChunkUtils;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.RecordUtils;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A Debezium binlog reader implementation that also support reads binlog and filter overlapping
+ * snapshot data that {@link SnapshotSplitReader} read.
+ */
+public class BinlogSplitReader implements DebeziumReader<SourceRecord, MySqlSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(BinlogSplitReader.class);
+    private final StatefulTaskContext statefulTaskContext;
+    private final ExecutorService executor;
+
+    private volatile ChangeEventQueue<DataChangeEvent> queue;
+    private volatile boolean currentTaskRunning;
+    private volatile Throwable readException;
+
+    private MySqlBinlogSplitReadTask binlogSplitReadTask;
+    private MySqlBinlogSplit currentBinlogSplit;
+    private Map<TableId, List<FinishedSnapshotSplitInfo>> finishedSplitsInfo;
+    // tableId -> the max splitHighWatermark
+    private Map<TableId, BinlogOffset> maxSplitHighWatermarkMap;
+    private Tables.TableFilter capturedTableFilter;
+
+    public BinlogSplitReader(StatefulTaskContext statefulTaskContext, int subTaskId) {
+        this.statefulTaskContext = statefulTaskContext;
+        ThreadFactory threadFactory =
+                new ThreadFactoryBuilder().setNameFormat("debezium-reader-" + subTaskId).build();
+        this.executor = Executors.newSingleThreadExecutor(threadFactory);
+        this.currentTaskRunning = true;
+    }
+
+    @Override
+    public void submitSplit(MySqlSplit mySqlSplit) {
+        this.currentBinlogSplit = mySqlSplit.asBinlogSplit();
+        configureFilter();
+        statefulTaskContext.configure(currentBinlogSplit);
+        this.capturedTableFilter =
+                statefulTaskContext.getConnectorConfig().getTableFilters().dataCollectionFilter();
+        this.queue = statefulTaskContext.getQueue();
+        final MySqlOffsetContext mySqlOffsetContext = statefulTaskContext.getOffsetContext();
+        this.binlogSplitReadTask =
+                new MySqlBinlogSplitReadTask(
+                        statefulTaskContext.getConnectorConfig(),
+                        mySqlOffsetContext,
+                        statefulTaskContext.getConnection(),
+                        statefulTaskContext.getDispatcher(),
+                        statefulTaskContext.getErrorHandler(),
+                        StatefulTaskContext.getClock(),
+                        statefulTaskContext.getTaskContext(),
+                        (MySqlStreamingChangeEventSourceMetrics)
+                                statefulTaskContext.getStreamingChangeEventSourceMetrics(),
+                        statefulTaskContext.getTopicSelector().getPrimaryTopic(),
+                        currentBinlogSplit);
+
+        executor.submit(
+                () -> {
+                    try {
+                        binlogSplitReadTask.execute(new BinlogSplitChangeEventSourceContextImpl());
+                    } catch (Exception e) {
+                        currentTaskRunning = false;
+                        LOG.error(
+                                String.format(
+                                        "Execute binlog read task for mysql split %s fail",
+                                        currentBinlogSplit),
+                                e);
+                        readException = e;
+                    }
+                });
+    }
+
+    private class BinlogSplitChangeEventSourceContextImpl
+            implements ChangeEventSource.ChangeEventSourceContext {
+        @Override
+        public boolean isRunning() {
+            return currentTaskRunning;
+        }
+    }
+
+    @Override
+    public boolean isFinished() {
+        return currentBinlogSplit == null || !currentTaskRunning;
+    }
+
+    @Nullable
+    @Override
+    public Iterator<SourceRecord> pollSplitRecords() throws InterruptedException {
+        checkReadException();
+        final List<SourceRecord> sourceRecords = new ArrayList<>();
+        if (currentTaskRunning) {
+            List<DataChangeEvent> batch = queue.poll();
+            for (DataChangeEvent event : batch) {
+                if (shouldEmit(event.getRecord())) {
+                    sourceRecords.add(event.getRecord());
+                }
+            }
+            return sourceRecords.iterator();
+        } else {
+            return null;
+        }
+    }
+
+    private void checkReadException() {
+        if (readException != null) {
+            throw new FlinkRuntimeException(
+                    String.format(
+                            "Read split %s error due to %s.",
+                            currentBinlogSplit, readException.getMessage()),
+                    readException);
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            if (statefulTaskContext.getConnection() != null) {
+                statefulTaskContext.getConnection().close();
+            }
+            if (statefulTaskContext.getBinaryLogClient() != null) {
+                statefulTaskContext.getBinaryLogClient().disconnect();
+            }
+        } catch (Exception e) {
+            LOG.error("Close binlog reader error", e);
+        }
+    }
+
+    /**
+     * Returns the record should emit or not.
+     *
+     * <p>The watermark signal algorithm is the binlog split reader only sends the binlog event that
+     * belongs to its finished snapshot splits. For each snapshot split, the binlog event is valid
+     * since the offset is after its high watermark.
+     *
+     * <pre> E.g: the data input is :
+     *    snapshot-split-0 info : [0,    1024) highWatermark0
+     *    snapshot-split-1 info : [1024, 2048) highWatermark1
+     *  the data output is:
+     *  only the binlog event belong to [0,    1024) and offset is after highWatermark0 should send,
+     *  only the binlog event belong to [1024, 2048) and offset is after highWatermark1 should send.
+     * </pre>
+     */
+    private boolean shouldEmit(SourceRecord sourceRecord) {
+        if (isDataChangeRecord(sourceRecord)) {
+            TableId tableId = getTableId(sourceRecord);
+            BinlogOffset position = getBinlogPosition(sourceRecord);
+            if (hasEnterPureBinlogPhase(tableId, position)) {
+                return true;
+            }
+            // only the table who captured snapshot splits need to filter
+            if (finishedSplitsInfo.containsKey(tableId)) {
+                RowType splitKeyType =
+                        ChunkUtils.getSplitType(
+                                statefulTaskContext.getDatabaseSchema().tableFor(tableId));
+                Object[] key =
+                        getSplitKey(
+                                splitKeyType,
+                                sourceRecord,
+                                statefulTaskContext.getSchemaNameAdjuster());
+                for (FinishedSnapshotSplitInfo splitInfo : finishedSplitsInfo.get(tableId)) {
+                    if (RecordUtils.splitKeyRangeContains(
+                                    key, splitInfo.getSplitStart(), splitInfo.getSplitEnd())
+                            && position.isAfter(splitInfo.getHighWatermark())) {
+                        return true;
+                    }
+                }
+            }
+            // not in the monitored splits scope, do not emit
+            return false;
+        }
+        // always send the schema change event and signal event
+        // we need record them to state of Flink
+        return true;
+    }
+
+    private boolean hasEnterPureBinlogPhase(TableId tableId, BinlogOffset position) {
+        // the existed tables those have finished snapshot reading
+        if (maxSplitHighWatermarkMap.containsKey(tableId)
+                && position.isAtOrAfter(maxSplitHighWatermarkMap.get(tableId))) {
+            return true;
+        }
+        // capture dynamically new added tables
+        // TODO: there is still very little chance that we can't capture new added table.
+        //  That the tables dynamically added after discovering captured tables in enumerator
+        //  and before the lowest binlog offset of all table splits. This interval should be
+        //  very short, so we don't support it for now.
+        return !maxSplitHighWatermarkMap.containsKey(tableId)
+                && capturedTableFilter.isIncluded(tableId);
+    }
+
+    private void configureFilter() {
+        List<FinishedSnapshotSplitInfo> finishedSplitInfos =
+                currentBinlogSplit.getFinishedSnapshotSplitInfos();
+        Map<TableId, List<FinishedSnapshotSplitInfo>> splitsInfoMap = new HashMap<>();
+        Map<TableId, BinlogOffset> tableIdBinlogPositionMap = new HashMap<>();
+        // latest-offset mode
+        if (finishedSplitInfos.isEmpty()) {
+            for (TableId tableId : currentBinlogSplit.getTableSchemas().keySet()) {
+                tableIdBinlogPositionMap.put(tableId, currentBinlogSplit.getStartingOffset());
+            }
+        } else {
+            for (FinishedSnapshotSplitInfo finishedSplitInfo : finishedSplitInfos) {
+                TableId tableId = finishedSplitInfo.getTableId();
+                List<FinishedSnapshotSplitInfo> list =
+                        splitsInfoMap.getOrDefault(tableId, new ArrayList<>());
+                list.add(finishedSplitInfo);
+                splitsInfoMap.put(tableId, list);
+
+                BinlogOffset highWatermark = finishedSplitInfo.getHighWatermark();
+                BinlogOffset maxHighWatermark = tableIdBinlogPositionMap.get(tableId);
+                if (maxHighWatermark == null || highWatermark.isAfter(maxHighWatermark)) {
+                    tableIdBinlogPositionMap.put(tableId, highWatermark);
+                }
+            }
+        }
+        this.finishedSplitsInfo = splitsInfoMap;
+        this.maxSplitHighWatermarkMap = tableIdBinlogPositionMap;
+    }
+
+    public void stopBinlogReadTask() {
+        this.currentTaskRunning = false;
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/reader/DebeziumReader.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/reader/DebeziumReader.java
new file mode 100644
index 000000000..e6799bb37
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/reader/DebeziumReader.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.reader;
+
+import java.util.Iterator;
+import javax.annotation.Nullable;
+
+/** Reader to read split of table, the split is either snapshot split or binlog split. */
+public interface DebeziumReader<T, Split> {
+
+    /** Return the current split of the reader is finished or not. */
+    boolean isFinished();
+
+    /**
+     * Add to split to read, this should call only the when reader is idle.
+     *
+     * @param splitToRead
+     */
+    void submitSplit(Split splitToRead);
+
+    /** Close the reader and releases all resources. */
+    void close();
+
+    /**
+     * Reads records from MySQL. The method should return null when reaching the end of the split,
+     * the empty {@link Iterator} will be returned if the data of split is on pulling.
+     */
+    @Nullable
+    Iterator<T> pollSplitRecords() throws InterruptedException;
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/reader/SnapshotSplitReader.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/reader/SnapshotSplitReader.java
new file mode 100644
index 000000000..50f7c7643
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/reader/SnapshotSplitReader.java
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.reader;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.RecordUtils.normalizedSplitRecords;
+
+import io.debezium.config.Configuration;
+import io.debezium.connector.base.ChangeEventQueue;
+import io.debezium.connector.mysql.MySqlConnectorConfig;
+import io.debezium.connector.mysql.MySqlOffsetContext;
+import io.debezium.connector.mysql.MySqlStreamingChangeEventSourceMetrics;
+import io.debezium.heartbeat.Heartbeat;
+import io.debezium.pipeline.DataChangeEvent;
+import io.debezium.pipeline.source.spi.ChangeEventSource;
+import io.debezium.pipeline.spi.SnapshotResult;
+import io.debezium.util.SchemaNameAdjuster;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicBoolean;
+import javax.annotation.Nullable;
+import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.dispatcher.SignalEventDispatcher;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.task.MySqlBinlogSplitReadTask;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.task.MySqlSnapshotSplitReadTask;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.task.context.StatefulTaskContext;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlBinlogSplit;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSnapshotSplit;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSplit;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.RecordUtils;
+import org.apache.kafka.connect.source.SourceRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A snapshot reader that reads data from Table in split level, the split is assigned by primary key
+ * range.
+ */
+public class SnapshotSplitReader implements DebeziumReader<SourceRecord, MySqlSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SnapshotSplitReader.class);
+    private final StatefulTaskContext statefulTaskContext;
+    private final ExecutorService executor;
+
+    private volatile ChangeEventQueue<DataChangeEvent> queue;
+    private volatile boolean currentTaskRunning;
+    private volatile Throwable readException;
+
+    // task to read snapshot for current split
+    private MySqlSnapshotSplitReadTask splitSnapshotReadTask;
+    private MySqlSnapshotSplit currentSnapshotSplit;
+    private SchemaNameAdjuster nameAdjuster;
+    public AtomicBoolean hasNextElement;
+    public AtomicBoolean reachEnd;
+
+    public SnapshotSplitReader(StatefulTaskContext statefulTaskContext, int subtaskId) {
+        this.statefulTaskContext = statefulTaskContext;
+        ThreadFactory threadFactory =
+                new ThreadFactoryBuilder().setNameFormat("debezium-reader-" + subtaskId).build();
+        this.executor = Executors.newSingleThreadExecutor(threadFactory);
+        this.currentTaskRunning = false;
+        this.hasNextElement = new AtomicBoolean(false);
+        this.reachEnd = new AtomicBoolean(false);
+    }
+
+    @Override
+    public void submitSplit(MySqlSplit mySqlSplit) {
+        this.currentSnapshotSplit = mySqlSplit.asSnapshotSplit();
+        statefulTaskContext.configure(currentSnapshotSplit);
+        this.queue = statefulTaskContext.getQueue();
+        this.nameAdjuster = statefulTaskContext.getSchemaNameAdjuster();
+        this.hasNextElement.set(true);
+        this.reachEnd.set(false);
+        this.splitSnapshotReadTask =
+                new MySqlSnapshotSplitReadTask(
+                        statefulTaskContext.getConnectorConfig(),
+                        statefulTaskContext.getOffsetContext(),
+                        statefulTaskContext.getSnapshotChangeEventSourceMetrics(),
+                        statefulTaskContext.getDatabaseSchema(),
+                        statefulTaskContext.getConnection(),
+                        statefulTaskContext.getDispatcher(),
+                        statefulTaskContext.getTopicSelector(),
+                        StatefulTaskContext.getClock(),
+                        currentSnapshotSplit);
+        executor.submit(
+                () -> {
+                    try {
+                        currentTaskRunning = true;
+                        // execute snapshot read task
+                        final SnapshotSplitChangeEventSourceContextImpl sourceContext =
+                                new SnapshotSplitChangeEventSourceContextImpl();
+                        SnapshotResult snapshotResult =
+                                splitSnapshotReadTask.execute(sourceContext);
+
+                        final MySqlBinlogSplit backfillBinlogSplit =
+                                createBackfillBinlogSplit(sourceContext);
+                        // optimization that skip the binlog read when the low watermark equals high
+                        // watermark
+                        final boolean binlogBackfillRequired =
+                                backfillBinlogSplit
+                                        .getEndingOffset()
+                                        .isAfter(backfillBinlogSplit.getStartingOffset());
+                        if (!binlogBackfillRequired) {
+                            dispatchBinlogEndEvent(backfillBinlogSplit);
+                            currentTaskRunning = false;
+                            return;
+                        }
+
+                        // execute binlog read task
+                        if (snapshotResult.isCompletedOrSkipped()) {
+                            final MySqlBinlogSplitReadTask backfillBinlogReadTask =
+                                    createBackfillBinlogReadTask(backfillBinlogSplit);
+                            backfillBinlogReadTask.execute(
+                                    new SnapshotBinlogSplitChangeEventSourceContextImpl());
+                        } else {
+                            readException =
+                                    new IllegalStateException(
+                                            String.format(
+                                                    "Read snapshot for mysql split %s fail",
+                                                    currentSnapshotSplit));
+                        }
+                    } catch (Exception e) {
+                        currentTaskRunning = false;
+                        LOG.error(
+                                String.format(
+                                        "Execute snapshot read task for mysql split %s fail",
+                                        currentSnapshotSplit),
+                                e);
+                        readException = e;
+                    }
+                });
+    }
+
+    private MySqlBinlogSplit createBackfillBinlogSplit(
+            SnapshotSplitChangeEventSourceContextImpl sourceContext) {
+        return new MySqlBinlogSplit(
+                currentSnapshotSplit.splitId(),
+                sourceContext.getLowWatermark(),
+                sourceContext.getHighWatermark(),
+                new ArrayList<>(),
+                currentSnapshotSplit.getTableSchemas(),
+                0);
+    }
+
+    private MySqlBinlogSplitReadTask createBackfillBinlogReadTask(
+            MySqlBinlogSplit backfillBinlogSplit) {
+        final MySqlOffsetContext.Loader loader =
+                new MySqlOffsetContext.Loader(statefulTaskContext.getConnectorConfig());
+        final MySqlOffsetContext mySqlOffsetContext =
+                (MySqlOffsetContext)
+                        loader.load(backfillBinlogSplit.getStartingOffset().getOffset());
+        // we should only capture events for the current table,
+        // otherwise, we may can't find corresponding schema
+        Configuration dezConf =
+                statefulTaskContext
+                        .getSourceConfig()
+                        .getDbzConfiguration()
+                        .edit()
+                        .with("table.include.list", currentSnapshotSplit.getTableId().toString())
+                        // Disable heartbeat event in snapshot split reader
+                        .with(Heartbeat.HEARTBEAT_INTERVAL, 0)
+                        .build();
+        // task to read binlog and backfill for current split
+        return new MySqlBinlogSplitReadTask(
+                new MySqlConnectorConfig(dezConf),
+                mySqlOffsetContext,
+                statefulTaskContext.getConnection(),
+                statefulTaskContext.getDispatcher(),
+                statefulTaskContext.getErrorHandler(),
+                StatefulTaskContext.getClock(),
+                statefulTaskContext.getTaskContext(),
+                (MySqlStreamingChangeEventSourceMetrics)
+                        statefulTaskContext.getStreamingChangeEventSourceMetrics(),
+                statefulTaskContext.getTopicSelector().getPrimaryTopic(),
+                backfillBinlogSplit);
+    }
+
+    private void dispatchBinlogEndEvent(MySqlBinlogSplit backFillBinlogSplit)
+            throws InterruptedException {
+        final SignalEventDispatcher signalEventDispatcher =
+                new SignalEventDispatcher(
+                        statefulTaskContext.getOffsetContext().getPartition(),
+                        statefulTaskContext.getTopicSelector().getPrimaryTopic(),
+                        statefulTaskContext.getDispatcher().getQueue());
+        signalEventDispatcher.dispatchWatermarkEvent(
+                backFillBinlogSplit,
+                backFillBinlogSplit.getEndingOffset(),
+                SignalEventDispatcher.WatermarkKind.BINLOG_END);
+    }
+
+    @Override
+    public boolean isFinished() {
+        return currentSnapshotSplit == null
+                || (!currentTaskRunning && !hasNextElement.get() && reachEnd.get());
+    }
+
+    @Nullable
+    @Override
+    public Iterator<SourceRecord> pollSplitRecords() throws InterruptedException {
+        checkReadException();
+
+        if (hasNextElement.get()) {
+            // data input: [low watermark event][snapshot events][high watermark event][binlog
+            // events][binlog-end event]
+            // data output: [low watermark event][normalized events][high watermark event]
+            boolean reachBinlogEnd = false;
+            final List<SourceRecord> sourceRecords = new ArrayList<>();
+            while (!reachBinlogEnd) {
+                List<DataChangeEvent> batch = queue.poll();
+                for (DataChangeEvent event : batch) {
+                    sourceRecords.add(event.getRecord());
+                    if (RecordUtils.isEndWatermarkEvent(event.getRecord())) {
+                        reachBinlogEnd = true;
+                        break;
+                    }
+                }
+            }
+            // snapshot split return its data once
+            hasNextElement.set(false);
+            return normalizedSplitRecords(currentSnapshotSplit, sourceRecords, nameAdjuster)
+                    .iterator();
+        }
+        // the data has been polled, no more data
+        reachEnd.compareAndSet(false, true);
+        return null;
+    }
+
+    private void checkReadException() {
+        if (readException != null) {
+            throw new FlinkRuntimeException(
+                    String.format(
+                            "Read split %s error due to %s.",
+                            currentSnapshotSplit, readException.getMessage()),
+                    readException);
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            if (statefulTaskContext.getConnection() != null) {
+                statefulTaskContext.getConnection().close();
+            }
+            if (statefulTaskContext.getBinaryLogClient() != null) {
+                statefulTaskContext.getBinaryLogClient().disconnect();
+            }
+        } catch (Exception e) {
+            LOG.error("Close snapshot reader error", e);
+        }
+    }
+
+    /**
+     * {@link ChangeEventSource.ChangeEventSourceContext} implementation that keeps low/high
+     * watermark for each {@link MySqlSnapshotSplit}.
+     */
+    public class SnapshotSplitChangeEventSourceContextImpl
+            implements ChangeEventSource.ChangeEventSourceContext {
+
+        private BinlogOffset lowWatermark;
+        private BinlogOffset highWatermark;
+
+        public BinlogOffset getLowWatermark() {
+            return lowWatermark;
+        }
+
+        public void setLowWatermark(BinlogOffset lowWatermark) {
+            this.lowWatermark = lowWatermark;
+        }
+
+        public BinlogOffset getHighWatermark() {
+            return highWatermark;
+        }
+
+        public void setHighWatermark(BinlogOffset highWatermark) {
+            this.highWatermark = highWatermark;
+        }
+
+        @Override
+        public boolean isRunning() {
+            return lowWatermark != null && highWatermark != null;
+        }
+    }
+
+    /**
+     * The {@link ChangeEventSource.ChangeEventSourceContext} implementation for bounded binlog task
+     * of a snapshot split task.
+     */
+    public class SnapshotBinlogSplitChangeEventSourceContextImpl
+            implements ChangeEventSource.ChangeEventSourceContext {
+
+        public void finished() {
+            currentTaskRunning = false;
+        }
+
+        @Override
+        public boolean isRunning() {
+            return currentTaskRunning;
+        }
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/MySqlBinlogSplitReadTask.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/MySqlBinlogSplitReadTask.java
new file mode 100644
index 000000000..07d0ca16d
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/MySqlBinlogSplitReadTask.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.task;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset.NO_STOPPING_OFFSET;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.RecordUtils.getBinlogPosition;
+
+import com.github.shyiko.mysql.binlog.event.Event;
+import io.debezium.DebeziumException;
+import io.debezium.connector.mysql.MySqlConnection;
+import io.debezium.connector.mysql.MySqlConnectorConfig;
+import io.debezium.connector.mysql.MySqlOffsetContext;
+import io.debezium.connector.mysql.MySqlStreamingChangeEventSource;
+import io.debezium.connector.mysql.MySqlStreamingChangeEventSourceMetrics;
+import io.debezium.connector.mysql.MySqlTaskContext;
+import io.debezium.pipeline.ErrorHandler;
+import io.debezium.relational.TableId;
+import io.debezium.util.Clock;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.dispatcher.EventDispatcherImpl;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.dispatcher.SignalEventDispatcher;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.reader.SnapshotSplitReader.SnapshotBinlogSplitChangeEventSourceContextImpl;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlBinlogSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Task to read all binlog for table and also supports read bounded (from lowWatermark to
+ * highWatermark) binlog.
+ */
+public class MySqlBinlogSplitReadTask extends MySqlStreamingChangeEventSource {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MySqlBinlogSplitReadTask.class);
+    private final MySqlBinlogSplit binlogSplit;
+    private final MySqlOffsetContext offsetContext;
+    private final EventDispatcherImpl<TableId> eventDispatcher;
+    private final SignalEventDispatcher signalEventDispatcher;
+    private final ErrorHandler errorHandler;
+    private ChangeEventSourceContext context;
+
+    public MySqlBinlogSplitReadTask(
+            MySqlConnectorConfig connectorConfig,
+            MySqlOffsetContext offsetContext,
+            MySqlConnection connection,
+            EventDispatcherImpl<TableId> dispatcher,
+            ErrorHandler errorHandler,
+            Clock clock,
+            MySqlTaskContext taskContext,
+            MySqlStreamingChangeEventSourceMetrics metrics,
+            String topic,
+            MySqlBinlogSplit binlogSplit) {
+        super(
+                connectorConfig,
+                offsetContext,
+                connection,
+                dispatcher,
+                errorHandler,
+                clock,
+                taskContext,
+                metrics);
+        this.binlogSplit = binlogSplit;
+        this.eventDispatcher = dispatcher;
+        this.offsetContext = offsetContext;
+        this.errorHandler = errorHandler;
+        this.signalEventDispatcher =
+                new SignalEventDispatcher(
+                        offsetContext.getPartition(), topic, eventDispatcher.getQueue());
+    }
+
+    @Override
+    public void execute(ChangeEventSourceContext context) throws InterruptedException {
+        this.context = context;
+        super.execute(context);
+    }
+
+    @Override
+    protected void handleEvent(Event event) {
+        super.handleEvent(event);
+        // check do we need to stop for read binlog for snapshot split.
+        if (isBoundedRead()) {
+            final BinlogOffset currentBinlogOffset = getBinlogPosition(offsetContext.getOffset());
+            // reach the high watermark, the binlog reader should finished
+            if (currentBinlogOffset.isAtOrAfter(binlogSplit.getEndingOffset())) {
+                // send binlog end event
+                try {
+                    signalEventDispatcher.dispatchWatermarkEvent(
+                            binlogSplit,
+                            currentBinlogOffset,
+                            SignalEventDispatcher.WatermarkKind.BINLOG_END);
+                } catch (InterruptedException e) {
+                    LOG.error("Send signal event error.", e);
+                    errorHandler.setProducerThrowable(
+                            new DebeziumException("Error processing binlog signal event", e));
+                }
+                // tell reader the binlog task finished
+                ((SnapshotBinlogSplitChangeEventSourceContextImpl) context).finished();
+            }
+        }
+    }
+
+    private boolean isBoundedRead() {
+        return !NO_STOPPING_OFFSET.equals(binlogSplit.getEndingOffset());
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/MySqlSnapshotSplitReadTask.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/MySqlSnapshotSplitReadTask.java
new file mode 100644
index 000000000..17949fb60
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/MySqlSnapshotSplitReadTask.java
@@ -0,0 +1,362 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.task;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.DebeziumUtils.currentBinlogOffset;
+
+import io.debezium.DebeziumException;
+import io.debezium.connector.mysql.MySqlConnection;
+import io.debezium.connector.mysql.MySqlConnectorConfig;
+import io.debezium.connector.mysql.MySqlDatabaseSchema;
+import io.debezium.connector.mysql.MySqlOffsetContext;
+import io.debezium.connector.mysql.MySqlValueConverters;
+import io.debezium.pipeline.EventDispatcher;
+import io.debezium.pipeline.source.AbstractSnapshotChangeEventSource;
+import io.debezium.pipeline.source.spi.SnapshotProgressListener;
+import io.debezium.pipeline.spi.ChangeRecordEmitter;
+import io.debezium.pipeline.spi.OffsetContext;
+import io.debezium.pipeline.spi.SnapshotResult;
+import io.debezium.relational.Column;
+import io.debezium.relational.RelationalSnapshotChangeEventSource;
+import io.debezium.relational.SnapshotChangeRecordEmitter;
+import io.debezium.relational.Table;
+import io.debezium.relational.TableId;
+import io.debezium.schema.TopicSelector;
+import io.debezium.util.Clock;
+import io.debezium.util.ColumnUtils;
+import io.debezium.util.Strings;
+import io.debezium.util.Threads;
+import java.io.UnsupportedEncodingException;
+import java.sql.Blob;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.time.Duration;
+import java.util.Calendar;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.dispatcher.EventDispatcherImpl;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.dispatcher.SignalEventDispatcher;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.reader.SnapshotSplitReader;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSnapshotSplit;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.StatementUtils;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Task to read snapshot split of table. */
+public class MySqlSnapshotSplitReadTask extends AbstractSnapshotChangeEventSource {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MySqlSnapshotSplitReadTask.class);
+
+    /** Interval for showing a log statement with the progress while scanning a single table. */
+    private static final Duration LOG_INTERVAL = Duration.ofMillis(10_000);
+
+    private final MySqlConnectorConfig connectorConfig;
+    private final MySqlDatabaseSchema databaseSchema;
+    private final MySqlConnection jdbcConnection;
+    private final EventDispatcherImpl<TableId> dispatcher;
+    private final Clock clock;
+    private final MySqlSnapshotSplit snapshotSplit;
+    private final MySqlOffsetContext offsetContext;
+    private final TopicSelector<TableId> topicSelector;
+    private final SnapshotProgressListener snapshotProgressListener;
+
+    public MySqlSnapshotSplitReadTask(
+            MySqlConnectorConfig connectorConfig,
+            MySqlOffsetContext previousOffset,
+            SnapshotProgressListener snapshotProgressListener,
+            MySqlDatabaseSchema databaseSchema,
+            MySqlConnection jdbcConnection,
+            EventDispatcherImpl<TableId> dispatcher,
+            TopicSelector<TableId> topicSelector,
+            Clock clock,
+            MySqlSnapshotSplit snapshotSplit) {
+        super(connectorConfig, previousOffset, snapshotProgressListener);
+        this.offsetContext = previousOffset;
+        this.connectorConfig = connectorConfig;
+        this.databaseSchema = databaseSchema;
+        this.jdbcConnection = jdbcConnection;
+        this.dispatcher = dispatcher;
+        this.clock = clock;
+        this.snapshotSplit = snapshotSplit;
+        this.topicSelector = topicSelector;
+        this.snapshotProgressListener = snapshotProgressListener;
+    }
+
+    @Override
+    public SnapshotResult execute(ChangeEventSourceContext context) throws InterruptedException {
+        SnapshottingTask snapshottingTask = getSnapshottingTask(previousOffset);
+        final SnapshotContext ctx;
+        try {
+            ctx = prepare(context);
+        } catch (Exception e) {
+            LOG.error("Failed to initialize snapshot context.", e);
+            throw new RuntimeException(e);
+        }
+        try {
+            return doExecute(context, ctx, snapshottingTask);
+        } catch (InterruptedException e) {
+            LOG.warn("Snapshot was interrupted before completion");
+            throw e;
+        } catch (Exception t) {
+            throw new DebeziumException(t);
+        }
+    }
+
+    @Override
+    protected SnapshotResult doExecute(
+            ChangeEventSourceContext context,
+            SnapshotContext snapshotContext,
+            SnapshottingTask snapshottingTask)
+            throws Exception {
+        final RelationalSnapshotChangeEventSource.RelationalSnapshotContext ctx =
+                (RelationalSnapshotChangeEventSource.RelationalSnapshotContext) snapshotContext;
+        ctx.offset = offsetContext;
+        final SignalEventDispatcher signalEventDispatcher =
+                new SignalEventDispatcher(
+                        offsetContext.getPartition(),
+                        topicSelector.topicNameFor(snapshotSplit.getTableId()),
+                        dispatcher.getQueue());
+
+        final BinlogOffset lowWatermark = currentBinlogOffset(jdbcConnection);
+        LOG.info(
+                "Snapshot step 1 - Determining low watermark {} for split {}",
+                lowWatermark,
+                snapshotSplit);
+        ((SnapshotSplitReader.SnapshotSplitChangeEventSourceContextImpl) (context))
+                .setLowWatermark(lowWatermark);
+        signalEventDispatcher.dispatchWatermarkEvent(
+                snapshotSplit, lowWatermark, SignalEventDispatcher.WatermarkKind.LOW);
+
+        LOG.info("Snapshot step 2 - Snapshotting data");
+        createDataEvents(ctx, snapshotSplit.getTableId());
+
+        final BinlogOffset highWatermark = currentBinlogOffset(jdbcConnection);
+        LOG.info(
+                "Snapshot step 3 - Determining high watermark {} for split {}",
+                highWatermark,
+                snapshotSplit);
+        signalEventDispatcher.dispatchWatermarkEvent(
+                snapshotSplit, highWatermark, SignalEventDispatcher.WatermarkKind.HIGH);
+        ((SnapshotSplitReader.SnapshotSplitChangeEventSourceContextImpl) (context))
+                .setHighWatermark(highWatermark);
+
+        return SnapshotResult.completed(ctx.offset);
+    }
+
+    @Override
+    protected SnapshottingTask getSnapshottingTask(OffsetContext previousOffset) {
+        return new SnapshottingTask(false, true);
+    }
+
+    @Override
+    protected SnapshotContext prepare(ChangeEventSourceContext changeEventSourceContext)
+            throws Exception {
+        return new MySqlSnapshotContext();
+    }
+
+    private static class MySqlSnapshotContext
+            extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext {
+
+        public MySqlSnapshotContext() throws SQLException {
+            super("");
+        }
+    }
+
+    private void createDataEvents(
+            RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext,
+            TableId tableId)
+            throws Exception {
+        EventDispatcher.SnapshotReceiver snapshotReceiver =
+                dispatcher.getSnapshotChangeEventReceiver();
+        LOG.debug("Snapshotting table {}", tableId);
+        createDataEventsForTable(
+                snapshotContext, snapshotReceiver, databaseSchema.tableFor(tableId));
+        snapshotReceiver.completeSnapshot();
+    }
+
+    /** Dispatches the data change events for the records of a single table. */
+    private void createDataEventsForTable(
+            RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext,
+            EventDispatcher.SnapshotReceiver snapshotReceiver,
+            Table table)
+            throws InterruptedException {
+
+        long exportStart = clock.currentTimeInMillis();
+        LOG.info("Exporting data from split '{}' of table {}", snapshotSplit.splitId(), table.id());
+
+        final String selectSql =
+                StatementUtils.buildSplitScanQuery(
+                        snapshotSplit.getTableId(),
+                        snapshotSplit.getSplitKeyType(),
+                        snapshotSplit.getSplitStart() == null,
+                        snapshotSplit.getSplitEnd() == null);
+        LOG.info(
+                "For split '{}' of table {} using select statement: '{}'",
+                snapshotSplit.splitId(),
+                table.id(),
+                selectSql);
+
+        try (PreparedStatement selectStatement =
+                        StatementUtils.readTableSplitDataStatement(
+                                jdbcConnection,
+                                selectSql,
+                                snapshotSplit.getSplitStart() == null,
+                                snapshotSplit.getSplitEnd() == null,
+                                snapshotSplit.getSplitStart(),
+                                snapshotSplit.getSplitEnd(),
+                                snapshotSplit.getSplitKeyType().getFieldCount(),
+                                connectorConfig.getQueryFetchSize());
+                ResultSet rs = selectStatement.executeQuery()) {
+
+            ColumnUtils.ColumnArray columnArray = ColumnUtils.toArray(rs, table);
+            long rows = 0;
+            Threads.Timer logTimer = getTableScanLogTimer();
+
+            while (rs.next()) {
+                rows++;
+                final Object[] row = new Object[columnArray.getGreatestColumnPosition()];
+                for (int i = 0; i < columnArray.getColumns().length; i++) {
+                    Column actualColumn = table.columns().get(i);
+                    row[columnArray.getColumns()[i].position() - 1] =
+                            readField(rs, i + 1, actualColumn, table);
+                }
+                if (logTimer.expired()) {
+                    long stop = clock.currentTimeInMillis();
+                    LOG.info(
+                            "Exported {} records for split '{}' after {}",
+                            rows,
+                            snapshotSplit.splitId(),
+                            Strings.duration(stop - exportStart));
+                    snapshotProgressListener.rowsScanned(table.id(), rows);
+                    logTimer = getTableScanLogTimer();
+                }
+                dispatcher.dispatchSnapshotEvent(
+                        table.id(),
+                        getChangeRecordEmitter(snapshotContext, table.id(), row),
+                        snapshotReceiver);
+            }
+            LOG.info(
+                    "Finished exporting {} records for split '{}', total duration '{}'",
+                    rows,
+                    snapshotSplit.splitId(),
+                    Strings.duration(clock.currentTimeInMillis() - exportStart));
+        } catch (SQLException e) {
+            throw new ConnectException("Snapshotting of table " + table.id() + " failed", e);
+        }
+    }
+
+    protected ChangeRecordEmitter getChangeRecordEmitter(
+            SnapshotContext snapshotContext, TableId tableId, Object[] row) {
+        snapshotContext.offset.event(tableId, clock.currentTime());
+        return new SnapshotChangeRecordEmitter(snapshotContext.offset, row, clock);
+    }
+
+    private Threads.Timer getTableScanLogTimer() {
+        return Threads.timer(clock, LOG_INTERVAL);
+    }
+
+    /**
+     * Read JDBC return value and deal special type like time, timestamp.
+     *
+     * <p>Note https://issues.redhat.com/browse/DBZ-3238 has fixed this issue, please remove this
+     * method once we bump Debezium version to 1.6
+     */
+    private Object readField(ResultSet rs, int fieldNo, Column actualColumn, Table actualTable)
+            throws SQLException {
+        if (actualColumn.jdbcType() == Types.TIME) {
+            return readTimeField(rs, fieldNo);
+        } else if (actualColumn.jdbcType() == Types.DATE) {
+            return readDateField(rs, fieldNo, actualColumn, actualTable);
+        } else if (actualColumn.jdbcType() == Types.TIMESTAMP) {
+            return readTimestampField(rs, fieldNo, actualColumn, actualTable);
+        } else if (actualColumn.jdbcType() == Types.TINYINT
+                || actualColumn.jdbcType() == Types.SMALLINT) {
+            return rs.getObject(fieldNo) == null ? null : rs.getInt(fieldNo);
+        } else if ("CHAR".equals(actualColumn.typeName())
+                || "VARCHAR".equals(actualColumn.typeName())
+                || "TEXT".equals(actualColumn.typeName())) {
+            return rs.getBytes(fieldNo);
+        } else {
+            return rs.getObject(fieldNo);
+        }
+    }
+
+    /**
+     * As MySQL connector/J implementation is broken for MySQL type "TIME" we have to use a
+     * binary-ish workaround. https://issues.jboss.org/browse/DBZ-342
+     */
+    private Object readTimeField(ResultSet rs, int fieldNo) throws SQLException {
+        Blob b = rs.getBlob(fieldNo);
+        if (b == null) {
+            return null; // Don't continue parsing time field if it is null
+        }
+
+        try {
+            return MySqlValueConverters.stringToDuration(
+                    new String(b.getBytes(1, (int) (b.length())), "UTF-8"));
+        } catch (UnsupportedEncodingException e) {
+            LOG.error("Could not read MySQL TIME value as UTF-8");
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * In non-string mode the date field can contain zero in any of the date part which we need to
+     * handle as all-zero.
+     */
+    private Object readDateField(ResultSet rs, int fieldNo, Column column, Table table)
+            throws SQLException {
+        Blob b = rs.getBlob(fieldNo);
+        if (b == null) {
+            return null; // Don't continue parsing date field if it is null
+        }
+
+        try {
+            return MySqlValueConverters.stringToLocalDate(
+                    new String(b.getBytes(1, (int) (b.length())), "UTF-8"), column, table);
+        } catch (UnsupportedEncodingException e) {
+            LOG.error("Could not read MySQL TIME value as UTF-8");
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * In non-string mode the time field can contain zero in any of the date part which we need to
+     * handle as all-zero.
+     */
+    private Object readTimestampField(ResultSet rs, int fieldNo, Column column, Table table)
+            throws SQLException {
+        Blob b = rs.getBlob(fieldNo);
+        if (b == null) {
+            return null; // Don't continue parsing timestamp field if it is null
+        }
+
+        try {
+            return MySqlValueConverters.containsZeroValuesInDatePart(
+                            (new String(b.getBytes(1, (int) (b.length())), "UTF-8")), column, table)
+                    ? null
+                    : rs.getTimestamp(fieldNo, Calendar.getInstance());
+        } catch (UnsupportedEncodingException e) {
+            LOG.error("Could not read MySQL TIME value as UTF-8");
+            throw new RuntimeException(e);
+        }
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/context/MySqlErrorHandler.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/context/MySqlErrorHandler.java
new file mode 100644
index 000000000..7907ea0d3
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/context/MySqlErrorHandler.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.task.context;
+
+import io.debezium.DebeziumException;
+import io.debezium.connector.base.ChangeEventQueue;
+import io.debezium.connector.mysql.MySqlConnector;
+import io.debezium.connector.mysql.MySqlTaskContext;
+import io.debezium.pipeline.ErrorHandler;
+import io.debezium.relational.TableId;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A subclass implementation of {@link ErrorHandler} which filter some {@link DebeziumException}, we
+ * use this class instead of {@link io.debezium.connector.mysql.MySqlErrorHandler}.
+ */
+public class MySqlErrorHandler extends ErrorHandler {
+    private static final Logger LOG = LoggerFactory.getLogger(MySqlErrorHandler.class);
+    private static final Pattern NOT_FOUND_TABLE_MSG_PATTERN =
+            Pattern.compile(
+                    "Encountered change event for table (.+)\\.(.+) whose schema isn't known to this connector");
+
+    MySqlTaskContext context;
+
+    public MySqlErrorHandler(
+            String logicalName, ChangeEventQueue<?> queue, MySqlTaskContext context) {
+        super(MySqlConnector.class, logicalName, queue);
+        this.context = context;
+    }
+
+    @Override
+    protected boolean isRetriable(Throwable throwable) {
+        return false;
+    }
+
+    @Override
+    public void setProducerThrowable(Throwable producerThrowable) {
+        if (producerThrowable.getCause() instanceof DebeziumException) {
+            DebeziumException e = (DebeziumException) producerThrowable.getCause();
+            String detailMessage = e.getMessage();
+            Matcher matcher = NOT_FOUND_TABLE_MSG_PATTERN.matcher(detailMessage);
+            if (matcher.find()) {
+                String databaseName = matcher.group(1);
+                String tableName = matcher.group(2);
+                TableId tableId = new TableId(databaseName, null, tableName);
+                if (context.getSchema().schemaFor(tableId) == null) {
+                    LOG.warn("Schema for table " + tableId + " is null");
+                    return;
+                }
+            }
+        }
+        super.setProducerThrowable(producerThrowable);
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/context/MySqlTaskContextImpl.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/context/MySqlTaskContextImpl.java
new file mode 100644
index 000000000..4cef227b5
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/context/MySqlTaskContextImpl.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.task.context;
+
+import com.github.shyiko.mysql.binlog.BinaryLogClient;
+import io.debezium.connector.mysql.MySqlConnectorConfig;
+import io.debezium.connector.mysql.MySqlDatabaseSchema;
+import io.debezium.connector.mysql.MySqlTaskContext;
+
+/** A subclass implementation of {@link MySqlTaskContext} which reuses one BinaryLogClient. */
+public class MySqlTaskContextImpl extends MySqlTaskContext {
+
+    private final BinaryLogClient reusedBinaryLogClient;
+
+    public MySqlTaskContextImpl(
+            MySqlConnectorConfig config,
+            MySqlDatabaseSchema schema,
+            BinaryLogClient reusedBinaryLogClient) {
+        super(config, schema);
+        this.reusedBinaryLogClient = reusedBinaryLogClient;
+    }
+
+    @Override
+    public BinaryLogClient getBinaryLogClient() {
+        return reusedBinaryLogClient;
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/context/StatefulTaskContext.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/context/StatefulTaskContext.java
new file mode 100644
index 000000000..021d2b30e
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/debezium/task/context/StatefulTaskContext.java
@@ -0,0 +1,322 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.task.context;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset.BINLOG_FILENAME_OFFSET_KEY;
+
+import com.github.shyiko.mysql.binlog.BinaryLogClient;
+import io.debezium.connector.AbstractSourceInfo;
+import io.debezium.connector.base.ChangeEventQueue;
+import io.debezium.connector.mysql.MySqlChangeEventSourceMetricsFactory;
+import io.debezium.connector.mysql.MySqlConnection;
+import io.debezium.connector.mysql.MySqlConnectorConfig;
+import io.debezium.connector.mysql.MySqlDatabaseSchema;
+import io.debezium.connector.mysql.MySqlOffsetContext;
+import io.debezium.connector.mysql.MySqlStreamingChangeEventSourceMetrics;
+import io.debezium.connector.mysql.MySqlTopicSelector;
+import io.debezium.data.Envelope;
+import io.debezium.pipeline.DataChangeEvent;
+import io.debezium.pipeline.ErrorHandler;
+import io.debezium.pipeline.metrics.SnapshotChangeEventSourceMetrics;
+import io.debezium.pipeline.metrics.StreamingChangeEventSourceMetrics;
+import io.debezium.pipeline.source.spi.EventMetadataProvider;
+import io.debezium.pipeline.spi.OffsetContext;
+import io.debezium.relational.TableId;
+import io.debezium.schema.DataCollectionId;
+import io.debezium.schema.TopicSelector;
+import io.debezium.util.Clock;
+import io.debezium.util.Collect;
+import io.debezium.util.SchemaNameAdjuster;
+import java.time.Instant;
+import java.util.List;
+import java.util.Map;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.DebeziumUtils;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.EmbeddedFlinkDatabaseHistory;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.dispatcher.EventDispatcherImpl;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfig;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSplit;
+import org.apache.kafka.connect.data.Struct;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A stateful task context that contains entries the debezium mysql connector task required.
+ *
+ * <p>The offset change and schema change should record to MySqlSplitState when emit the record,
+ * thus the Flink's state mechanism can help to store/restore when failover happens.
+ */
+public class StatefulTaskContext {
+
+    private static final Logger LOG = LoggerFactory.getLogger(StatefulTaskContext.class);
+    private static final Clock clock = Clock.SYSTEM;
+
+    private final MySqlSourceConfig sourceConfig;
+    private final MySqlConnectorConfig connectorConfig;
+    private final MySqlEventMetadataProvider metadataProvider;
+    private final SchemaNameAdjuster schemaNameAdjuster;
+    private final MySqlConnection connection;
+    private final BinaryLogClient binaryLogClient;
+
+    private MySqlDatabaseSchema databaseSchema;
+    private MySqlTaskContextImpl taskContext;
+    private MySqlOffsetContext offsetContext;
+    private TopicSelector<TableId> topicSelector;
+    private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics;
+    private StreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics;
+    private EventDispatcherImpl<TableId> dispatcher;
+    private ChangeEventQueue<DataChangeEvent> queue;
+    private ErrorHandler errorHandler;
+
+    public StatefulTaskContext(
+            MySqlSourceConfig sourceConfig,
+            BinaryLogClient binaryLogClient,
+            MySqlConnection connection) {
+        this.sourceConfig = sourceConfig;
+        this.connectorConfig = sourceConfig.getMySqlConnectorConfig();
+        this.schemaNameAdjuster = SchemaNameAdjuster.create();
+        this.metadataProvider = new MySqlEventMetadataProvider();
+        this.binaryLogClient = binaryLogClient;
+        this.connection = connection;
+    }
+
+    public void configure(MySqlSplit mySqlSplit) {
+        // initial stateful objects
+        final boolean tableIdCaseInsensitive = connection.isTableIdCaseSensitive();
+        this.topicSelector = MySqlTopicSelector.defaultSelector(connectorConfig);
+        EmbeddedFlinkDatabaseHistory.registerHistory(
+                sourceConfig
+                        .getDbzConfiguration()
+                        .getString(EmbeddedFlinkDatabaseHistory.DATABASE_HISTORY_INSTANCE_NAME),
+                mySqlSplit.getTableSchemas().values());
+        this.databaseSchema =
+                DebeziumUtils.createMySqlDatabaseSchema(connectorConfig, tableIdCaseInsensitive);
+        this.offsetContext =
+                loadStartingOffsetState(new MySqlOffsetContext.Loader(connectorConfig), mySqlSplit);
+        validateAndLoadDatabaseHistory(offsetContext, databaseSchema);
+
+        this.taskContext =
+                new MySqlTaskContextImpl(connectorConfig, databaseSchema, binaryLogClient);
+        final int queueSize =
+                mySqlSplit.isSnapshotSplit()
+                        ? Integer.MAX_VALUE
+                        : connectorConfig.getMaxQueueSize();
+        this.queue =
+                new ChangeEventQueue.Builder<DataChangeEvent>()
+                        .pollInterval(connectorConfig.getPollInterval())
+                        .maxBatchSize(connectorConfig.getMaxBatchSize())
+                        .maxQueueSize(queueSize)
+                        .maxQueueSizeInBytes(connectorConfig.getMaxQueueSizeInBytes())
+                        .loggingContextSupplier(
+                                () ->
+                                        taskContext.configureLoggingContext(
+                                                "mysql-cdc-connector-task"))
+                        // do not buffer any element, we use signal event
+                        // .buffering()
+                        .build();
+        this.dispatcher =
+                new EventDispatcherImpl<>(
+                        connectorConfig,
+                        topicSelector,
+                        databaseSchema,
+                        queue,
+                        connectorConfig.getTableFilters().dataCollectionFilter(),
+                        DataChangeEvent::new,
+                        metadataProvider,
+                        schemaNameAdjuster);
+
+        final MySqlChangeEventSourceMetricsFactory changeEventSourceMetricsFactory =
+                new MySqlChangeEventSourceMetricsFactory(
+                        new MySqlStreamingChangeEventSourceMetrics(
+                                taskContext, queue, metadataProvider));
+        this.snapshotChangeEventSourceMetrics =
+                changeEventSourceMetricsFactory.getSnapshotMetrics(
+                        taskContext, queue, metadataProvider);
+        this.streamingChangeEventSourceMetrics =
+                changeEventSourceMetricsFactory.getStreamingMetrics(
+                        taskContext, queue, metadataProvider);
+        this.errorHandler =
+                new MySqlErrorHandler(connectorConfig.getLogicalName(), queue, taskContext);
+    }
+
+    private void validateAndLoadDatabaseHistory(
+            MySqlOffsetContext offset, MySqlDatabaseSchema schema) {
+        schema.initializeStorage();
+        schema.recover(offset);
+    }
+
+    /** Loads the connector's persistent offset (if present) via the given loader. */
+    private MySqlOffsetContext loadStartingOffsetState(
+            OffsetContext.Loader loader, MySqlSplit mySqlSplit) {
+        BinlogOffset offset =
+                mySqlSplit.isSnapshotSplit()
+                        ? BinlogOffset.INITIAL_OFFSET
+                        : mySqlSplit.asBinlogSplit().getStartingOffset();
+
+        MySqlOffsetContext mySqlOffsetContext =
+                (MySqlOffsetContext) loader.load(offset.getOffset());
+
+        if (!isBinlogAvailable(mySqlOffsetContext)) {
+            throw new IllegalStateException(
+                    "The connector is trying to read binlog starting at "
+                            + mySqlOffsetContext.getSourceInfo()
+                            + ", but this is no longer "
+                            + "available on the server. Reconfigure the connector to use a snapshot when needed.");
+        }
+        return mySqlOffsetContext;
+    }
+
+    private boolean isBinlogAvailable(MySqlOffsetContext offset) {
+        String binlogFilename = offset.getSourceInfo().getString(BINLOG_FILENAME_OFFSET_KEY);
+        if (binlogFilename == null) {
+            return true; // start at current position
+        }
+        if (binlogFilename.equals("")) {
+            return true; // start at beginning
+        }
+
+        // Accumulate the available binlog filenames ...
+        List<String> logNames = connection.availableBinlogFiles();
+
+        // And compare with the one we're supposed to use ...
+        boolean found = logNames.stream().anyMatch(binlogFilename::equals);
+        if (!found) {
+            LOG.info(
+                    "Connector requires binlog file '{}', but MySQL only has {}",
+                    binlogFilename,
+                    String.join(", ", logNames));
+        } else {
+            LOG.info("MySQL has the binlog file '{}' required by the connector", binlogFilename);
+        }
+        return found;
+    }
+
+    /** Copied from debezium for accessing here. */
+    public static class MySqlEventMetadataProvider implements EventMetadataProvider {
+        public static final String SERVER_ID_KEY = "server_id";
+
+        public static final String GTID_KEY = "gtid";
+        public static final String BINLOG_FILENAME_OFFSET_KEY = "file";
+        public static final String BINLOG_POSITION_OFFSET_KEY = "pos";
+        public static final String BINLOG_ROW_IN_EVENT_OFFSET_KEY = "row";
+        public static final String THREAD_KEY = "thread";
+        public static final String QUERY_KEY = "query";
+
+        @Override
+        public Instant getEventTimestamp(
+                DataCollectionId source, OffsetContext offset, Object key, Struct value) {
+            if (value == null) {
+                return null;
+            }
+            final Struct sourceInfo = value.getStruct(Envelope.FieldName.SOURCE);
+            if (source == null) {
+                return null;
+            }
+            final Long timestamp = sourceInfo.getInt64(AbstractSourceInfo.TIMESTAMP_KEY);
+            return timestamp == null ? null : Instant.ofEpochMilli(timestamp);
+        }
+
+        @Override
+        public Map<String, String> getEventSourcePosition(
+                DataCollectionId source, OffsetContext offset, Object key, Struct value) {
+            if (value == null) {
+                return null;
+            }
+            final Struct sourceInfo = value.getStruct(Envelope.FieldName.SOURCE);
+            if (source == null) {
+                return null;
+            }
+            return Collect.hashMapOf(
+                    BINLOG_FILENAME_OFFSET_KEY,
+                    sourceInfo.getString(BINLOG_FILENAME_OFFSET_KEY),
+                    BINLOG_POSITION_OFFSET_KEY,
+                    Long.toString(sourceInfo.getInt64(BINLOG_POSITION_OFFSET_KEY)),
+                    BINLOG_ROW_IN_EVENT_OFFSET_KEY,
+                    Integer.toString(sourceInfo.getInt32(BINLOG_ROW_IN_EVENT_OFFSET_KEY)));
+        }
+
+        @Override
+        public String getTransactionId(
+                DataCollectionId source, OffsetContext offset, Object key, Struct value) {
+            return ((MySqlOffsetContext) offset).getTransactionId();
+        }
+    }
+
+    public static Clock getClock() {
+        return clock;
+    }
+
+    public MySqlSourceConfig getSourceConfig() {
+        return sourceConfig;
+    }
+
+    public MySqlConnectorConfig getConnectorConfig() {
+        return connectorConfig;
+    }
+
+    public MySqlConnection getConnection() {
+        return connection;
+    }
+
+    public BinaryLogClient getBinaryLogClient() {
+        return binaryLogClient;
+    }
+
+    public MySqlDatabaseSchema getDatabaseSchema() {
+        return databaseSchema;
+    }
+
+    public MySqlTaskContextImpl getTaskContext() {
+        return taskContext;
+    }
+
+    public EventDispatcherImpl<TableId> getDispatcher() {
+        return dispatcher;
+    }
+
+    public ChangeEventQueue<DataChangeEvent> getQueue() {
+        return queue;
+    }
+
+    public ErrorHandler getErrorHandler() {
+        return errorHandler;
+    }
+
+    public MySqlOffsetContext getOffsetContext() {
+        return offsetContext;
+    }
+
+    public TopicSelector<TableId> getTopicSelector() {
+        return topicSelector;
+    }
+
+    public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() {
+        snapshotChangeEventSourceMetrics.reset();
+        return snapshotChangeEventSourceMetrics;
+    }
+
+    public StreamingChangeEventSourceMetrics getStreamingChangeEventSourceMetrics() {
+        streamingChangeEventSourceMetrics.reset();
+        return streamingChangeEventSourceMetrics;
+    }
+
+    public SchemaNameAdjuster getSchemaNameAdjuster() {
+        return schemaNameAdjuster;
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/schema/MySqlFieldDefinition.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/schema/MySqlFieldDefinition.java
new file mode 100644
index 000000000..cff1fa361
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/schema/MySqlFieldDefinition.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.schema;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.StatementUtils.quote;
+
+import org.apache.commons.lang3.StringUtils;
+
+/** used to generate field definition in ddl with "desc table". */
+class MySqlFieldDefinition {
+    private String columnName;
+    private String columnType;
+    private boolean nullable;
+    private boolean key;
+    private String defaultValue;
+    private String extra;
+    private boolean unique;
+
+    public String getColumnName() {
+        return columnName;
+    }
+
+    public void setColumnName(String columnName) {
+        this.columnName = columnName;
+    }
+
+    public String getColumnType() {
+        return columnType;
+    }
+
+    public void setColumnType(String columnType) {
+        this.columnType = columnType;
+    }
+
+    public void setNullable(boolean nullable) {
+        this.nullable = nullable;
+    }
+
+    public String getDefaultValue() {
+        return StringUtils.isEmpty(defaultValue) ? "" : "DEFAULT " + defaultValue;
+    }
+
+    public void setDefaultValue(String defaultValue) {
+        this.defaultValue = defaultValue;
+    }
+
+    public boolean isUnsigned() {
+        return StringUtils.containsIgnoreCase(columnType, "unsigned");
+    }
+
+    public boolean isNullable() {
+        return nullable;
+    }
+
+    public boolean isKey() {
+        return key;
+    }
+
+    public void setKey(boolean key) {
+        this.key = key;
+    }
+
+    public String getExtra() {
+        return extra;
+    }
+
+    public void setExtra(String extra) {
+        this.extra = extra;
+    }
+
+    public boolean isUnique() {
+        return unique;
+    }
+
+    public void setUnique(boolean unique) {
+        this.unique = unique;
+    }
+
+    public String toDdl() {
+        return quote(columnName) + " " + columnType + " " + (nullable ? "" : "NOT NULL");
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/schema/MySqlSchema.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/schema/MySqlSchema.java
new file mode 100644
index 000000000..1d7fe6df3
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/schema/MySqlSchema.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.schema;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.DebeziumUtils.createMySqlDatabaseSchema;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.StatementUtils.quote;
+
+import io.debezium.connector.mysql.MySqlConnectorConfig;
+import io.debezium.connector.mysql.MySqlDatabaseSchema;
+import io.debezium.connector.mysql.MySqlOffsetContext;
+import io.debezium.jdbc.JdbcConnection;
+import io.debezium.relational.TableId;
+import io.debezium.relational.history.TableChanges.TableChange;
+import io.debezium.schema.SchemaChangeEvent;
+import java.sql.SQLException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfig;
+
+/** A component used to get schema by table path. */
+public class MySqlSchema {
+    private static final String SHOW_CREATE_TABLE = "SHOW CREATE TABLE ";
+    private static final String DESC_TABLE = "DESC ";
+
+    private final MySqlConnectorConfig connectorConfig;
+    private final MySqlDatabaseSchema databaseSchema;
+    private final Map<TableId, TableChange> schemasByTableId;
+
+    public MySqlSchema(MySqlSourceConfig sourceConfig, boolean isTableIdCaseSensitive) {
+        this.connectorConfig = sourceConfig.getMySqlConnectorConfig();
+        this.databaseSchema = createMySqlDatabaseSchema(connectorConfig, isTableIdCaseSensitive);
+        this.schemasByTableId = new HashMap<>();
+    }
+
+    /**
+     * Gets table schema for the given table path. It will request to MySQL server by running `SHOW
+     * CREATE TABLE` if cache missed.
+     */
+    public TableChange getTableSchema(JdbcConnection jdbc, TableId tableId) {
+        // read schema from cache first
+        TableChange schema = schemasByTableId.get(tableId);
+        if (schema == null) {
+            schema = buildTableSchema(jdbc, tableId);
+            schemasByTableId.put(tableId, schema);
+        }
+        return schema;
+    }
+
+    // ------------------------------------------------------------------------------------------
+    // Helpers
+    // ------------------------------------------------------------------------------------------
+
+    private TableChange buildTableSchema(JdbcConnection jdbc, TableId tableId) {
+        final Map<TableId, TableChange> tableChangeMap = new HashMap<>();
+        String showCreateTable = SHOW_CREATE_TABLE + quote(tableId);
+        buildSchemaByShowCreateTable(jdbc, tableId, tableChangeMap);
+        if (!tableChangeMap.containsKey(tableId)) {
+            // fallback to desc table
+            String descTable = DESC_TABLE + quote(tableId);
+            buildSchemaByDescTable(jdbc, descTable, tableId, tableChangeMap);
+            if (!tableChangeMap.containsKey(tableId)) {
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Can't obtain schema for table %s by running %s and %s ",
+                                tableId, showCreateTable, descTable));
+            }
+        }
+        return tableChangeMap.get(tableId);
+    }
+
+    private void buildSchemaByShowCreateTable(
+            JdbcConnection jdbc, TableId tableId, Map<TableId, TableChange> tableChangeMap) {
+        final String sql = SHOW_CREATE_TABLE + quote(tableId);
+        try {
+            jdbc.query(
+                    sql,
+                    rs -> {
+                        if (rs.next()) {
+                            final String ddl = rs.getString(2);
+                            parseSchemaByDdl(ddl, tableId, tableChangeMap);
+                        }
+                    });
+        } catch (SQLException e) {
+            throw new FlinkRuntimeException(
+                    String.format("Failed to read schema for table %s by running %s", tableId, sql),
+                    e);
+        }
+    }
+
+    private void parseSchemaByDdl(
+            String ddl, TableId tableId, Map<TableId, TableChange> tableChangeMap) {
+        final MySqlOffsetContext offsetContext = MySqlOffsetContext.initial(connectorConfig);
+        List<SchemaChangeEvent> schemaChangeEvents =
+                databaseSchema.parseSnapshotDdl(
+                        ddl, tableId.catalog(), offsetContext, Instant.now());
+        for (SchemaChangeEvent schemaChangeEvent : schemaChangeEvents) {
+            for (TableChange tableChange : schemaChangeEvent.getTableChanges()) {
+                tableChangeMap.put(tableId, tableChange);
+            }
+        }
+    }
+
+    private void buildSchemaByDescTable(
+            JdbcConnection jdbc,
+            String descTable,
+            TableId tableId,
+            Map<TableId, TableChange> tableChangeMap) {
+        List<MySqlFieldDefinition> fieldMetas = new ArrayList<>();
+        List<String> primaryKeys = new ArrayList<>();
+        try {
+            jdbc.query(
+                    descTable,
+                    rs -> {
+                        while (rs.next()) {
+                            MySqlFieldDefinition meta = new MySqlFieldDefinition();
+                            meta.setColumnName(rs.getString("Field"));
+                            meta.setColumnType(rs.getString("Type"));
+                            meta.setNullable(
+                                    StringUtils.equalsIgnoreCase(rs.getString("Null"), "YES"));
+                            meta.setKey("PRI".equalsIgnoreCase(rs.getString("Key")));
+                            meta.setUnique("UNI".equalsIgnoreCase(rs.getString("Key")));
+                            meta.setDefaultValue(rs.getString("Default"));
+                            meta.setExtra(rs.getString("Extra"));
+                            if (meta.isKey()) {
+                                primaryKeys.add(meta.getColumnName());
+                            }
+                            fieldMetas.add(meta);
+                        }
+                    });
+            parseSchemaByDdl(
+                    new MySqlTableDefinition(tableId, fieldMetas, primaryKeys).toDdl(),
+                    tableId,
+                    tableChangeMap);
+        } catch (SQLException e) {
+            throw new FlinkRuntimeException(
+                    String.format(
+                            "Failed to read schema for table %s by running %s", tableId, descTable),
+                    e);
+        }
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/schema/MySqlTableDefinition.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/schema/MySqlTableDefinition.java
new file mode 100644
index 000000000..460291f86
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/schema/MySqlTableDefinition.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.schema;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.StatementUtils.quote;
+
+import io.debezium.relational.TableId;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.flink.util.CollectionUtil;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.StatementUtils;
+
+/** used to generate table definition in ddl with "desc table". */
+public class MySqlTableDefinition {
+    TableId tableId;
+    List<MySqlFieldDefinition> fieldDefinitions;
+    List<String> primaryKeys;
+
+    public MySqlTableDefinition(
+            TableId tableId,
+            List<MySqlFieldDefinition> fieldDefinitions,
+            List<String> primaryKeys) {
+        this.tableId = tableId;
+        this.fieldDefinitions = fieldDefinitions;
+        this.primaryKeys = primaryKeys;
+    }
+
+    String toDdl() {
+        return String.format(
+                "CREATE TABLE %s (\n\t %s %s );",
+                quote(tableId), fieldDefinitions(), pkDefinition());
+    }
+
+    private String fieldDefinitions() {
+        return fieldDefinitions.stream()
+                .map(MySqlFieldDefinition::toDdl)
+                .collect(Collectors.joining(", \n\t"));
+    }
+
+    private String pkDefinition() {
+        StringBuilder pkDefinition = new StringBuilder();
+        if (!CollectionUtil.isNullOrEmpty(primaryKeys)) {
+            pkDefinition.append(",");
+            pkDefinition.append(
+                    String.format(
+                            "PRIMARY KEY ( %s )",
+                            primaryKeys.stream()
+                                    .map(StatementUtils::quote)
+                                    .collect(Collectors.joining(","))));
+        }
+        return pkDefinition.toString();
+    }
+}
+
+
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/schema/MySqlTypeUtils.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/schema/MySqlTypeUtils.java
new file mode 100644
index 000000000..2c17900b7
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/schema/MySqlTypeUtils.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.schema;
+
+import io.debezium.relational.Column;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.types.DataType;
+
+/** Utilities for converting from MySQL types to Flink types. */
+public class MySqlTypeUtils {
+
+    // ------ MySQL Type ------
+    private static final String BIT = "BIT";
+    private static final String TINYINT = "TINYINT";
+    private static final String TINYINT_UNSIGNED = "TINYINT UNSIGNED";
+    private static final String SMALLINT = "SMALLINT";
+    private static final String SMALLINT_UNSIGNED = "SMALLINT UNSIGNED";
+    private static final String MEDIUMINT = "MEDIUMINT";
+    private static final String MEDIUMINT_UNSIGNED = "MEDIUMINT UNSIGNED";
+    private static final String INT = "INT";
+    private static final String INT_UNSIGNED = "INT UNSIGNED";
+    private static final String BIGINT = "BIGINT";
+    private static final String BIGINT_UNSIGNED = "BIGINT UNSIGNED";
+    private static final String FLOAT = "FLOAT";
+    private static final String FLOAT_UNSIGNED = "FLOAT UNSIGNED";
+    private static final String DOUBLE = "DOUBLE";
+    private static final String DOUBLE_UNSIGNED = "DOUBLE UNSIGNED";
+    private static final String DECIMAL = "DECIMAL";
+    private static final String DECIMAL_UNSIGNED = "DECIMAL UNSIGNED";
+    private static final String CHAR = "CHAR";
+    private static final String VARCHAR = "VARCHAR";
+    private static final String TINYTEXT = "TINYTEXT";
+    private static final String MEDIUMTEXT = "MEDIUMTEXT";
+    private static final String TEXT = "TEXT";
+    private static final String LONGTEXT = "LONGTEXT";
+    private static final String DATE = "DATE";
+    private static final String TIME = "TIME";
+    private static final String DATETIME = "DATETIME";
+    private static final String TIMESTAMP = "TIMESTAMP";
+    private static final String YEAR = "YEAR";
+    private static final String BINARY = "BINARY";
+    private static final String VARBINARY = "VARBINARY";
+    private static final String TINYBLOB = "TINYBLOB";
+    private static final String MEDIUMBLOB = "MEDIUMBLOB";
+    private static final String BLOB = "BLOB";
+    private static final String LONGBLOB = "LONGBLOB";
+    private static final String JSON = "JSON";
+    private static final String SET = "SET";
+    private static final String ENUM = "ENUM";
+    private static final String GEOMETRY = "GEOMETRY";
+    private static final String UNKNOWN = "UNKNOWN";
+
+    /** Returns a corresponding Flink data type from a debezium {@link Column}. */
+    public static DataType fromDbzColumn(Column column) {
+        DataType dataType = convertFromColumn(column);
+        if (column.isOptional()) {
+            return dataType;
+        } else {
+            return dataType.notNull();
+        }
+    }
+
+    /**
+     * Returns a corresponding Flink data type from a debezium {@link Column} with nullable always
+     * be true.
+     */
+    private static DataType convertFromColumn(Column column) {
+        String typeName = column.typeName();
+        switch (typeName) {
+            case TINYINT:
+                return column.length() == 1 ? DataTypes.BOOLEAN() : DataTypes.TINYINT();
+            case TINYINT_UNSIGNED:
+            case SMALLINT:
+                return DataTypes.SMALLINT();
+            case SMALLINT_UNSIGNED:
+            case INT:
+            case MEDIUMINT:
+                return DataTypes.INT();
+            case INT_UNSIGNED:
+            case MEDIUMINT_UNSIGNED:
+            case BIGINT:
+                return DataTypes.BIGINT();
+            case BIGINT_UNSIGNED:
+                return DataTypes.DECIMAL(20, 0);
+            case FLOAT:
+                return DataTypes.FLOAT();
+            case DOUBLE:
+                return DataTypes.DOUBLE();
+            case DECIMAL:
+                return DataTypes.DECIMAL(column.length(), column.scale().orElse(0));
+            case TIME:
+                return column.length() >= 0 ? DataTypes.TIME(column.length()) : DataTypes.TIME();
+            case DATE:
+                return DataTypes.DATE();
+            case DATETIME:
+            case TIMESTAMP:
+                return column.length() >= 0
+                        ? DataTypes.TIMESTAMP(column.length())
+                        : DataTypes.TIMESTAMP();
+            case CHAR:
+                return DataTypes.CHAR(column.length());
+            case VARCHAR:
+                return DataTypes.VARCHAR(column.length());
+            case TEXT:
+                return DataTypes.STRING();
+            case BINARY:
+                return DataTypes.BINARY(column.length());
+            case VARBINARY:
+                return DataTypes.VARBINARY(column.length());
+            case BLOB:
+                return DataTypes.BYTES();
+            default:
+                throw new UnsupportedOperationException(
+                        String.format("Don't support MySQL type '%s' yet.", typeName));
+        }
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/MySqlSource.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/MySqlSource.java
new file mode 100644
index 000000000..442c11c4a
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/MySqlSource.java
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.DebeziumUtils.discoverCapturedTables;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.DebeziumUtils.openJdbcConnection;
+
+import io.debezium.jdbc.JdbcConnection;
+import io.debezium.relational.TableId;
+import java.lang.reflect.Method;
+import java.util.List;
+import java.util.function.Supplier;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumDeserializationSchema;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.MySqlValidator;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.DebeziumUtils;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.MySqlBinlogSplitAssigner;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.MySqlHybridSplitAssigner;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.MySqlSplitAssigner;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state.BinlogPendingSplitsState;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state.HybridPendingSplitsState;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state.PendingSplitsState;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state.PendingSplitsStateSerializer;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfig;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfigFactory;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.enumerator.MySqlSourceEnumerator;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.metrics.MySqlSourceReaderMetrics;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.reader.MySqlRecordEmitter;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.reader.MySqlSourceReader;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.reader.MySqlSourceReaderContext;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.reader.MySqlSplitReader;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSplit;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSplitSerializer;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.table.StartupMode;
+import org.apache.kafka.connect.source.SourceRecord;
+
+/**
+ * The MySQL CDC Source based on FLIP-27 and Watermark Signal Algorithm which supports parallel
+ * reading snapshot of table and then continue to capture data change from binlog.
+ *
+ * <pre>
+ *     1. The source supports parallel capturing table change.
+ *     2. The source supports checkpoint in split level when read snapshot data.
+ *     3. The source doesn't need apply any lock of MySQL.
+ * </pre>
+ *
+ * <pre>{@code
+ * MySqlSource
+ *     .<String>builder()
+ *     .hostname("localhost")
+ *     .port(3306)
+ *     .databaseList("mydb")
+ *     .tableList("mydb.users")
+ *     .username(username)
+ *     .password(password)
+ *     .serverId(5400)
+ *     .deserializer(new JsonDebeziumDeserializationSchema())
+ *     .build();
+ * }</pre>
+ *
+ * <p>See {@link MySqlSourceBuilder} for more details.
+ *
+ * @param <T> the output type of the source.
+ */
+@Internal
+public class MySqlSource<T>
+        implements Source<T, MySqlSplit, PendingSplitsState>, ResultTypeQueryable<T> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final MySqlSourceConfigFactory configFactory;
+    private final DebeziumDeserializationSchema<T> deserializationSchema;
+
+    /**
+     * Get a MySqlParallelSourceBuilder to build a {@link MySqlSource}.
+     *
+     * @return a MySql parallel source builder.
+     */
+    @PublicEvolving
+    public static <T> MySqlSourceBuilder<T> builder() {
+        return new MySqlSourceBuilder<>();
+    }
+
+    MySqlSource(
+            MySqlSourceConfigFactory configFactory,
+            DebeziumDeserializationSchema<T> deserializationSchema) {
+        this.configFactory = configFactory;
+        this.deserializationSchema = deserializationSchema;
+    }
+
+    public MySqlSourceConfigFactory getConfigFactory() {
+        return configFactory;
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        return Boundedness.CONTINUOUS_UNBOUNDED;
+    }
+
+    @Override
+    public SourceReader<T, MySqlSplit> createReader(SourceReaderContext readerContext)
+            throws Exception {
+        // create source config for the given subtask (e.g. unique server id)
+        MySqlSourceConfig sourceConfig =
+                configFactory.createConfig(readerContext.getIndexOfSubtask());
+        FutureCompletingBlockingQueue<RecordsWithSplitIds<SourceRecord>> elementsQueue =
+                new FutureCompletingBlockingQueue<>();
+
+        final Method metricGroupMethod = readerContext.getClass().getMethod("metricGroup");
+        metricGroupMethod.setAccessible(true);
+        final MetricGroup metricGroup = (MetricGroup) metricGroupMethod.invoke(readerContext);
+
+        final MySqlSourceReaderMetrics sourceReaderMetrics =
+                new MySqlSourceReaderMetrics(metricGroup);
+        sourceReaderMetrics.registerMetrics();
+        MySqlSourceReaderContext mySqlSourceReaderContext =
+                new MySqlSourceReaderContext(readerContext);
+        Supplier<MySqlSplitReader> splitReaderSupplier =
+                () ->
+                        new MySqlSplitReader(
+                                sourceConfig,
+                                readerContext.getIndexOfSubtask(),
+                                mySqlSourceReaderContext);
+        return new MySqlSourceReader<>(
+                elementsQueue,
+                splitReaderSupplier,
+                new MySqlRecordEmitter<>(
+                        deserializationSchema,
+                        sourceReaderMetrics,
+                        sourceConfig.isIncludeSchemaChanges()),
+                readerContext.getConfiguration(),
+                mySqlSourceReaderContext,
+                sourceConfig);
+    }
+
+    @Override
+    public SplitEnumerator<MySqlSplit, PendingSplitsState> createEnumerator(
+            SplitEnumeratorContext<MySqlSplit> enumContext) {
+        MySqlSourceConfig sourceConfig = configFactory.createConfig(0);
+
+        final MySqlValidator validator = new MySqlValidator(sourceConfig);
+        validator.validate();
+
+        final MySqlSplitAssigner splitAssigner;
+        if (sourceConfig.getStartupOptions().startupMode == StartupMode.INITIAL) {
+            try (JdbcConnection jdbc = openJdbcConnection(sourceConfig)) {
+                final List<TableId> remainingTables = discoverCapturedTables(jdbc, sourceConfig);
+                boolean isTableIdCaseSensitive = DebeziumUtils.isTableIdCaseSensitive(jdbc);
+                splitAssigner =
+                        new MySqlHybridSplitAssigner(
+                                sourceConfig,
+                                enumContext.currentParallelism(),
+                                remainingTables,
+                                isTableIdCaseSensitive);
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(
+                        "Failed to discover captured tables for enumerator", e);
+            }
+        } else {
+            splitAssigner = new MySqlBinlogSplitAssigner(sourceConfig);
+        }
+
+        return new MySqlSourceEnumerator(enumContext, sourceConfig, splitAssigner);
+    }
+
+    @Override
+    public SplitEnumerator<MySqlSplit, PendingSplitsState> restoreEnumerator(
+            SplitEnumeratorContext<MySqlSplit> enumContext, PendingSplitsState checkpoint) {
+        MySqlSourceConfig sourceConfig = configFactory.createConfig(0);
+
+        final MySqlSplitAssigner splitAssigner;
+        if (checkpoint instanceof HybridPendingSplitsState) {
+            splitAssigner =
+                    new MySqlHybridSplitAssigner(
+                            sourceConfig,
+                            enumContext.currentParallelism(),
+                            (HybridPendingSplitsState) checkpoint);
+        } else if (checkpoint instanceof BinlogPendingSplitsState) {
+            splitAssigner =
+                    new MySqlBinlogSplitAssigner(
+                            sourceConfig, (BinlogPendingSplitsState) checkpoint);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported restored PendingSplitsState: " + checkpoint);
+        }
+        return new MySqlSourceEnumerator(enumContext, sourceConfig, splitAssigner);
+    }
+
+    @Override
+    public SimpleVersionedSerializer<MySqlSplit> getSplitSerializer() {
+        return MySqlSplitSerializer.INSTANCE;
+    }
+
+    @Override
+    public SimpleVersionedSerializer<PendingSplitsState> getEnumeratorCheckpointSerializer() {
+        return new PendingSplitsStateSerializer(getSplitSerializer());
+    }
+
+    @Override
+    public TypeInformation<T> getProducedType() {
+        return deserializationSchema.getProducedType();
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/MySqlSourceBuilder.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/MySqlSourceBuilder.java
new file mode 100644
index 000000000..1202ff930
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/MySqlSourceBuilder.java
@@ -0,0 +1,238 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+import java.time.Duration;
+import java.util.Properties;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.inlong.sort.singletenant.flink.cdc.debezium.DebeziumDeserializationSchema;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfigFactory;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.table.StartupOptions;
+
+/**
+ * The builder class for {@link MySqlSource} to make it easier for the users to construct a {@link
+ * MySqlSource}.
+ *
+ * <pre>{@code
+ * MySqlSource
+ *     .<String>builder()
+ *     .hostname("localhost")
+ *     .port(3306)
+ *     .databaseList("mydb")
+ *     .tableList("mydb.users")
+ *     .username(username)
+ *     .password(password)
+ *     .serverId(5400)
+ *     .deserializer(new JsonDebeziumDeserializationSchema())
+ *     .build();
+ * }</pre>
+ *
+ * <p>Check the Java docs of each individual method to learn more about the settings to build a
+ * {@link MySqlSource}.
+ */
+@PublicEvolving
+public class MySqlSourceBuilder<T> {
+    private final MySqlSourceConfigFactory configFactory = new MySqlSourceConfigFactory();
+    private DebeziumDeserializationSchema<T> deserializer;
+
+    public MySqlSourceBuilder<T> hostname(String hostname) {
+        this.configFactory.hostname(hostname);
+        return this;
+    }
+
+    /** Integer port number of the MySQL database server. */
+    public MySqlSourceBuilder<T> port(int port) {
+        this.configFactory.port(port);
+        return this;
+    }
+
+    /**
+     * An required list of regular expressions that match database names to be monitored; any
+     * database name not included in the whitelist will be excluded from monitoring.
+     */
+    public MySqlSourceBuilder<T> databaseList(String... databaseList) {
+        this.configFactory.databaseList(databaseList);
+        return this;
+    }
+
+    /**
+     * An required list of regular expressions that match fully-qualified table identifiers for
+     * tables to be monitored; any table not included in the list will be excluded from monitoring.
+     * Each identifier is of the form {@code <databaseName>.<tableName>}.
+     */
+    public MySqlSourceBuilder<T> tableList(String... tableList) {
+        this.configFactory.tableList(tableList);
+        return this;
+    }
+
+    /** Name of the MySQL database to use when connecting to the MySQL database server. */
+    public MySqlSourceBuilder<T> username(String username) {
+        this.configFactory.username(username);
+        return this;
+    }
+
+    /** Password to use when connecting to the MySQL database server. */
+    public MySqlSourceBuilder<T> password(String password) {
+        this.configFactory.password(password);
+        return this;
+    }
+
+    /**
+     * A numeric ID or a numeric ID range of this database client, The numeric ID syntax is like
+     * '5400', the numeric ID range syntax is like '5400-5408', The numeric ID range syntax is
+     * required when 'scan.incremental.snapshot.enabled' enabled. Every ID must be unique across all
+     * currently-running database processes in the MySQL cluster. This connector joins the MySQL
+     * cluster as another server (with this unique ID) so it can read the binlog. By default, a
+     * random number is generated between 5400 and 6400, though we recommend setting an explicit
+     * value."
+     */
+    public MySqlSourceBuilder<T> serverId(String serverId) {
+        this.configFactory.serverId(serverId);
+        return this;
+    }
+
+    /**
+     * The session time zone in database server, e.g. "America/Los_Angeles". It controls how the
+     * TIMESTAMP type in MYSQL converted to STRING. See more
+     * https://debezium.io/documentation/reference/1.5/connectors/mysql.html#mysql-temporal-types
+     */
+    public MySqlSourceBuilder<T> serverTimeZone(String timeZone) {
+        this.configFactory.serverTimeZone(timeZone);
+        return this;
+    }
+
+    /**
+     * The split size (number of rows) of table snapshot, captured tables are split into multiple
+     * splits when read the snapshot of table.
+     */
+    public MySqlSourceBuilder<T> splitSize(int splitSize) {
+        this.configFactory.splitSize(splitSize);
+        return this;
+    }
+
+    /**
+     * The group size of split meta, if the meta size exceeds the group size, the meta will be will
+     * be divided into multiple groups.
+     */
+    public MySqlSourceBuilder<T> splitMetaGroupSize(int splitMetaGroupSize) {
+        this.configFactory.splitMetaGroupSize(splitMetaGroupSize);
+        return this;
+    }
+
+    /**
+     * The upper bound of split key evenly distribution factor, the factor is used to determine
+     * whether the table is evenly distribution or not.
+     */
+    public MySqlSourceBuilder<T> distributionFactorUpper(double distributionFactorUpper) {
+        this.configFactory.distributionFactorUpper(distributionFactorUpper);
+        return this;
+    }
+
+    /**
+     * The lower bound of split key evenly distribution factor, the factor is used to determine
+     * whether the table is evenly distribution or not.
+     */
+    public MySqlSourceBuilder<T> distributionFactorLower(double distributionFactorLower) {
+        this.configFactory.distributionFactorLower(distributionFactorLower);
+        return this;
+    }
+
+    /** The maximum fetch size for per poll when read table snapshot. */
+    public MySqlSourceBuilder<T> fetchSize(int fetchSize) {
+        this.configFactory.fetchSize(fetchSize);
+        return this;
+    }
+
+    /**
+     * The maximum time that the connector should wait after trying to connect to the MySQL database
+     * server before timing out.
+     */
+    public MySqlSourceBuilder<T> connectTimeout(Duration connectTimeout) {
+        this.configFactory.connectTimeout(connectTimeout);
+        return this;
+    }
+
+    /** The max retry times to get connection. */
+    public MySqlSourceBuilder<T> connectMaxRetries(int connectMaxRetries) {
+        this.configFactory.connectMaxRetries(connectMaxRetries);
+        return this;
+    }
+
+    /** The connection pool size. */
+    public MySqlSourceBuilder<T> connectionPoolSize(int connectionPoolSize) {
+        this.configFactory.connectionPoolSize(connectionPoolSize);
+        return this;
+    }
+
+    /** Whether the {@link MySqlSource} should output the schema changes or not. */
+    public MySqlSourceBuilder<T> includeSchemaChanges(boolean includeSchemaChanges) {
+        this.configFactory.includeSchemaChanges(includeSchemaChanges);
+        return this;
+    }
+
+    /** Whether the {@link MySqlSource} should scan the newly added tables or not. */
+    public MySqlSourceBuilder<T> scanNewlyAddedTableEnabled(boolean scanNewlyAddedTableEnabled) {
+        this.configFactory.scanNewlyAddedTableEnabled(scanNewlyAddedTableEnabled);
+        return this;
+    }
+
+    /** Specifies the startup options. */
+    public MySqlSourceBuilder<T> startupOptions(StartupOptions startupOptions) {
+        this.configFactory.startupOptions(startupOptions);
+        return this;
+    }
+
+    /** Custom properties that will overwrite the default JDBC connection URL. */
+    public MySqlSourceBuilder<T> jdbcProperties(Properties jdbcProperties) {
+        this.configFactory.jdbcProperties(jdbcProperties);
+        return this;
+    }
+
+    /** The Debezium MySQL connector properties. For example, "snapshot.mode". */
+    public MySqlSourceBuilder<T> debeziumProperties(Properties properties) {
+        this.configFactory.debeziumProperties(properties);
+        return this;
+    }
+
+    /**
+     * The deserializer used to convert from consumed {@link
+     * org.apache.kafka.connect.source.SourceRecord}.
+     */
+    public MySqlSourceBuilder<T> deserializer(DebeziumDeserializationSchema<T> deserializer) {
+        this.deserializer = deserializer;
+        return this;
+    }
+
+    /** The interval of heartbeat event. */
+    public MySqlSourceBuilder<T> heartbeatInterval(Duration heartbeatInterval) {
+        this.configFactory.heartbeatInterval(heartbeatInterval);
+        return this;
+    }
+
+    /**
+     * Build the {@link MySqlSource}.
+     *
+     * @return a MySqlParallelSource with the settings made for this builder.
+     */
+    public MySqlSource<T> build() {
+        return new MySqlSource<>(configFactory, checkNotNull(deserializer));
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/AssignerStatus.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/AssignerStatus.java
new file mode 100644
index 000000000..a77df359a
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/AssignerStatus.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners;
+
+import static java.lang.String.format;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The state of split assigner finite state machine, tips: we use word status instead of word state
+ * to avoid conflict with Flink state keyword. The assigner finite state machine goes this way.
+ *
+ * <pre>
+ *        INITIAL_ASSIGNING(start)
+ *              |
+ *              |
+ *          onFinish()
+ *              |
+ *              ↓
+ *    INITIAL_ASSIGNING_FINISHED(end)
+ *              |
+ *              |
+ *        suspend() // found newly added tables
+ *              |
+ *              ↓
+ *          SUSPENDED --- wakeup() --→ NEWLY_ADDED_ASSIGNING --- onFinish() --→ NEWLY_ADDED_ASSIGNING_FINISHED(end)
+ *              ↑                                                                  |
+ *              |                                                                  |
+ *              |----------------- suspend() //found newly added tables -----------|
+ * </pre>
+ */
+public enum AssignerStatus {
+    INITIAL_ASSIGNING(0) {
+        @Override
+        public AssignerStatus getNextStatus() {
+            return INITIAL_ASSIGNING_FINISHED;
+        }
+
+        @Override
+        public AssignerStatus onFinish() {
+            LOG.info(
+                    "Assigner status changes from INITIAL_ASSIGNING to INITIAL_ASSIGNING_FINISHED");
+            return this.getNextStatus();
+        }
+    },
+    INITIAL_ASSIGNING_FINISHED(1) {
+        @Override
+        public AssignerStatus getNextStatus() {
+            return SUSPENDED;
+        }
+
+        @Override
+        public AssignerStatus suspend() {
+            LOG.info("Assigner status changes from INITIAL_ASSIGNING_FINISHED to SUSPENDED");
+            return this.getNextStatus();
+        }
+    },
+    SUSPENDED(2) {
+        @Override
+        public AssignerStatus getNextStatus() {
+            return NEWLY_ADDED_ASSIGNING;
+        }
+
+        @Override
+        public AssignerStatus wakeup() {
+            LOG.info("Assigner status changes from SUSPENDED to NEWLY_ADDED_ASSIGNING");
+            return this.getNextStatus();
+        }
+    },
+    NEWLY_ADDED_ASSIGNING(3) {
+        @Override
+        public AssignerStatus getNextStatus() {
+            return NEWLY_ADDED_ASSIGNING_FINISHED;
+        }
+
+        @Override
+        public AssignerStatus onFinish() {
+            LOG.info(
+                    "Assigner status changes from NEWLY_ADDED_ASSIGNING to NEWLY_ADDED_ASSIGNING_FINISHED");
+            return this.getNextStatus();
+        }
+    },
+    NEWLY_ADDED_ASSIGNING_FINISHED(4) {
+        @Override
+        public AssignerStatus getNextStatus() {
+            return SUSPENDED;
+        }
+
+        @Override
+        public AssignerStatus suspend() {
+            LOG.info("Assigner status changes from NEWLY_ADDED_ASSIGNING_FINISHED to SUSPENDED");
+            return this.getNextStatus();
+        }
+    };
+
+    private static final Logger LOG = LoggerFactory.getLogger(AssignerStatus.class);
+    private final int statusCode;
+
+    AssignerStatus(int statusCode) {
+        this.statusCode = statusCode;
+    }
+
+    public int getStatusCode() {
+        return statusCode;
+    }
+
+    public abstract AssignerStatus getNextStatus();
+
+    public AssignerStatus onFinish() {
+        throw new IllegalStateException(
+                format(
+                        "Invalid call, assigner under %s state can not call onFinish()",
+                        fromStatusCode(this.getStatusCode())));
+    }
+
+    public AssignerStatus suspend() {
+        throw new IllegalStateException(
+                format(
+                        "Invalid call, assigner under %s state can not call suspend()",
+                        fromStatusCode(this.getStatusCode())));
+    }
+
+    public AssignerStatus wakeup() {
+        throw new IllegalStateException(
+                format(
+                        "Invalid call, assigner under %s state can not call wakeup()",
+                        fromStatusCode(this.getStatusCode())));
+    }
+
+    // --------------------------------------------------------------------------------------------
+    // Utilities
+    // --------------------------------------------------------------------------------------------
+
+    /** Gets the {@link AssignerStatus} from status code. */
+    public static AssignerStatus fromStatusCode(int statusCode) {
+        switch (statusCode) {
+            case 0:
+                return INITIAL_ASSIGNING;
+            case 1:
+                return INITIAL_ASSIGNING_FINISHED;
+            case 2:
+                return SUSPENDED;
+            case 3:
+                return NEWLY_ADDED_ASSIGNING;
+            case 4:
+                return NEWLY_ADDED_ASSIGNING_FINISHED;
+            default:
+                throw new IllegalStateException(
+                        format(
+                                "Invalid status code %s,the valid code range is [0, 4]",
+                                statusCode));
+        }
+    }
+
+    /** Returns whether the split assigner state is suspended. */
+    public static boolean isSuspended(AssignerStatus assignerStatus) {
+        return assignerStatus == SUSPENDED;
+    }
+
+    /**
+     * Returns whether the split assigner has assigned all snapshot splits, which indicates there is
+     * no more splits and all records of splits have been completely processed in the pipeline.
+     */
+    public static boolean isAssigningFinished(AssignerStatus assignerStatus) {
+        return assignerStatus == INITIAL_ASSIGNING_FINISHED
+                || assignerStatus == NEWLY_ADDED_ASSIGNING_FINISHED;
+    }
+
+    /** Returns whether the split assigner is assigning snapshot splits. */
+    public static boolean isAssigning(AssignerStatus assignerStatus) {
+        return assignerStatus == INITIAL_ASSIGNING || assignerStatus == NEWLY_ADDED_ASSIGNING;
+    }
+
+    /** Returns whether the split assigner is assigning newly added snapshot splits. */
+    public static boolean isNewlyAddedAssigning(AssignerStatus assignerStatus) {
+        return assignerStatus == NEWLY_ADDED_ASSIGNING;
+    }
+
+    /** Returns whether the split assigner has finished its initial tables assignment. */
+    public static boolean isInitialAssigningFinished(AssignerStatus assignerStatus) {
+        return assignerStatus == INITIAL_ASSIGNING_FINISHED;
+    }
+
+    /** Returns whether the split assigner has finished its newly added tables assignment. */
+    public static boolean isNewlyAddedAssigningFinished(AssignerStatus assignerStatus) {
+        return assignerStatus == NEWLY_ADDED_ASSIGNING_FINISHED;
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/ChunkRange.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/ChunkRange.java
new file mode 100644
index 000000000..1de32c9a5
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/ChunkRange.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+import java.util.Objects;
+import javax.annotation.Nullable;
+
+/**
+ * An internal structure describes a chunk range with a chunk start (inclusive) and chunk end
+ * (exclusive). Note that {@code null} represents unbounded chunk start/end.
+ */
+class ChunkRange {
+    private final @Nullable Object chunkStart;
+    private final @Nullable Object chunkEnd;
+
+    /**
+     * Returns a {@link ChunkRange} which represents a full table scan with unbounded chunk start
+     * and chunk end.
+     */
+    public static ChunkRange all() {
+        return new ChunkRange(null, null);
+    }
+
+    /** Returns a {@link ChunkRange} with the given chunk start and chunk end. */
+    public static ChunkRange of(Object chunkStart, Object chunkEnd) {
+        return new ChunkRange(chunkStart, chunkEnd);
+    }
+
+    private ChunkRange(@Nullable Object chunkStart, @Nullable Object chunkEnd) {
+        if (chunkStart != null || chunkEnd != null) {
+            checkArgument(
+                    !Objects.equals(chunkStart, chunkEnd),
+                    "Chunk start %s shouldn't be equal to chunk end %s",
+                    chunkStart,
+                    chunkEnd);
+        }
+        this.chunkStart = chunkStart;
+        this.chunkEnd = chunkEnd;
+    }
+
+    @Nullable
+    public Object getChunkStart() {
+        return chunkStart;
+    }
+
+    @Nullable
+    public Object getChunkEnd() {
+        return chunkEnd;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        ChunkRange that = (ChunkRange) o;
+        return Objects.equals(chunkStart, that.chunkStart)
+                && Objects.equals(chunkEnd, that.chunkEnd);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(chunkStart, chunkEnd);
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/ChunkSplitter.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/ChunkSplitter.java
new file mode 100644
index 000000000..6b634aee4
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/ChunkSplitter.java
@@ -0,0 +1,325 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.DebeziumUtils.openJdbcConnection;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.ObjectUtils.doubleCompare;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.StatementUtils.queryApproximateRowCnt;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.StatementUtils.queryMin;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.StatementUtils.queryMinMax;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.StatementUtils.queryNextChunkMax;
+import static java.math.BigDecimal.ROUND_CEILING;
+
+import io.debezium.jdbc.JdbcConnection;
+import io.debezium.relational.Column;
+import io.debezium.relational.Table;
+import io.debezium.relational.TableId;
+import io.debezium.relational.history.TableChanges.TableChange;
+import java.math.BigDecimal;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.schema.MySqlSchema;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.schema.MySqlTypeUtils;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfig;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSnapshotSplit;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.ChunkUtils;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.ObjectUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The {@code ChunkSplitter}'s task is to split table into a set of chunks or called splits (i.e.
+ * {@link MySqlSnapshotSplit}).
+ */
+class ChunkSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ChunkSplitter.class);
+
+    private final MySqlSourceConfig sourceConfig;
+    private final MySqlSchema mySqlSchema;
+
+    public ChunkSplitter(MySqlSchema mySqlSchema, MySqlSourceConfig sourceConfig) {
+        this.mySqlSchema = mySqlSchema;
+        this.sourceConfig = sourceConfig;
+    }
+
+    /** Generates all snapshot splits (chunks) for the give table path. */
+    public Collection<MySqlSnapshotSplit> generateSplits(TableId tableId) {
+        try (JdbcConnection jdbc = openJdbcConnection(sourceConfig)) {
+
+            LOG.info("Start splitting table {} into chunks...", tableId);
+            long start = System.currentTimeMillis();
+
+            Table table = mySqlSchema.getTableSchema(jdbc, tableId).getTable();
+            Column splitColumn = ChunkUtils.getSplitColumn(table);
+            final List<ChunkRange> chunks;
+            try {
+                chunks = splitTableIntoChunks(jdbc, tableId, splitColumn);
+            } catch (SQLException e) {
+                throw new FlinkRuntimeException("Failed to split chunks for table " + tableId, e);
+            }
+
+            // convert chunks into splits
+            List<MySqlSnapshotSplit> splits = new ArrayList<>();
+            RowType splitType = ChunkUtils.getSplitType(splitColumn);
+            for (int i = 0; i < chunks.size(); i++) {
+                ChunkRange chunk = chunks.get(i);
+                MySqlSnapshotSplit split =
+                        createSnapshotSplit(
+                                jdbc,
+                                tableId,
+                                i,
+                                splitType,
+                                chunk.getChunkStart(),
+                                chunk.getChunkEnd());
+                splits.add(split);
+            }
+
+            long end = System.currentTimeMillis();
+            LOG.info(
+                    "Split table {} into {} chunks, time cost: {}ms.",
+                    tableId,
+                    splits.size(),
+                    end - start);
+            return splits;
+        } catch (Exception e) {
+            throw new FlinkRuntimeException(
+                    String.format("Generate Splits for table %s error", tableId), e);
+        }
+    }
+
+    // --------------------------------------------------------------------------------------------
+    // Utilities
+    // --------------------------------------------------------------------------------------------
+
+    /**
+     * We can use evenly-sized chunks or unevenly-sized chunks when split table into chunks, using
+     * evenly-sized chunks which is much efficient, using unevenly-sized chunks which will request
+     * many queries and is not efficient.
+     */
+    private List<ChunkRange> splitTableIntoChunks(
+            JdbcConnection jdbc, TableId tableId, Column splitColumn) throws SQLException {
+        final String splitColumnName = splitColumn.name();
+        final Object[] minMaxOfSplitColumn = queryMinMax(jdbc, tableId, splitColumnName);
+        final Object min = minMaxOfSplitColumn[0];
+        final Object max = minMaxOfSplitColumn[1];
+        if (min == null || max == null || min.equals(max)) {
+            // empty table, or only one row, return full table scan as a chunk
+            return Collections.singletonList(ChunkRange.all());
+        }
+
+        final int chunkSize = sourceConfig.getSplitSize();
+        final double distributionFactorUpper = sourceConfig.getDistributionFactorUpper();
+        final double distributionFactorLower = sourceConfig.getDistributionFactorLower();
+
+        if (isEvenlySplitColumn(splitColumn)) {
+            long approximateRowCnt = queryApproximateRowCnt(jdbc, tableId);
+            double distributionFactor =
+                    calculateDistributionFactor(tableId, min, max, approximateRowCnt);
+
+            boolean dataIsEvenlyDistributed =
+                    doubleCompare(distributionFactor, distributionFactorLower) >= 0
+                            && doubleCompare(distributionFactor, distributionFactorUpper) <= 0;
+
+            if (dataIsEvenlyDistributed) {
+                // the minimum dynamic chunk size is at least 1
+                final int dynamicChunkSize = Math.max((int) (distributionFactor * chunkSize), 1);
+                return splitEvenlySizedChunks(
+                        tableId, min, max, approximateRowCnt, dynamicChunkSize);
+            } else {
+                return splitUnevenlySizedChunks(
+                        jdbc, tableId, splitColumnName, min, max, chunkSize);
+            }
+        } else {
+            return splitUnevenlySizedChunks(jdbc, tableId, splitColumnName, min, max, chunkSize);
+        }
+    }
+
+    /**
+     * Split table into evenly sized chunks based on the numeric min and max value of split column,
+     * and tumble chunks in step size.
+     */
+    private List<ChunkRange> splitEvenlySizedChunks(
+            TableId tableId, Object min, Object max, long approximateRowCnt, int chunkSize) {
+        LOG.info(
+                "Use evenly-sized chunk optimization for table {}, "
+                    + "the approximate row count is {}, the chunk size is {}",
+                tableId,
+                approximateRowCnt,
+                chunkSize);
+        if (approximateRowCnt <= chunkSize) {
+            // there is no more than one chunk, return full table as a chunk
+            return Collections.singletonList(ChunkRange.all());
+        }
+
+        final List<ChunkRange> splits = new ArrayList<>();
+        Object chunkStart = null;
+        Object chunkEnd = ObjectUtils.plus(min, chunkSize);
+        while (ObjectUtils.compare(chunkEnd, max) <= 0) {
+            splits.add(ChunkRange.of(chunkStart, chunkEnd));
+            chunkStart = chunkEnd;
+            chunkEnd = ObjectUtils.plus(chunkEnd, chunkSize);
+        }
+        // add the ending split
+        splits.add(ChunkRange.of(chunkStart, null));
+        return splits;
+    }
+
+    /** Split table into unevenly sized chunks by continuously calculating next chunk max value. */
+    private List<ChunkRange> splitUnevenlySizedChunks(
+            JdbcConnection jdbc,
+            TableId tableId,
+            String splitColumnName,
+            Object min,
+            Object max,
+            int chunkSize)
+            throws SQLException {
+        LOG.info(
+                "Use unevenly-sized chunks for table {}, the chunk size is {}", tableId, chunkSize);
+        final List<ChunkRange> splits = new ArrayList<>();
+        Object chunkStart = null;
+        Object chunkEnd = nextChunkEnd(jdbc, min, tableId, splitColumnName, max, chunkSize);
+        int count = 0;
+        while (chunkEnd != null && ObjectUtils.compare(chunkEnd, max) <= 0) {
+            // we start from [null, min + chunk_size) and avoid [null, min)
+            splits.add(ChunkRange.of(chunkStart, chunkEnd));
+            // may sleep a while to avoid DDOS on MySQL server
+            maySleep(count++, tableId);
+            chunkStart = chunkEnd;
+            chunkEnd = nextChunkEnd(jdbc, chunkEnd, tableId, splitColumnName, max, chunkSize);
+        }
+        // add the ending split
+        splits.add(ChunkRange.of(chunkStart, null));
+        return splits;
+    }
+
+    private Object nextChunkEnd(
+            JdbcConnection jdbc,
+            Object previousChunkEnd,
+            TableId tableId,
+            String splitColumnName,
+            Object max,
+            int chunkSize)
+            throws SQLException {
+        // chunk end might be null when max values are removed
+        Object chunkEnd =
+                queryNextChunkMax(jdbc, tableId, splitColumnName, chunkSize, previousChunkEnd);
+        if (Objects.equals(previousChunkEnd, chunkEnd)) {
+            // we don't allow equal chunk start and end,
+            // should query the next one larger than chunkEnd
+            chunkEnd = queryMin(jdbc, tableId, splitColumnName, chunkEnd);
+        }
+        if (ObjectUtils.compare(chunkEnd, max) >= 0) {
+            return null;
+        } else {
+            return chunkEnd;
+        }
+    }
+
+    private MySqlSnapshotSplit createSnapshotSplit(
+            JdbcConnection jdbc,
+            TableId tableId,
+            int chunkId,
+            RowType splitKeyType,
+            Object chunkStart,
+            Object chunkEnd) {
+        // currently, we only support single split column
+        Object[] splitStart = chunkStart == null ? null : new Object[] {chunkStart};
+        Object[] splitEnd = chunkEnd == null ? null : new Object[] {chunkEnd};
+        Map<TableId, TableChange> schema = new HashMap<>();
+        schema.put(tableId, mySqlSchema.getTableSchema(jdbc, tableId));
+        return new MySqlSnapshotSplit(
+                tableId,
+                splitId(tableId, chunkId),
+                splitKeyType,
+                splitStart,
+                splitEnd,
+                null,
+                schema);
+    }
+
+    // ------------------------------------------------------------------------------------------
+
+    /** Checks whether split column is evenly distributed across its range. */
+    private static boolean isEvenlySplitColumn(Column splitColumn) {
+        DataType flinkType = MySqlTypeUtils.fromDbzColumn(splitColumn);
+        LogicalTypeRoot typeRoot = flinkType.getLogicalType().getTypeRoot();
+
+        // currently, we only support the optimization that split column with type BIGINT, INT,
+        // DECIMAL
+        return typeRoot == LogicalTypeRoot.BIGINT
+                || typeRoot == LogicalTypeRoot.INTEGER
+                || typeRoot == LogicalTypeRoot.DECIMAL;
+    }
+
+    /** Returns the distribution factor of the given table. */
+    private double calculateDistributionFactor(
+            TableId tableId, Object min, Object max, long approximateRowCnt) {
+
+        if (!min.getClass().equals(max.getClass())) {
+            throw new IllegalStateException(
+                    String.format(
+                            "Unsupported operation type, the MIN value type %s is different with MAX value type %s.",
+                            min.getClass().getSimpleName(), max.getClass().getSimpleName()));
+        }
+        if (approximateRowCnt == 0) {
+            return Double.MAX_VALUE;
+        }
+        BigDecimal difference = ObjectUtils.minus(max, min);
+        // factor = (max - min + 1) / rowCount
+        final BigDecimal subRowCnt = difference.add(BigDecimal.valueOf(1));
+        double distributionFactor =
+                subRowCnt.divide(new BigDecimal(approximateRowCnt), 4, ROUND_CEILING).doubleValue();
+        LOG.info(
+                "The distribution factor of table {} is {} according to "
+                    + "the min split key {}, max split key {} and approximate row count {}",
+                tableId,
+                distributionFactor,
+                min,
+                max,
+                approximateRowCnt);
+        return distributionFactor;
+    }
+
+    private static String splitId(TableId tableId, int chunkId) {
+        return tableId.toString() + ":" + chunkId;
+    }
+
+    private static void maySleep(int count, TableId tableId) {
+        // every 100 queries to sleep 1s
+        if (count % 10 == 0) {
+            try {
+                Thread.sleep(100);
+            } catch (InterruptedException e) {
+                // nothing to do
+            }
+            LOG.info("ChunkSplitter has split {} chunks for table {}", count, tableId);
+        }
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/MySqlBinlogSplitAssigner.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/MySqlBinlogSplitAssigner.java
new file mode 100644
index 000000000..c2b918d24
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/MySqlBinlogSplitAssigner.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners;
+
+import static com.ververica.cdc.connectors.mysql.source.utils.TableDiscoveryUtils.listTables;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.DebeziumUtils.currentBinlogOffset;
+
+import com.ververica.cdc.connectors.mysql.schema.MySqlSchema;
+import com.ververica.cdc.connectors.mysql.source.MySqlSourceOptions;
+import io.debezium.connector.mysql.MySqlConnection;
+import io.debezium.jdbc.JdbcConnection;
+import io.debezium.relational.RelationalTableFilters;
+import io.debezium.relational.TableId;
+import io.debezium.relational.history.TableChanges.TableChange;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.DebeziumUtils;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state.BinlogPendingSplitsState;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state.PendingSplitsState;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfig;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.FinishedSnapshotSplitInfo;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlBinlogSplit;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link MySqlSplitAssigner} which only read binlog from current binlog position.
+ */
+public class MySqlBinlogSplitAssigner implements MySqlSplitAssigner {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MySqlBinlogSplitAssigner.class);
+    private static final String BINLOG_SPLIT_ID = "binlog-split";
+
+    private final MySqlSourceConfig sourceConfig;
+    private final RelationalTableFilters tableFilters;
+
+    private MySqlConnection jdbc;
+    private boolean isBinlogSplitAssigned;
+
+    public MySqlBinlogSplitAssigner(MySqlSourceConfig sourceConfig) {
+        this(sourceConfig, false);
+    }
+
+    public MySqlBinlogSplitAssigner(
+            MySqlSourceConfig sourceConfig, BinlogPendingSplitsState checkpoint) {
+        this(sourceConfig, checkpoint.isBinlogSplitAssigned());
+    }
+
+    private MySqlBinlogSplitAssigner(
+            MySqlSourceConfig sourceConfig, boolean isBinlogSplitAssigned) {
+        this.sourceConfig = sourceConfig;
+        this.tableFilters = DebeziumUtils.createTableFilters(sourceConfig);
+        this.isBinlogSplitAssigned = isBinlogSplitAssigned;
+    }
+
+    @Override
+    public void open() {
+        jdbc = DebeziumUtils.createMySqlConnection(sourceConfig);
+    }
+
+    @Override
+    public Optional<MySqlSplit> getNext() {
+        if (isBinlogSplitAssigned) {
+            return Optional.empty();
+        } else {
+            isBinlogSplitAssigned = true;
+            return Optional.of(createBinlogSplit());
+        }
+    }
+
+    @Override
+    public boolean waitingForFinishedSplits() {
+        return false;
+    }
+
+    @Override
+    public List<FinishedSnapshotSplitInfo> getFinishedSplitInfos() {
+        return Collections.EMPTY_LIST;
+    }
+
+    @Override
+    public void onFinishedSplits(Map<String, BinlogOffset> splitFinishedOffsets) {
+        // do nothing
+    }
+
+    @Override
+    public void addSplits(Collection<MySqlSplit> splits) {
+        // we don't store the split, but will re-create binlog split later
+        isBinlogSplitAssigned = false;
+    }
+
+    @Override
+    public PendingSplitsState snapshotState(long checkpointId) {
+        return new BinlogPendingSplitsState(isBinlogSplitAssigned);
+    }
+
+    @Override
+    public void notifyCheckpointComplete(long checkpointId) {
+        // nothing to do
+    }
+
+    @Override
+    public AssignerStatus getAssignerStatus() {
+        return AssignerStatus.INITIAL_ASSIGNING_FINISHED;
+    }
+
+    @Override
+    public void suspend() {
+
+    }
+
+    @Override
+    public void wakeup() {
+
+    }
+
+    @Override
+    public void close() {
+
+    }
+
+    // ------------------------------------------------------------------------------------------
+
+    private MySqlBinlogSplit createBinlogSplit() {
+        Map<TableId, TableChange> tableSchemas = discoverCapturedTableSchemas();
+        try (JdbcConnection jdbc = DebeziumUtils.openJdbcConnection(sourceConfig)) {
+            return new MySqlBinlogSplit(
+                    BINLOG_SPLIT_ID,
+                    currentBinlogOffset(jdbc),
+                    BinlogOffset.NO_STOPPING_OFFSET,
+                    new ArrayList<>(),
+                    tableSchemas,
+                    0);
+        } catch (Exception e) {
+            throw new FlinkRuntimeException("Read the binlog offset error", e);
+        }
+    }
+
+    private Map<TableId, TableChange> discoverCapturedTableSchemas() {
+        final List<TableId> capturedTableIds;
+        try {
+            capturedTableIds = listTables(jdbc, tableFilters);
+        } catch (SQLException e) {
+            throw new FlinkRuntimeException("Failed to discover captured tables", e);
+        }
+        if (capturedTableIds.isEmpty()) {
+            throw new IllegalArgumentException(
+                    String.format(
+                            "Can't find any matched tables,"
+                                    + " please check your configured database-name: %s and table-name: %s",
+                            sourceConfig.getDbzConfiguration().getString(MySqlSourceOptions.DATABASE_NAME.key()),
+                            sourceConfig.getDbzConfiguration().getString(MySqlSourceOptions.TABLE_NAME.key())));
+        }
+
+        // fetch table schemas
+        MySqlSchema mySqlSchema = new MySqlSchema(sourceConfig.getDbzConfiguration(), jdbc);
+        Map<TableId, TableChange> tableSchemas = new HashMap<>();
+        for (TableId tableId : capturedTableIds) {
+            TableChange tableSchema = mySqlSchema.getTableSchema(tableId);
+            tableSchemas.put(tableId, tableSchema);
+        }
+        return tableSchemas;
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/MySqlHybridSplitAssigner.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/MySqlHybridSplitAssigner.java
new file mode 100644
index 000000000..712b3a086
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/MySqlHybridSplitAssigner.java
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.AssignerStatus.isInitialAssigningFinished;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.AssignerStatus.isNewlyAddedAssigningFinished;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.AssignerStatus.isSuspended;
+
+import io.debezium.relational.TableId;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state.HybridPendingSplitsState;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state.PendingSplitsState;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfig;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.FinishedSnapshotSplitInfo;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlBinlogSplit;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSnapshotSplit;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link MySqlSplitAssigner} that splits tables into small chunk splits based on primary key
+ * range and chunk size and also continue with a binlog split.
+ */
+public class MySqlHybridSplitAssigner implements MySqlSplitAssigner {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MySqlHybridSplitAssigner.class);
+    private static final String BINLOG_SPLIT_ID = "binlog-split";
+
+    private final int splitMetaGroupSize;
+
+    private boolean isBinlogSplitAssigned;
+
+    private final MySqlSnapshotSplitAssigner snapshotSplitAssigner;
+
+    public MySqlHybridSplitAssigner(
+            MySqlSourceConfig sourceConfig,
+            int currentParallelism,
+            List<TableId> remainingTables,
+            boolean isTableIdCaseSensitive) {
+        this(
+                new MySqlSnapshotSplitAssigner(
+                        sourceConfig, currentParallelism, remainingTables, isTableIdCaseSensitive),
+                false,
+                sourceConfig.getSplitMetaGroupSize());
+    }
+
+    public MySqlHybridSplitAssigner(
+            MySqlSourceConfig sourceConfig,
+            int currentParallelism,
+            HybridPendingSplitsState checkpoint) {
+        this(
+                new MySqlSnapshotSplitAssigner(
+                        sourceConfig, currentParallelism, checkpoint.getSnapshotPendingSplits()),
+                checkpoint.isBinlogSplitAssigned(),
+                sourceConfig.getSplitMetaGroupSize());
+    }
+
+    private MySqlHybridSplitAssigner(
+            MySqlSnapshotSplitAssigner snapshotSplitAssigner,
+            boolean isBinlogSplitAssigned,
+            int splitMetaGroupSize) {
+        this.snapshotSplitAssigner = snapshotSplitAssigner;
+        this.isBinlogSplitAssigned = isBinlogSplitAssigned;
+        this.splitMetaGroupSize = splitMetaGroupSize;
+    }
+
+    @Override
+    public void open() {
+        snapshotSplitAssigner.open();
+    }
+
+    @Override
+    public Optional<MySqlSplit> getNext() {
+        if (isSuspended(getAssignerStatus())) {
+            // do not assign split until the assigner received SuspendBinlogReaderAckEvent
+            return Optional.empty();
+        }
+        if (snapshotSplitAssigner.noMoreSplits()) {
+            // binlog split assigning
+            if (isBinlogSplitAssigned) {
+                // no more splits for the assigner
+                return Optional.empty();
+            } else if (isInitialAssigningFinished(snapshotSplitAssigner.getAssignerStatus())) {
+                // we need to wait snapshot-assigner to be finished before
+                // assigning the binlog split. Otherwise, records emitted from binlog split
+                // might be out-of-order in terms of same primary key with snapshot splits.
+                isBinlogSplitAssigned = true;
+                return Optional.of(createBinlogSplit());
+            } else if (isNewlyAddedAssigningFinished(snapshotSplitAssigner.getAssignerStatus())) {
+                // do not need to create binlog, but send event to wake up the binlog reader
+                isBinlogSplitAssigned = true;
+                return Optional.empty();
+            } else {
+                // binlog split is not ready by now
+                return Optional.empty();
+            }
+        } else {
+            // snapshot assigner still have remaining splits, assign split from it
+            return snapshotSplitAssigner.getNext();
+        }
+    }
+
+    @Override
+    public boolean waitingForFinishedSplits() {
+        return snapshotSplitAssigner.waitingForFinishedSplits();
+    }
+
+    @Override
+    public List<FinishedSnapshotSplitInfo> getFinishedSplitInfos() {
+        return snapshotSplitAssigner.getFinishedSplitInfos();
+    }
+
+    @Override
+    public void onFinishedSplits(Map<String, BinlogOffset> splitFinishedOffsets) {
+        snapshotSplitAssigner.onFinishedSplits(splitFinishedOffsets);
+    }
+
+    @Override
+    public void addSplits(Collection<MySqlSplit> splits) {
+        List<MySqlSplit> snapshotSplits = new ArrayList<>();
+        for (MySqlSplit split : splits) {
+            if (split.isSnapshotSplit()) {
+                snapshotSplits.add(split);
+            } else {
+                // we don't store the split, but will re-create binlog split later
+                isBinlogSplitAssigned = false;
+            }
+        }
+        snapshotSplitAssigner.addSplits(snapshotSplits);
+    }
+
+    @Override
+    public PendingSplitsState snapshotState(long checkpointId) {
+        return new HybridPendingSplitsState(
+                snapshotSplitAssigner.snapshotState(checkpointId), isBinlogSplitAssigned);
+    }
+
+    @Override
+    public void notifyCheckpointComplete(long checkpointId) {
+        snapshotSplitAssigner.notifyCheckpointComplete(checkpointId);
+    }
+
+    @Override
+    public AssignerStatus getAssignerStatus() {
+        return snapshotSplitAssigner.getAssignerStatus();
+    }
+
+    @Override
+    public void suspend() {
+        snapshotSplitAssigner.suspend();
+    }
+
+    @Override
+    public void wakeup() {
+        snapshotSplitAssigner.wakeup();
+    }
+
+    @Override
+    public void close() {
+        snapshotSplitAssigner.close();
+    }
+
+    // --------------------------------------------------------------------------------------------
+
+    private MySqlBinlogSplit createBinlogSplit() {
+        final List<MySqlSnapshotSplit> assignedSnapshotSplit =
+                snapshotSplitAssigner.getAssignedSplits().values().stream()
+                        .sorted(Comparator.comparing(MySqlSplit::splitId))
+                        .collect(Collectors.toList());
+
+        Map<String, BinlogOffset> splitFinishedOffsets =
+                snapshotSplitAssigner.getSplitFinishedOffsets();
+        final List<FinishedSnapshotSplitInfo> finishedSnapshotSplitInfos = new ArrayList<>();
+
+        BinlogOffset minBinlogOffset = null;
+        for (MySqlSnapshotSplit split : assignedSnapshotSplit) {
+            // find the min binlog offset
+            BinlogOffset binlogOffset = splitFinishedOffsets.get(split.splitId());
+            if (minBinlogOffset == null || binlogOffset.isBefore(minBinlogOffset)) {
+                minBinlogOffset = binlogOffset;
+            }
+            finishedSnapshotSplitInfos.add(
+                    new FinishedSnapshotSplitInfo(
+                            split.getTableId(),
+                            split.splitId(),
+                            split.getSplitStart(),
+                            split.getSplitEnd(),
+                            binlogOffset));
+        }
+
+        // the finishedSnapshotSplitInfos is too large for transmission, divide it to groups and
+        // then transfer them
+
+        boolean divideMetaToGroups = finishedSnapshotSplitInfos.size() > splitMetaGroupSize;
+        return new MySqlBinlogSplit(
+                BINLOG_SPLIT_ID,
+                minBinlogOffset == null ? BinlogOffset.INITIAL_OFFSET : minBinlogOffset,
+                BinlogOffset.NO_STOPPING_OFFSET,
+                divideMetaToGroups ? new ArrayList<>() : finishedSnapshotSplitInfos,
+                new HashMap<>(),
+                finishedSnapshotSplitInfos.size());
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/MySqlSnapshotSplitAssigner.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/MySqlSnapshotSplitAssigner.java
new file mode 100644
index 000000000..65c415d6c
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/MySqlSnapshotSplitAssigner.java
@@ -0,0 +1,404 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.DebeziumUtils.discoverCapturedTables;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.DebeziumUtils.openJdbcConnection;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.AssignerStatus.isAssigningFinished;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.AssignerStatus.isSuspended;
+
+import io.debezium.jdbc.JdbcConnection;
+import io.debezium.relational.TableId;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.DebeziumUtils;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.schema.MySqlSchema;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state.SnapshotPendingSplitsState;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfig;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceOptions;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.FinishedSnapshotSplitInfo;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSnapshotSplit;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link MySqlSplitAssigner} that splits tables into small chunk splits based on primary key
+ * range and chunk size.
+ *
+ * @see MySqlSourceOptions#SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE
+ */
+public class MySqlSnapshotSplitAssigner implements MySqlSplitAssigner {
+    private static final Logger LOG = LoggerFactory.getLogger(MySqlSnapshotSplitAssigner.class);
+
+    private final List<TableId> alreadyProcessedTables;
+    private final List<MySqlSnapshotSplit> remainingSplits;
+    private final Map<String, MySqlSnapshotSplit> assignedSplits;
+    private final Map<String, BinlogOffset> splitFinishedOffsets;
+    private final MySqlSourceConfig sourceConfig;
+    private final int currentParallelism;
+    private final List<TableId> remainingTables;
+    private final boolean isRemainingTablesCheckpointed;
+
+    private AssignerStatus assignerStatus;
+    private ChunkSplitter chunkSplitter;
+    private boolean isTableIdCaseSensitive;
+
+    private ExecutorService executor;
+    private Object lock;
+
+    @Nullable private Long checkpointIdToFinish;
+
+    public MySqlSnapshotSplitAssigner(
+            MySqlSourceConfig sourceConfig,
+            int currentParallelism,
+            List<TableId> remainingTables,
+            boolean isTableIdCaseSensitive) {
+        this(
+                sourceConfig,
+                currentParallelism,
+                new ArrayList<>(),
+                new ArrayList<>(),
+                new HashMap<>(),
+                new HashMap<>(),
+                AssignerStatus.INITIAL_ASSIGNING,
+                remainingTables,
+                isTableIdCaseSensitive,
+                true);
+    }
+
+    public MySqlSnapshotSplitAssigner(
+            MySqlSourceConfig sourceConfig,
+            int currentParallelism,
+            SnapshotPendingSplitsState checkpoint) {
+        this(
+                sourceConfig,
+                currentParallelism,
+                checkpoint.getAlreadyProcessedTables(),
+                checkpoint.getRemainingSplits(),
+                checkpoint.getAssignedSplits(),
+                checkpoint.getSplitFinishedOffsets(),
+                checkpoint.getSnapshotAssignerStatus(),
+                checkpoint.getRemainingTables(),
+                checkpoint.isTableIdCaseSensitive(),
+                checkpoint.isRemainingTablesCheckpointed());
+    }
+
+    private MySqlSnapshotSplitAssigner(
+            MySqlSourceConfig sourceConfig,
+            int currentParallelism,
+            List<TableId> alreadyProcessedTables,
+            List<MySqlSnapshotSplit> remainingSplits,
+            Map<String, MySqlSnapshotSplit> assignedSplits,
+            Map<String, BinlogOffset> splitFinishedOffsets,
+            AssignerStatus assignerStatus,
+            List<TableId> remainingTables,
+            boolean isTableIdCaseSensitive,
+            boolean isRemainingTablesCheckpointed) {
+        this.sourceConfig = sourceConfig;
+        this.currentParallelism = currentParallelism;
+        this.alreadyProcessedTables = alreadyProcessedTables;
+        this.remainingSplits = new CopyOnWriteArrayList<>(remainingSplits);
+        this.assignedSplits = assignedSplits;
+        this.splitFinishedOffsets = splitFinishedOffsets;
+        this.assignerStatus = assignerStatus;
+        this.remainingTables = new CopyOnWriteArrayList<>(remainingTables);
+        this.isRemainingTablesCheckpointed = isRemainingTablesCheckpointed;
+        this.isTableIdCaseSensitive = isTableIdCaseSensitive;
+    }
+
+    @Override
+    public void open() {
+        lock = new Object();
+        chunkSplitter = createChunkSplitter(sourceConfig, isTableIdCaseSensitive);
+
+        // the legacy state didn't snapshot remaining tables, discovery remaining table here
+        if (!isRemainingTablesCheckpointed && !isAssigningFinished(assignerStatus)) {
+            try (JdbcConnection jdbc = openJdbcConnection(sourceConfig)) {
+                final List<TableId> discoverTables = discoverCapturedTables(jdbc, sourceConfig);
+                discoverTables.removeAll(alreadyProcessedTables);
+                this.remainingTables.addAll(discoverTables);
+                this.isTableIdCaseSensitive = DebeziumUtils.isTableIdCaseSensitive(jdbc);
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(
+                        "Failed to discover remaining tables to capture", e);
+            }
+        }
+        captureNewlyAddedTables();
+        startAsynchronouslySplit();
+    }
+
+    private void captureNewlyAddedTables() {
+        if (sourceConfig.isScanNewlyAddedTableEnabled()) {
+            // check whether we got newly added tables
+            try (JdbcConnection jdbc = openJdbcConnection(sourceConfig)) {
+                final List<TableId> newlyAddedTables = discoverCapturedTables(jdbc, sourceConfig);
+                newlyAddedTables.removeAll(alreadyProcessedTables);
+                newlyAddedTables.removeAll(remainingTables);
+                if (!newlyAddedTables.isEmpty()) {
+                    // if job is still in snapshot reading phase, directly add all newly added
+                    // tables
+                    LOG.info("Found newly added tables, start capture newly added tables process");
+                    remainingTables.addAll(newlyAddedTables);
+                    if (isAssigningFinished(assignerStatus)) {
+                        // start the newly added tables process under binlog reading phase
+                        LOG.info(
+                                "Found newly added tables, start capture "
+                                    + "newly added tables process under binlog reading phase");
+                        this.suspend();
+                    }
+                }
+            } catch (Exception e) {
+                throw new FlinkRuntimeException(
+                        "Failed to discover remaining tables to capture", e);
+            }
+        }
+    }
+
+    private void startAsynchronouslySplit() {
+        if (!remainingTables.isEmpty()) {
+            if (executor == null) {
+                ThreadFactory threadFactory =
+                        new ThreadFactoryBuilder().setNameFormat("snapshot-splitting").build();
+                this.executor = Executors.newSingleThreadExecutor(threadFactory);
+            }
+
+            executor.submit(
+                    () -> {
+                        Iterator<TableId> iterator = remainingTables.iterator();
+                        while (iterator.hasNext()) {
+                            TableId nextTable = iterator.next();
+                            // split the given table into chunks (snapshot splits)
+                            Collection<MySqlSnapshotSplit> splits =
+                                    chunkSplitter.generateSplits(nextTable);
+                            synchronized (lock) {
+                                remainingSplits.addAll(splits);
+                                remainingTables.remove(nextTable);
+                                lock.notify();
+                            }
+                        }
+                    });
+        }
+    }
+
+    @Override
+    public Optional<MySqlSplit> getNext() {
+        synchronized (lock) {
+            if (!remainingSplits.isEmpty()) {
+                // return remaining splits firstly
+                Iterator<MySqlSnapshotSplit> iterator = remainingSplits.iterator();
+                MySqlSnapshotSplit split = iterator.next();
+                remainingSplits.remove(split);
+                assignedSplits.put(split.splitId(), split);
+                addAlreadyProcessedTablesIfNotExists(split.getTableId());
+                return Optional.of(split);
+            } else if (!remainingTables.isEmpty()) {
+                try {
+                    // wait for the asynchronous split to complete
+                    lock.wait();
+                } catch (InterruptedException e) {
+                    throw new FlinkRuntimeException(
+                            "InterruptedException while waiting for asynchronously snapshot split");
+                }
+                return getNext();
+            } else {
+                closeExecutorService();
+                return Optional.empty();
+            }
+        }
+    }
+
+    @Override
+    public boolean waitingForFinishedSplits() {
+        return !allSplitsFinished();
+    }
+
+    @Override
+    public List<FinishedSnapshotSplitInfo> getFinishedSplitInfos() {
+        if (waitingForFinishedSplits()) {
+            LOG.error(
+                    "The assigner is not ready to offer finished split information, this should not be called");
+            throw new FlinkRuntimeException(
+                    "The assigner is not ready to offer finished split information, this should not be called");
+        }
+        final List<MySqlSnapshotSplit> assignedSnapshotSplit =
+                assignedSplits.values().stream()
+                        .sorted(Comparator.comparing(MySqlSplit::splitId))
+                        .collect(Collectors.toList());
+        List<FinishedSnapshotSplitInfo> finishedSnapshotSplitInfos = new ArrayList<>();
+        for (MySqlSnapshotSplit split : assignedSnapshotSplit) {
+            BinlogOffset binlogOffset = splitFinishedOffsets.get(split.splitId());
+            finishedSnapshotSplitInfos.add(
+                    new FinishedSnapshotSplitInfo(
+                            split.getTableId(),
+                            split.splitId(),
+                            split.getSplitStart(),
+                            split.getSplitEnd(),
+                            binlogOffset));
+        }
+        return finishedSnapshotSplitInfos;
+    }
+
+    @Override
+    public void onFinishedSplits(Map<String, BinlogOffset> splitFinishedOffsets) {
+        this.splitFinishedOffsets.putAll(splitFinishedOffsets);
+        if (allSplitsFinished() && AssignerStatus.isAssigning(assignerStatus)) {
+            // Skip the waiting checkpoint when current parallelism is 1 which means we do not need
+            // to care about the global output data order of snapshot splits and binlog split.
+            if (currentParallelism == 1) {
+                assignerStatus = assignerStatus.onFinish();
+                LOG.info(
+                        "Snapshot split assigner received all splits finished "
+                            + "and the job parallelism is 1, snapshot split assigner is turn into finished status.");
+            } else {
+                LOG.info(
+                        "Snapshot split assigner received all splits finished, "
+                            + "waiting for a complete checkpoint to mark the assigner finished.");
+            }
+        }
+    }
+
+    @Override
+    public void addSplits(Collection<MySqlSplit> splits) {
+        for (MySqlSplit split : splits) {
+            remainingSplits.add(split.asSnapshotSplit());
+            // we should remove the add-backed splits from the assigned list,
+            // because they are failed
+            assignedSplits.remove(split.splitId());
+            splitFinishedOffsets.remove(split.splitId());
+        }
+    }
+
+    @Override
+    public SnapshotPendingSplitsState snapshotState(long checkpointId) {
+        SnapshotPendingSplitsState state =
+                new SnapshotPendingSplitsState(
+                        alreadyProcessedTables,
+                        remainingSplits,
+                        assignedSplits,
+                        splitFinishedOffsets,
+                        assignerStatus,
+                        remainingTables,
+                        isTableIdCaseSensitive,
+                        true);
+        // we need a complete checkpoint before mark this assigner to be finished, to wait for all
+        // records of snapshot splits are completely processed
+        if (checkpointIdToFinish == null
+                && !isAssigningFinished(assignerStatus)
+                && allSplitsFinished()) {
+            checkpointIdToFinish = checkpointId;
+        }
+        return state;
+    }
+
+    @Override
+    public void notifyCheckpointComplete(long checkpointId) {
+        // we have waited for at-least one complete checkpoint after all snapshot-splits are
+        // finished, then we can mark snapshot assigner as finished.
+        if (checkpointIdToFinish != null
+                && !isAssigningFinished(assignerStatus)
+                && allSplitsFinished()) {
+            if (checkpointId >= checkpointIdToFinish) {
+                assignerStatus = assignerStatus.onFinish();
+            }
+            LOG.info("Snapshot split assigner is turn into finished status.");
+        }
+    }
+
+    @Override
+    public AssignerStatus getAssignerStatus() {
+        return assignerStatus;
+    }
+
+    @Override
+    public void suspend() {
+        Preconditions.checkState(
+                isAssigningFinished(assignerStatus), "Invalid assigner status {}", assignerStatus);
+        assignerStatus = assignerStatus.suspend();
+    }
+
+    @Override
+    public void wakeup() {
+        Preconditions.checkState(
+                isSuspended(assignerStatus), "Invalid assigner status {}", assignerStatus);
+        assignerStatus = assignerStatus.wakeup();
+    }
+
+    @Override
+    public void close() {
+        closeExecutorService();
+    }
+
+    private void closeExecutorService() {
+        if (executor != null) {
+            executor.shutdown();
+        }
+    }
+
+    private void addAlreadyProcessedTablesIfNotExists(TableId tableId) {
+        if (!alreadyProcessedTables.contains(tableId)) {
+            alreadyProcessedTables.add(tableId);
+        }
+    }
+
+    /** Indicates there is no more splits available in this assigner. */
+    public boolean noMoreSplits() {
+        return remainingTables.isEmpty() && remainingSplits.isEmpty();
+    }
+
+    public Map<String, MySqlSnapshotSplit> getAssignedSplits() {
+        return assignedSplits;
+    }
+
+    public Map<String, BinlogOffset> getSplitFinishedOffsets() {
+        return splitFinishedOffsets;
+    }
+
+    // -------------------------------------------------------------------------------------------
+
+    /**
+     * Returns whether all splits are finished which means no more splits and all assigned splits
+     * are finished.
+     */
+    private boolean allSplitsFinished() {
+        return noMoreSplits() && assignedSplits.size() == splitFinishedOffsets.size();
+    }
+
+    private static ChunkSplitter createChunkSplitter(
+            MySqlSourceConfig sourceConfig, boolean isTableIdCaseSensitive) {
+        MySqlSchema mySqlSchema = new MySqlSchema(sourceConfig, isTableIdCaseSensitive);
+        return new ChunkSplitter(mySqlSchema, sourceConfig);
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/MySqlSplitAssigner.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/MySqlSplitAssigner.java
new file mode 100644
index 000000000..8ce6b3334
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/MySqlSplitAssigner.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state.PendingSplitsState;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.FinishedSnapshotSplitInfo;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSplit;
+
+/**
+ * The {@code MySqlSplitAssigner} is responsible for deciding what split should be processed. It
+ * determines split processing order.
+ */
+public interface MySqlSplitAssigner {
+
+    /**
+     * Called to open the assigner to acquire any resources, like threads or network connections.
+     */
+    void open();
+
+    /**
+     * Gets the next split.
+     *
+     * <p>When this method returns an empty {@code Optional}, then the set of splits is assumed to
+     * be done and the source will finish once the readers finished their current splits.
+     */
+    Optional<MySqlSplit> getNext();
+
+    /**
+     * Whether the split assigner is still waiting for callback of finished splits, i.e. {@link
+     * #onFinishedSplits(Map)}.
+     */
+    boolean waitingForFinishedSplits();
+
+    /**
+     * Gets the finished splits information. This is useful meta data to generate a binlog split
+     * that considering finished snapshot splits.
+     */
+    List<FinishedSnapshotSplitInfo> getFinishedSplitInfos();
+
+    /**
+     * Callback to handle the finished splits with finished binlog offset. This is useful for
+     * determine when to generate binlog split and what binlog split to generate.
+     */
+    void onFinishedSplits(Map<String, BinlogOffset> splitFinishedOffsets);
+
+    /**
+     * Adds a set of splits to this assigner. This happens for example when some split processing
+     * failed and the splits need to be re-added.
+     */
+    void addSplits(Collection<MySqlSplit> splits);
+
+    /**
+     * Creates a snapshot of the state of this split assigner, to be stored in a checkpoint.
+     *
+     * <p>The snapshot should contain the latest state of the assigner: It should assume that all
+     * operations that happened before the snapshot have successfully completed. For example all
+     * splits assigned to readers via {@link #getNext()} don't need to be included in the snapshot
+     * anymore.
+     *
+     * <p>This method takes the ID of the checkpoint for which the state is snapshotted. Most
+     * implementations should be able to ignore this parameter, because for the contents of the
+     * snapshot, it doesn't matter for which checkpoint it gets created. This parameter can be
+     * interesting for source connectors with external systems where those systems are themselves
+     * aware of checkpoints; for example in cases where the enumerator notifies that system about a
+     * specific checkpoint being triggered.
+     *
+     * @param checkpointId The ID of the checkpoint for which the snapshot is created.
+     * @return an object containing the state of the split enumerator.
+     */
+    PendingSplitsState snapshotState(long checkpointId);
+
+    /**
+     * Notifies the listener that the checkpoint with the given {@code checkpointId} completed and
+     * was committed.
+     *
+     * @see CheckpointListener#notifyCheckpointComplete(long)
+     */
+    void notifyCheckpointComplete(long checkpointId);
+
+    /** Gets the split assigner status, see {@code AssignerStatus}. */
+    AssignerStatus getAssignerStatus();
+
+    /**
+     * Suspends the assigner under {@link AssignerStatus#INITIAL_ASSIGNING_FINISHED} or {@link
+     * AssignerStatus#NEWLY_ADDED_ASSIGNING_FINISHED}.
+     */
+    void suspend();
+
+    /** Wakes up the assigner under {@link AssignerStatus#SUSPENDED}. */
+    void wakeup();
+
+    /**
+     * Called to close the assigner, in case it holds on to any resources, like threads or network
+     * connections.
+     */
+    void close();
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/ConnectionPoolId.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/BinlogPendingSplitsState.java
similarity index 57%
copy from inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/ConnectionPoolId.java
copy to inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/BinlogPendingSplitsState.java
index 8fcef7de9..4d6ea8828 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/ConnectionPoolId.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/BinlogPendingSplitsState.java
@@ -16,21 +16,21 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.sort.singletenant.flink.cdc.mysql.connection;
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state;
 
-import java.io.Serializable;
 import java.util.Objects;
 
-/** The connection pool identifier. */
-public class ConnectionPoolId implements Serializable {
+/** A {@link PendingSplitsState} for pending binlog splits. */
+public class BinlogPendingSplitsState extends PendingSplitsState {
 
-    private static final long serialVersionUID = 1L;
-    private final String host;
-    private final int port;
+    private final boolean isBinlogSplitAssigned;
 
-    public ConnectionPoolId(String host, int port) {
-        this.host = host;
-        this.port = port;
+    public BinlogPendingSplitsState(boolean isBinlogSplitAssigned) {
+        this.isBinlogSplitAssigned = isBinlogSplitAssigned;
+    }
+
+    public boolean isBinlogSplitAssigned() {
+        return isBinlogSplitAssigned;
     }
 
     @Override
@@ -38,20 +38,20 @@ public class ConnectionPoolId implements Serializable {
         if (this == o) {
             return true;
         }
-        if (!(o instanceof ConnectionPoolId)) {
+        if (o == null || getClass() != o.getClass()) {
             return false;
         }
-        ConnectionPoolId that = (ConnectionPoolId) o;
-        return Objects.equals(host, that.host) && Objects.equals(port, that.port);
+        BinlogPendingSplitsState that = (BinlogPendingSplitsState) o;
+        return isBinlogSplitAssigned == that.isBinlogSplitAssigned;
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(host, port);
+        return Objects.hash(isBinlogSplitAssigned);
     }
 
     @Override
     public String toString() {
-        return host + ':' + port;
+        return "BinlogPendingSplitsState{" + "isBinlogSplitAssigned=" + isBinlogSplitAssigned + '}';
     }
 }
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/HybridPendingSplitsState.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/HybridPendingSplitsState.java
new file mode 100644
index 000000000..1a2cabe7e
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/HybridPendingSplitsState.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state;
+
+import java.util.Objects;
+
+/** A {@link PendingSplitsState} for pending hybrid (snapshot & binlog) splits. */
+public class HybridPendingSplitsState extends PendingSplitsState {
+    private final SnapshotPendingSplitsState snapshotPendingSplits;
+    private final boolean isBinlogSplitAssigned;
+
+    public HybridPendingSplitsState(
+            SnapshotPendingSplitsState snapshotPendingSplits, boolean isBinlogSplitAssigned) {
+        this.snapshotPendingSplits = snapshotPendingSplits;
+        this.isBinlogSplitAssigned = isBinlogSplitAssigned;
+    }
+
+    public SnapshotPendingSplitsState getSnapshotPendingSplits() {
+        return snapshotPendingSplits;
+    }
+
+    public boolean isBinlogSplitAssigned() {
+        return isBinlogSplitAssigned;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        HybridPendingSplitsState that = (HybridPendingSplitsState) o;
+        return isBinlogSplitAssigned == that.isBinlogSplitAssigned
+                && Objects.equals(snapshotPendingSplits, that.snapshotPendingSplits);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(snapshotPendingSplits, isBinlogSplitAssigned);
+    }
+
+    @Override
+    public String toString() {
+        return "HybridPendingSplitsState{"
+                + "snapshotPendingSplits="
+                + snapshotPendingSplits
+                + ", isBinlogSplitAssigned="
+                + isBinlogSplitAssigned
+                + '}';
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/MetadataConverter.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/PendingSplitsState.java
similarity index 58%
copy from inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/MetadataConverter.java
copy to inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/PendingSplitsState.java
index 8c3470841..5dfb8676a 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/debezium/table/MetadataConverter.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/PendingSplitsState.java
@@ -16,21 +16,18 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.sort.singletenant.flink.cdc.debezium.table;
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state;
 
-import io.debezium.relational.history.TableChanges;
-import java.io.Serializable;
 import javax.annotation.Nullable;
-import org.apache.flink.annotation.Internal;
-import org.apache.kafka.connect.source.SourceRecord;
 
-/** A converter converts {@link SourceRecord} metadata into Flink internal data structures. */
-@FunctionalInterface
-@Internal
-public interface MetadataConverter extends Serializable {
-    Object read(SourceRecord record);
-
-    default Object read(SourceRecord record, @Nullable TableChanges.TableChange tableSchema) {
-        return read(record);
-    }
+/**
+ * A checkpoint of the current state of the containing the currently pending splits that are not yet
+ * assigned.
+ */
+public abstract class PendingSplitsState {
+    /**
+     * The splits are frequently serialized into checkpoints. Caching the byte representation makes
+     * repeated serialization cheap. This field is used by {@link PendingSplitsStateSerializer}.
+     */
+    @Nullable transient byte[] serializedFormCache;
 }
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/PendingSplitsStateSerializer.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/PendingSplitsStateSerializer.java
new file mode 100644
index 000000000..1a20399f7
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/PendingSplitsStateSerializer.java
@@ -0,0 +1,350 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.SerializerUtils.readBinlogPosition;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.utils.SerializerUtils.writeBinlogPosition;
+
+import io.debezium.relational.TableId;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.core.memory.DataInputDeserializer;
+import org.apache.flink.core.memory.DataOutputSerializer;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.AssignerStatus;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSnapshotSplit;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSplit;
+
+/**
+ * The {@link SimpleVersionedSerializer Serializer} for the {@link PendingSplitsState} of MySQL CDC
+ * source.
+ */
+public class PendingSplitsStateSerializer implements SimpleVersionedSerializer<PendingSplitsState> {
+
+    private static final int VERSION = 3;
+    private static final ThreadLocal<DataOutputSerializer> SERIALIZER_CACHE =
+            ThreadLocal.withInitial(() -> new DataOutputSerializer(64));
+
+    private static final int SNAPSHOT_PENDING_SPLITS_STATE_FLAG = 1;
+    private static final int BINLOG_PENDING_SPLITS_STATE_FLAG = 2;
+    private static final int HYBRID_PENDING_SPLITS_STATE_FLAG = 3;
+
+    private final SimpleVersionedSerializer<MySqlSplit> splitSerializer;
+
+    public PendingSplitsStateSerializer(SimpleVersionedSerializer<MySqlSplit> splitSerializer) {
+        this.splitSerializer = splitSerializer;
+    }
+
+    @Override
+    public int getVersion() {
+        return VERSION;
+    }
+
+    @Override
+    public byte[] serialize(PendingSplitsState state) throws IOException {
+        // optimization: the splits lazily cache their own serialized form
+        if (state.serializedFormCache != null) {
+            return state.serializedFormCache;
+        }
+        final DataOutputSerializer out = SERIALIZER_CACHE.get();
+
+        out.writeInt(splitSerializer.getVersion());
+        if (state instanceof SnapshotPendingSplitsState) {
+            out.writeInt(SNAPSHOT_PENDING_SPLITS_STATE_FLAG);
+            serializeSnapshotPendingSplitsState((SnapshotPendingSplitsState) state, out);
+        } else if (state instanceof BinlogPendingSplitsState) {
+            out.writeInt(BINLOG_PENDING_SPLITS_STATE_FLAG);
+            serializeBinlogPendingSplitsState((BinlogPendingSplitsState) state, out);
+        } else if (state instanceof HybridPendingSplitsState) {
+            out.writeInt(HYBRID_PENDING_SPLITS_STATE_FLAG);
+            serializeHybridPendingSplitsState((HybridPendingSplitsState) state, out);
+        } else {
+            throw new IOException(
+                    "Unsupported to serialize PendingSplitsState class: "
+                            + state.getClass().getName());
+        }
+
+        final byte[] result = out.getCopyOfBuffer();
+        // optimization: cache the serialized from, so we avoid the byte work during repeated
+        // serialization
+        state.serializedFormCache = result;
+        out.clear();
+        return result;
+    }
+
+    @Override
+    public PendingSplitsState deserialize(int version, byte[] serialized) throws IOException {
+        switch (version) {
+            case 1:
+            case 2:
+                return deserializeLegacyPendingSplitsState(serialized);
+            case 3:
+            case 4:
+                return deserializePendingSplitsState(serialized);
+            default:
+                throw new IOException("Unknown version: " + version);
+        }
+    }
+
+    public PendingSplitsState deserializeLegacyPendingSplitsState(byte[] serialized)
+            throws IOException {
+        final DataInputDeserializer in = new DataInputDeserializer(serialized);
+        final int splitVersion = in.readInt();
+        final int stateFlag = in.readInt();
+        if (stateFlag == SNAPSHOT_PENDING_SPLITS_STATE_FLAG) {
+            return deserializeLegacySnapshotPendingSplitsState(splitVersion, in);
+        } else if (stateFlag == HYBRID_PENDING_SPLITS_STATE_FLAG) {
+            return deserializeLegacyHybridPendingSplitsState(splitVersion, in);
+        } else if (stateFlag == BINLOG_PENDING_SPLITS_STATE_FLAG) {
+            return deserializeBinlogPendingSplitsState(in);
+        } else {
+            throw new IOException(
+                    "Unsupported to deserialize PendingSplitsState flag: " + stateFlag);
+        }
+    }
+
+    public PendingSplitsState deserializePendingSplitsState(byte[] serialized) throws IOException {
+        final DataInputDeserializer in = new DataInputDeserializer(serialized);
+        final int splitVersion = in.readInt();
+        final int stateFlag = in.readInt();
+        if (stateFlag == SNAPSHOT_PENDING_SPLITS_STATE_FLAG) {
+            return deserializeSnapshotPendingSplitsState(splitVersion, in);
+        } else if (stateFlag == HYBRID_PENDING_SPLITS_STATE_FLAG) {
+            return deserializeHybridPendingSplitsState(splitVersion, in);
+        } else if (stateFlag == BINLOG_PENDING_SPLITS_STATE_FLAG) {
+            return deserializeBinlogPendingSplitsState(in);
+        } else {
+            throw new IOException(
+                    "Unsupported to deserialize PendingSplitsState flag: " + stateFlag);
+        }
+    }
+
+    // ------------------------------------------------------------------------------------------
+    // Serialize
+    // ------------------------------------------------------------------------------------------
+
+    private void serializeSnapshotPendingSplitsState(
+            SnapshotPendingSplitsState state, DataOutputSerializer out) throws IOException {
+        writeTableIds(state.getAlreadyProcessedTables(), out);
+        writeMySqlSplits(state.getRemainingSplits(), out);
+        writeAssignedSnapshotSplits(state.getAssignedSplits(), out);
+        writeFinishedOffsets(state.getSplitFinishedOffsets(), out);
+        out.writeInt(state.getSnapshotAssignerStatus().getStatusCode());
+        writeTableIds(state.getRemainingTables(), out);
+        out.writeBoolean(state.isTableIdCaseSensitive());
+    }
+
+    private void serializeHybridPendingSplitsState(
+            HybridPendingSplitsState state, DataOutputSerializer out) throws IOException {
+        serializeSnapshotPendingSplitsState(state.getSnapshotPendingSplits(), out);
+        out.writeBoolean(state.isBinlogSplitAssigned());
+    }
+
+    private void serializeBinlogPendingSplitsState(
+            BinlogPendingSplitsState state, DataOutputSerializer out) throws IOException {
+        out.writeBoolean(state.isBinlogSplitAssigned());
+    }
+
+    // ------------------------------------------------------------------------------------------
+    // Deserialize
+    // ------------------------------------------------------------------------------------------
+
+    private SnapshotPendingSplitsState deserializeLegacySnapshotPendingSplitsState(
+            int splitVersion, DataInputDeserializer in) throws IOException {
+        List<TableId> alreadyProcessedTables = readTableIds(in);
+        List<MySqlSnapshotSplit> remainingSplits = readMySqlSnapshotSplits(splitVersion, in);
+        Map<String, MySqlSnapshotSplit> assignedSnapshotSplits =
+                readAssignedSnapshotSplits(splitVersion, in);
+        Map<String, BinlogOffset> finishedOffsets = readFinishedOffsets(splitVersion, in);
+        AssignerStatus assignerStatus;
+        boolean isAssignerFinished = in.readBoolean();
+        if (isAssignerFinished) {
+            assignerStatus = AssignerStatus.INITIAL_ASSIGNING_FINISHED;
+        } else {
+            assignerStatus = AssignerStatus.INITIAL_ASSIGNING;
+        }
+
+        return new SnapshotPendingSplitsState(
+                alreadyProcessedTables,
+                remainingSplits,
+                assignedSnapshotSplits,
+                finishedOffsets,
+                assignerStatus,
+                new ArrayList<>(),
+                false,
+                false);
+    }
+
+    private HybridPendingSplitsState deserializeLegacyHybridPendingSplitsState(
+            int splitVersion, DataInputDeserializer in) throws IOException {
+        SnapshotPendingSplitsState snapshotPendingSplitsState =
+                deserializeLegacySnapshotPendingSplitsState(splitVersion, in);
+        boolean isBinlogSplitAssigned = in.readBoolean();
+        return new HybridPendingSplitsState(snapshotPendingSplitsState, isBinlogSplitAssigned);
+    }
+
+    private SnapshotPendingSplitsState deserializeSnapshotPendingSplitsState(
+            int splitVersion, DataInputDeserializer in) throws IOException {
+        List<TableId> alreadyProcessedTables = readTableIds(in);
+        List<MySqlSnapshotSplit> remainingSplits = readMySqlSnapshotSplits(splitVersion, in);
+        Map<String, MySqlSnapshotSplit> assignedSnapshotSplits =
+                readAssignedSnapshotSplits(splitVersion, in);
+        Map<String, BinlogOffset> finishedOffsets = readFinishedOffsets(splitVersion, in);
+        AssignerStatus assignerStatus;
+        if (splitVersion < 4) {
+            boolean isAssignerFinished = in.readBoolean();
+            if (isAssignerFinished) {
+                assignerStatus = AssignerStatus.INITIAL_ASSIGNING_FINISHED;
+            } else {
+                assignerStatus = AssignerStatus.INITIAL_ASSIGNING;
+            }
+        } else {
+            assignerStatus = AssignerStatus.fromStatusCode(in.readInt());
+        }
+        List<TableId> remainingTableIds = readTableIds(in);
+        boolean isTableIdCaseSensitive = in.readBoolean();
+        return new SnapshotPendingSplitsState(
+                alreadyProcessedTables,
+                remainingSplits,
+                assignedSnapshotSplits,
+                finishedOffsets,
+                assignerStatus,
+                remainingTableIds,
+                isTableIdCaseSensitive,
+                true);
+    }
+
+    private HybridPendingSplitsState deserializeHybridPendingSplitsState(
+            int splitVersion, DataInputDeserializer in) throws IOException {
+        SnapshotPendingSplitsState snapshotPendingSplitsState =
+                deserializeSnapshotPendingSplitsState(splitVersion, in);
+        boolean isBinlogSplitAssigned = in.readBoolean();
+        return new HybridPendingSplitsState(snapshotPendingSplitsState, isBinlogSplitAssigned);
+    }
+
+    private BinlogPendingSplitsState deserializeBinlogPendingSplitsState(DataInputDeserializer in)
+            throws IOException {
+        return new BinlogPendingSplitsState(in.readBoolean());
+    }
+
+    // ------------------------------------------------------------------------------------------
+    // Utilities
+    // ------------------------------------------------------------------------------------------
+
+    private void writeFinishedOffsets(
+            Map<String, BinlogOffset> splitsInfo, DataOutputSerializer out) throws IOException {
+        final int size = splitsInfo.size();
+        out.writeInt(size);
+        for (Map.Entry<String, BinlogOffset> splitInfo : splitsInfo.entrySet()) {
+            out.writeUTF(splitInfo.getKey());
+            writeBinlogPosition(splitInfo.getValue(), out);
+        }
+    }
+
+    private Map<String, BinlogOffset> readFinishedOffsets(
+            int offsetVersion, DataInputDeserializer in) throws IOException {
+        Map<String, BinlogOffset> splitsInfo = new HashMap<>();
+        final int size = in.readInt();
+        for (int i = 0; i < size; i++) {
+            String splitId = in.readUTF();
+            BinlogOffset binlogOffset = readBinlogPosition(offsetVersion, in);
+            splitsInfo.put(splitId, binlogOffset);
+        }
+        return splitsInfo;
+    }
+
+    private void writeAssignedSnapshotSplits(
+            Map<String, MySqlSnapshotSplit> assignedSplits, DataOutputSerializer out)
+            throws IOException {
+        final int size = assignedSplits.size();
+        out.writeInt(size);
+        for (Map.Entry<String, MySqlSnapshotSplit> entry : assignedSplits.entrySet()) {
+            out.writeUTF(entry.getKey());
+            byte[] splitBytes = splitSerializer.serialize(entry.getValue());
+            out.writeInt(splitBytes.length);
+            out.write(splitBytes);
+        }
+    }
+
+    private Map<String, MySqlSnapshotSplit> readAssignedSnapshotSplits(
+            int splitVersion, DataInputDeserializer in) throws IOException {
+        Map<String, MySqlSnapshotSplit> assignedSplits = new HashMap<>();
+        final int size = in.readInt();
+        for (int i = 0; i < size; i++) {
+            String splitId = in.readUTF();
+            MySqlSnapshotSplit mySqlSplit = readMySqlSplit(splitVersion, in).asSnapshotSplit();
+            assignedSplits.put(splitId, mySqlSplit);
+        }
+        return assignedSplits;
+    }
+
+    private <T extends MySqlSplit> void writeMySqlSplits(
+            Collection<T> mySqlSplits, DataOutputSerializer out) throws IOException {
+        final int size = mySqlSplits.size();
+        out.writeInt(size);
+        for (MySqlSplit split : mySqlSplits) {
+            byte[] splitBytes = splitSerializer.serialize(split);
+            out.writeInt(splitBytes.length);
+            out.write(splitBytes);
+        }
+    }
+
+    private List<MySqlSnapshotSplit> readMySqlSnapshotSplits(
+            int splitVersion, DataInputDeserializer in) throws IOException {
+        List<MySqlSnapshotSplit> mySqlSplits = new ArrayList<>();
+        final int size = in.readInt();
+        for (int i = 0; i < size; i++) {
+            MySqlSnapshotSplit mySqlSplit = readMySqlSplit(splitVersion, in).asSnapshotSplit();
+            mySqlSplits.add(mySqlSplit);
+        }
+        return mySqlSplits;
+    }
+
+    private MySqlSplit readMySqlSplit(int splitVersion, DataInputDeserializer in)
+            throws IOException {
+        int splitBytesLen = in.readInt();
+        byte[] splitBytes = new byte[splitBytesLen];
+        in.read(splitBytes);
+        return splitSerializer.deserialize(splitVersion, splitBytes);
+    }
+
+    private void writeTableIds(Collection<TableId> tableIds, DataOutputSerializer out)
+            throws IOException {
+        final int size = tableIds.size();
+        out.writeInt(size);
+        for (TableId tableId : tableIds) {
+            out.writeUTF(tableId.toString());
+        }
+    }
+
+    private List<TableId> readTableIds(DataInputDeserializer in) throws IOException {
+        List<TableId> tableIds = new ArrayList<>();
+        final int size = in.readInt();
+        for (int i = 0; i < size; i++) {
+            String tableIdStr = in.readUTF();
+            tableIds.add(TableId.parse(tableIdStr));
+        }
+        return tableIds;
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/SnapshotPendingSplitsState.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/SnapshotPendingSplitsState.java
new file mode 100644
index 000000000..40ff96dd8
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/assigners/state/SnapshotPendingSplitsState.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state;
+
+import io.debezium.relational.TableId;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.AssignerStatus;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.enumerator.MySqlSourceEnumerator;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.reader.MySqlSplitReader;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSnapshotSplit;
+
+/** A {@link PendingSplitsState} for pending snapshot splits. */
+public class SnapshotPendingSplitsState extends PendingSplitsState {
+
+    /** The tables in the checkpoint. */
+    private final List<TableId> remainingTables;
+
+    /**
+     * The paths that are no longer in the enumerator checkpoint, but have been processed before and
+     * should this be ignored. Relevant only for sources in continuous monitoring mode.
+     */
+    private final List<TableId> alreadyProcessedTables;
+
+    /** The splits in the checkpoint. */
+    private final List<MySqlSnapshotSplit> remainingSplits;
+
+    /**
+     * The snapshot splits that the {@link MySqlSourceEnumerator} has assigned to {@link
+     * MySqlSplitReader}s.
+     */
+    private final Map<String, MySqlSnapshotSplit> assignedSplits;
+
+    /**
+     * The offsets of finished (snapshot) splits that the {@link MySqlSourceEnumerator} has received
+     * from {@link MySqlSplitReader}s.
+     */
+    private final Map<String, BinlogOffset> splitFinishedOffsets;
+
+    /** The {@link AssignerStatus} that indicates the snapshot assigner status. */
+    private final AssignerStatus assignerStatus;
+
+    /** Whether the table identifier is case-sensitive. */
+    private final boolean isTableIdCaseSensitive;
+
+    /** Whether the remaining tables are keep when snapshot state. */
+    private final boolean isRemainingTablesCheckpointed;
+
+    public SnapshotPendingSplitsState(
+            List<TableId> alreadyProcessedTables,
+            List<MySqlSnapshotSplit> remainingSplits,
+            Map<String, MySqlSnapshotSplit> assignedSplits,
+            Map<String, BinlogOffset> splitFinishedOffsets,
+            AssignerStatus assignerStatus,
+            List<TableId> remainingTables,
+            boolean isTableIdCaseSensitive,
+            boolean isRemainingTablesCheckpointed) {
+        this.alreadyProcessedTables = alreadyProcessedTables;
+        this.remainingSplits = remainingSplits;
+        this.assignedSplits = assignedSplits;
+        this.splitFinishedOffsets = splitFinishedOffsets;
+        this.assignerStatus = assignerStatus;
+        this.remainingTables = remainingTables;
+        this.isTableIdCaseSensitive = isTableIdCaseSensitive;
+        this.isRemainingTablesCheckpointed = isRemainingTablesCheckpointed;
+    }
+
+    public List<TableId> getAlreadyProcessedTables() {
+        return alreadyProcessedTables;
+    }
+
+    public List<MySqlSnapshotSplit> getRemainingSplits() {
+        return remainingSplits;
+    }
+
+    public Map<String, MySqlSnapshotSplit> getAssignedSplits() {
+        return assignedSplits;
+    }
+
+    public Map<String, BinlogOffset> getSplitFinishedOffsets() {
+        return splitFinishedOffsets;
+    }
+
+    public AssignerStatus getSnapshotAssignerStatus() {
+        return assignerStatus;
+    }
+
+    public List<TableId> getRemainingTables() {
+        return remainingTables;
+    }
+
+    public boolean isTableIdCaseSensitive() {
+        return isTableIdCaseSensitive;
+    }
+
+    public boolean isRemainingTablesCheckpointed() {
+        return isRemainingTablesCheckpointed;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (!(o instanceof SnapshotPendingSplitsState)) {
+            return false;
+        }
+        SnapshotPendingSplitsState that = (SnapshotPendingSplitsState) o;
+        return assignerStatus == that.assignerStatus
+                && isTableIdCaseSensitive == that.isTableIdCaseSensitive
+                && isRemainingTablesCheckpointed == that.isRemainingTablesCheckpointed
+                && Objects.equals(remainingTables, that.remainingTables)
+                && Objects.equals(alreadyProcessedTables, that.alreadyProcessedTables)
+                && Objects.equals(remainingSplits, that.remainingSplits)
+                && Objects.equals(assignedSplits, that.assignedSplits)
+                && Objects.equals(splitFinishedOffsets, that.splitFinishedOffsets);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                remainingTables,
+                alreadyProcessedTables,
+                remainingSplits,
+                assignedSplits,
+                splitFinishedOffsets,
+                assignerStatus,
+                isTableIdCaseSensitive,
+                isRemainingTablesCheckpointed);
+    }
+
+    @Override
+    public String toString() {
+        return "SnapshotPendingSplitsState{"
+                + "remainingTables="
+                + remainingTables
+                + ", alreadyProcessedTables="
+                + alreadyProcessedTables
+                + ", remainingSplits="
+                + remainingSplits
+                + ", assignedSplits="
+                + assignedSplits
+                + ", splitFinishedOffsets="
+                + splitFinishedOffsets
+                + ", assignerStatus="
+                + assignerStatus
+                + ", isTableIdCaseSensitive="
+                + isTableIdCaseSensitive
+                + ", isRemainingTablesCheckpointed="
+                + isRemainingTablesCheckpointed
+                + '}';
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/config/MySqlSourceConfig.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/config/MySqlSourceConfig.java
similarity index 96%
rename from inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/config/MySqlSourceConfig.java
rename to inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/config/MySqlSourceConfig.java
index 835778fed..b8f89e9a9 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/config/MySqlSourceConfig.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/config/MySqlSourceConfig.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.sort.singletenant.flink.cdc.mysql.config;
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -28,8 +28,10 @@ import java.time.Duration;
 import java.util.List;
 import java.util.Properties;
 import javax.annotation.Nullable;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.MySqlSource;
 import org.apache.inlong.sort.singletenant.flink.cdc.mysql.table.StartupOptions;
 
+/** A MySql Source configuration which is used by {@link MySqlSource}. */
 public class MySqlSourceConfig implements Serializable {
     private static final long serialVersionUID = 1L;
 
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/config/MySqlSourceConfigFactory.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/config/MySqlSourceConfigFactory.java
new file mode 100644
index 000000000..b01e60427
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/config/MySqlSourceConfigFactory.java
@@ -0,0 +1,334 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceOptions.CHUNK_META_GROUP_SIZE;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceOptions.CONNECTION_POOL_SIZE;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceOptions.CONNECT_MAX_RETRIES;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceOptions.CONNECT_TIMEOUT;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceOptions.HEARTBEAT_INTERVAL;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceOptions.SCAN_SNAPSHOT_FETCH_SIZE;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceOptions.SERVER_TIME_ZONE;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+import org.apache.flink.annotation.Internal;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.debezium.EmbeddedFlinkDatabaseHistory;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.MySqlSource;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.table.StartupOptions;
+
+/** A factory to construct {@link MySqlSourceConfig}. */
+@Internal
+public class MySqlSourceConfigFactory implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private int port = 3306; // default 3306 port
+    private String hostname;
+    private String username;
+    private String password;
+    private ServerIdRange serverIdRange;
+    private List<String> databaseList;
+    private List<String> tableList;
+    private String serverTimeZone = SERVER_TIME_ZONE.defaultValue();
+    private StartupOptions startupOptions = StartupOptions.initial();
+    private int splitSize = SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE.defaultValue();
+    private int splitMetaGroupSize = CHUNK_META_GROUP_SIZE.defaultValue();
+    private int fetchSize = SCAN_SNAPSHOT_FETCH_SIZE.defaultValue();
+    private Duration connectTimeout = CONNECT_TIMEOUT.defaultValue();
+    private int connectMaxRetries = CONNECT_MAX_RETRIES.defaultValue();
+    private int connectionPoolSize = CONNECTION_POOL_SIZE.defaultValue();
+    private double distributionFactorUpper =
+            SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND.defaultValue();
+    private double distributionFactorLower =
+            SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND.defaultValue();
+    private boolean includeSchemaChanges = false;
+    private boolean scanNewlyAddedTableEnabled = false;
+    private Properties jdbcProperties;
+    private Duration heartbeatInterval = HEARTBEAT_INTERVAL.defaultValue();
+    private Properties dbzProperties;
+
+    public MySqlSourceConfigFactory hostname(String hostname) {
+        this.hostname = hostname;
+        return this;
+    }
+
+    /** Integer port number of the MySQL database server. */
+    public MySqlSourceConfigFactory port(int port) {
+        this.port = port;
+        return this;
+    }
+
+    /**
+     * An optional list of regular expressions that match database names to be monitored; any
+     * database name not included in the whitelist will be excluded from monitoring. By default all
+     * databases will be monitored.
+     */
+    public MySqlSourceConfigFactory databaseList(String... databaseList) {
+        this.databaseList = Arrays.asList(databaseList);
+        return this;
+    }
+
+    /**
+     * An optional list of regular expressions that match fully-qualified table identifiers for
+     * tables to be monitored; any table not included in the list will be excluded from monitoring.
+     * Each identifier is of the form databaseName.tableName. By default the connector will monitor
+     * every non-system table in each monitored database.
+     */
+    public MySqlSourceConfigFactory tableList(String... tableList) {
+        this.tableList = Arrays.asList(tableList);
+        return this;
+    }
+
+    /** Name of the MySQL database to use when connecting to the MySQL database server. */
+    public MySqlSourceConfigFactory username(String username) {
+        this.username = username;
+        return this;
+    }
+
+    /** Password to use when connecting to the MySQL database server. */
+    public MySqlSourceConfigFactory password(String password) {
+        this.password = password;
+        return this;
+    }
+
+    /**
+     * A numeric ID or a numeric ID range of this database client, The numeric ID syntax is like
+     * '5400', the numeric ID range syntax is like '5400-5408', The numeric ID range syntax is
+     * required when 'scan.incremental.snapshot.enabled' enabled. Every ID must be unique across all
+     * currently-running database processes in the MySQL cluster. This connector joins the MySQL
+     * cluster as another server (with this unique ID) so it can read the binlog. By default, a
+     * random number is generated between 5400 and 6400, though we recommend setting an explicit
+     * value."
+     */
+    public MySqlSourceConfigFactory serverId(String serverId) {
+        this.serverIdRange = ServerIdRange.from(serverId);
+        return this;
+    }
+
+    /**
+     * The session time zone in database server, e.g. "America/Los_Angeles". It controls how the
+     * TIMESTAMP type in MYSQL converted to STRING. See more
+     * https://debezium.io/documentation/reference/1.5/connectors/mysql.html#mysql-temporal-types
+     */
+    public MySqlSourceConfigFactory serverTimeZone(String timeZone) {
+        this.serverTimeZone = timeZone;
+        return this;
+    }
+
+    /**
+     * The split size (number of rows) of table snapshot, captured tables are split into multiple
+     * splits when read the snapshot of table.
+     */
+    public MySqlSourceConfigFactory splitSize(int splitSize) {
+        this.splitSize = splitSize;
+        return this;
+    }
+
+    /**
+     * The group size of split meta, if the meta size exceeds the group size, the meta will be will
+     * be divided into multiple groups.
+     */
+    public MySqlSourceConfigFactory splitMetaGroupSize(int splitMetaGroupSize) {
+        this.splitMetaGroupSize = splitMetaGroupSize;
+        return this;
+    }
+
+    /**
+     * The upper bound of split key evenly distribution factor, the factor is used to determine
+     * whether the table is evenly distribution or not.
+     */
+    public MySqlSourceConfigFactory distributionFactorUpper(double distributionFactorUpper) {
+        this.distributionFactorUpper = distributionFactorUpper;
+        return this;
+    }
+
+    /**
+     * The lower bound of split key evenly distribution factor, the factor is used to determine
+     * whether the table is evenly distribution or not.
+     */
+    public MySqlSourceConfigFactory distributionFactorLower(double distributionFactorLower) {
+        this.distributionFactorLower = distributionFactorLower;
+        return this;
+    }
+
+    /** The maximum fetch size for per poll when read table snapshot. */
+    public MySqlSourceConfigFactory fetchSize(int fetchSize) {
+        this.fetchSize = fetchSize;
+        return this;
+    }
+
+    /**
+     * The maximum time that the connector should wait after trying to connect to the MySQL database
+     * server before timing out.
+     */
+    public MySqlSourceConfigFactory connectTimeout(Duration connectTimeout) {
+        this.connectTimeout = connectTimeout;
+        return this;
+    }
+
+    /** The connection pool size. */
+    public MySqlSourceConfigFactory connectionPoolSize(int connectionPoolSize) {
+        this.connectionPoolSize = connectionPoolSize;
+        return this;
+    }
+
+    /** The max retry times to get connection. */
+    public MySqlSourceConfigFactory connectMaxRetries(int connectMaxRetries) {
+        this.connectMaxRetries = connectMaxRetries;
+        return this;
+    }
+
+    /** Whether the {@link MySqlSource} should output the schema changes or not. */
+    public MySqlSourceConfigFactory includeSchemaChanges(boolean includeSchemaChanges) {
+        this.includeSchemaChanges = includeSchemaChanges;
+        return this;
+    }
+
+    /** Whether the {@link MySqlSource} should scan the newly added tables or not. */
+    public MySqlSourceConfigFactory scanNewlyAddedTableEnabled(boolean scanNewlyAddedTableEnabled) {
+        this.scanNewlyAddedTableEnabled = scanNewlyAddedTableEnabled;
+        return this;
+    }
+
+    /** Custom properties that will overwrite the default JDBC connection URL. */
+    public MySqlSourceConfigFactory jdbcProperties(Properties jdbcProperties) {
+        this.jdbcProperties = jdbcProperties;
+        return this;
+    }
+
+    /** Specifies the startup options. */
+    public MySqlSourceConfigFactory startupOptions(StartupOptions startupOptions) {
+        switch (startupOptions.startupMode) {
+            case INITIAL:
+            case LATEST_OFFSET:
+                break;
+            default:
+                throw new UnsupportedOperationException(
+                        "Unsupported startup mode: " + startupOptions.startupMode);
+        }
+        this.startupOptions = startupOptions;
+        return this;
+    }
+
+    public MySqlSourceConfigFactory heartbeatInterval(Duration heartbeatInterval) {
+        this.heartbeatInterval = heartbeatInterval;
+        return this;
+    }
+
+    /** The Debezium MySQL connector properties. For example, "snapshot.mode". */
+    public MySqlSourceConfigFactory debeziumProperties(Properties properties) {
+        this.dbzProperties = properties;
+        return this;
+    }
+
+    /** Creates a new {@link MySqlSourceConfig} for the given subtask {@code subtaskId}. */
+    public MySqlSourceConfig createConfig(int subtaskId) {
+        Properties props = new Properties();
+        // hard code server name, because we don't need to distinguish it, docs:
+        // Logical name that identifies and provides a namespace for the particular
+        // MySQL database server/cluster being monitored. The logical name should be
+        // unique across all other connectors, since it is used as a prefix for all
+        // Kafka topic names emanating from this connector.
+        // Only alphanumeric characters and underscores should be used.
+        props.setProperty("database.server.name", "mysql_binlog_source");
+        props.setProperty("database.hostname", checkNotNull(hostname));
+        props.setProperty("database.user", checkNotNull(username));
+        props.setProperty("database.password", checkNotNull(password));
+        props.setProperty("database.port", String.valueOf(port));
+        props.setProperty("database.fetchSize", String.valueOf(fetchSize));
+        props.setProperty("database.responseBuffering", "adaptive");
+        props.setProperty("database.serverTimezone", serverTimeZone);
+        // database history
+        props.setProperty(
+                "database.history", EmbeddedFlinkDatabaseHistory.class.getCanonicalName());
+        props.setProperty(
+                "database.history.instance.name", UUID.randomUUID().toString() + "_" + subtaskId);
+        props.setProperty("database.history.skip.unparseable.ddl", String.valueOf(true));
+        props.setProperty("database.history.refer.ddl", String.valueOf(true));
+        props.setProperty("connect.timeout.ms", String.valueOf(connectTimeout.toMillis()));
+        // the underlying debezium reader should always capture the schema changes and forward them.
+        // Note: the includeSchemaChanges parameter is used to control emitting the schema record,
+        // only DataStream API program need to emit the schema record, the Table API need not
+        props.setProperty("include.schema.changes", String.valueOf(true));
+        // disable the offset flush totally
+        props.setProperty("offset.flush.interval.ms", String.valueOf(Long.MAX_VALUE));
+        // disable tombstones
+        props.setProperty("tombstones.on.delete", String.valueOf(false));
+        props.setProperty("heartbeat.interval.ms", String.valueOf(heartbeatInterval.toMillis()));
+        // debezium use "long" mode to handle unsigned bigint by default,
+        // but it'll cause lose of precise when the value is larger than 2^63,
+        // so use "precise" mode to avoid it.
+        props.put("bigint.unsigned.handling.mode", "precise");
+
+        if (serverIdRange != null) {
+            int serverId = serverIdRange.getServerId(subtaskId);
+            props.setProperty("database.server.id", String.valueOf(serverId));
+        }
+        if (databaseList != null) {
+            props.setProperty("database.include.list", String.join(",", databaseList));
+        }
+        if (tableList != null) {
+            props.setProperty("table.include.list", String.join(",", tableList));
+        }
+        if (serverTimeZone != null) {
+            props.setProperty("database.serverTimezone", serverTimeZone);
+        }
+
+        // override the user-defined debezium properties
+        if (dbzProperties != null) {
+            props.putAll(dbzProperties);
+        }
+
+        if (jdbcProperties == null) {
+            jdbcProperties = new Properties();
+        }
+
+        return new MySqlSourceConfig(
+                hostname,
+                port,
+                username,
+                password,
+                databaseList,
+                tableList,
+                serverIdRange,
+                startupOptions,
+                splitSize,
+                splitMetaGroupSize,
+                fetchSize,
+                serverTimeZone,
+                connectTimeout,
+                connectMaxRetries,
+                connectionPoolSize,
+                distributionFactorUpper,
+                distributionFactorLower,
+                includeSchemaChanges,
+                scanNewlyAddedTableEnabled,
+                props,
+                jdbcProperties);
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/config/MySqlSourceOptions.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/config/MySqlSourceOptions.java
similarity index 98%
rename from inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/config/MySqlSourceOptions.java
rename to inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/config/MySqlSourceOptions.java
index 9885cf74f..8cc49ff93 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/config/MySqlSourceOptions.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/config/MySqlSourceOptions.java
@@ -16,13 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.sort.singletenant.flink.cdc.mysql.config;
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config;
 
 import java.time.Duration;
 import org.apache.flink.annotation.Experimental;
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.ConfigOptions;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.MySqlSource;
 
+/**
+ * Configurations for {@link MySqlSource}.
+ */
 public class MySqlSourceOptions {
 
     public static final ConfigOption<String> HOSTNAME =
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/config/ServerIdRange.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/config/ServerIdRange.java
similarity index 98%
rename from inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/config/ServerIdRange.java
rename to inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/config/ServerIdRange.java
index aff76f2aa..7caf48c92 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/config/ServerIdRange.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/config/ServerIdRange.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.sort.singletenant.flink.cdc.mysql.config;
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
 
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/ConnectionPoolId.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/ConnectionPoolId.java
similarity index 95%
rename from inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/ConnectionPoolId.java
rename to inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/ConnectionPoolId.java
index 8fcef7de9..947f56c64 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/ConnectionPoolId.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/ConnectionPoolId.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.sort.singletenant.flink.cdc.mysql.connection;
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.connection;
 
 import java.io.Serializable;
 import java.util.Objects;
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/ConnectionPools.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/ConnectionPools.java
similarity index 88%
rename from inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/ConnectionPools.java
rename to inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/ConnectionPools.java
index ab3a6b36d..b57e85327 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/ConnectionPools.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/ConnectionPools.java
@@ -16,11 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.sort.singletenant.flink.cdc.mysql.connection;
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.connection;
 
 import com.zaxxer.hikari.HikariDataSource;
 import org.apache.flink.annotation.Internal;
-import org.apache.inlong.sort.singletenant.flink.cdc.mysql.config.MySqlSourceConfig;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfig;
 
 /** A JDBC connection pools that consists of {@link HikariDataSource}. */
 @Internal
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/JdbcConnectionFactory.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/JdbcConnectionFactory.java
similarity index 94%
rename from inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/JdbcConnectionFactory.java
rename to inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/JdbcConnectionFactory.java
index 6454cb50e..238218eb5 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/JdbcConnectionFactory.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/JdbcConnectionFactory.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.sort.singletenant.flink.cdc.mysql.connection;
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.connection;
 
 import com.zaxxer.hikari.HikariDataSource;
 import io.debezium.jdbc.JdbcConfiguration;
@@ -24,7 +24,7 @@ import io.debezium.jdbc.JdbcConnection;
 import java.sql.Connection;
 import java.sql.SQLException;
 import org.apache.flink.util.FlinkRuntimeException;
-import org.apache.inlong.sort.singletenant.flink.cdc.mysql.config.MySqlSourceConfig;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/JdbcConnectionPools.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/JdbcConnectionPools.java
similarity index 89%
rename from inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/JdbcConnectionPools.java
rename to inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/JdbcConnectionPools.java
index 72801b509..47a531540 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/JdbcConnectionPools.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/JdbcConnectionPools.java
@@ -16,14 +16,14 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.sort.singletenant.flink.cdc.mysql.connection;
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.connection;
 
-import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.connection.PooledDataSourceFactory.createPooledDataSource;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.connection.PooledDataSourceFactory.createPooledDataSource;
 
 import com.zaxxer.hikari.HikariDataSource;
 import java.util.HashMap;
 import java.util.Map;
-import org.apache.inlong.sort.singletenant.flink.cdc.mysql.config.MySqlSourceConfig;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/PooledDataSourceFactory.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/PooledDataSourceFactory.java
similarity index 95%
rename from inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/PooledDataSourceFactory.java
rename to inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/PooledDataSourceFactory.java
index eafe67dbf..b780666cd 100644
--- a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/connection/PooledDataSourceFactory.java
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/connection/PooledDataSourceFactory.java
@@ -16,13 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.inlong.sort.singletenant.flink.cdc.mysql.connection;
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.connection;
 
 import com.zaxxer.hikari.HikariConfig;
 import com.zaxxer.hikari.HikariDataSource;
 import io.debezium.connector.mysql.MySqlConnectorConfig;
 import java.util.Properties;
-import org.apache.inlong.sort.singletenant.flink.cdc.mysql.config.MySqlSourceConfig;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfig;
 
 /** A connection pool factory to create pooled DataSource {@link HikariDataSource}. */
 public class PooledDataSourceFactory {
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/enumerator/MySqlSourceEnumerator.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/enumerator/MySqlSourceEnumerator.java
new file mode 100644
index 000000000..3925a95d2
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/enumerator/MySqlSourceEnumerator.java
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.enumerator;
+
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.AssignerStatus.isAssigning;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.AssignerStatus.isAssigningFinished;
+import static org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.AssignerStatus.isSuspended;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.MySqlHybridSplitAssigner;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.MySqlSplitAssigner;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.assigners.state.PendingSplitsState;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.config.MySqlSourceConfig;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.events.BinlogSplitMetaEvent;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.events.BinlogSplitMetaRequestEvent;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.events.FinishedSnapshotSplitsAckEvent;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.events.FinishedSnapshotSplitsReportEvent;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.events.FinishedSnapshotSplitsRequestEvent;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.events.LatestFinishedSplitsSizeEvent;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.events.LatestFinishedSplitsSizeRequestEvent;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.events.SuspendBinlogReaderAckEvent;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.events.SuspendBinlogReaderEvent;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.events.WakeupReaderEvent;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.offset.BinlogOffset;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.FinishedSnapshotSplitInfo;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.MySqlSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A MySQL CDC source enumerator that enumerates receive the split request and assign the split to
+ * source readers.
+ */
+@Internal
+public class MySqlSourceEnumerator implements SplitEnumerator<MySqlSplit, PendingSplitsState> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceEnumerator.class);
+    private static final long CHECK_EVENT_INTERVAL = 30_000L;
+
+    private final SplitEnumeratorContext<MySqlSplit> context;
+    private final MySqlSourceConfig sourceConfig;
+    private final MySqlSplitAssigner splitAssigner;
+
+    // using TreeSet to prefer assigning binlog split to task-0 for easier debug
+    private final TreeSet<Integer> readersAwaitingSplit;
+    private List<List<FinishedSnapshotSplitInfo>> binlogSplitMeta;
+    private boolean binlogReaderIsSuspended = false;
+
+    public MySqlSourceEnumerator(
+            SplitEnumeratorContext<MySqlSplit> context,
+            MySqlSourceConfig sourceConfig,
+            MySqlSplitAssigner splitAssigner) {
+        this.context = context;
+        this.sourceConfig = sourceConfig;
+        this.splitAssigner = splitAssigner;
+        this.readersAwaitingSplit = new TreeSet<>();
+
+        // when restored from state, if the split assigner is assigning snapshot
+        // splits or has already assigned all splits, send wakeup event to
+        // SourceReader, SourceReader can omit the event based on its own status.
+        if (isAssigning(splitAssigner.getAssignerStatus())
+                || isAssigningFinished(splitAssigner.getAssignerStatus())) {
+            binlogReaderIsSuspended = true;
+        }
+    }
+
+    @Override
+    public void start() {
+        splitAssigner.open();
+        suspendBinlogReaderIfNeed();
+        wakeupBinlogReaderIfNeed();
+        this.context.callAsync(
+                this::getRegisteredReader,
+                this::syncWithReaders,
+                CHECK_EVENT_INTERVAL,
+                CHECK_EVENT_INTERVAL);
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        if (!context.registeredReaders().containsKey(subtaskId)) {
+            // reader failed between sending the request and now. skip this request.
+            return;
+        }
+
+        readersAwaitingSplit.add(subtaskId);
+        assignSplits();
+    }
+
+    @Override
+    public void addSplitsBack(List<MySqlSplit> splits, int subtaskId) {
+        LOG.debug("MySQL Source Enumerator adds splits back: {}", splits);
+        splitAssigner.addSplits(splits);
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        // send SuspendBinlogReaderEvent to source reader if the assigner's status is
+        // suspended
+        if (isSuspended(splitAssigner.getAssignerStatus())) {
+            context.sendEventToSourceReader(subtaskId, new SuspendBinlogReaderEvent());
+        }
+    }
+
+    @Override
+    public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) {
+        if (sourceEvent instanceof FinishedSnapshotSplitsReportEvent) {
+            LOG.info(
+                    "The enumerator receives finished split offsets {} from subtask {}.",
+                    sourceEvent,
+                    subtaskId);
+            FinishedSnapshotSplitsReportEvent reportEvent =
+                    (FinishedSnapshotSplitsReportEvent) sourceEvent;
+            Map<String, BinlogOffset> finishedOffsets = reportEvent.getFinishedOffsets();
+
+            splitAssigner.onFinishedSplits(finishedOffsets);
+
+            wakeupBinlogReaderIfNeed();
+
+            // send acknowledge event
+            FinishedSnapshotSplitsAckEvent ackEvent =
+                    new FinishedSnapshotSplitsAckEvent(new ArrayList<>(finishedOffsets.keySet()));
+            context.sendEventToSourceReader(subtaskId, ackEvent);
+        } else if (sourceEvent instanceof BinlogSplitMetaRequestEvent) {
+            LOG.debug(
+                    "The enumerator receives request for binlog split meta from subtask {}.",
+                    subtaskId);
+            sendBinlogMeta(subtaskId, (BinlogSplitMetaRequestEvent) sourceEvent);
+        } else if (sourceEvent instanceof SuspendBinlogReaderAckEvent) {
+            LOG.info(
+                    "The enumerator receives event that the binlog split reader has been suspended from subtask {}. ",
+                    subtaskId);
+            handleSuspendBinlogReaderAckEvent(subtaskId);
+        } else if (sourceEvent instanceof LatestFinishedSplitsSizeRequestEvent) {
+            handleLatestFinishedSplitSizeRequest(subtaskId);
+        }
+    }
+
+    @Override
+    public PendingSplitsState snapshotState(long checkpointId) {
+        return splitAssigner.snapshotState(checkpointId);
+    }
+
+    @Override
+    public void notifyCheckpointComplete(long checkpointId) {
+        splitAssigner.notifyCheckpointComplete(checkpointId);
+        // binlog split may be available after checkpoint complete
+        assignSplits();
+    }
+
+    @Override
+    public void close() {
+        LOG.info("Closing enumerator...");
+        splitAssigner.close();
+    }
+
+    // ------------------------------------------------------------------------------------------
+
+    private void assignSplits() {
+        final Iterator<Integer> awaitingReader = readersAwaitingSplit.iterator();
+
+        while (awaitingReader.hasNext()) {
+            int nextAwaiting = awaitingReader.next();
+            // if the reader that requested another split has failed in the meantime, remove
+            // it from the list of waiting readers
+            if (!context.registeredReaders().containsKey(nextAwaiting)) {
+                awaitingReader.remove();
+                continue;
+            }
+
+            Optional<MySqlSplit> split = splitAssigner.getNext();
+            if (split.isPresent()) {
+                final MySqlSplit mySqlSplit = split.get();
+                context.assignSplit(mySqlSplit, nextAwaiting);
+                awaitingReader.remove();
+                LOG.info("Assign split {} to subtask {}", mySqlSplit, nextAwaiting);
+            } else {
+                // there is no available splits by now, skip assigning
+                wakeupBinlogReaderIfNeed();
+                break;
+            }
+        }
+    }
+
+    private int[] getRegisteredReader() {
+        return this.context.registeredReaders().keySet().stream()
+                .mapToInt(Integer::intValue)
+                .toArray();
+    }
+
+    private void syncWithReaders(int[] subtaskIds, Throwable t) {
+        if (t != null) {
+            throw new FlinkRuntimeException("Failed to list obtain registered readers due to:", t);
+        }
+        // when the SourceEnumerator restores or the communication failed between
+        // SourceEnumerator and SourceReader, it may missed some notification event.
+        // tell all SourceReader(s) to report there finished but unacked splits.
+        if (splitAssigner.waitingForFinishedSplits()) {
+            for (int subtaskId : subtaskIds) {
+                context.sendEventToSourceReader(
+                        subtaskId, new FinishedSnapshotSplitsRequestEvent());
+            }
+        }
+
+        suspendBinlogReaderIfNeed();
+        wakeupBinlogReaderIfNeed();
+    }
+
+    private void suspendBinlogReaderIfNeed() {
+        if (isSuspended(splitAssigner.getAssignerStatus())) {
+            for (int subtaskId : getRegisteredReader()) {
+                context.sendEventToSourceReader(subtaskId, new SuspendBinlogReaderEvent());
+            }
+            binlogReaderIsSuspended = true;
+        }
+    }
+
+    private void wakeupBinlogReaderIfNeed() {
+        if (isAssigningFinished(splitAssigner.getAssignerStatus()) && binlogReaderIsSuspended) {
+            for (int subtaskId : getRegisteredReader()) {
+                context.sendEventToSourceReader(
+                        subtaskId,
+                        new WakeupReaderEvent(WakeupReaderEvent.WakeUpTarget.BINLOG_READER));
+            }
+            binlogReaderIsSuspended = false;
+        }
+    }
+
+    private void sendBinlogMeta(int subTask, BinlogSplitMetaRequestEvent requestEvent) {
+        // initialize once
+        if (binlogSplitMeta == null) {
+            final List<FinishedSnapshotSplitInfo> finishedSnapshotSplitInfos =
+                    splitAssigner.getFinishedSplitInfos();
+            if (finishedSnapshotSplitInfos.isEmpty()) {
+                LOG.error(
+                        "The assigner offer empty finished split information, this should not happen");
+                throw new FlinkRuntimeException(
+                        "The assigner offer empty finished split information, this should not happen");
+            }
+            binlogSplitMeta =
+                    Lists.partition(
+                            finishedSnapshotSplitInfos, sourceConfig.getSplitMetaGroupSize());
+        }
+        final int requestMetaGroupId = requestEvent.getRequestMetaGroupId();
+
+        if (binlogSplitMeta.size() > requestMetaGroupId) {
+            List<FinishedSnapshotSplitInfo> metaToSend = binlogSplitMeta.get(requestMetaGroupId);
+            BinlogSplitMetaEvent metadataEvent =
+                    new BinlogSplitMetaEvent(
+                            requestEvent.getSplitId(),
+                            requestMetaGroupId,
+                            metaToSend.stream()
+                                    .map(FinishedSnapshotSplitInfo::serialize)
+                                    .collect(Collectors.toList()));
+            context.sendEventToSourceReader(subTask, metadataEvent);
+        } else {
+            LOG.error(
+                    "Received invalid request meta group id {}, the invalid meta group id range is [0, {}]",
+                    requestMetaGroupId,
+                    binlogSplitMeta.size() - 1);
+        }
+    }
+
+    private void handleSuspendBinlogReaderAckEvent(int subTask) {
+        LOG.info(
+                "Received event that the binlog split reader has been suspended from subtask {}. ",
+                subTask);
+        splitAssigner.wakeup();
+        if (splitAssigner instanceof MySqlHybridSplitAssigner) {
+            for (int subtaskId : this.getRegisteredReader()) {
+                context.sendEventToSourceReader(
+                        subtaskId,
+                        new WakeupReaderEvent(WakeupReaderEvent.WakeUpTarget.SNAPSHOT_READER));
+            }
+        }
+    }
+
+    private void handleLatestFinishedSplitSizeRequest(int subTask) {
+        if (splitAssigner instanceof MySqlHybridSplitAssigner) {
+            context.sendEventToSourceReader(
+                    subTask,
+                    new LatestFinishedSplitsSizeEvent(
+                            splitAssigner.getFinishedSplitInfos().size()));
+        }
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/events/BinlogSplitMetaEvent.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/events/BinlogSplitMetaEvent.java
new file mode 100644
index 000000000..c492263fd
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/events/BinlogSplitMetaEvent.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.events;
+
+import java.util.List;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.enumerator.MySqlSourceEnumerator;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.reader.MySqlSourceReader;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.split.FinishedSnapshotSplitInfo;
+
+/**
+ * The {@link SourceEvent} that {@link MySqlSourceEnumerator} sends to {@link MySqlSourceReader} to
+ * pass binlog meta data, i.e. {@link FinishedSnapshotSplitInfo}.
+ */
+public class BinlogSplitMetaEvent implements SourceEvent {
+
+    private static final long serialVersionUID = 1L;
+
+    private final String splitId;
+
+    /** The meta data of binlog split is divided to multiple groups. */
+    private final int metaGroupId;
+    /**
+     * The serialized meta data of binlog split, it's serialized/deserialize by {@link
+     * FinishedSnapshotSplitInfo#serialize(FinishedSnapshotSplitInfo)} and {@link
+     * FinishedSnapshotSplitInfo#deserialize(byte[])}.
+     */
+    private final List<byte[]> metaGroup;
+
+    public BinlogSplitMetaEvent(String splitId, int metaGroupId, List<byte[]> metaGroup) {
+        this.splitId = splitId;
+        this.metaGroupId = metaGroupId;
+        this.metaGroup = metaGroup;
+    }
+
+    public String getSplitId() {
+        return splitId;
+    }
+
+    public int getMetaGroupId() {
+        return metaGroupId;
+    }
+
+    public List<byte[]> getMetaGroup() {
+        return metaGroup;
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/events/BinlogSplitMetaRequestEvent.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/events/BinlogSplitMetaRequestEvent.java
new file mode 100644
index 000000000..428cbc883
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/events/BinlogSplitMetaRequestEvent.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.events;
+
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.enumerator.MySqlSourceEnumerator;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.reader.MySqlSourceReader;
+
+/**
+ * The {@link SourceEvent} that {@link MySqlSourceReader} sends to {@link MySqlSourceEnumerator} to
+ * pull binlog meta data, i.e. sending {@link BinlogSplitMetaEvent}.
+ */
+public class BinlogSplitMetaRequestEvent implements SourceEvent {
+
+    private static final long serialVersionUID = 1L;
+
+    private final String splitId;
+    private final int requestMetaGroupId;
+
+    public BinlogSplitMetaRequestEvent(String splitId, int requestMetaGroupId) {
+        this.splitId = splitId;
+        this.requestMetaGroupId = requestMetaGroupId;
+    }
+
+    public String getSplitId() {
+        return splitId;
+    }
+
+    public int getRequestMetaGroupId() {
+        return requestMetaGroupId;
+    }
+}
diff --git a/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/events/FinishedSnapshotSplitsAckEvent.java b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/events/FinishedSnapshotSplitsAckEvent.java
new file mode 100644
index 000000000..fcc3a51b6
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/main/java/org/apache/inlong/sort/singletenant/flink/cdc/mysql/source/events/FinishedSnapshotSplitsAckEvent.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.events;
+
+import java.util.List;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.inlong.sort.singletenant.flink.cdc.mysql.source.enumerator.MySqlSourceEnumerator;
... 6983 lines suppressed ...