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