You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by go...@apache.org on 2021/07/01 08:35:13 UTC

[incubator-inlong] branch INLONG-613 updated: [INLONG-633] Add inlong-sort subdirectory (#490)

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

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


The following commit(s) were added to refs/heads/INLONG-613 by this push:
     new 9274325  [INLONG-633] Add inlong-sort subdirectory (#490)
9274325 is described below

commit 92743255840068a877f1423ad75ed58dc63914d5
Author: chantccc <52...@users.noreply.github.com>
AuthorDate: Thu Jul 1 16:35:08 2021 +0800

    [INLONG-633] Add inlong-sort subdirectory (#490)
    
    Co-authored-by: tianqiwan <ti...@tencent.com>
---
 inlong-sort/README.md                              |  35 +
 inlong-sort/api/pom.xml                            |  53 ++
 .../main/java/org/apache/inlong/sort/ZkTools.java  | 143 ++++
 inlong-sort/common/pom.xml                         |  75 ++
 .../inlong/sort/configuration/ConfigOption.java    | 270 ++++++
 .../inlong/sort/configuration/ConfigOptions.java   | 120 +++
 .../inlong/sort/configuration/Configuration.java   | 915 +++++++++++++++++++++
 .../inlong/sort/configuration/Constants.java       | 224 +++++
 .../inlong/sort/configuration/FallbackKey.java     |  82 ++
 .../apache/inlong/sort/protocol/DataFlowInfo.java  |  65 ++
 .../inlong/sort/protocol/DataFlowStorageInfo.java  |  71 ++
 .../org/apache/inlong/sort/protocol/FieldInfo.java |  47 ++
 .../deserialization/CsvDeserializationInfo.java    |  42 +
 .../deserialization/DeserializationInfo.java       |  38 +
 .../deserialization/KvDeserializationInfo.java     |  51 ++
 .../TDMsgCsv2DeserializationInfo.java              |  44 +
 .../TDMsgCsvDeserializationInfo.java               |  62 ++
 .../deserialization/TDMsgDeserializationInfo.java  |  55 ++
 .../TDMsgKvDeserializationInfo.java                |  52 ++
 .../TDMsgTlogCsvDeserializationInfo.java           |  44 +
 .../TDMsgTlogKvDeserializationInfo.java            |  61 ++
 .../sort/protocol/sink/ClickHouseSinkInfo.java     | 160 ++++
 .../inlong/sort/protocol/sink/HiveSinkInfo.java    | 252 ++++++
 .../apache/inlong/sort/protocol/sink/SinkInfo.java |  56 ++
 .../sort/protocol/source/PulsarSourceInfo.java     |  74 ++
 .../inlong/sort/protocol/source/SourceInfo.java    |  66 ++
 .../sort/protocol/source/TubeSourceInfo.java       |  75 ++
 .../apache/inlong/sort/util/InstantiationUtil.java | 268 ++++++
 .../org/apache/inlong/sort/util/TestLogger.java    |  86 ++
 .../apache/inlong/sort/util/ZooKeeperUtils.java    | 266 ++++++
 .../sort/configuration/ConfigOptionTest.java       | 118 +++
 .../sort/configuration/ConfigurationTest.java      | 462 +++++++++++
 .../sort/protocol/DeserializationInfoTest.java     |  36 +
 .../apache/inlong/sort/protocol/FieldInfoTest.java |  35 +
 inlong-sort/core/pom.xml                           | 211 +++++
 .../org/apache/inlong/sort/flink/Entrance.java     |  95 +++
 .../java/org/apache/inlong/sort/flink/Record.java  |  78 ++
 .../apache/inlong/sort/flink/SerializedRecord.java |  57 ++
 .../inlong/sort/flink/TDMsgSerializedRecord.java   |  50 ++
 .../clickhouse/ClickHouseConnectionProvider.java   | 172 ++++
 .../clickhouse/ClickHouseMultiSinkFunction.java    | 165 ++++
 .../flink/clickhouse/ClickHouseRowConverter.java   |  90 ++
 .../flink/clickhouse/ClickHouseSinkFunction.java   |  86 ++
 .../clickhouse/ClickHouseStatementFactory.java     |  73 ++
 .../executor/ClickHouseAppendExecutor.java         | 145 ++++
 .../clickhouse/executor/ClickHouseExecutor.java    |  38 +
 .../executor/ClickHouseExecutorFactory.java        |  43 +
 .../executor/ClickHouseUpsertExecutor.java         | 182 ++++
 .../output/AbstractClickHouseOutputFormat.java     |  36 +
 .../output/ClickHouseBatchOutputFormat.java        | 123 +++
 .../output/ClickHouseOutputFormatFactory.java      |  86 ++
 .../output/ClickHouseShardOutputFormat.java        | 174 ++++
 .../clickhouse/partitioner/BalancePartitioner.java |  34 +
 .../partitioner/ClickHousePartitioner.java         |  28 +
 .../clickhouse/partitioner/HashPartitioner.java    |  40 +
 .../clickhouse/partitioner/RandomPartitioner.java  |  33 +
 .../flink/deserialization/CallbackCollector.java   |  50 ++
 .../deserialization/DeserializationSchema.java     | 132 +++
 .../sort/flink/deserialization/Deserializer.java   |  24 +
 .../MultiTenancyTDMsgMixedDeserializer.java        | 151 ++++
 .../deserialization/TDMsgMixedDeserializer.java    | 113 +++
 .../inlong/sort/flink/filesystem/Bucket.java       | 399 +++++++++
 .../sort/flink/filesystem/BucketAssigner.java      |  77 ++
 .../sort/flink/filesystem/BucketFactory.java       |  47 ++
 .../inlong/sort/flink/filesystem/BucketState.java  | 118 +++
 .../flink/filesystem/BucketStateSerializer.java    | 168 ++++
 .../inlong/sort/flink/filesystem/Buckets.java      | 364 ++++++++
 .../sort/flink/filesystem/BulkPartWriter.java      | 103 +++
 .../flink/filesystem/DefaultBucketFactoryImpl.java |  70 ++
 .../flink/filesystem/DefaultRollingPolicy.java     | 140 ++++
 .../inlong/sort/flink/filesystem/PartFileInfo.java |  47 ++
 .../sort/flink/filesystem/PartFileWriter.java      | 130 +++
 .../sort/flink/filesystem/RollingPolicy.java       |  54 ++
 .../sort/flink/filesystem/StreamingFileSink.java   | 220 +++++
 .../inlong/sort/flink/hive/HiveCommitter.java      | 114 +++
 .../sort/flink/hive/HiveMultiTenantCommitter.java  | 161 ++++
 .../sort/flink/hive/HiveMultiTenantWriter.java     | 201 +++++
 .../flink/hive/HivePartitionBucketAssigner.java    |  49 ++
 .../inlong/sort/flink/hive/HiveSinkHelper.java     |  74 ++
 .../apache/inlong/sort/flink/hive/HiveWriter.java  | 207 +++++
 .../sort/flink/hive/formats/ParquetRowWriter.java  | 181 ++++
 .../hive/formats/ParquetRowWriterBuilder.java      | 113 +++
 .../flink/hive/formats/ParquetSchemaConverter.java | 615 ++++++++++++++
 .../sort/flink/hive/formats/TextRowWriter.java     |  77 ++
 .../sort/flink/hive/partition/HivePartition.java   |  97 +++
 .../partition/JdbcHivePartitionCommitPolicy.java   | 121 +++
 .../hive/partition/JdbcHivePartitionTool.java      |  86 ++
 .../flink/hive/partition/PartitionCommitInfo.java  |  63 ++
 .../hive/partition/PartitionCommitPolicy.java      |  62 ++
 .../flink/hive/partition/PartitionComputer.java    |  49 ++
 .../flink/hive/partition/PartitionPathUtils.java   |  74 ++
 .../flink/hive/partition/RowPartitionComputer.java | 133 +++
 .../MultiTenantFunctionInitializationContext.java  | 133 +++
 .../transformation/FieldMappingTransformer.java    | 107 +++
 .../flink/transformation/RecordTransformer.java    | 147 ++++
 .../flink/tubemq/MultiTenancyTubeConsumer.java     | 425 ++++++++++
 .../flink/tubemq/MultiTopicTubeSourceFunction.java | 343 ++++++++
 .../flink/tubemq/TubeSubscriptionDescription.java  | 203 +++++
 .../org/apache/inlong/sort/meta/MetaManager.java   | 280 +++++++
 .../org/apache/inlong/sort/util/CommonUtils.java   |  86 ++
 .../apache/inlong/sort/util/CountBasedSampler.java |  51 ++
 .../org/apache/inlong/sort/util/ParameterTool.java | 581 +++++++++++++
 .../java/org/apache/inlong/sort/util/Sampler.java  |  33 +
 .../org/apache/inlong/sort/util/TestingUtils.java  |  86 ++
 .../apache/inlong/sort/util/ZookeeperWatcher.java  | 340 ++++++++
 .../MultiTenancyTDMsgMixedDeserializerTest.java    | 102 +++
 .../TDMsgMixedDeserializerTest.java                | 154 ++++
 .../inlong/sort/flink/hive/HiveSinkITCase.java     | 463 +++++++++++
 .../FieldMappingTransformerTest.java               |  64 ++
 .../transformation/RecordTransformerTest.java      | 191 +++++
 .../flink/tubemq/MultiTenancyTubeConsumerTest.java | 296 +++++++
 .../tubemq/MultiTopicTubeSourceFunctionTest.java   | 186 +++++
 .../tubemq/TubeSubscriptionDescriptionTest.java    | 104 +++
 .../apache/inlong/sort/meta/MetaManagerTest.java   | 181 ++++
 .../inlong/sort/util/ZooKeeperTestEnvironment.java | 158 ++++
 .../core/src/test/resources/log4j-test.properties  |  26 +
 inlong-sort/formats/format-base/pom.xml            |  67 ++
 .../base/DefaultTableFormatDeserializer.java       | 119 +++
 .../formats/base/DefaultTableFormatSerializer.java | 111 +++
 .../ProjectedDeserializationSchemaFactory.java     |  45 +
 .../base/ProjectedSerializationSchemaFactory.java  |  45 +
 .../sort/formats/base/TableFormatConstants.java    |  41 +
 .../sort/formats/base/TableFormatDeserializer.java |  30 +
 .../base/TableFormatDeserializerFactory.java       |  41 +
 .../sort/formats/base/TableFormatSerializer.java   |  29 +
 .../formats/base/TableFormatSerializerFactory.java |  41 +
 .../inlong/sort/formats/base/TableFormatUtils.java | 591 +++++++++++++
 .../TableFormatUtilsTest.java                      | 233 ++++++
 inlong-sort/formats/format-common/pom.xml          |  43 +
 .../sort/formats/common/ArrayFormatInfo.java       |  83 ++
 .../inlong/sort/formats/common/ArrayTypeInfo.java  |  77 ++
 .../sort/formats/common/BasicFormatInfo.java       |  43 +
 .../sort/formats/common/BooleanFormatInfo.java     |  63 ++
 .../sort/formats/common/BooleanTypeInfo.java       |  48 ++
 .../inlong/sort/formats/common/ByteFormatInfo.java |  63 ++
 .../inlong/sort/formats/common/ByteTypeInfo.java   |  48 ++
 .../inlong/sort/formats/common/DateFormatInfo.java | 157 ++++
 .../inlong/sort/formats/common/DateTypeInfo.java   |  48 ++
 .../sort/formats/common/DecimalFormatInfo.java     |  65 ++
 .../sort/formats/common/DecimalTypeInfo.java       |  48 ++
 .../sort/formats/common/DoubleFormatInfo.java      |  63 ++
 .../inlong/sort/formats/common/DoubleTypeInfo.java |  48 ++
 .../sort/formats/common/FloatFormatInfo.java       |  63 ++
 .../inlong/sort/formats/common/FloatTypeInfo.java  |  48 ++
 .../inlong/sort/formats/common/FormatInfo.java     |  56 ++
 .../inlong/sort/formats/common/FormatUtils.java    |  74 ++
 .../inlong/sort/formats/common/IntFormatInfo.java  |  63 ++
 .../inlong/sort/formats/common/IntTypeInfo.java    |  48 ++
 .../inlong/sort/formats/common/LongFormatInfo.java |  63 ++
 .../inlong/sort/formats/common/LongTypeInfo.java   |  48 ++
 .../inlong/sort/formats/common/MapFormatInfo.java  | 101 +++
 .../inlong/sort/formats/common/MapTypeInfo.java    |  93 +++
 .../inlong/sort/formats/common/RowFormatInfo.java  | 124 +++
 .../inlong/sort/formats/common/RowTypeInfo.java    | 114 +++
 .../sort/formats/common/ShortFormatInfo.java       |  63 ++
 .../inlong/sort/formats/common/ShortTypeInfo.java  |  48 ++
 .../sort/formats/common/StringFormatInfo.java      |  63 ++
 .../inlong/sort/formats/common/StringTypeInfo.java |  48 ++
 .../inlong/sort/formats/common/TimeFormatInfo.java | 157 ++++
 .../inlong/sort/formats/common/TimeTypeInfo.java   |  48 ++
 .../sort/formats/common/TimestampFormatInfo.java   | 158 ++++
 .../sort/formats/common/TimestampTypeInfo.java     |  48 ++
 .../inlong/sort/formats/common/TypeInfo.java       |  48 ++
 .../inlong/sort/formats/util/StringUtils.java      | 414 ++++++++++
 .../sort/formats/common/ArrayFormatInfoTest.java   |  49 ++
 .../sort/formats/common/BooleanFormatInfoTest.java |  46 ++
 .../sort/formats/common/ByteFormatInfoTest.java    |  46 ++
 .../sort/formats/common/DateFormatInfoTest.java    |  72 ++
 .../sort/formats/common/DecimalFormatInfoTest.java |  57 ++
 .../sort/formats/common/DoubleFormatInfoTest.java  |  52 ++
 .../sort/formats/common/FloatFormatInfoTest.java   |  52 ++
 .../sort/formats/common/FormatInfoTestBase.java    |  53 ++
 .../sort/formats/common/IntFormatInfoTest.java     |  52 ++
 .../sort/formats/common/LongFormatInfoTest.java    |  52 ++
 .../sort/formats/common/MapFormatInfoTest.java     |  54 ++
 .../sort/formats/common/RowFormatInfoTest.java     |  92 +++
 .../sort/formats/common/ShortFormatInfoTest.java   |  52 ++
 .../sort/formats/common/StringFormatInfoTest.java  |  46 ++
 .../sort/formats/common/TimeFormatInfoTest.java    |  72 ++
 .../formats/common/TimestampFormatInfoTest.java    |  72 ++
 inlong-sort/formats/format-csv/pom.xml             | 116 +++
 .../org/apache/inlong/sort/formats/csv/Csv.java    | 143 ++++
 .../sort/formats/csv/CsvDeserializationSchema.java | 240 ++++++
 .../inlong/sort/formats/csv/CsvFormatFactory.java  | 250 ++++++
 .../sort/formats/csv/CsvSerializationSchema.java   | 221 +++++
 .../inlong/sort/formats/csv/CsvValidator.java      |  41 +
 .../org.apache.flink.table.factories.TableFactory  |  33 +
 .../formats/csv/CsvDeserializationSchemaTest.java  | 240 ++++++
 .../sort/formats/csv/CsvFormatFactoryTest.java     | 314 +++++++
 .../formats/csv/CsvSerializationSchemaTest.java    | 212 +++++
 .../apache/inlong/sort/formats/csv/CsvTest.java    | 117 +++
 .../inlong/sort/formats/csv/CsvUtilsTest.java      | 126 +++
 .../src/test/resources/log4j-test.properties       |  22 +
 inlong-sort/formats/format-kv/pom.xml              | 116 +++
 .../java/org/apache/inlong/sort/formats/kv/Kv.java | 149 ++++
 .../sort/formats/kv/KvDeserializationSchema.java   | 257 ++++++
 .../inlong/sort/formats/kv/KvFormatFactory.java    | 257 ++++++
 .../sort/formats/kv/KvSerializationSchema.java     | 249 ++++++
 .../apache/inlong/sort/formats/kv/KvValidator.java |  42 +
 .../org.apache.flink.table.factories.TableFactory  |  33 +
 .../formats/kv/KvDeserializationSchemaTest.java    | 256 ++++++
 .../sort/formats/kv/KvFormatFactoryTest.java       | 323 ++++++++
 .../sort/formats/kv/KvSerializationSchemaTest.java | 212 +++++
 .../org/apache/inlong/sort/formats/kv/KvTest.java  | 121 +++
 .../apache/inlong/sort/formats/kv/KvUtilsTest.java | 234 ++++++
 .../src/test/resources/log4j-test.properties       |  22 +
 inlong-sort/formats/format-tdmsg-base/pom.xml      | 127 +++
 .../tdmsg/AbstractTDMsgFormatDeserializer.java     | 148 ++++
 .../AbstractTDMsgMixedFormatDeserializer.java      |  32 +
 .../inlong/sort/formats/tdmsg/TDMsgBody.java       | 106 +++
 .../inlong/sort/formats/tdmsg/TDMsgHead.java       | 114 +++
 .../formats/tdmsg/TDMsgMixedFormatConverter.java   |  31 +
 .../formats/tdmsg/TDMsgMixedFormatFactory.java     |  52 ++
 .../sort/formats/tdmsg/TDMsgMixedValidator.java    |  41 +
 .../inlong/sort/formats/tdmsg/TDMsgUtils.java      | 289 +++++++
 .../inlong/sort/formats/tdmsg/TDMsgValidator.java  |  44 +
 inlong-sort/formats/format-tdmsg-csv/pom.xml       | 130 +++
 .../inlong/sort/formats/tdmsgcsv/TDMsgCsv.java     | 175 ++++
 .../tdmsgcsv/TDMsgCsvFormatDeserializer.java       | 204 +++++
 .../formats/tdmsgcsv/TDMsgCsvFormatFactory.java    | 230 ++++++
 .../tdmsgcsv/TDMsgCsvMixedFormatConverter.java     | 140 ++++
 .../tdmsgcsv/TDMsgCsvMixedFormatDeserializer.java  | 151 ++++
 .../sort/formats/tdmsgcsv/TDMsgCsvUtils.java       | 193 +++++
 .../org.apache.flink.table.factories.TableFactory  |  33 +
 .../tdmsgcsv/TDMsgCsvFormatDeserializerTest.java   | 507 ++++++++++++
 .../tdmsgcsv/TDMsgCsvFormatFactoryTest.java        | 127 +++
 .../inlong/sort/formats/tdmsgcsv/TDMsgCsvTest.java | 128 +++
 .../src/test/resources/log4j-test.properties       |  22 +
 inlong-sort/formats/pom.xml                        | 243 ++++++
 inlong-sort/pom.xml                                | 323 ++++++++
 inlong-sort/quick_start.md                         |  60 ++
 pom.xml                                            |   1 +
 232 files changed, 29362 insertions(+)

diff --git a/inlong-sort/README.md b/inlong-sort/README.md
new file mode 100644
index 0000000..4e5a2ff
--- /dev/null
+++ b/inlong-sort/README.md
@@ -0,0 +1,35 @@
+# Description
+## overview
+Inlong-sort is used to extract data from different source systems, then transforms the data and finally loads the data into diffrent storage systems.
+Inlong-sort is simply an Flink application, and relys on Inlong-manager to manage meta data(such as the source informations and storage informations)
+
+##features
+### multi-tenancy
+Inlong-sort is an multi-tenancy system, which means you can extract data from different sources(these sources must be of the same source type) and load data into different sinks(these sinks must be of the same storage type).
+e.g. you can extract data form different topics of inlong-tubemq and the load them to different hive clusters.
+
+### change meta data without restart
+Inlong-sort uses zookeeper to manage its meta data, every time you change meta data on zk, inlong-sort application will be informed immediately.
+e.g if you want to change the schema of your data, just change the meta data on zk without restart your inlong-sort application.
+
+## supported sources
+**inlong-tubemq**
+
+## supported storages
+**hive**
+Currently we just support parquet file format in hive
+
+**clickhouse**
+
+## limitations
+Currently, we just support extracting specified fields in the stage of **Transform**.
+
+# Plans in the future
+## More kinds of source systems
+pulsar, kafka and etc
+
+## More kinds of storage systems
+Hbase, Elastic Search, and etc
+
+## More kinds of file format in hive sink
+sequence file, orc
diff --git a/inlong-sort/api/pom.xml b/inlong-sort/api/pom.xml
new file mode 100644
index 0000000..e8977a0
--- /dev/null
+++ b/inlong-sort/api/pom.xml
@@ -0,0 +1,53 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+        xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+        xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.inlong</groupId>
+        <artifactId>inlong-sort</artifactId>
+        <version>1.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>inlong-sort-api</artifactId>
+    <name>Apache InLong - Sort Api</name>
+    <packaging>jar</packaging>
+
+    <properties>
+        <!-- deploy this module to repo -->
+        <maven.deploy.skip>false</maven.deploy.skip>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>inlong-sort-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+        </dependency>
+    </dependencies>
+
+</project>
diff --git a/inlong-sort/api/src/main/java/org/apache/inlong/sort/ZkTools.java b/inlong-sort/api/src/main/java/org/apache/inlong/sort/ZkTools.java
new file mode 100644
index 0000000..462435e
--- /dev/null
+++ b/inlong-sort/api/src/main/java/org/apache/inlong/sort/ZkTools.java
@@ -0,0 +1,143 @@
+/*
+ * 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;
+
+import static org.apache.inlong.sort.protocol.DataFlowStorageInfo.StorageType.ZK;
+
+import org.apache.inlong.sort.configuration.Configuration;
+import org.apache.inlong.sort.configuration.Constants;
+import org.apache.inlong.sort.protocol.DataFlowInfo;
+import org.apache.inlong.sort.protocol.DataFlowStorageInfo;
+import org.apache.inlong.sort.util.ZooKeeperUtils;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.zookeeper.CreateMode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This tools of ZK is provided as a SDK tool for TDM.
+ *
+ * <p> TODO, reorganize the maven structure to provide this as a separate API module. </p>
+ */
+public class ZkTools {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ZkTools.class);
+
+    public static void addDataFlowToCluster(
+            String cluster,
+            long dataFlowId,
+            String zkQuorum,
+            String zkRoot) throws Exception {
+        final Configuration config = new Configuration();
+        config.setString(Constants.ZOOKEEPER_QUORUM, zkQuorum);
+        config.setString(Constants.ZOOKEEPER_ROOT, zkRoot);
+
+        final String watchingPath = getNodePathOfDataFlowStorageInfoInCluster(cluster, dataFlowId);
+
+        try (CuratorFramework zkClient = ZooKeeperUtils.startCuratorFramework(config)) {
+            if (zkClient.checkExists().forPath(watchingPath) == null) {
+                ZooKeeperUtils.createRecursive(zkClient, watchingPath, null, CreateMode.PERSISTENT);
+            }
+        }
+
+        LOG.info("Add dataFlow with id {} to cluster {} successfully", dataFlowId, cluster);
+    }
+
+    public static void removeDataFlowFromCluster(
+            String cluster,
+            long dataFlowId,
+            String zkQuorum,
+            String zkRoot) throws Exception {
+        final Configuration config = new Configuration();
+        config.setString(Constants.ZOOKEEPER_QUORUM, zkQuorum);
+        config.setString(Constants.ZOOKEEPER_ROOT, zkRoot);
+
+        final String watchingPath = getNodePathOfDataFlowStorageInfoInCluster(cluster, dataFlowId);
+
+        try (CuratorFramework zkClient = ZooKeeperUtils.startCuratorFramework(config)) {
+            if (zkClient.checkExists().forPath(watchingPath) != null) {
+                zkClient.delete().forPath(watchingPath);
+            }
+        }
+
+        LOG.info("Remove dataFlow with id {} to cluster {} successfully", dataFlowId, cluster);
+    }
+
+    /**
+     * Update DataFlowInfo.
+     * @param dataFlowInfo DataFlowInfo to be updated to.
+     * @param cluster cluster name of the etl job.
+     * @param dataFlowId dataFlow id
+     * @param zkQuorum zk quorum
+     * @param zkRoot zk root path
+     * @throws Exception
+     */
+    public static void updateDataFlowInfo(
+            DataFlowInfo dataFlowInfo,
+            String cluster,
+            long dataFlowId,
+            String zkQuorum,
+            String zkRoot) throws Exception {
+        final Configuration config = new Configuration();
+        config.setString(Constants.ZOOKEEPER_QUORUM, zkQuorum);
+        config.setString(Constants.ZOOKEEPER_ROOT, zkRoot);
+
+        final ObjectMapper objectMapper = new ObjectMapper();
+        final String dataFlowInfoPath = getNodePathOfDataFlowInfo(dataFlowId);
+        final String dataFlowStorageInfoPath = getNodePathOfDataFlowStorageInfoInCluster(cluster, dataFlowId);
+
+        try (CuratorFramework zkClient = ZooKeeperUtils.startCuratorFramework(config)) {
+            // update DataFlowInfo
+            final byte[] dataFlowInfoData = objectMapper.writeValueAsBytes(dataFlowInfo);
+            if (zkClient.checkExists().forPath(dataFlowInfoPath) == null) {
+                ZooKeeperUtils.createRecursive(zkClient, dataFlowInfoPath, dataFlowInfoData, CreateMode.PERSISTENT);
+            } else {
+                zkClient.setData().forPath(dataFlowInfoPath, dataFlowInfoData);
+            }
+
+            // update DataFlowStorageInfo
+            DataFlowStorageInfo dataFlowStorageInfo = new DataFlowStorageInfo(ZK, dataFlowInfoPath);
+            final byte[] dataFlowStorageInfoData = objectMapper.writeValueAsBytes(dataFlowStorageInfo);
+            if (zkClient.checkExists().forPath(dataFlowStorageInfoPath) == null) {
+                ZooKeeperUtils.createRecursive(
+                        zkClient, dataFlowStorageInfoPath, dataFlowStorageInfoData, CreateMode.PERSISTENT);
+            } else {
+                zkClient.setData().forPath(dataFlowStorageInfoPath, dataFlowStorageInfoData);
+            }
+        }
+
+        LOG.info("Update DataFlowInfo with id {} on zk successfully", dataFlowId);
+    }
+
+    /**
+     * Get node path which stores the meta data of DataFlowStorageInfo.
+     * If you want to change the path rule here, please change
+     * MetaManager#getWatchingPathOfDataFlowsInCluster in core too.
+     */
+    public static String getNodePathOfDataFlowStorageInfoInCluster(String cluster, long dataFlowId) {
+        return "/clusters/" + cluster + "/dataflows/" + dataFlowId;
+    }
+
+    /**
+     * Get node path which stores the meta data of DataFlowInfo.
+     */
+    public static String getNodePathOfDataFlowInfo(long dataFlowId) {
+        return "/dataflows/" + dataFlowId;
+    }
+}
diff --git a/inlong-sort/common/pom.xml b/inlong-sort/common/pom.xml
new file mode 100644
index 0000000..98a4cd2
--- /dev/null
+++ b/inlong-sort/common/pom.xml
@@ -0,0 +1,75 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+        xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+        xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.inlong</groupId>
+        <artifactId>inlong-sort</artifactId>
+        <version>1.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>inlong-sort-common</artifactId>
+    <name>Apache InLong - Sort Common</name>
+    <packaging>jar</packaging>
+
+    <properties>
+        <!-- deploy this module to repo -->
+        <maven.deploy.skip>false</maven.deploy.skip>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>inlong-sort-format-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-shaded-jackson</artifactId>
+            <version>${flink.jackson.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>jsr305</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+        </dependency>
+
+    </dependencies>
+
+</project>
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/ConfigOption.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/ConfigOption.java
new file mode 100644
index 0000000..ec35627
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/ConfigOption.java
@@ -0,0 +1,270 @@
+/*
+ * 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.configuration;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A {@code ConfigOption} describes a configuration parameter. It encapsulates the configuration
+ * key, deprecated older versions of the key, and an optional default value for the configuration
+ * parameter.
+ *
+ * <p>{@code ConfigOptions} are built via the {@link ConfigOptions} class.
+ * Once created, a config option is immutable.</p>
+ *
+ * <p>Copied from Flink project(org.apache.flink/flink-core) with a bit of changing.</p>
+ *
+ * @param <T> The type of value associated with the configuration option.
+ */
+public class ConfigOption<T> {
+
+    private static final FallbackKey[] EMPTY = new FallbackKey[0];
+
+    // ------------------------------------------------------------------------
+
+    /**
+     * The current key for that config option.
+     */
+    private final String key;
+
+    /**
+     * The list of deprecated keys, in the order to be checked.
+     */
+    private final FallbackKey[] fallbackKeys;
+
+    /**
+     * The default value for this option.
+     */
+    private final T defaultValue;
+
+    /**
+     * The description for this option.
+     */
+    private final String description;
+
+    // ------------------------------------------------------------------------
+
+    /**
+     * Creates a new config option with no fallback keys.
+     *
+     * @param key The current key for that config option
+     * @param defaultValue The default value for this option
+     */
+    ConfigOption(String key, T defaultValue) {
+        this.key = checkNotNull(key);
+        this.description = "";
+        this.defaultValue = defaultValue;
+        this.fallbackKeys = EMPTY;
+    }
+
+    /**
+     * Creates a new config option with fallback keys.
+     *
+     * @param key The current key for that config option
+     * @param description Description for that option
+     * @param defaultValue The default value for this option
+     * @param fallbackKeys The list of fallback keys, in the order to be checked
+     */
+    ConfigOption(String key, String description, T defaultValue, FallbackKey... fallbackKeys) {
+        this.key = checkNotNull(key);
+        this.description = checkNotNull(description);
+        this.defaultValue = defaultValue;
+        this.fallbackKeys = fallbackKeys == null || fallbackKeys.length == 0 ? EMPTY : fallbackKeys;
+    }
+
+    // ------------------------------------------------------------------------
+
+    /**
+     * Creates a new config option, using this option's key and default value, and adding the given
+     * fallback keys.
+     *
+     * <p>When obtaining a value from the configuration via
+     * {@link Configuration#getValue(ConfigOption)}, the fallback keys will be checked in the order
+     * provided to this method. The first key for which a value is found will be used - that value
+     * will be returned.</p>
+     *
+     * @param fallbackKeys The fallback keys, in the order in which they should be checked.
+     * @return A new config options, with the given fallback keys.
+     */
+    public ConfigOption<T> withFallbackKeys(String... fallbackKeys) {
+        final Stream<FallbackKey> newFallbackKeys = Arrays.stream(fallbackKeys)
+                .map(FallbackKey::createFallbackKey);
+        final Stream<FallbackKey> currentAlternativeKeys = Arrays.stream(this.fallbackKeys);
+
+        // put fallback keys first so that they are prioritized
+        final FallbackKey[] mergedAlternativeKeys = Stream
+                .concat(newFallbackKeys, currentAlternativeKeys)
+                .toArray(FallbackKey[]::new);
+        return new ConfigOption<>(key, description, defaultValue, mergedAlternativeKeys);
+    }
+
+    /**
+     * Creates a new config option, using this option's key and default value, and adding the given
+     * deprecated keys.
+     *
+     * <p>When obtaining a value from the configuration via
+     * {@link Configuration#getValue(ConfigOption)}, the deprecated keys will be checked in the
+     * order provided to this method. The first key for which a value is found will be used - that
+     * value will be returned.</p>
+     *
+     * @param deprecatedKeys The deprecated keys, in the order in which they should be
+     *         checked.
+     * @return A new config options, with the given deprecated keys.
+     */
+    public ConfigOption<T> withDeprecatedKeys(String... deprecatedKeys) {
+        final Stream<FallbackKey> newDeprecatedKeys = Arrays.stream(deprecatedKeys)
+                .map(FallbackKey::createDeprecatedKey);
+        final Stream<FallbackKey> currentAlternativeKeys = Arrays.stream(this.fallbackKeys);
+
+        // put deprecated keys last so that they are de-prioritized
+        final FallbackKey[] mergedAlternativeKeys = Stream
+                .concat(currentAlternativeKeys, newDeprecatedKeys)
+                .toArray(FallbackKey[]::new);
+        return new ConfigOption<>(key, description, defaultValue, mergedAlternativeKeys);
+    }
+
+    /**
+     * Creates a new config option, using this option's key and default value, and adding the given
+     * description. The given description is used when generation the configuration documention.
+     *
+     * @param description The description for this option.
+     * @return A new config option, with given description.
+     */
+    public ConfigOption<T> withDescription(final String description) {
+        return new ConfigOption<>(key, description, defaultValue, fallbackKeys);
+    }
+
+    // ------------------------------------------------------------------------
+
+    /**
+     * Gets the configuration key.
+     *
+     * @return The configuration key
+     */
+    public String key() {
+        return key;
+    }
+
+    /**
+     * Checks if this option has a default value.
+     *
+     * @return True if it has a default value, false if not.
+     */
+    public boolean hasDefaultValue() {
+        return defaultValue != null;
+    }
+
+    /**
+     * Returns the default value, or null, if there is no default value.
+     *
+     * @return The default value, or null.
+     */
+    public T defaultValue() {
+        return defaultValue;
+    }
+
+    /**
+     * Checks whether this option has deprecated keys.
+     *
+     * @return True if the option has deprecated keys, false if not.
+     * @deprecated Replaced by {@link #hasFallbackKeys()}
+     */
+    @Deprecated
+    public boolean hasDeprecatedKeys() {
+        return fallbackKeys != EMPTY && Arrays.stream(fallbackKeys)
+                .anyMatch(FallbackKey::isDeprecated);
+    }
+
+    /**
+     * Gets the deprecated keys, in the order to be checked.
+     *
+     * @return The option's deprecated keys.
+     * @deprecated Replaced by {@link #fallbackKeys()}
+     */
+    @Deprecated
+    public Iterable<String> deprecatedKeys() {
+        return fallbackKeys == EMPTY ? Collections.<String>emptyList() :
+                Arrays.stream(fallbackKeys)
+                        .filter(FallbackKey::isDeprecated)
+                        .map(FallbackKey::getKey)
+                        .collect(Collectors.toList());
+    }
+
+    /**
+     * Checks whether this option has fallback keys.
+     *
+     * @return True if the option has fallback keys, false if not.
+     */
+    public boolean hasFallbackKeys() {
+        return fallbackKeys != EMPTY;
+    }
+
+    /**
+     * Gets the fallback keys, in the order to be checked.
+     *
+     * @return The option's fallback keys.
+     */
+    public Iterable<FallbackKey> fallbackKeys() {
+        return (fallbackKeys == EMPTY) ? Collections.emptyList() : Arrays.asList(fallbackKeys);
+    }
+
+    /**
+     * Returns the description of this option.
+     *
+     * @return The option's description.
+     */
+    public String description() {
+        return description;
+    }
+
+    // ------------------------------------------------------------------------
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        } else if (o != null && o.getClass() == ConfigOption.class) {
+            ConfigOption<?> that = (ConfigOption<?>) o;
+            return this.key.equals(that.key)
+                    && Arrays.equals(this.fallbackKeys, that.fallbackKeys)
+                    && (this.defaultValue == null ? that.defaultValue == null :
+                    (that.defaultValue != null && this.defaultValue.equals(that.defaultValue)));
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        return 31 * key.hashCode()
+                + 17 * Arrays.hashCode(fallbackKeys)
+                + (defaultValue != null ? defaultValue.hashCode() : 0);
+    }
+
+    @Override
+    public String toString() {
+        return String.format("Key: '%s' , default: %s (fallback keys: %s)",
+                key, defaultValue, Arrays.toString(fallbackKeys));
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/ConfigOptions.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/ConfigOptions.java
new file mode 100644
index 0000000..b5bfede
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/ConfigOptions.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.configuration;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * {@code ConfigOptions} are used to build a {@link ConfigOption}. The option is typically built in
+ * one of the following pattern:
+ *
+ * <pre>{@code
+ * // simple string-valued option with a default value
+ * ConfigOption<String> tempDirs = ConfigOptions
+ *     .key("tmp.dir")
+ *     .defaultValue("/tmp");
+ *
+ * // simple integer-valued option with a default value
+ * ConfigOption<Integer> parallelism = ConfigOptions
+ *     .key("application.parallelism")
+ *     .defaultValue(100);
+ *
+ * // option with no default value
+ * ConfigOption<String> userName = ConfigOptions
+ *     .key("user.name")
+ *     .noDefaultValue();
+ *
+ * // option with deprecated keys to check
+ * ConfigOption<Double> threshold = ConfigOptions
+ *     .key("cpu.utilization.threshold")
+ *     .defaultValue(0.9).
+ *     .withDeprecatedKeys("cpu.threshold");
+ * }</pre>
+ *
+ * <p>Copied from Flink project(org.apache.flink/flink-core) with a bit of changing.</p>
+ */
+public class ConfigOptions {
+
+    /**
+     * Starts building a new {@link ConfigOption}.
+     *
+     * @param key The key for the config option.
+     * @return The builder for the config option with the given key.
+     */
+    public static OptionBuilder key(String key) {
+        checkNotNull(key);
+        return new OptionBuilder(key);
+    }
+
+    // ------------------------------------------------------------------------
+
+    /**
+     * The option builder is used to create a {@link ConfigOption}. It is instantiated via {@link
+     * ConfigOptions#key(String)}.
+     */
+    public static final class OptionBuilder {
+
+        /**
+         * The key for the config option.
+         */
+        private final String key;
+
+        /**
+         * Creates a new OptionBuilder.
+         *
+         * @param key The key for the config option
+         */
+        OptionBuilder(String key) {
+            this.key = key;
+        }
+
+        /**
+         * Creates a ConfigOption with the given default value.
+         *
+         * <p>This method does not accept "null". For options with no default value, choose
+         * one of the {@code noDefaultValue} methods.</p>
+         *
+         * @param value The default value for the config option
+         * @param <T> The type of the default value.
+         * @return The config option with the default value.
+         */
+        public <T> ConfigOption<T> defaultValue(T value) {
+            checkNotNull(value);
+            return new ConfigOption<>(key, value);
+        }
+
+        /**
+         * Creates a string-valued option with no default value. String-valued options are the only
+         * ones that can have no default value.
+         *
+         * @return The created ConfigOption.
+         */
+        public ConfigOption<String> noDefaultValue() {
+            return new ConfigOption<>(key, null);
+        }
+    }
+
+    // ------------------------------------------------------------------------
+
+    /**
+     * Not intended to be instantiated.
+     */
+    private ConfigOptions() {
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/Configuration.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/Configuration.java
new file mode 100644
index 0000000..f68ed60
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/Configuration.java
@@ -0,0 +1,915 @@
+/*
+ * 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.configuration;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Lightweight configuration object which stores key/value pairs.
+ *
+ * <p>Copied from Flink project(org.apache.flink/flink-core) with a bit of changing.</p>
+ */
+public class Configuration implements java.io.Serializable, Cloneable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final byte TYPE_STRING = 0;
+    private static final byte TYPE_INT = 1;
+    private static final byte TYPE_LONG = 2;
+    private static final byte TYPE_BOOLEAN = 3;
+    private static final byte TYPE_FLOAT = 4;
+    private static final byte TYPE_DOUBLE = 5;
+    private static final byte TYPE_BYTES = 6;
+
+    /**
+     * The log object used for debugging.
+     */
+    private static final Logger LOG = LoggerFactory.getLogger(Configuration.class);
+
+    /**
+     * Stores the concrete key/value pairs of this configuration object.
+     */
+    protected final HashMap<String, Object> confData;
+
+    // --------------------------------------------------------------------------------------------
+
+    /**
+     * Creates a new empty configuration.
+     */
+    public Configuration() {
+        confData = new HashMap<>();
+    }
+
+    /**
+     * Creates a new configuration with the copy of the given configuration.
+     *
+     * @param other The configuration to copy the entries from.
+     */
+    public Configuration(Configuration other) {
+        this.confData = new HashMap<>(other.confData);
+    }
+
+    // --------------------------------------------------------------------------------------------
+
+    /**
+     * Returns the class associated with the given key as a string.
+     *
+     * @param <T> The type of the class to return.
+     * @param key The key pointing to the associated value
+     * @param defaultValue The optional default value returned if no entry exists
+     * @param classLoader The class loader used to resolve the class.
+     * @return The value associated with the given key, or the default value, if to entry for the
+     *         key exists.
+     */
+    @SuppressWarnings("unchecked")
+    public <T> Class<T> getClass(String key, Class<? extends T> defaultValue,
+            ClassLoader classLoader) throws ClassNotFoundException {
+        Object o = getRawValue(key);
+        if (o == null) {
+            return (Class<T>) defaultValue;
+        }
+
+        if (o.getClass() == String.class) {
+            return (Class<T>) Class.forName((String) o, true, classLoader);
+        }
+
+        LOG.warn("Configuration cannot evaluate value " + o + " as a class name");
+        return (Class<T>) defaultValue;
+    }
+
+    /**
+     * Adds the given key/value pair to the configuration object. The class can be retrieved by
+     * invoking {@link #getClass(String, Class, ClassLoader)} if it is in the scope of the class
+     * loader on the caller.
+     *
+     * @param key The key of the pair to be added
+     * @param klazz The value of the pair to be added
+     * @see #getClass(String, Class, ClassLoader)
+     */
+    public void setClass(String key, Class<?> klazz) {
+        setValueInternal(key, klazz.getName());
+    }
+
+    /**
+     * Returns the value associated with the given key as a string.
+     *
+     * @param key the key pointing to the associated value
+     * @param defaultValue the default value which is returned in case there is no value
+     *         associated with the given key
+     * @return the (default) value associated with the given key
+     */
+    public String getString(String key, String defaultValue) {
+        Object o = getRawValue(key);
+        if (o == null) {
+            return defaultValue;
+        } else {
+            return o.toString();
+        }
+    }
+
+    /**
+     * Returns the value associated with the given config option as a string.
+     *
+     * @param configOption The configuration option
+     * @return the (default) value associated with the given config option
+     */
+    public String getString(ConfigOption<String> configOption) {
+        Object o = getValueOrDefaultFromOption(configOption);
+        return o == null ? null : o.toString();
+    }
+
+    /**
+     * Returns the value associated with the given config option as a string. If no value is mapped
+     * under any key of the option, it returns the specified default instead of the option's default
+     * value.
+     *
+     * @param configOption The configuration option
+     * @return the (default) value associated with the given config option
+     */
+    public String getString(ConfigOption<String> configOption, String overrideDefault) {
+        Object o = getRawValueFromOption(configOption);
+        return o == null ? overrideDefault : o.toString();
+    }
+
+    /**
+     * Adds the given key/value pair to the configuration object.
+     *
+     * @param key the key of the key/value pair to be added
+     * @param value the value of the key/value pair to be added
+     */
+    public void setString(String key, String value) {
+        setValueInternal(key, value);
+    }
+
+    /**
+     * Adds the given value to the configuration object. The main key of the config option will be
+     * used to map the value.
+     *
+     * @param key the option specifying the key to be added
+     * @param value the value of the key/value pair to be added
+     */
+    public void setString(ConfigOption<String> key, String value) {
+        setValueInternal(key.key(), value);
+    }
+
+    /**
+     * Returns the value associated with the given key as an integer.
+     *
+     * @param key the key pointing to the associated value
+     * @param defaultValue the default value which is returned in case there is no value
+     *         associated with the given key
+     * @return the (default) value associated with the given key
+     */
+    public int getInteger(String key, int defaultValue) {
+        Object o = getRawValue(key);
+        if (o == null) {
+            return defaultValue;
+        }
+
+        return convertToInt(o, defaultValue);
+    }
+
+    /**
+     * Returns the value associated with the given config option as an integer.
+     *
+     * @param configOption The configuration option
+     * @return the (default) value associated with the given config option
+     */
+    public int getInteger(ConfigOption<Integer> configOption) {
+        Object o = getValueOrDefaultFromOption(configOption);
+        return convertToInt(o, configOption.defaultValue());
+    }
+
+    /**
+     * Returns the value associated with the given config option as an integer. If no value is
+     * mapped under any key of the option, it returns the specified default instead of the option's
+     * default value.
+     *
+     * @param configOption The configuration option
+     * @param overrideDefault The value to return if no value was mapper for any key of the
+     *         option
+     * @return the configured value associated with the given config option, or the overrideDefault
+     */
+    public int getInteger(ConfigOption<Integer> configOption, int overrideDefault) {
+        Object o = getRawValueFromOption(configOption);
+        if (o == null) {
+            return overrideDefault;
+        }
+        return convertToInt(o, configOption.defaultValue());
+    }
+
+    /**
+     * Adds the given key/value pair to the configuration object.
+     *
+     * @param key the key of the key/value pair to be added
+     * @param value the value of the key/value pair to be added
+     */
+    public void setInteger(String key, int value) {
+        setValueInternal(key, value);
+    }
+
+    /**
+     * Adds the given value to the configuration object. The main key of the config option will be
+     * used to map the value.
+     *
+     * @param key the option specifying the key to be added
+     * @param value the value of the key/value pair to be added
+     */
+    public void setInteger(ConfigOption<Integer> key, int value) {
+        setValueInternal(key.key(), value);
+    }
+
+    /**
+     * Returns the value associated with the given key as a long.
+     *
+     * @param key the key pointing to the associated value
+     * @param defaultValue the default value which is returned in case there is no value
+     *         associated with the given key
+     * @return the (default) value associated with the given key
+     */
+    public long getLong(String key, long defaultValue) {
+        Object o = getRawValue(key);
+        if (o == null) {
+            return defaultValue;
+        }
+
+        return convertToLong(o, defaultValue);
+    }
+
+    /**
+     * Returns the value associated with the given config option as a long integer.
+     *
+     * @param configOption The configuration option
+     * @return the (default) value associated with the given config option
+     */
+    public long getLong(ConfigOption<Long> configOption) {
+        Object o = getValueOrDefaultFromOption(configOption);
+        return convertToLong(o, configOption.defaultValue());
+    }
+
+    /**
+     * Returns the value associated with the given config option as a long integer. If no value is
+     * mapped under any key of the option, it returns the specified default instead of the option's
+     * default value.
+     *
+     * @param configOption The configuration option
+     * @param overrideDefault The value to return if no value was mapper for any key of the
+     *         option
+     * @return the configured value associated with the given config option, or the overrideDefault
+     */
+    public long getLong(ConfigOption<Long> configOption, long overrideDefault) {
+        Object o = getRawValueFromOption(configOption);
+        if (o == null) {
+            return overrideDefault;
+        }
+        return convertToLong(o, configOption.defaultValue());
+    }
+
+    /**
+     * Adds the given key/value pair to the configuration object.
+     *
+     * @param key the key of the key/value pair to be added
+     * @param value the value of the key/value pair to be added
+     */
+    public void setLong(String key, long value) {
+        setValueInternal(key, value);
+    }
+
+    /**
+     * Adds the given value to the configuration object. The main key of the config option will be
+     * used to map the value.
+     *
+     * @param key the option specifying the key to be added
+     * @param value the value of the key/value pair to be added
+     */
+    public void setLong(ConfigOption<Long> key, long value) {
+        setValueInternal(key.key(), value);
+    }
+
+    /**
+     * Returns the value associated with the given key as a boolean.
+     *
+     * @param key the key pointing to the associated value
+     * @param defaultValue the default value which is returned in case there is no value
+     *         associated with the given key
+     * @return the (default) value associated with the given key
+     */
+    public boolean getBoolean(String key, boolean defaultValue) {
+        Object o = getRawValue(key);
+        if (o == null) {
+            return defaultValue;
+        }
+
+        return convertToBoolean(o);
+    }
+
+    /**
+     * Returns the value associated with the given config option as a boolean.
+     *
+     * @param configOption The configuration option
+     * @return the (default) value associated with the given config option
+     */
+    public boolean getBoolean(ConfigOption<Boolean> configOption) {
+        Object o = getValueOrDefaultFromOption(configOption);
+        return convertToBoolean(o);
+    }
+
+    /**
+     * Returns the value associated with the given config option as a boolean. If no value is mapped
+     * under any key of the option, it returns the specified default instead of the option's default
+     * value.
+     *
+     * @param configOption The configuration option
+     * @param overrideDefault The value to return if no value was mapper for any key of the
+     *         option
+     * @return the configured value associated with the given config option, or the overrideDefault
+     */
+    public boolean getBoolean(ConfigOption<Boolean> configOption, boolean overrideDefault) {
+        Object o = getRawValueFromOption(configOption);
+        if (o == null) {
+            return overrideDefault;
+        }
+        return convertToBoolean(o);
+    }
+
+    /**
+     * Adds the given key/value pair to the configuration object.
+     *
+     * @param key the key of the key/value pair to be added
+     * @param value the value of the key/value pair to be added
+     */
+    public void setBoolean(String key, boolean value) {
+        setValueInternal(key, value);
+    }
+
+    /**
+     * Adds the given value to the configuration object. The main key of the config option will be
+     * used to map the value.
+     *
+     * @param key the option specifying the key to be added
+     * @param value the value of the key/value pair to be added
+     */
+    public void setBoolean(ConfigOption<Boolean> key, boolean value) {
+        setValueInternal(key.key(), value);
+    }
+
+    /**
+     * Returns the value associated with the given key as a float.
+     *
+     * @param key the key pointing to the associated value
+     * @param defaultValue the default value which is returned in case there is no value
+     *         associated with the given key
+     * @return the (default) value associated with the given key
+     */
+    public float getFloat(String key, float defaultValue) {
+        Object o = getRawValue(key);
+        if (o == null) {
+            return defaultValue;
+        }
+
+        return convertToFloat(o, defaultValue);
+    }
+
+    /**
+     * Returns the value associated with the given config option as a float.
+     *
+     * @param configOption The configuration option
+     * @return the (default) value associated with the given config option
+     */
+    public float getFloat(ConfigOption<Float> configOption) {
+        Object o = getValueOrDefaultFromOption(configOption);
+        return convertToFloat(o, configOption.defaultValue());
+    }
+
+    /**
+     * Returns the value associated with the given config option as a float. If no value is mapped
+     * under any key of the option, it returns the specified default instead of the option's default
+     * value.
+     *
+     * @param configOption The configuration option
+     * @param overrideDefault The value to return if no value was mapper for any key of the
+     *         option
+     * @return the configured value associated with the given config option, or the overrideDefault
+     */
+    public float getFloat(ConfigOption<Float> configOption, float overrideDefault) {
+        Object o = getRawValueFromOption(configOption);
+        if (o == null) {
+            return overrideDefault;
+        }
+        return convertToFloat(o, configOption.defaultValue());
+    }
+
+    /**
+     * Adds the given key/value pair to the configuration object.
+     *
+     * @param key the key of the key/value pair to be added
+     * @param value the value of the key/value pair to be added
+     */
+    public void setFloat(String key, float value) {
+        setValueInternal(key, value);
+    }
+
+    /**
+     * Adds the given value to the configuration object. The main key of the config option will be
+     * used to map the value.
+     *
+     * @param key the option specifying the key to be added
+     * @param value the value of the key/value pair to be added
+     */
+    public void setFloat(ConfigOption<Float> key, float value) {
+        setValueInternal(key.key(), value);
+    }
+
+    /**
+     * Returns the value associated with the given key as a double.
+     *
+     * @param key the key pointing to the associated value
+     * @param defaultValue the default value which is returned in case there is no value
+     *         associated with the given key
+     * @return the (default) value associated with the given key
+     */
+    public double getDouble(String key, double defaultValue) {
+        Object o = getRawValue(key);
+        if (o == null) {
+            return defaultValue;
+        }
+
+        return convertToDouble(o, defaultValue);
+    }
+
+    /**
+     * Returns the value associated with the given config option as a {@code double}.
+     *
+     * @param configOption The configuration option
+     * @return the (default) value associated with the given config option
+     */
+    public double getDouble(ConfigOption<Double> configOption) {
+        Object o = getValueOrDefaultFromOption(configOption);
+        return convertToDouble(o, configOption.defaultValue());
+    }
+
+    /**
+     * Returns the value associated with the given config option as a {@code double}. If no value is
+     * mapped under any key of the option, it returns the specified default instead of the option's
+     * default value.
+     *
+     * @param configOption The configuration option
+     * @param overrideDefault The value to return if no value was mapper for any key of the
+     *         option
+     * @return the configured value associated with the given config option, or the overrideDefault
+     */
+    public double getDouble(ConfigOption<Double> configOption, double overrideDefault) {
+        Object o = getRawValueFromOption(configOption);
+        if (o == null) {
+            return overrideDefault;
+        }
+        return convertToDouble(o, configOption.defaultValue());
+    }
+
+    /**
+     * Adds the given key/value pair to the configuration object.
+     *
+     * @param key the key of the key/value pair to be added
+     * @param value the value of the key/value pair to be added
+     */
+    public void setDouble(String key, double value) {
+        setValueInternal(key, value);
+    }
+
+    /**
+     * Adds the given value to the configuration object. The main key of the config option will be
+     * used to map the value.
+     *
+     * @param key the option specifying the key to be added
+     * @param value the value of the key/value pair to be added
+     */
+    public void setDouble(ConfigOption<Double> key, double value) {
+        setValueInternal(key.key(), value);
+    }
+
+    /**
+     * Returns the value associated with the given key as a byte array.
+     *
+     * @param key The key pointing to the associated value.
+     * @param defaultValue The default value which is returned in case there is no value
+     *         associated with the given key.
+     * @return the (default) value associated with the given key.
+     */
+    @SuppressWarnings("EqualsBetweenInconvertibleTypes")
+    public byte[] getBytes(String key, byte[] defaultValue) {
+
+        Object o = getRawValue(key);
+        if (o == null) {
+            return defaultValue;
+        } else if (o.getClass().equals(byte[].class)) {
+            return (byte[]) o;
+        } else {
+            LOG.warn("Configuration cannot evaluate value {} as a byte[] value", o);
+            return defaultValue;
+        }
+    }
+
+    /**
+     * Adds the given byte array to the configuration object. If key is <code>null</code> then
+     * nothing is added.
+     *
+     * @param key The key under which the bytes are added.
+     * @param bytes The bytes to be added.
+     */
+    public void setBytes(String key, byte[] bytes) {
+        setValueInternal(key, bytes);
+    }
+
+    /**
+     * Returns the value associated with the given config option as a string.
+     *
+     * @param configOption The configuration option
+     * @return the (default) value associated with the given config option
+     */
+    public String getValue(ConfigOption<?> configOption) {
+        Object o = getValueOrDefaultFromOption(configOption);
+        return o == null ? null : o.toString();
+    }
+
+    /**
+     * Returns the value associated with the given config option as an enum.
+     *
+     * @param enumClass The return enum class
+     * @param configOption The configuration option
+     * @throws IllegalArgumentException If the string associated with the given config
+     *         option cannot be parsed as a value of the provided enum class.
+     */
+    public <T extends Enum<T>> T getEnum(
+            final Class<T> enumClass,
+            final ConfigOption<String> configOption) {
+        checkNotNull(enumClass, "enumClass must not be null");
+        checkNotNull(configOption, "configOption must not be null");
+
+        final String configValue = getString(configOption);
+        try {
+            return Enum.valueOf(enumClass, configValue.toUpperCase(Locale.ROOT));
+        } catch (final IllegalArgumentException | NullPointerException e) {
+            final String errorMessage = String
+                    .format("Value for config option %s must be one of %s (was %s)",
+                            configOption.key(),
+                            Arrays.toString(enumClass.getEnumConstants()),
+                            configValue);
+            throw new IllegalArgumentException(errorMessage, e);
+        }
+    }
+
+    // --------------------------------------------------------------------------------------------
+
+    /**
+     * Returns the keys of all key/value pairs stored inside this configuration object.
+     *
+     * @return the keys of all key/value pairs stored inside this configuration object
+     */
+    public Set<String> keySet() {
+        synchronized (this.confData) {
+            return new HashSet<>(this.confData.keySet());
+        }
+    }
+
+    /**
+     * Adds all entries in this {@code Configuration} to the given {@link Properties}.
+     */
+    public void addAllToProperties(Properties props) {
+        synchronized (this.confData) {
+            for (Map.Entry<String, Object> entry : this.confData.entrySet()) {
+                props.put(entry.getKey(), entry.getValue());
+            }
+        }
+    }
+
+    public void addAll(Configuration other) {
+        synchronized (this.confData) {
+            synchronized (other.confData) {
+                this.confData.putAll(other.confData);
+            }
+        }
+    }
+
+    /**
+     * Adds all entries from the given configuration into this configuration. The keys are prepended
+     * with the given prefix.
+     *
+     * @param other The configuration whose entries are added to this configuration.
+     * @param prefix The prefix to prepend.
+     */
+    public void addAll(Configuration other, String prefix) {
+        final StringBuilder bld = new StringBuilder();
+        bld.append(prefix);
+        final int pl = bld.length();
+
+        synchronized (this.confData) {
+            synchronized (other.confData) {
+                for (Map.Entry<String, Object> entry : other.confData.entrySet()) {
+                    bld.setLength(pl);
+                    bld.append(entry.getKey());
+                    this.confData.put(bld.toString(), entry.getValue());
+                }
+            }
+        }
+    }
+
+    @Override
+    public Configuration clone() {
+        Configuration config = new Configuration();
+        config.addAll(this);
+
+        return config;
+    }
+
+    /**
+     * Checks whether there is an entry with the specified key.
+     *
+     * @param key key of entry
+     * @return true if the key is stored, false otherwise
+     */
+    public boolean containsKey(String key) {
+        synchronized (this.confData) {
+            return this.confData.containsKey(key);
+        }
+    }
+
+    /**
+     * Checks whether there is an entry for the given config option.
+     *
+     * @param configOption The configuration option
+     * @return <tt>true</tt> if a valid (current or deprecated) key of the config option is stored,
+     *         <tt>false</tt> otherwise
+     */
+    public boolean contains(ConfigOption<?> configOption) {
+        synchronized (this.confData) {
+            // first try the current key
+            if (this.confData.containsKey(configOption.key())) {
+                return true;
+            } else if (configOption.hasFallbackKeys()) {
+                // try the fallback keys
+                for (FallbackKey fallbackKey : configOption.fallbackKeys()) {
+                    if (this.confData.containsKey(fallbackKey.getKey())) {
+                        loggingFallback(fallbackKey, configOption);
+                        return true;
+                    }
+                }
+            }
+
+            return false;
+        }
+    }
+
+    // --------------------------------------------------------------------------------------------
+
+    public Map<String, String> toMap() {
+        synchronized (this.confData) {
+            Map<String, String> ret = new HashMap<>(this.confData.size());
+            for (Map.Entry<String, Object> entry : confData.entrySet()) {
+                ret.put(entry.getKey(), entry.getValue().toString());
+            }
+            return ret;
+        }
+    }
+
+    /**
+     * Removes given config option from the configuration.
+     *
+     * @param configOption config option to remove
+     * @param <T> Type of the config option
+     * @return true is config has been removed, false otherwise
+     */
+    public <T> boolean removeConfig(ConfigOption<T> configOption) {
+        synchronized (this.confData) {
+            // try the current key
+            Object oldValue = this.confData.remove(configOption.key());
+            if (oldValue == null) {
+                for (FallbackKey fallbackKey : configOption.fallbackKeys()) {
+                    oldValue = this.confData.remove(fallbackKey.getKey());
+                    if (oldValue != null) {
+                        loggingFallback(fallbackKey, configOption);
+                        return true;
+                    }
+                }
+                return false;
+            }
+            return true;
+        }
+    }
+
+    // --------------------------------------------------------------------------------------------
+
+    <T> void setValueInternal(String key, T value) {
+        if (key == null) {
+            throw new NullPointerException("Key must not be null.");
+        }
+        if (value == null) {
+            throw new NullPointerException("Value must not be null.");
+        }
+
+        synchronized (this.confData) {
+            this.confData.put(key, value);
+        }
+    }
+
+    private Object getRawValue(String key) {
+        if (key == null) {
+            throw new NullPointerException("Key must not be null.");
+        }
+
+        synchronized (this.confData) {
+            return this.confData.get(key);
+        }
+    }
+
+    private Object getRawValueFromOption(ConfigOption<?> configOption) {
+        // first try the current key
+        Object o = getRawValue(configOption.key());
+
+        if (o != null) {
+            // found a value for the current proper key
+            return o;
+        } else if (configOption.hasFallbackKeys()) {
+            // try the deprecated keys
+            for (FallbackKey fallbackKey : configOption.fallbackKeys()) {
+                Object oo = getRawValue(fallbackKey.getKey());
+                if (oo != null) {
+                    loggingFallback(fallbackKey, configOption);
+                    return oo;
+                }
+            }
+        }
+
+        return null;
+    }
+
+    private Object getValueOrDefaultFromOption(ConfigOption<?> configOption) {
+        Object o = getRawValueFromOption(configOption);
+        return o != null ? o : configOption.defaultValue();
+    }
+
+    private void loggingFallback(FallbackKey fallbackKey, ConfigOption<?> configOption) {
+        if (fallbackKey.isDeprecated()) {
+            LOG.warn("Config uses deprecated configuration key '{}' instead of proper key '{}'",
+                    fallbackKey.getKey(), configOption.key());
+        } else {
+            LOG.info("Config uses fallback configuration key '{}' instead of key '{}'",
+                    fallbackKey.getKey(), configOption.key());
+        }
+    }
+
+    // --------------------------------------------------------------------------------------------
+    //  Type conversion
+    // --------------------------------------------------------------------------------------------
+
+    private int convertToInt(Object o, int defaultValue) {
+        if (o.getClass() == Integer.class) {
+            return (Integer) o;
+        } else if (o.getClass() == Long.class) {
+            long value = (Long) o;
+            if (value <= Integer.MAX_VALUE && value >= Integer.MIN_VALUE) {
+                return (int) value;
+            } else {
+                LOG.warn("Configuration value {} overflows/underflows the integer type.", value);
+                return defaultValue;
+            }
+        } else {
+            try {
+                return Integer.parseInt(o.toString());
+            } catch (NumberFormatException e) {
+                LOG.warn("Configuration cannot evaluate value {} as an integer number", o);
+                return defaultValue;
+            }
+        }
+    }
+
+    private long convertToLong(Object o, long defaultValue) {
+        if (o.getClass() == Long.class) {
+            return (Long) o;
+        } else if (o.getClass() == Integer.class) {
+            return ((Integer) o).longValue();
+        } else {
+            try {
+                return Long.parseLong(o.toString());
+            } catch (NumberFormatException e) {
+                LOG.warn("Configuration cannot evaluate value " + o + " as a long integer number");
+                return defaultValue;
+            }
+        }
+    }
+
+    private boolean convertToBoolean(Object o) {
+        if (o.getClass() == Boolean.class) {
+            return (Boolean) o;
+        } else {
+            return Boolean.parseBoolean(o.toString());
+        }
+    }
+
+    private float convertToFloat(Object o, float defaultValue) {
+        if (o.getClass() == Float.class) {
+            return (Float) o;
+        } else if (o.getClass() == Double.class) {
+            double value = ((Double) o);
+            if (value == 0.0
+                    || (value >= Float.MIN_VALUE && value <= Float.MAX_VALUE)
+                    || (value >= -Float.MAX_VALUE && value <= -Float.MIN_VALUE)) {
+                return (float) value;
+            } else {
+                LOG.warn("Configuration value {} overflows/underflows the float type.", value);
+                return defaultValue;
+            }
+        } else {
+            try {
+                return Float.parseFloat(o.toString());
+            } catch (NumberFormatException e) {
+                LOG.warn("Configuration cannot evaluate value {} as a float value", o);
+                return defaultValue;
+            }
+        }
+    }
+
+    private double convertToDouble(Object o, double defaultValue) {
+        if (o.getClass() == Double.class) {
+            return (Double) o;
+        } else if (o.getClass() == Float.class) {
+            return ((Float) o).doubleValue();
+        } else {
+            try {
+                return Double.parseDouble(o.toString());
+            } catch (NumberFormatException e) {
+                LOG.warn("Configuration cannot evaluate value {} as a double value", o);
+                return defaultValue;
+            }
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        int hash = 0;
+        for (String s : this.confData.keySet()) {
+            hash ^= s.hashCode();
+        }
+        return hash;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) {
+            return true;
+        } else if (obj instanceof Configuration) {
+            Map<String, Object> otherConf = ((Configuration) obj).confData;
+
+            for (Map.Entry<String, Object> e : this.confData.entrySet()) {
+                Object thisVal = e.getValue();
+                Object otherVal = otherConf.get(e.getKey());
+
+                if (!thisVal.getClass().equals(byte[].class)) {
+                    if (!thisVal.equals(otherVal)) {
+                        return false;
+                    }
+                } else if (otherVal.getClass().equals(byte[].class)) {
+                    if (!Arrays.equals((byte[]) thisVal, (byte[]) otherVal)) {
+                        return false;
+                    }
+                } else {
+                    return false;
+                }
+            }
+
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public String toString() {
+        return this.confData.toString();
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/Constants.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/Constants.java
new file mode 100644
index 0000000..dfc077a
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/Constants.java
@@ -0,0 +1,224 @@
+/*
+ * 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.configuration;
+
+import static org.apache.inlong.sort.configuration.ConfigOptions.key;
+
+import java.time.Duration;
+
+public class Constants {
+    public static final long UNKNOWN_DATAFLOW_ID = -1L;
+
+    public static final String SOURCE_TYPE_TUBE = "tubemq";
+
+    public static final String SOURCE_TYPE_PULSAR = "pulsar";
+
+    public static final String SINK_TYPE_CLICKHOUSE = "clickhouse";
+
+    public static final String SINK_TYPE_HIVE = "hive";
+
+    // ------------------------------------------------------------------------
+    //  Operator uid
+    // ------------------------------------------------------------------------
+    public static final String SOURCE_UID = "source_uid";
+
+    public static final String DESERIALIZATION_SCHEMA_UID = "deserialization_schema_uid";
+
+    public static final String SINK_UID = "sink_uid";
+
+    // ------------------------------------------------------------------------
+    //  Common configs
+    // ------------------------------------------------------------------------
+    /**
+     * The ID of the cluster, used to separate multiple clusters.
+     */
+    public static final ConfigOption<String> CLUSTER_ID =
+            key("cluster-id")
+                    .noDefaultValue()
+                    .withDescription("The ID of the cluster, used to separate multiple clusters.");
+
+    /**
+     * The ZooKeeper quorum to use.
+     */
+    public static final ConfigOption<String> ZOOKEEPER_QUORUM =
+            key("zookeeper.quorum")
+                    .noDefaultValue()
+                    .withDescription("The ZooKeeper quorum to use");
+
+    /**
+     * The root path under which it stores its entries in ZooKeeper.
+     */
+    public static final ConfigOption<String> ZOOKEEPER_ROOT =
+            key("zookeeper.path.root")
+                    .defaultValue("/inlong-sort")
+                    .withDescription("The root path in ZooKeeper.");
+
+    public static final ConfigOption<Integer> ETL_RECORD_SERIALIZATION_BUFFER_SIZE =
+            key("etl.record.serialization.buffer.size")
+                    .defaultValue(1024);
+
+    public static final ConfigOption<String> SOURCE_TYPE =
+            key("source.type")
+                    .noDefaultValue()
+                    .withDescription("The type of source, currently only 'tubemq' is supported");
+
+    public static final ConfigOption<String> SINK_TYPE =
+            key("sink.type")
+                    .noDefaultValue()
+                    .withDescription("The type of sink, currently only 'clickhouse' and 'iceberg' are supported");
+
+    // ------------------------------------------------------------------------
+    //  Operator parallelism configs
+    // ------------------------------------------------------------------------
+    public static final ConfigOption<Integer> SOURCE_PARALLELISM =
+            key("source.parallelism")
+                    .defaultValue(1);
+
+    public static final ConfigOption<Integer> DESERIALIZATION_PARALLELISM =
+            key("deserialization.parallelism")
+                    .defaultValue(1);
+
+    public static final ConfigOption<Integer> SINK_PARALLELISM =
+            key("sink.parallelism")
+                    .defaultValue(1);
+
+    public static final ConfigOption<Integer> COMMITTER_PARALLELISM =
+            key("committer.parallelism")
+                    .defaultValue(1);
+
+    // ------------------------------------------------------------------------
+    //  TubeMQ source configs
+    // ------------------------------------------------------------------------
+    public static final ConfigOption<String> TUBE_MASTER_ADDRESS =
+            key("tubemq.master.address")
+                    .noDefaultValue()
+                    .withDescription("The address of tubeMQ master.");
+
+    public static final ConfigOption<String> TUBE_SESSION_KEY =
+            key("tubemq.session.key")
+                    .defaultValue("inlong-sort")
+                    .withDescription("The session key of tubeMQ consumer.");
+
+    public static final ConfigOption<Boolean> TUBE_BOOTSTRAP_FROM_MAX =
+            key("tubemq.bootstrap.from.max")
+                    .defaultValue(false)
+                    .withDescription("Consume tubeMQ from max offset.");
+
+    public static final ConfigOption<String> TUBE_MESSAGE_NOT_FOUND_WAIT_PERIOD =
+            key("tubemq.message.not.found.wait.period")
+                    .defaultValue("350ms")
+                    .withDescription("The time of waiting period if "
+                            + "tubeMQ broker return message not found.");
+
+    public static final ConfigOption<Long> TUBE_SUBSCRIBE_RETRY_TIMEOUT =
+            key("tubemq.subscribe.retry.timeout")
+                    .defaultValue(300000L)
+                    .withDescription("The time of subscribing tubeMQ timeout, in millisecond");
+
+    public static final ConfigOption<Integer> SOURCE_EVENT_QUEUE_CAPACITY =
+            key("source.event.queue.capacity")
+                    .defaultValue(1024);
+
+    // ------------------------------------------------------------------------
+    //  ZooKeeper Client Settings
+    // ------------------------------------------------------------------------
+
+    public static final ConfigOption<Integer> ZOOKEEPER_SESSION_TIMEOUT =
+            key("zookeeper.client.session-timeout")
+                    .defaultValue(60000)
+                    .withDescription(
+                            "Defines the session timeout for the ZooKeeper session in ms.");
+
+    public static final ConfigOption<Integer> ZOOKEEPER_CONNECTION_TIMEOUT =
+            key("zookeeper.client.connection-timeout")
+                    .defaultValue(15000)
+                    .withDescription("Defines the connection timeout for ZooKeeper in ms.");
+
+    public static final ConfigOption<Integer> ZOOKEEPER_RETRY_WAIT =
+            key("zookeeper.client.retry-wait")
+                    .defaultValue(5000)
+                    .withDescription("Defines the pause between consecutive retries in ms.");
+
+    public static final ConfigOption<Integer> ZOOKEEPER_MAX_RETRY_ATTEMPTS =
+            key("zookeeper.client.max-retry-attempts")
+                    .defaultValue(3)
+                    .withDescription(
+                            "Defines the number of connection retries before the client gives up.");
+
+    public static final ConfigOption<String> ZOOKEEPER_CLIENT_ACL =
+            key("zookeeper.client.acl")
+                    .defaultValue("open")
+                    .withDescription(
+                            "Defines the ACL (open|creator) to be configured on ZK node. The "
+                                    + "configuration value can be"
+                                    + " set to “creator” if the ZooKeeper server configuration "
+                                    + "has the “authProvider” property mapped to use"
+                                    + " SASLAuthenticationProvider and the cluster is configured "
+                                    + "to run in secure mode (Kerberos).");
+
+    public static final ConfigOption<Boolean> ZOOKEEPER_SASL_DISABLE =
+            key("zookeeper.sasl.disable")
+                    .defaultValue(false);
+
+    // ------------------------------------------------------------------------
+    //  Hive sink related configs
+    // ------------------------------------------------------------------------
+    public static final ConfigOption<Long> HIKARICP_IDLE_TIMEOUT_MS =
+            key("hikaricp.idle.timeout.ms")
+                    .defaultValue(10000L);
+
+    public static final ConfigOption<Long> HIKARICP_CONNECTION_TIMEOUT_MS =
+            key("hikaricp.connection.timeout.ms")
+                    .defaultValue(30000L);
+
+    public static final ConfigOption<Integer> HIKARICP_MAXIMUM_POOL_SIZE =
+            key("hikaricp.maximum.pool.size")
+                    .defaultValue(10);
+
+    public static final ConfigOption<Long> HIKARICP_MAXIMUM_LIFETIME_MS =
+            key("hikaricp.maximum.lifetime.ms")
+                    .defaultValue(30000L);
+
+    // ------------------------------------------------------------------------
+    //  Hive sink related configs (initial version, refactor later)
+    // ------------------------------------------------------------------------
+
+    public static final ConfigOption<Integer> SINK_HIVE_COMMITTED_PARTITIONS_CACHE_SIZE =
+            key("sink.hive.committed.partitions.cache.size")
+                    .defaultValue(1024);
+
+    public static final ConfigOption<Long> SINK_HIVE_ROLLING_POLICY_FILE_SIZE =
+            key("sink.hive.rolling-policy.file-size")
+                    .defaultValue(128L << 20)
+                    .withDescription("The maximum part file size before rolling.");
+
+    public static final ConfigOption<Long> SINK_HIVE_ROLLING_POLICY_ROLLOVER_INTERVAL =
+            key("sink.hive.rolling-policy.rollover-interval")
+                    .defaultValue(Duration.ofMinutes(30).toMillis())
+                    .withDescription("The maximum time duration a part file can stay open before rolling"
+                            + " (by default long enough to avoid too many small files). The frequency at which"
+                            + " this is checked is controlled by the 'sink.rolling-policy.check-interval' option.");
+
+    public static final ConfigOption<Long> SINK_HIVE_ROLLING_POLICY_CHECK_INTERVAL =
+            key("sink.hive.rolling-policy.check-interval")
+                    .defaultValue(Duration.ofMinutes(1).toMillis())
+                    .withDescription("The interval for checking time based rolling policies. "
+                            + "This controls the frequency to check whether a part file should rollover based on"
+                            + " 'sink.rolling-policy.rollover-interval'.");
+
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/FallbackKey.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/FallbackKey.java
new file mode 100644
index 0000000..523db7a
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/configuration/FallbackKey.java
@@ -0,0 +1,82 @@
+/*
+ * 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.configuration;
+
+/**
+ * A key with FallbackKeys will fall back to the FallbackKeys if it itself is not configured.
+ *
+ * <p>Copied from Flink project(org.apache.flink/flink-core) with a bit of changing.</p>
+ */
+public class FallbackKey {
+
+    // -------------------------
+    //  Factory methods
+    // -------------------------
+
+    static FallbackKey createFallbackKey(String key) {
+        return new FallbackKey(key, false);
+    }
+
+    static FallbackKey createDeprecatedKey(String key) {
+        return new FallbackKey(key, true);
+    }
+
+    // ------------------------------------------------------------------------
+
+    private final String key;
+
+    private final boolean isDeprecated;
+
+    public String getKey() {
+        return key;
+    }
+
+    public boolean isDeprecated() {
+        return isDeprecated;
+    }
+
+    private FallbackKey(String key, boolean isDeprecated) {
+        this.key = key;
+        this.isDeprecated = isDeprecated;
+    }
+
+    // ------------------------------------------------------------------------
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        } else if (o != null && o.getClass() == FallbackKey.class) {
+            FallbackKey that = (FallbackKey) o;
+            return this.key.equals(that.key) && (this.isDeprecated == that.isDeprecated);
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        return 31 * key.hashCode() + (isDeprecated ? 1 : 0);
+    }
+
+    @Override
+    public String toString() {
+        return String.format("{key=%s, isDeprecated=%s}", key, isDeprecated);
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/DataFlowInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/DataFlowInfo.java
new file mode 100644
index 0000000..22b1811
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/DataFlowInfo.java
@@ -0,0 +1,65 @@
+/*
+ * 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;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.inlong.sort.protocol.sink.SinkInfo;
+import org.apache.inlong.sort.protocol.source.SourceInfo;
+import java.io.Serializable;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Data flow protocol.
+ */
+public class DataFlowInfo implements Serializable {
+
+    private static final long serialVersionUID = 6549803844655330833L;
+
+    private final long id;
+
+    private final SourceInfo sourceInfo;
+
+    private final SinkInfo sinkInfo;
+
+    @JsonCreator
+    public DataFlowInfo(
+            @JsonProperty("id") long id,
+            @JsonProperty("source_info") SourceInfo sourceInfo,
+            @JsonProperty("sink_info") SinkInfo sinkInfo) {
+        this.id = id;
+        this.sourceInfo = checkNotNull(sourceInfo);
+        this.sinkInfo = checkNotNull(sinkInfo);
+    }
+
+    @JsonProperty("id")
+    public long getId() {
+        return id;
+    }
+
+    @JsonProperty("source_info")
+    public SourceInfo getSourceInfo() {
+        return sourceInfo;
+    }
+
+    @JsonProperty("sink_info")
+    public SinkInfo getSinkInfo() {
+        return sinkInfo;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/DataFlowStorageInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/DataFlowStorageInfo.java
new file mode 100644
index 0000000..95d99d1
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/DataFlowStorageInfo.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.protocol;
+
+import com.google.common.base.Preconditions;
+import java.util.Objects;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+
+public class DataFlowStorageInfo {
+
+    public enum StorageType {
+        ZK,
+        HDFS
+    }
+
+    @JsonProperty("storage_type")
+    private final StorageType storageType;
+
+    @JsonProperty("path")
+    private final String path;
+
+    @JsonCreator
+    public DataFlowStorageInfo(
+            @JsonProperty("storage_type") StorageType storageType,
+            @JsonProperty("path") String path) {
+        this.storageType = Preconditions.checkNotNull(storageType);
+        this.path = Preconditions.checkNotNull(path);
+    }
+
+    @JsonProperty("storage_type")
+    public StorageType getStorageType() {
+        return storageType;
+    }
+
+    @JsonProperty("path")
+    public String getPath() {
+        return path;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        DataFlowStorageInfo that = (DataFlowStorageInfo) o;
+
+        return Objects.equals(storageType, that.storageType)
+                && Objects.equals(path, that.path);
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/FieldInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/FieldInfo.java
new file mode 100644
index 0000000..1d31050
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/FieldInfo.java
@@ -0,0 +1,47 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import org.apache.inlong.sort.formats.common.FormatInfo;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+
+public class FieldInfo {
+    @JsonProperty("name")
+    private final String name;
+
+    @JsonProperty("format_info")
+    private final FormatInfo formatInfo;
+
+    @JsonCreator
+    public FieldInfo(
+            @JsonProperty("name") String name,
+            @JsonProperty("format_info") FormatInfo formatInfo) {
+        this.name = Preconditions.checkNotNull(name);
+        this.formatInfo = Preconditions.checkNotNull(formatInfo);
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public FormatInfo getFormatInfo() {
+        return formatInfo;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/CsvDeserializationInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/CsvDeserializationInfo.java
new file mode 100644
index 0000000..fafae28
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/CsvDeserializationInfo.java
@@ -0,0 +1,42 @@
+/*
+ * 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.deserialization;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * .
+ */
+public class CsvDeserializationInfo implements DeserializationInfo {
+
+    private final char splitter;
+
+    // TODO: support mapping index to field
+
+    @JsonCreator
+    public CsvDeserializationInfo(
+            @JsonProperty("splitter") char splitter) {
+        this.splitter = splitter;
+    }
+
+    @JsonProperty("splitter")
+    public char getSplitter() {
+        return splitter;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/DeserializationInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/DeserializationInfo.java
new file mode 100644
index 0000000..187d4da
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/DeserializationInfo.java
@@ -0,0 +1,38 @@
+
+/*
+ * 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.deserialization;
+
+import java.io.Serializable;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes.Type;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeInfo;
+
+/**
+ * Interface of deserialization info.
+ */
+@JsonTypeInfo(
+        use = JsonTypeInfo.Id.NAME,
+        include = JsonTypeInfo.As.PROPERTY,
+        property = "type")
+@JsonSubTypes({
+        @Type(value = CsvDeserializationInfo.class, name = "csv"),
+})
+public interface DeserializationInfo extends Serializable {
+
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/KvDeserializationInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/KvDeserializationInfo.java
new file mode 100644
index 0000000..aadde75
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/KvDeserializationInfo.java
@@ -0,0 +1,51 @@
+/*
+ * 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.deserialization;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * .
+ */
+public class KvDeserializationInfo implements DeserializationInfo {
+
+    private final char entrySplitter;
+
+    private final char kvSplitter;
+
+    @JsonCreator
+    public KvDeserializationInfo(
+            @JsonProperty("entry_splitter") char entrySplitter,
+            @JsonProperty("kv_splitter") char kvSplitter) {
+        this.entrySplitter = checkNotNull(entrySplitter);
+        this.kvSplitter = checkNotNull(kvSplitter);
+    }
+
+    @JsonProperty("entry_splitter")
+    public char getEntrySplitter() {
+        return entrySplitter;
+    }
+
+    @JsonProperty("kv_splitter")
+    public char getKvSplitter() {
+        return kvSplitter;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgCsv2DeserializationInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgCsv2DeserializationInfo.java
new file mode 100644
index 0000000..4039ee8
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgCsv2DeserializationInfo.java
@@ -0,0 +1,44 @@
+/*
+ * 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.deserialization;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * It represents CSV2 format of TDMsg(m=9).
+ */
+public class TDMsgCsv2DeserializationInfo extends TDMsgDeserializationInfo {
+
+    private static final long serialVersionUID = 2188769102604850019L;
+
+    private final char delimiter;
+
+    @JsonCreator
+    public TDMsgCsv2DeserializationInfo(
+            @JsonProperty("tid") String tid,
+            @JsonProperty("delimiter") char delimiter) {
+        super(tid);
+        this.delimiter = delimiter;
+    }
+
+    @JsonProperty("delimiter")
+    public char getDelimiter() {
+        return delimiter;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgCsvDeserializationInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgCsvDeserializationInfo.java
new file mode 100644
index 0000000..322a8d0
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgCsvDeserializationInfo.java
@@ -0,0 +1,62 @@
+/*
+ * 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.deserialization;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+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.JsonProperty;
+
+/**
+ * It represents CSV format of TDMsg(m=0).
+ */
+public class TDMsgCsvDeserializationInfo extends TDMsgDeserializationInfo {
+
+    private static final long serialVersionUID = 1499370571949888870L;
+
+    private final char delimiter;
+
+    @JsonInclude(Include.NON_NULL)
+    private final boolean deleteHeadDelimiter;
+
+    public TDMsgCsvDeserializationInfo(
+            @JsonProperty("tid") String tid,
+            @JsonProperty("delimiter") char delimiter) {
+        this(tid, delimiter, true);
+    }
+
+    @JsonCreator
+    public TDMsgCsvDeserializationInfo(
+            @JsonProperty("tid") String tid,
+            @JsonProperty("delimiter") char delimiter,
+            @JsonProperty("delete_head_delimiter") boolean deleteHeadDelimiter) {
+        super(tid);
+        this.delimiter = delimiter;
+        this.deleteHeadDelimiter = deleteHeadDelimiter;
+    }
+
+    @JsonProperty("delimiter")
+    public char getDelimiter() {
+        return delimiter;
+    }
+
+    @JsonProperty("delete_head_delimiter")
+    public boolean isDeleteHeadDelimiter() {
+        return deleteHeadDelimiter;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgDeserializationInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgDeserializationInfo.java
new file mode 100644
index 0000000..f00a7d1
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgDeserializationInfo.java
@@ -0,0 +1,55 @@
+/*
+ * 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.deserialization;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes.Type;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeInfo;
+
+/**
+ * .
+ */
+@JsonTypeInfo(
+        use = JsonTypeInfo.Id.NAME,
+        include = JsonTypeInfo.As.PROPERTY,
+        property = "type")
+@JsonSubTypes({
+        @Type(value = TDMsgCsvDeserializationInfo.class, name = "tdmsg_csv"),
+        @Type(value = TDMsgCsv2DeserializationInfo.class, name = "tdmsg_csv2"),
+        @Type(value = TDMsgKvDeserializationInfo.class, name = "tdmsg_kv"),
+        @Type(value = TDMsgTlogCsvDeserializationInfo.class, name = "tdmsg_tlog_csv"),
+        @Type(value = TDMsgTlogKvDeserializationInfo.class, name = "tdmsg_tlog_kv"),
+})
+public abstract class TDMsgDeserializationInfo implements DeserializationInfo {
+
+    private static final long serialVersionUID = 3707412713264864315L;
+
+    private final String tid;
+
+    public TDMsgDeserializationInfo(@JsonProperty("tid") String tid) {
+        this.tid = checkNotNull(tid);
+    }
+
+    @JsonProperty("tid")
+    public String getTid() {
+        return tid;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgKvDeserializationInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgKvDeserializationInfo.java
new file mode 100644
index 0000000..068b25e
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgKvDeserializationInfo.java
@@ -0,0 +1,52 @@
+/*
+ * 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.deserialization;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * It represents KV format of TDMsg(m=5).
+ */
+public class TDMsgKvDeserializationInfo extends TDMsgDeserializationInfo {
+
+    private static final long serialVersionUID = 8431516458466278968L;
+
+    private final char entryDelimiter;
+
+    private final char kvDelimiter;
+
+    public TDMsgKvDeserializationInfo(
+            @JsonProperty("tid") String tid,
+            @JsonProperty("entry_delimiter") char entryDelimiter,
+            @JsonProperty("kv_delimiter") char kvDelimiter
+    ) {
+        super(tid);
+        this.entryDelimiter = entryDelimiter;
+        this.kvDelimiter = kvDelimiter;
+    }
+
+    @JsonProperty("entry_delimiter")
+    public char getEntryDelimiter() {
+        return entryDelimiter;
+    }
+
+    @JsonProperty("kv_delimiter")
+    public char getKvDelimiter() {
+        return kvDelimiter;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgTlogCsvDeserializationInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgTlogCsvDeserializationInfo.java
new file mode 100644
index 0000000..34822a8
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgTlogCsvDeserializationInfo.java
@@ -0,0 +1,44 @@
+/*
+ * 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.deserialization;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * It represents TLog CSV format of TDMsg(m=10).
+ */
+public class TDMsgTlogCsvDeserializationInfo extends TDMsgDeserializationInfo {
+
+    private static final long serialVersionUID = -6585242216925992303L;
+
+    private final char delimiter;
+
+    @JsonCreator
+    public TDMsgTlogCsvDeserializationInfo(
+            @JsonProperty("tid") String tid,
+            @JsonProperty("delimiter") char delimiter) {
+        super(tid);
+        this.delimiter = delimiter;
+    }
+
+    @JsonProperty("delimiter")
+    public char getDelimiter() {
+        return delimiter;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgTlogKvDeserializationInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgTlogKvDeserializationInfo.java
new file mode 100644
index 0000000..fb28f8e
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/deserialization/TDMsgTlogKvDeserializationInfo.java
@@ -0,0 +1,61 @@
+/*
+ * 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.deserialization;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * It represents TLog KV format of TDMsg(m=15).
+ */
+public class TDMsgTlogKvDeserializationInfo extends TDMsgDeserializationInfo {
+
+    private static final long serialVersionUID = 3299931901024581425L;
+
+    private final char delimiter;
+
+    private final char entryDelimiter;
+
+    private final char kvDelimiter;
+
+    public TDMsgTlogKvDeserializationInfo(
+            @JsonProperty("tid") String tid,
+            @JsonProperty("delimiter") char delimiter,
+            @JsonProperty("entry_delimiter") char entryDelimiter,
+            @JsonProperty("kv_delimiter") char kvDelimiter
+    ) {
+        super(tid);
+        this.delimiter = delimiter;
+        this.entryDelimiter = entryDelimiter;
+        this.kvDelimiter = kvDelimiter;
+    }
+
+    @JsonProperty("delimiter")
+    public char getDelimiter() {
+        return delimiter;
+    }
+
+    @JsonProperty("entry_delimiter")
+    public char getEntryDelimiter() {
+        return entryDelimiter;
+    }
+
+    @JsonProperty("kv_delimiter")
+    public char getKvDelimiter() {
+        return kvDelimiter;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/sink/ClickHouseSinkInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/sink/ClickHouseSinkInfo.java
new file mode 100644
index 0000000..4db6e63
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/sink/ClickHouseSinkInfo.java
@@ -0,0 +1,160 @@
+/*
+ * 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.sink;
+
+import com.google.common.base.Preconditions;
+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.inlong.sort.protocol.FieldInfo;
+
+public class ClickHouseSinkInfo extends SinkInfo {
+
+    public enum PartitionStrategy {
+        BALANCE,
+        RANDOM,
+        HASH
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    @JsonProperty("url")
+    private final String url;
+
+    @JsonProperty("database")
+    private final String databaseName;
+
+    @JsonProperty("table")
+    private final String tableName;
+
+    @JsonProperty("username")
+    private final String username;
+
+    @JsonProperty("password")
+    private final String password;
+
+    @JsonProperty("distributed_table")
+    private final boolean isDistributedTable;
+
+    @JsonProperty("partition_strategy")
+    private final PartitionStrategy partitionStrategy;
+
+    @JsonProperty("partition_key")
+    private final String partitionKey;
+
+    @JsonProperty("key_field_names")
+    private final String[] keyFieldNames;
+
+    @JsonProperty("flush_interval")
+    private final int flushInterval;
+
+    @JsonProperty("flush_record_number")
+    private final int flushRecordNumber;
+
+    @JsonProperty("write_max_retry_times")
+    private final int writeMaxRetryTimes;
+
+    @JsonCreator
+    public ClickHouseSinkInfo(
+            @JsonProperty("url") String url,
+            @JsonProperty("database") String databaseName,
+            @JsonProperty("table") String tableName,
+            @JsonProperty("username") String username,
+            @JsonProperty("password") String password,
+            @JsonProperty("distributed_table") boolean isDistributedTable,
+            @JsonProperty("partition_strategy") PartitionStrategy partitionStrategy,
+            @JsonProperty("partition_key") String partitionKey,
+            @JsonProperty("fields") FieldInfo[] fields,
+            @JsonProperty("key_field_names") String[] keyFieldNames,
+            @JsonProperty("flush_interval") int flushInterval,
+            @JsonProperty("flush_record_number") int flushRecordNumber,
+            @JsonProperty("write_max_retry_times") int writeMaxRetryTimes) {
+        super(fields);
+        this.url = Preconditions.checkNotNull(url);
+        this.databaseName = Preconditions.checkNotNull(databaseName);
+        this.tableName = Preconditions.checkNotNull(tableName);
+        this.username = username;
+        this.password = password;
+        this.isDistributedTable = isDistributedTable;
+        this.partitionStrategy = Preconditions.checkNotNull(partitionStrategy);
+        this.partitionKey = Preconditions.checkNotNull(partitionKey);
+        this.keyFieldNames = Preconditions.checkNotNull(keyFieldNames);
+        this.flushInterval = flushInterval;
+        this.flushRecordNumber = flushRecordNumber;
+        this.writeMaxRetryTimes = writeMaxRetryTimes;
+    }
+
+    @JsonProperty("url")
+    public String getUrl() {
+        return url;
+    }
+
+    @JsonProperty("database")
+    public String getDatabaseName() {
+        return databaseName;
+    }
+
+    @JsonProperty("table")
+    public String getTableName() {
+        return tableName;
+    }
+
+    @JsonProperty("username")
+    public String getUsername() {
+        return username;
+    }
+
+    @JsonProperty("password")
+    public String getPassword() {
+        return password;
+    }
+
+    @JsonProperty("distributed_table")
+    public boolean isDistributedTable() {
+        return isDistributedTable;
+    }
+
+    @JsonProperty("partition_strategy")
+    public PartitionStrategy getPartitionStrategy() {
+        return partitionStrategy;
+    }
+
+    @JsonProperty("partition_key")
+    public String getPartitionKey() {
+        return partitionKey;
+    }
+
+    @JsonProperty("key_field_names")
+    public String[] getKeyFieldNames() {
+        return keyFieldNames;
+    }
+
+    @JsonProperty("flush_interval")
+    public int getFlushInterval() {
+        return flushInterval;
+    }
+
+    @JsonProperty("flush_record_number")
+    public int getFlushRecordNumber() {
+        return flushRecordNumber;
+    }
+
+    @JsonProperty("write_max_retry_times")
+    public int getWriteMaxRetryTimes() {
+        return writeMaxRetryTimes;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/sink/HiveSinkInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/sink/HiveSinkInfo.java
new file mode 100644
index 0000000..158a914
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/sink/HiveSinkInfo.java
@@ -0,0 +1,252 @@
+/*
+ * 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.sink;
+
+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.JsonProperty;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes.Type;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeInfo;
+import org.apache.inlong.sort.protocol.FieldInfo;
+
+import javax.annotation.Nullable;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class HiveSinkInfo extends SinkInfo {
+
+    private static final long serialVersionUID = -333999329240274826L;
+
+    @JsonProperty("hive_server_jdbc_url")
+    private final String hiveServerJdbcUrl;
+
+    @JsonProperty("database")
+    private final String databaseName;
+
+    @JsonProperty("table")
+    private final String tableName;
+
+    @JsonInclude(Include.NON_NULL)
+    @JsonProperty("username")
+    private final String username;
+
+    @JsonInclude(Include.NON_NULL)
+    @JsonProperty("password")
+    private final String password;
+
+    @JsonProperty("data_path")
+    private final String dataPath;
+
+    @JsonProperty("partitions")
+    private final HivePartitionInfo[] partitions;
+
+    @JsonProperty("file_format")
+    private final HiveFileFormat hiveFileFormat;
+
+    public HiveSinkInfo(
+            @JsonProperty("fields") FieldInfo[] fields,
+            @JsonProperty("hive_server_jdbc_url") String hiveServerJdbcUrl,
+            @JsonProperty("database") String databaseName,
+            @JsonProperty("table") String tableName,
+            @JsonProperty("username") @Nullable String username,
+            @JsonProperty("password") @Nullable String password,
+            @JsonProperty("data_path") String dataPath,
+            @JsonProperty("partitions") HivePartitionInfo[] partitions,
+            @JsonProperty("file_format") HiveFileFormat hiveFileFormat) {
+        super(fields);
+        this.hiveServerJdbcUrl = checkNotNull(hiveServerJdbcUrl);
+        this.databaseName = checkNotNull(databaseName);
+        this.tableName = checkNotNull(tableName);
+        this.username = username;
+        this.password = password;
+        this.dataPath = checkNotNull(dataPath);
+        this.partitions = checkNotNull(partitions);
+        this.hiveFileFormat = checkNotNull(hiveFileFormat);
+    }
+
+    @JsonProperty("hive_server_jdbc_url")
+    public String getHiveServerJdbcUrl() {
+        return hiveServerJdbcUrl;
+    }
+
+    @JsonProperty("database")
+    public String getDatabaseName() {
+        return databaseName;
+    }
+
+    @JsonProperty("table")
+    public String getTableName() {
+        return tableName;
+    }
+
+    @Nullable
+    @JsonProperty("username")
+    public String getUsername() {
+        return username;
+    }
+
+    @Nullable
+    @JsonProperty("password")
+    public String getPassword() {
+        return password;
+    }
+
+    @JsonProperty("data_path")
+    public String getDataPath() {
+        return dataPath;
+    }
+
+    @JsonProperty("partitions")
+    public HivePartitionInfo[] getPartitions() {
+        return partitions;
+    }
+
+    @JsonProperty("file_format")
+    public HiveFileFormat getHiveFileFormat() {
+        return hiveFileFormat;
+    }
+
+    /**
+     * HivePartitionInfo.
+     */
+    @JsonTypeInfo(
+            use = JsonTypeInfo.Id.NAME,
+            property = "type")
+    @JsonSubTypes({
+            @Type(value = HiveTimePartitionInfo.class, name = "time"),
+            @Type(value = HiveFieldPartitionInfo.class, name = "field")})
+    public abstract static class HivePartitionInfo {
+        @JsonProperty("field_name")
+        private final String fieldName;
+
+        public HivePartitionInfo(
+                @JsonProperty("field_name") String fieldName) {
+            this.fieldName = fieldName;
+        }
+
+        @JsonProperty("field_name")
+        public String getFieldName() {
+            return fieldName;
+        }
+    }
+
+    public static class HiveTimePartitionInfo extends HivePartitionInfo {
+
+        @JsonProperty("date_format")
+        private final String format;
+
+        public HiveTimePartitionInfo(
+                @JsonProperty("field_name") String fieldName,
+                @JsonProperty("date_format") String format) {
+            super(fieldName);
+            this.format = format;
+        }
+
+        @JsonProperty("date_format")
+        public String getFormat() {
+            return format;
+        }
+    }
+
+    public static class HiveFieldPartitionInfo extends HivePartitionInfo {
+
+        public HiveFieldPartitionInfo(
+                @JsonProperty("field_name") String fieldName) {
+            super(fieldName);
+        }
+    }
+
+    /**
+     * HiveFileFormat.
+     */
+    @JsonTypeInfo(
+            use = JsonTypeInfo.Id.NAME,
+            property = "type")
+    @JsonSubTypes({
+            @Type(value = TextFileFormat.class, name = "text"),
+            @Type(value = OrcFileFormat.class, name = "orc"),
+            @Type(value = SequenceFileFormat.class, name = "sequence"),
+            @Type(value = ParquetFileFormat.class, name = "parquet"),})
+    public interface HiveFileFormat {
+    }
+
+    public static class TextFileFormat implements HiveFileFormat {
+
+        @JsonProperty("splitter")
+        private final Character splitter;
+
+        public TextFileFormat(
+                @JsonProperty("splitter") Character splitter) {
+            this.splitter = splitter;
+        }
+
+        @JsonProperty("splitter")
+        public Character getSplitter() {
+            return splitter;
+        }
+    }
+
+    public static class OrcFileFormat implements HiveFileFormat {
+
+        @JsonProperty("batch_size")
+        private final int batchSize;
+
+        public OrcFileFormat(
+                @JsonProperty("batch_size") int batchSize) {
+            this.batchSize = batchSize;
+        }
+
+        @JsonProperty("batch_size")
+        public int getBatchSize() {
+            return batchSize;
+        }
+    }
+
+    public static class SequenceFileFormat implements HiveFileFormat {
+
+        @JsonProperty("splitter")
+        private final Character splitter;
+
+        @JsonProperty("buffer_size")
+        private final int bufferSize;
+
+        public SequenceFileFormat(
+                @JsonProperty("splitter") Character splitter,
+                @JsonProperty("buffer_size") int bufferSize) {
+            this.splitter = splitter;
+            this.bufferSize = bufferSize;
+        }
+
+        @JsonProperty("splitter")
+        public Character getSplitter() {
+            return splitter;
+        }
+
+        @JsonProperty("buffer_size")
+        public int getBufferSize() {
+            return bufferSize;
+        }
+    }
+
+    public static class ParquetFileFormat implements HiveFileFormat {
+
+        public ParquetFileFormat() {
+        }
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/sink/SinkInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/sink/SinkInfo.java
new file mode 100644
index 0000000..12714e0
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/sink/SinkInfo.java
@@ -0,0 +1,56 @@
+/*
+ * 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.sink;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.Serializable;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes.Type;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeInfo;
+import org.apache.inlong.sort.configuration.Constants;
+import org.apache.inlong.sort.protocol.FieldInfo;
+
+/**
+ * The base class of the data sink in the metadata.
+ */
+@JsonTypeInfo(
+        use = JsonTypeInfo.Id.NAME,
+        include = JsonTypeInfo.As.PROPERTY,
+        property = "type")
+@JsonSubTypes({
+        @Type(value = ClickHouseSinkInfo.class, name = Constants.SINK_TYPE_CLICKHOUSE),
+        @Type(value = HiveSinkInfo.class, name = Constants.SINK_TYPE_HIVE)}
+)
+public abstract class SinkInfo implements Serializable {
+
+    private static final long serialVersionUID = 1485856855405721745L;
+
+    @JsonProperty("fields")
+    private final FieldInfo[] fields;
+
+    public SinkInfo(@JsonProperty("fields") FieldInfo[] fields) {
+        this.fields = checkNotNull(fields);
+    }
+
+    @JsonProperty("fields")
+    public FieldInfo[] getFields() {
+        return fields;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/source/PulsarSourceInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/source/PulsarSourceInfo.java
new file mode 100644
index 0000000..c5da028
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/source/PulsarSourceInfo.java
@@ -0,0 +1,74 @@
+/*
+ * 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.source;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+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.inlong.sort.protocol.FieldInfo;
+import org.apache.inlong.sort.protocol.deserialization.DeserializationInfo;
+
+public class PulsarSourceInfo extends SourceInfo {
+
+    private static final long serialVersionUID = -3354319497859179290L;
+
+    private final String adminUrl;
+
+    private final String serviceUrl;
+
+    private final String topic;
+
+    private final String subscriptionName;
+
+    @JsonCreator
+    public PulsarSourceInfo(
+            @JsonProperty("admin_url") String adminUrl,
+            @JsonProperty("service_url") String serviceUrl,
+            @JsonProperty("topic") String topic,
+            @JsonProperty("subscription_name") String subscriptionName,
+            @JsonProperty("deserialization_info") DeserializationInfo deserializationInfo,
+            @JsonProperty("fields") FieldInfo[] fields) {
+        super(fields, deserializationInfo);
+
+        this.adminUrl = checkNotNull(adminUrl);
+        this.serviceUrl = checkNotNull(serviceUrl);
+        this.topic = checkNotNull(topic);
+        this.subscriptionName = checkNotNull(subscriptionName);
+    }
+
+    @JsonProperty("admin_url")
+    public String getAdminUrl() {
+        return adminUrl;
+    }
+
+    @JsonProperty("service_url")
+    public String getServiceUrl() {
+        return serviceUrl;
+    }
+
+    @JsonProperty("topic")
+    public String getTopic() {
+        return topic;
+    }
+
+    @JsonProperty("subscription_name")
+    public String getSubscriptionName() {
+        return subscriptionName;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/source/SourceInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/source/SourceInfo.java
new file mode 100644
index 0000000..ca1b3e2
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/source/SourceInfo.java
@@ -0,0 +1,66 @@
+/*
+ * 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.source;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.Serializable;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes.Type;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeInfo;
+import org.apache.inlong.sort.configuration.Constants;
+import org.apache.inlong.sort.protocol.FieldInfo;
+import org.apache.inlong.sort.protocol.deserialization.DeserializationInfo;
+
+/**
+ * The base class of the data source in the metadata.
+ */
+@JsonTypeInfo(
+        use = JsonTypeInfo.Id.NAME,
+        include = JsonTypeInfo.As.PROPERTY,
+        property = "type")
+@JsonSubTypes({
+        @Type(value = TubeSourceInfo.class, name = Constants.SOURCE_TYPE_TUBE),
+        @Type(value = PulsarSourceInfo.class, name = Constants.SOURCE_TYPE_PULSAR),
+})
+public abstract class SourceInfo implements Serializable {
+
+    private static final long serialVersionUID = 701717917118317501L;
+
+    private final FieldInfo[] fields;
+
+    private final DeserializationInfo deserializationInfo;
+
+    public SourceInfo(
+            @JsonProperty("fields") FieldInfo[] fields,
+            @JsonProperty("deserialization_info") DeserializationInfo deserializationInfo) {
+        this.fields = checkNotNull(fields);
+        this.deserializationInfo = checkNotNull(deserializationInfo);
+    }
+
+    @JsonProperty("fields")
+    public FieldInfo[] getFields() {
+        return fields;
+    }
+
+    @JsonProperty("deserialization_info")
+    public DeserializationInfo getDeserializationInfo() {
+        return deserializationInfo;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/source/TubeSourceInfo.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/source/TubeSourceInfo.java
new file mode 100644
index 0000000..3481bc0
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/protocol/source/TubeSourceInfo.java
@@ -0,0 +1,75 @@
+/*
+ * 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.source;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.inlong.sort.protocol.FieldInfo;
+import org.apache.inlong.sort.protocol.deserialization.DeserializationInfo;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+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.JsonProperty;
+
+/**
+ * TubeMQ source information.
+ */
+public class TubeSourceInfo extends SourceInfo {
+
+    private static final long serialVersionUID = 3559859089154486836L;
+
+    private final String topic;
+
+    private final String masterAddress;
+
+    @JsonInclude(Include.NON_NULL)
+    private final String consumerGroup;
+
+    @JsonCreator
+    public TubeSourceInfo(
+            @JsonProperty("topic") String topic,
+            @JsonProperty("master_address") String masterAddress,
+            @JsonProperty("consumer_group") @Nullable String consumerGroup,
+            @JsonProperty("deserialization_info") DeserializationInfo deserializationInfo,
+            @JsonProperty("fields") FieldInfo[] fields) {
+        super(fields, deserializationInfo);
+
+        this.topic = checkNotNull(topic);
+        this.masterAddress = checkNotNull(masterAddress);
+        this.consumerGroup = consumerGroup;
+    }
+
+    @JsonProperty("topic")
+    public String getTopic() {
+        return topic;
+    }
+
+    @JsonProperty("master_address")
+    public String getMasterAddress() {
+        return masterAddress;
+    }
+
+
+    @JsonProperty("consumer_group")
+    @Nullable
+    public String getConsumerGroup() {
+        return consumerGroup;
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/util/InstantiationUtil.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/util/InstantiationUtil.java
new file mode 100644
index 0000000..a15d1a1
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/util/InstantiationUtil.java
@@ -0,0 +1,268 @@
+/*
+ * 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.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamClass;
+import java.io.OutputStream;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Proxy;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Set;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class to create instances from class objects and checking failure reasons.
+ *
+ * <p>Copied from Flink project.</p>
+ */
+public class InstantiationUtil {
+
+    private static final Logger LOG = LoggerFactory.getLogger(InstantiationUtil.class);
+
+    public static byte[] serializeObject(Object o) throws IOException {
+        try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+                ObjectOutputStream oos = new ObjectOutputStream(baos)) {
+            oos.writeObject(o);
+            oos.flush();
+            return baos.toByteArray();
+        }
+    }
+
+    public static void serializeObject(OutputStream out, Object o) throws IOException {
+        ObjectOutputStream oos = new ObjectOutputStream(out);
+        oos.writeObject(o);
+    }
+
+    public static boolean isSerializable(Object o) {
+        try {
+            serializeObject(o);
+        } catch (IOException e) {
+            return false;
+        }
+
+        return true;
+    }
+
+    @SuppressWarnings("unchecked")
+    public static <T> T deserializeObject(byte[] bytes, ClassLoader cl)
+            throws IOException, ClassNotFoundException {
+        return deserializeObject(bytes, cl, false);
+    }
+
+    @SuppressWarnings("unchecked")
+    public static <T> T deserializeObject(InputStream in, ClassLoader cl)
+            throws IOException, ClassNotFoundException {
+        return deserializeObject(in, cl, false);
+    }
+
+    @SuppressWarnings("unchecked")
+    public static <T> T deserializeObject(byte[] bytes, ClassLoader cl, boolean isFailureTolerant)
+            throws IOException, ClassNotFoundException {
+
+        return deserializeObject(new ByteArrayInputStream(bytes), cl, isFailureTolerant);
+    }
+
+    @SuppressWarnings("unchecked")
+    public static <T> T deserializeObject(InputStream in, ClassLoader cl, boolean isFailureTolerant)
+            throws IOException, ClassNotFoundException {
+
+        final ClassLoader old = Thread.currentThread().getContextClassLoader();
+        // not using resource try to avoid AutoClosable's close() on the given stream
+        try {
+            ObjectInputStream oois = isFailureTolerant
+                    ? new InstantiationUtil.FailureTolerantObjectInputStream(in, cl)
+                    : new InstantiationUtil.ClassLoaderObjectInputStream(in, cl);
+            Thread.currentThread().setContextClassLoader(cl);
+            return (T) oois.readObject();
+        } finally {
+            Thread.currentThread().setContextClassLoader(old);
+        }
+    }
+
+    /**
+     * A custom ObjectInputStream that can load classes using a specific ClassLoader.
+     */
+    public static class ClassLoaderObjectInputStream extends ObjectInputStream {
+
+        protected final ClassLoader classLoader;
+
+        public ClassLoaderObjectInputStream(InputStream in, ClassLoader classLoader)
+                throws IOException {
+            super(in);
+            this.classLoader = classLoader;
+        }
+
+        @Override
+        protected Class<?> resolveClass(ObjectStreamClass desc)
+                throws IOException, ClassNotFoundException {
+            if (classLoader != null) {
+                String name = desc.getName();
+                try {
+                    return Class.forName(name, false, classLoader);
+                } catch (ClassNotFoundException ex) {
+                    // check if class is a primitive class
+                    Class<?> cl = primitiveClasses.get(name);
+                    if (cl != null) {
+                        // return primitive class
+                        return cl;
+                    } else {
+                        // throw ClassNotFoundException
+                        throw ex;
+                    }
+                }
+            }
+
+            return super.resolveClass(desc);
+        }
+
+        @Override
+        protected Class<?> resolveProxyClass(String[] interfaces)
+                throws IOException, ClassNotFoundException {
+            if (classLoader != null) {
+                ClassLoader nonPublicLoader = null;
+                boolean hasNonPublicInterface = false;
+
+                // define proxy in class loader of non-public interface(s), if any
+                Class<?>[] classObjs = new Class<?>[interfaces.length];
+                for (int i = 0; i < interfaces.length; i++) {
+                    Class<?> cl = Class.forName(interfaces[i], false, classLoader);
+                    if ((cl.getModifiers() & Modifier.PUBLIC) == 0) {
+                        if (hasNonPublicInterface) {
+                            if (nonPublicLoader != cl.getClassLoader()) {
+                                throw new IllegalAccessError(
+                                        "conflicting non-public interface class loaders");
+                            }
+                        } else {
+                            nonPublicLoader = cl.getClassLoader();
+                            hasNonPublicInterface = true;
+                        }
+                    }
+                    classObjs[i] = cl;
+                }
+                try {
+                    return Proxy.getProxyClass(
+                            hasNonPublicInterface ? nonPublicLoader : classLoader, classObjs);
+                } catch (IllegalArgumentException e) {
+                    throw new ClassNotFoundException(null, e);
+                }
+            }
+
+            return super.resolveProxyClass(interfaces);
+        }
+
+        // ------------------------------------------------
+
+        private static final HashMap<String, Class<?>> primitiveClasses = new HashMap<>(9);
+
+        static {
+            primitiveClasses.put("boolean", boolean.class);
+            primitiveClasses.put("byte", byte.class);
+            primitiveClasses.put("char", char.class);
+            primitiveClasses.put("short", short.class);
+            primitiveClasses.put("int", int.class);
+            primitiveClasses.put("long", long.class);
+            primitiveClasses.put("float", float.class);
+            primitiveClasses.put("double", double.class);
+            primitiveClasses.put("void", void.class);
+        }
+    }
+
+    /**
+     * An {@link ObjectInputStream} that ignores serialVersionUID mismatches when deserializing
+     * objects of anonymous classes or our Scala serializer classes and also replaces occurences of
+     * GenericData.Array (from Avro) by a dummy class so that the KryoSerializer can still be
+     * deserialized without Avro being on the classpath.
+     *
+     * <p>Uses this specific object input stream to read serializers,
+     * so that mismatching serialVersionUIDs of anonymous classes / Scala serializers are ignored.
+     * This is a required workaround to maintain backwards compatibility for our pre-1.3 Scala
+     * serializers. See FLINK-6869 for details.</p>
+     *
+     * @see <a href="https://issues.apache.org/jira/browse/FLINK-6869">FLINK-6869</a>
+     */
+    public static class FailureTolerantObjectInputStream extends
+            InstantiationUtil.ClassLoaderObjectInputStream {
+
+        // initialize this set to support compatible manually
+        private static final Set<String> compatibleClasses = new HashSet<>();
+
+        public FailureTolerantObjectInputStream(InputStream in, ClassLoader cl) throws IOException {
+            super(in, cl);
+        }
+
+        @Override
+        protected ObjectStreamClass readClassDescriptor()
+                throws IOException, ClassNotFoundException {
+            ObjectStreamClass streamClassDescriptor = super.readClassDescriptor();
+
+            try {
+                Class.forName(streamClassDescriptor.getName(), false, classLoader);
+            } catch (ClassNotFoundException ignored) {
+                // ignore this exception
+            }
+
+            final Class localClass = resolveClass(streamClassDescriptor);
+            final String name = localClass.getName();
+            if (compatibleClasses.contains(name) || isAnonymousClass(localClass)) {
+                final ObjectStreamClass localClassDescriptor = ObjectStreamClass.lookup(localClass);
+                if (localClassDescriptor != null
+                        && localClassDescriptor.getSerialVersionUID() != streamClassDescriptor
+                        .getSerialVersionUID()) {
+                    LOG.warn("Ignoring serialVersionUID mismatch for class {}; was {}, now {}.",
+                            streamClassDescriptor.getName(),
+                            streamClassDescriptor.getSerialVersionUID(),
+                            localClassDescriptor.getSerialVersionUID());
+
+                    streamClassDescriptor = localClassDescriptor;
+                }
+            }
+
+            return streamClassDescriptor;
+        }
+
+    }
+
+    private static boolean isAnonymousClass(Class clazz) {
+        final String name = clazz.getName();
+
+        // isAnonymousClass does not work for anonymous Scala classes
+        // additionally check by class name
+        if (name.contains("$anon$") || name.contains("$anonfun") || name.contains("$macro$")) {
+            return true;
+        }
+
+        // calling isAnonymousClass or getSimpleName can throw InternalError for certain Scala
+        // types, see https://issues.scala-lang.org/browse/SI-2034
+        // until we move to JDK 9, this try-catch is necessary
+        try {
+            return clazz.isAnonymousClass();
+        } catch (InternalError e) {
+            return false;
+        }
+    }
+
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/util/TestLogger.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/util/TestLogger.java
new file mode 100644
index 0000000..11e1671
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/util/TestLogger.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.util;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import org.junit.Rule;
+import org.junit.rules.TestRule;
+import org.junit.rules.TestWatcher;
+import org.junit.runner.Description;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Adds automatic test name logging. Every test which wants to log which test is currently executed
+ * and why it failed, simply has to extend this class.
+ *
+ * <p>Copied from Flink project.</p>
+ */
+public class TestLogger {
+
+    protected final Logger log = LoggerFactory.getLogger(getClass());
+
+    @Rule
+    public TestRule watchman = new TestWatcher() {
+
+        @Override
+        public void starting(Description description) {
+            log.info(
+                    "\n============================================================================"
+                            + "\nTest {} is running."
+                            + "\n-----------------------------------------------------------------",
+                    description);
+        }
+
+        @Override
+        public void succeeded(Description description) {
+            log.info(
+                    "\n----------------------------------------------------------------------------"
+                            + "\nTest {} successfully run."
+                            + "\n=================================================================",
+                    description);
+        }
+
+        @Override
+        public void failed(Throwable e, Description description) {
+            log.error(
+                    "\n----------------------------------------------------------------------------"
+                            + "\nTest {} failed with:\n{}"
+                            + "\n=================================================================",
+                    description, exceptionToString(e));
+        }
+    };
+
+    private static String exceptionToString(Throwable t) {
+        if (t == null) {
+            return "(null)";
+        }
+
+        try {
+            StringWriter stm = new StringWriter();
+            PrintWriter wrt = new PrintWriter(stm);
+            t.printStackTrace(wrt);
+            wrt.close();
+            return stm.toString();
+        } catch (Throwable ignored) {
+            return t.getClass().getName() + " (error while printing stack trace)";
+        }
+    }
+}
diff --git a/inlong-sort/common/src/main/java/org/apache/inlong/sort/util/ZooKeeperUtils.java b/inlong-sort/common/src/main/java/org/apache/inlong/sort/util/ZooKeeperUtils.java
new file mode 100644
index 0000000..5ef3c5d
--- /dev/null
+++ b/inlong-sort/common/src/main/java/org/apache/inlong/sort/util/ZooKeeperUtils.java
@@ -0,0 +1,266 @@
+/*
+ * 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.util;
+
+import com.google.common.base.Preconditions;
+import org.apache.inlong.sort.configuration.Configuration;
+import org.apache.inlong.sort.configuration.Constants;
+import java.util.List;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.ACLProvider;
+import org.apache.curator.framework.imps.DefaultACLProvider;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.Code;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class containing helper functions to interact with ZooKeeper.
+ *
+ * <p>Copied from Flink project.</p>
+ */
+public class ZooKeeperUtils {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperUtils.class);
+
+    /**
+     * Starts a {@link CuratorFramework} instance and connects it to the given ZooKeeper quorum.
+     *
+     * @param configuration {@link Configuration} object containing the configuration
+     *         values
+     * @return {@link CuratorFramework} instance
+     */
+    public static CuratorFramework startCuratorFramework(Configuration configuration) {
+        Preconditions.checkNotNull(configuration, "configuration");
+        final String zkQuorum = configuration.getValue(Constants.ZOOKEEPER_QUORUM);
+
+        if (zkQuorum == null || StringUtils.isBlank(zkQuorum)) {
+            throw new RuntimeException("No valid ZooKeeper quorum has been specified. "
+                    + "You can specify the quorum via the configuration key '"
+                    + Constants.ZOOKEEPER_QUORUM.key() + "'.");
+        }
+
+        final int sessionTimeout = configuration
+                .getInteger(Constants.ZOOKEEPER_SESSION_TIMEOUT);
+
+        final int connectionTimeout = configuration
+                .getInteger(Constants.ZOOKEEPER_CONNECTION_TIMEOUT);
+
+        final int retryWait = configuration.getInteger(Constants.ZOOKEEPER_RETRY_WAIT);
+
+        final int maxRetryAttempts = configuration
+                .getInteger(Constants.ZOOKEEPER_MAX_RETRY_ATTEMPTS);
+
+        final boolean disableSaslClient = configuration
+                .getBoolean(Constants.ZOOKEEPER_SASL_DISABLE);
+
+        ACLProvider aclProvider;
+
+        final ZkClientACLMode aclMode = ZkClientACLMode.fromConfig(configuration);
+
+        if (disableSaslClient && aclMode == ZkClientACLMode.CREATOR) {
+            String errorMessage =
+                    "Cannot set ACL role to " + aclMode + "  since SASL authentication is "
+                            + "disabled through the " + Constants.ZOOKEEPER_SASL_DISABLE.key()
+                            + " property";
+            LOG.warn(errorMessage);
+            throw new IllegalStateException(errorMessage);
+        }
+
+        if (aclMode == ZkClientACLMode.CREATOR) {
+            LOG.info("Enforcing creator for ZK connections");
+            aclProvider = new SecureAclProvider();
+        } else {
+            LOG.info("Enforcing default ACL for ZK connections");
+            aclProvider = new DefaultACLProvider();
+        }
+
+        final String root = configuration.getValue(Constants.ZOOKEEPER_ROOT);
+
+        LOG.info("Using '{}' as Zookeeper namespace.", root);
+
+        CuratorFramework cf = CuratorFrameworkFactory.builder()
+                .connectString(zkQuorum)
+                .sessionTimeoutMs(sessionTimeout)
+                .connectionTimeoutMs(connectionTimeout)
+                .retryPolicy(new ExponentialBackoffRetry(retryWait, maxRetryAttempts))
+                // Curator prepends a '/' manually and throws an Exception if the
+                // namespace starts with a '/'.
+                .namespace(root.startsWith("/") ? root.substring(1)
+                        : root)
+                .aclProvider(aclProvider)
+                .build();
+
+        cf.start();
+
+        return cf;
+    }
+
+    /**
+     * Returns the configured ZooKeeper quorum (and removes whitespace, because ZooKeeper does not
+     * tolerate it).
+     */
+    public static String getZooKeeperEnsemble(Configuration config)
+            throws IllegalStateException {
+
+        String zkQuorum = config.getValue(Constants.ZOOKEEPER_QUORUM);
+
+        if (zkQuorum == null || StringUtils.isBlank(zkQuorum)) {
+            throw new IllegalStateException("No ZooKeeper quorum specified in config.");
+        }
+
+        // Remove all whitespace
+        zkQuorum = zkQuorum.replaceAll("\\s+", "");
+
+        return zkQuorum;
+    }
+
+    public static String generateZookeeperPath(String root, String namespace) {
+        if (!namespace.startsWith("/")) {
+            namespace = '/' + namespace;
+        }
+
+        if (namespace.endsWith("/")) {
+            namespace = namespace.substring(0, namespace.length() - 1);
+        }
+
+        if (root.endsWith("/")) {
+            root = root.substring(0, root.length() - 1);
+        }
+
+        return root + namespace;
+    }
+
+    /**
+     * Returns a facade of the client that uses the specified namespace, and ensures that all nodes
+     * in the path exist.
+     *
+     * @param client ZK client
+     * @param path the new namespace
+     * @return ZK Client that uses the new namespace
+     * @throws Exception ZK errors
+     */
+    public static CuratorFramework useNamespaceAndEnsurePath(final CuratorFramework client,
+            final String path) throws Exception {
+        Preconditions.checkNotNull(client, "client must not be null");
+        Preconditions.checkNotNull(path, "path must not be null");
+
+        // Ensure that the checkpoints path exists
+        client.newNamespaceAwareEnsurePath(path)
+                .ensure(client.getZookeeperClient());
+
+        // All operations will have the path as root
+        return client.usingNamespace(generateZookeeperPath(client.getNamespace(), path));
+    }
+
+    /**
+     * Create zk path with data.
+     * @param zkClient zk client
+     * @param nodePath the path to be created
+     * @param data the data in the path to be created
+     * @param createMode {@link CreateMode}
+     */
+    public static String createRecursive(
+            CuratorFramework zkClient,
+            String nodePath,
+            byte[] data,
+            CreateMode createMode) throws Exception {
+
+        if (!nodePath.startsWith("/")) {
+            throw KeeperException.create(Code.NONODE, "path " + nodePath
+                    + " is invalid");
+        }
+
+        if (nodePath.endsWith("/")) {
+            nodePath = nodePath.substring(0, nodePath.length() - 1);
+        }
+
+        int lastsp = nodePath.lastIndexOf("/");
+
+        if (lastsp == 0) {
+            return zkClient.create().withMode(createMode).forPath(nodePath);
+        }
+
+        String parent = nodePath.substring(0, lastsp);
+        if (zkClient.checkExists().forPath(parent) == null) {
+            createRecursive(zkClient, parent, null, CreateMode.PERSISTENT);
+        }
+
+        return zkClient.create().withMode(createMode).forPath(nodePath, data);
+    }
+
+
+    /**
+     * Secure {@link ACLProvider} implementation.
+     */
+    public static class SecureAclProvider implements ACLProvider {
+
+        @Override
+        public List<ACL> getDefaultAcl() {
+            return ZooDefs.Ids.CREATOR_ALL_ACL;
+        }
+
+        @Override
+        public List<ACL> getAclForPath(String path) {
+            return ZooDefs.Ids.CREATOR_ALL_ACL;
+        }
+    }
+
+    /**
+     * ZooKeeper client ACL mode enum.
+     */
+    public enum ZkClientACLMode {
+        CREATOR,
+        OPEN;
+
+        /**
+         * Return the configured {@link ZkClientACLMode}.
+         *
+         * @param config The config to parse
+         * @return Configured ACL mode or the default defined by
+         * {@link Constants#ZOOKEEPER_CLIENT_ACL}
+         *         if not configured.
+         */
+        public static ZkClientACLMode fromConfig(Configuration config) {
+            String aclMode = config.getString(Constants.ZOOKEEPER_CLIENT_ACL);
+            if (aclMode == null || aclMode.equalsIgnoreCase(ZkClientACLMode.OPEN.name())) {
+                return ZkClientACLMode.OPEN;
+            } else if (aclMode.equalsIgnoreCase(ZkClientACLMode.CREATOR.name())) {
+                return ZkClientACLMode.CREATOR;
+            } else {
+                String message = "Unsupported ACL option: [" + aclMode + "] provided";
+                LOG.error(message);
+                throw new IllegalStateException(message);
+            }
+        }
+    }
+
+    /**
+     * Private constructor to prevent instantiation.
+     */
+    private ZooKeeperUtils() {
+        throw new RuntimeException();
+    }
+}
diff --git a/inlong-sort/common/src/test/java/org/apache/inlong/sort/configuration/ConfigOptionTest.java b/inlong-sort/common/src/test/java/org/apache/inlong/sort/configuration/ConfigOptionTest.java
new file mode 100644
index 0000000..9f70870
--- /dev/null
+++ b/inlong-sort/common/src/test/java/org/apache/inlong/sort/configuration/ConfigOptionTest.java
@@ -0,0 +1,118 @@
+/*
+ * 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.configuration;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.collect.Sets;
+import org.apache.inlong.sort.util.TestLogger;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.junit.Test;
+
+/**
+ * Tests for the {@link ConfigOption}.
+ */
+public class ConfigOptionTest extends TestLogger {
+
+    @Test
+    public void testDeprecationFlagForDeprecatedKeys() {
+        final ConfigOption<Integer> optionWithDeprecatedKeys = ConfigOptions
+                .key("key")
+                .defaultValue(0)
+                .withDeprecatedKeys("deprecated1", "deprecated2");
+
+        assertTrue(optionWithDeprecatedKeys.hasFallbackKeys());
+        for (final FallbackKey fallbackKey : optionWithDeprecatedKeys.fallbackKeys()) {
+            assertTrue("deprecated key not flagged as deprecated", fallbackKey.isDeprecated());
+        }
+    }
+
+    @Test
+    public void testDeprecationFlagForFallbackKeys() {
+        final ConfigOption<Integer> optionWithFallbackKeys = ConfigOptions
+                .key("key")
+                .defaultValue(0)
+                .withFallbackKeys("fallback1", "fallback2");
+
+        assertTrue(optionWithFallbackKeys.hasFallbackKeys());
+        for (final FallbackKey fallbackKey : optionWithFallbackKeys.fallbackKeys()) {
+            assertFalse("falback key flagged as deprecated", fallbackKey.isDeprecated());
+        }
+    }
+
+    @Test
+    public void testDeprecationFlagForMixedAlternativeKeys() {
+        final ConfigOption<Integer> optionWithMixedKeys = ConfigOptions
+                .key("key")
+                .defaultValue(0)
+                .withDeprecatedKeys("deprecated1", "deprecated2")
+                .withFallbackKeys("fallback1", "fallback2");
+
+        final List<String> fallbackKeys = new ArrayList<>(2);
+        final List<String> deprecatedKeys = new ArrayList<>(2);
+        for (final FallbackKey alternativeKey : optionWithMixedKeys.fallbackKeys()) {
+            if (alternativeKey.isDeprecated()) {
+                deprecatedKeys.add(alternativeKey.getKey());
+            } else {
+                fallbackKeys.add(alternativeKey.getKey());
+            }
+        }
+
+        assertEquals(2, fallbackKeys.size());
+        assertEquals(2, deprecatedKeys.size());
+
+        assertThat(fallbackKeys, containsInAnyOrder("fallback1", "fallback2"));
+        assertThat(deprecatedKeys, containsInAnyOrder("deprecated1", "deprecated2"));
+    }
+
+    @Test
+    public void testDeprecationForDeprecatedKeys() {
+        String[] deprecatedKeys = new String[]{"deprecated1", "deprecated2"};
+        final Set<String> expectedDeprecatedKeys = new HashSet<>(Arrays.asList(deprecatedKeys));
+
+        final ConfigOption<Integer> optionWithDeprecatedKeys = ConfigOptions
+                .key("key")
+                .defaultValue(0)
+                .withDeprecatedKeys(deprecatedKeys)
+                .withFallbackKeys("fallback1");
+
+        assertTrue(optionWithDeprecatedKeys.hasDeprecatedKeys());
+        assertEquals(expectedDeprecatedKeys,
+                Sets.newHashSet(optionWithDeprecatedKeys.deprecatedKeys()));
+    }
+
+    @Test
+    public void testNoDeprecationForFallbackKeysWithoutDeprecated() {
+        final ConfigOption<Integer> optionWithFallbackKeys = ConfigOptions
+                .key("key")
+                .defaultValue(0)
+                .withFallbackKeys("fallback1");
+
+        assertFalse(optionWithFallbackKeys.hasDeprecatedKeys());
+    }
+
+}
diff --git a/inlong-sort/common/src/test/java/org/apache/inlong/sort/configuration/ConfigurationTest.java b/inlong-sort/common/src/test/java/org/apache/inlong/sort/configuration/ConfigurationTest.java
new file mode 100644
index 0000000..2de21e9
--- /dev/null
+++ b/inlong-sort/common/src/test/java/org/apache/inlong/sort/configuration/ConfigurationTest.java
@@ -0,0 +1,462 @@
+/*
+ * 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.configuration;
+
+import static org.hamcrest.Matchers.containsString;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.inlong.sort.util.InstantiationUtil;
+import org.apache.inlong.sort.util.TestLogger;
+import org.junit.Test;
+
+/**
+ * This class contains test for the configuration package. In particular, the serialization of
+ * {@link Configuration} objects is tested.
+ */
+public class ConfigurationTest extends TestLogger {
+
+    private static final byte[] EMPTY_BYTES = new byte[0];
+    private static final long TOO_LONG = Integer.MAX_VALUE + 10L;
+    private static final double TOO_LONG_DOUBLE = Double.MAX_VALUE;
+
+    /**
+     * This test checks the serialization/deserialization of configuration objects.
+     */
+    @Test
+    public void testConfigurationSerializationAndGetters() {
+        try {
+            final Configuration orig = new Configuration();
+            orig.setString("mykey", "myvalue");
+            orig.setInteger("mynumber", 100);
+            orig.setLong("longvalue", 478236947162389746L);
+            orig.setFloat("PI", 3.1415926f);
+            orig.setDouble("E", Math.E);
+            orig.setBoolean("shouldbetrue", true);
+            orig.setBytes("bytes sequence", new byte[]{1, 2, 3, 4, 5});
+            orig.setClass("myclass", this.getClass());
+
+            final Configuration copy = InstantiationUtil
+                    .deserializeObject(InstantiationUtil.serializeObject(orig),
+                            Thread.currentThread().getContextClassLoader());
+            assertEquals("myvalue", copy.getString("mykey", "null"));
+            assertEquals(100, copy.getInteger("mynumber", 0));
+            assertEquals(478236947162389746L, copy.getLong("longvalue", 0L));
+            assertEquals(3.1415926f, copy.getFloat("PI", 3.1415926f), 0.0);
+            assertEquals(Math.E, copy.getDouble("E", 0.0), 0.0);
+            assertEquals(true, copy.getBoolean("shouldbetrue", false));
+            assertArrayEquals(new byte[]{1, 2, 3, 4, 5}, copy.getBytes("bytes sequence", null));
+            assertEquals(getClass(), copy.getClass("myclass", null, getClass().getClassLoader()));
+
+            assertEquals(orig, copy);
+            assertEquals(orig.keySet(), copy.keySet());
+            assertEquals(orig.hashCode(), copy.hashCode());
+
+        } catch (Exception e) {
+            e.printStackTrace();
+            fail(e.getMessage());
+        }
+    }
+
+    @Test
+    public void testConversions() {
+        try {
+            Configuration pc = new Configuration();
+
+            pc.setInteger("int", 5);
+            pc.setLong("long", 15);
+            pc.setLong("too_long", TOO_LONG);
+            pc.setFloat("float", 2.1456775f);
+            pc.setDouble("double", Math.PI);
+            pc.setDouble("negative_double", -1.0);
+            pc.setDouble("zero", 0.0);
+            pc.setDouble("too_long_double", TOO_LONG_DOUBLE);
+            pc.setString("string", "42");
+            pc.setString("non_convertible_string", "bcdefg&&");
+            pc.setBoolean("boolean", true);
+
+            // as integer
+            assertEquals(5, pc.getInteger("int", 0));
+            assertEquals(5L, pc.getLong("int", 0));
+            assertEquals(5f, pc.getFloat("int", 0), 0.0);
+            assertEquals(5.0, pc.getDouble("int", 0), 0.0);
+            assertEquals(false, pc.getBoolean("int", true));
+            assertEquals("5", pc.getString("int", "0"));
+            assertArrayEquals(EMPTY_BYTES, pc.getBytes("int", EMPTY_BYTES));
+
+            // as long
+            assertEquals(15, pc.getInteger("long", 0));
+            assertEquals(15L, pc.getLong("long", 0));
+            assertEquals(15f, pc.getFloat("long", 0), 0.0);
+            assertEquals(15.0, pc.getDouble("long", 0), 0.0);
+            assertEquals(false, pc.getBoolean("long", true));
+            assertEquals("15", pc.getString("long", "0"));
+            assertArrayEquals(EMPTY_BYTES, pc.getBytes("long", EMPTY_BYTES));
+
+            // as too long
+            assertEquals(0, pc.getInteger("too_long", 0));
+            assertEquals(TOO_LONG, pc.getLong("too_long", 0));
+            assertEquals((float) TOO_LONG, pc.getFloat("too_long", 0), 10.0);
+            assertEquals((double) TOO_LONG, pc.getDouble("too_long", 0), 10.0);
+            assertEquals(false, pc.getBoolean("too_long", true));
+            assertEquals(String.valueOf(TOO_LONG), pc.getString("too_long", "0"));
+            assertArrayEquals(EMPTY_BYTES, pc.getBytes("too_long", EMPTY_BYTES));
+
+            // as float
+            assertEquals(0, pc.getInteger("float", 0));
+            assertEquals(0L, pc.getLong("float", 0));
+            assertEquals(2.1456775f, pc.getFloat("float", 0), 0.0);
+            assertEquals(2.1456775, pc.getDouble("float", 0), 0.0000001);
+            assertEquals(false, pc.getBoolean("float", true));
+            assertTrue(pc.getString("float", "0").startsWith("2.145677"));
+            assertArrayEquals(EMPTY_BYTES, pc.getBytes("float", EMPTY_BYTES));
+
+            // as double
+            assertEquals(0, pc.getInteger("double", 0));
+            assertEquals(0L, pc.getLong("double", 0));
+            assertEquals(3.141592f, pc.getFloat("double", 0), 0.000001);
+            assertEquals(Math.PI, pc.getDouble("double", 0), 0.0);
+            assertEquals(false, pc.getBoolean("double", true));
+            assertTrue(pc.getString("double", "0").startsWith("3.1415926535"));
+            assertArrayEquals(EMPTY_BYTES, pc.getBytes("double", EMPTY_BYTES));
+
+            // as negative double
+            assertEquals(0, pc.getInteger("negative_double", 0));
+            assertEquals(0L, pc.getLong("negative_double", 0));
+            assertEquals(-1f, pc.getFloat("negative_double", 0), 0.000001);
+            assertEquals(-1, pc.getDouble("negative_double", 0), 0.0);
+            assertEquals(false, pc.getBoolean("negative_double", true));
+            assertTrue(pc.getString("negative_double", "0").startsWith("-1"));
+            assertArrayEquals(EMPTY_BYTES, pc.getBytes("negative_double", EMPTY_BYTES));
+
+            // as zero
+            assertEquals(-1, pc.getInteger("zero", -1));
+            assertEquals(-1L, pc.getLong("zero", -1));
+            assertEquals(0f, pc.getFloat("zero", -1), 0.000001);
+            assertEquals(0.0, pc.getDouble("zero", -1), 0.0);
+            assertEquals(false, pc.getBoolean("zero", true));
+            assertTrue(pc.getString("zero", "-1").startsWith("0"));
+            assertArrayEquals(EMPTY_BYTES, pc.getBytes("zero", EMPTY_BYTES));
+
+            // as too long double
+            assertEquals(0, pc.getInteger("too_long_double", 0));
+            assertEquals(0L, pc.getLong("too_long_double", 0));
+            assertEquals(0f, pc.getFloat("too_long_double", 0f), 0.000001);
+            assertEquals(TOO_LONG_DOUBLE, pc.getDouble("too_long_double", 0), 0.0);
+            assertEquals(false, pc.getBoolean("too_long_double", true));
+            assertEquals(String.valueOf(TOO_LONG_DOUBLE), pc.getString("too_long_double", "0"));
+            assertArrayEquals(EMPTY_BYTES, pc.getBytes("too_long_double", EMPTY_BYTES));
+
+            // as string
+            assertEquals(42, pc.getInteger("string", 0));
+            assertEquals(42L, pc.getLong("string", 0));
+            assertEquals(42f, pc.getFloat("string", 0f), 0.000001);
+            assertEquals(42.0, pc.getDouble("string", 0), 0.0);
+            assertEquals(false, pc.getBoolean("string", true));
+            assertEquals("42", pc.getString("string", "0"));
+            assertArrayEquals(EMPTY_BYTES, pc.getBytes("string", EMPTY_BYTES));
+
+            // as non convertible string
+            assertEquals(0, pc.getInteger("non_convertible_string", 0));
+            assertEquals(0L, pc.getLong("non_convertible_string", 0));
+            assertEquals(0f, pc.getFloat("non_convertible_string", 0f), 0.000001);
+            assertEquals(0.0, pc.getDouble("non_convertible_string", 0), 0.0);
+            assertEquals(false, pc.getBoolean("non_convertible_string", true));
+            assertEquals("bcdefg&&", pc.getString("non_convertible_string", "0"));
+            assertArrayEquals(EMPTY_BYTES, pc.getBytes("non_convertible_string", EMPTY_BYTES));
+
+            // as boolean
+            assertEquals(0, pc.getInteger("boolean", 0));
+            assertEquals(0L, pc.getLong("boolean", 0));
+            assertEquals(0f, pc.getFloat("boolean", 0f), 0.000001);
+            assertEquals(0.0, pc.getDouble("boolean", 0), 0.0);
+            assertEquals(true, pc.getBoolean("boolean", false));
+            assertEquals("true", pc.getString("boolean", "0"));
+            assertArrayEquals(EMPTY_BYTES, pc.getBytes("boolean", EMPTY_BYTES));
+        } catch (Exception e) {
+            e.printStackTrace();
+            fail(e.getMessage());
+        }
+    }
+
+    @Test
+    public void testCopyConstructor() {
+        try {
+            final String key = "theKey";
+
+            Configuration cfg1 = new Configuration();
+            cfg1.setString(key, "value");
+
+            Configuration cfg2 = new Configuration(cfg1);
+            cfg2.setString(key, "another value");
+
+            assertEquals("value", cfg1.getString(key, ""));
+        } catch (Exception e) {
+            e.printStackTrace();
+            fail(e.getMessage());
+        }
+    }
+
+    @Test
+    public void testOptionWithDefault() {
+        Configuration cfg = new Configuration();
+        cfg.setInteger("int-key", 11);
+        cfg.setString("string-key", "abc");
+
+        ConfigOption<String> presentStringOption = ConfigOptions.key("string-key")
+                .defaultValue("my-beautiful-default");
+        ConfigOption<Integer> presentIntOption = ConfigOptions.key("int-key").defaultValue(87);
+
+        assertEquals("abc", cfg.getString(presentStringOption));
+        assertEquals("abc", cfg.getValue(presentStringOption));
+
+        assertEquals(11, cfg.getInteger(presentIntOption));
+        assertEquals("11", cfg.getValue(presentIntOption));
+
+        // test getting default when no value is present
+
+        ConfigOption<String> stringOption = ConfigOptions.key("test")
+                .defaultValue("my-beautiful-default");
+        final ConfigOption<Integer> intOption = ConfigOptions.key("test2").defaultValue(87);
+
+        // getting strings with default value should work
+        assertEquals("my-beautiful-default", cfg.getValue(stringOption));
+        assertEquals("my-beautiful-default", cfg.getString(stringOption));
+
+        // overriding the default should work
+        assertEquals("override", cfg.getString(stringOption, "override"));
+
+        // getting a primitive with a default value should work
+        assertEquals(87, cfg.getInteger(intOption));
+        assertEquals("87", cfg.getValue(intOption));
+    }
+
+    @Test
+    public void testOptionWithNoDefault() {
+        Configuration cfg = new Configuration();
+        cfg.setInteger("int-key", 11);
+        cfg.setString("string-key", "abc");
+
+        ConfigOption<String> presentStringOption = ConfigOptions.key("string-key").noDefaultValue();
+
+        assertEquals("abc", cfg.getString(presentStringOption));
+        assertEquals("abc", cfg.getValue(presentStringOption));
+
+        // test getting default when no value is present
+
+        ConfigOption<String> stringOption = ConfigOptions.key("test").noDefaultValue();
+
+        // getting strings for null should work
+        assertNull(cfg.getValue(stringOption));
+        assertNull(cfg.getString(stringOption));
+
+        // overriding the null default should work
+        assertEquals("override", cfg.getString(stringOption, "override"));
+    }
+
+    @Test
+    public void testDeprecatedKeys() {
+        Configuration cfg = new Configuration();
+        cfg.setInteger("the-key", 11);
+        cfg.setInteger("old-key", 12);
+        cfg.setInteger("older-key", 13);
+
+        ConfigOption<Integer> matchesFirst = ConfigOptions
+                .key("the-key")
+                .defaultValue(-1)
+                .withDeprecatedKeys("old-key", "older-key");
+
+        ConfigOption<Integer> matchesSecond = ConfigOptions
+                .key("does-not-exist")
+                .defaultValue(-1)
+                .withDeprecatedKeys("old-key", "older-key");
+
+        ConfigOption<Integer> matchesThird = ConfigOptions
+                .key("does-not-exist")
+                .defaultValue(-1)
+                .withDeprecatedKeys("foo", "older-key");
+
+        final ConfigOption<Integer> notContained = ConfigOptions
+                .key("does-not-exist")
+                .defaultValue(-1)
+                .withDeprecatedKeys("not-there", "also-not-there");
+
+        assertEquals(11, cfg.getInteger(matchesFirst));
+        assertEquals(12, cfg.getInteger(matchesSecond));
+        assertEquals(13, cfg.getInteger(matchesThird));
+        assertEquals(-1, cfg.getInteger(notContained));
+    }
+
+    @Test
+    public void testFallbackKeys() {
+        Configuration cfg = new Configuration();
+        cfg.setInteger("the-key", 11);
+        cfg.setInteger("old-key", 12);
+        cfg.setInteger("older-key", 13);
+
+        ConfigOption<Integer> matchesFirst = ConfigOptions
+                .key("the-key")
+                .defaultValue(-1)
+                .withFallbackKeys("old-key", "older-key");
+
+        ConfigOption<Integer> matchesSecond = ConfigOptions
+                .key("does-not-exist")
+                .defaultValue(-1)
+                .withFallbackKeys("old-key", "older-key");
+
+        ConfigOption<Integer> matchesThird = ConfigOptions
+                .key("does-not-exist")
+                .defaultValue(-1)
+                .withFallbackKeys("foo", "older-key");
+
+        final ConfigOption<Integer> notContained = ConfigOptions
+                .key("does-not-exist")
+                .defaultValue(-1)
+                .withFallbackKeys("not-there", "also-not-there");
+
+        assertEquals(11, cfg.getInteger(matchesFirst));
+        assertEquals(12, cfg.getInteger(matchesSecond));
+        assertEquals(13, cfg.getInteger(matchesThird));
+        assertEquals(-1, cfg.getInteger(notContained));
+    }
+
+    @Test
+    public void testFallbackAndDeprecatedKeys() {
+        final ConfigOption<Integer> fallback = ConfigOptions
+                .key("fallback")
+                .defaultValue(-1);
+
+        final ConfigOption<Integer> deprecated = ConfigOptions
+                .key("deprecated")
+                .defaultValue(-1);
+
+        final ConfigOption<Integer> mainOption = ConfigOptions
+                .key("main")
+                .defaultValue(-1)
+                .withFallbackKeys(fallback.key())
+                .withDeprecatedKeys(deprecated.key());
+
+        final Configuration fallbackCfg = new Configuration();
+        fallbackCfg.setInteger(fallback, 1);
+        assertEquals(1, fallbackCfg.getInteger(mainOption));
+
+        final Configuration deprecatedCfg = new Configuration();
+        deprecatedCfg.setInteger(deprecated, 2);
+        assertEquals(2, deprecatedCfg.getInteger(mainOption));
+
+        // reverse declaration of fallback and deprecated keys, fallback keys should always be
+        // used first
+        final ConfigOption<Integer> reversedMainOption = ConfigOptions
+                .key("main")
+                .defaultValue(-1)
+                .withDeprecatedKeys(deprecated.key())
+                .withFallbackKeys(fallback.key());
+
+        final Configuration deprecatedAndFallBackConfig = new Configuration();
+        deprecatedAndFallBackConfig.setInteger(fallback, 1);
+        deprecatedAndFallBackConfig.setInteger(deprecated, 2);
+        assertEquals(1, deprecatedAndFallBackConfig.getInteger(mainOption));
+        assertEquals(1, deprecatedAndFallBackConfig.getInteger(reversedMainOption));
+    }
+
+    @Test
+    public void testRemove() {
+        Configuration cfg = new Configuration();
+        cfg.setInteger("a", 1);
+        cfg.setInteger("b", 2);
+
+        ConfigOption<Integer> validOption = ConfigOptions
+                .key("a")
+                .defaultValue(-1);
+
+        final ConfigOption<Integer> deprecatedOption = ConfigOptions
+                .key("c")
+                .defaultValue(-1)
+                .withDeprecatedKeys("d", "b");
+
+        final ConfigOption<Integer> unexistedOption = ConfigOptions
+                .key("e")
+                .defaultValue(-1)
+                .withDeprecatedKeys("f", "g", "j");
+
+        assertEquals("Wrong expectation about size", cfg.keySet().size(), 2);
+        assertTrue("Expected 'validOption' is removed", cfg.removeConfig(validOption));
+        assertEquals("Wrong expectation about size", cfg.keySet().size(), 1);
+        assertTrue("Expected 'existedOption' is removed", cfg.removeConfig(deprecatedOption));
+        assertEquals("Wrong expectation about size", cfg.keySet().size(), 0);
+        assertFalse("Expected 'unexistedOption' is not removed", cfg.removeConfig(unexistedOption));
+    }
+
+    @Test
+    public void testShouldParseValidStringToEnum() {
+        final ConfigOption<String> configOption = createStringConfigOption();
+
+        final Configuration configuration = new Configuration();
+        configuration.setString(configOption.key(), TestEnum.VALUE1.toString());
+
+        final TestEnum parsedEnumValue = configuration.getEnum(TestEnum.class, configOption);
+        assertEquals(TestEnum.VALUE1, parsedEnumValue);
+    }
+
+    @Test
+    public void testShouldParseValidStringToEnumIgnoringCase() {
+        final ConfigOption<String> configOption = createStringConfigOption();
+
+        final Configuration configuration = new Configuration();
+        configuration.setString(configOption.key(), TestEnum.VALUE1.toString().toLowerCase());
+
+        final TestEnum parsedEnumValue = configuration.getEnum(TestEnum.class, configOption);
+        assertEquals(TestEnum.VALUE1, parsedEnumValue);
+    }
+
+    @Test
+    public void testThrowsExceptionIfTryingToParseInvalidStringForEnum() {
+        final ConfigOption<String> configOption = createStringConfigOption();
+
+        final Configuration configuration = new Configuration();
+        final String invalidValueForTestEnum = "InvalidValueForTestEnum";
+        configuration.setString(configOption.key(), invalidValueForTestEnum);
+
+        try {
+            configuration.getEnum(TestEnum.class, configOption);
+            fail("Expected exception not thrown");
+        } catch (IllegalArgumentException e) {
+            final String expectedMessage = "Value for config option "
+                    + configOption.key() + " must be one of [VALUE1, VALUE2] (was "
+                    + invalidValueForTestEnum + ")";
+            assertThat(e.getMessage(), containsString(expectedMessage));
+        }
+    }
+
+    enum TestEnum {
+        VALUE1,
+        VALUE2
+    }
+
+    private static ConfigOption<String> createStringConfigOption() {
+        return ConfigOptions
+                .key("test-string-key")
+                .noDefaultValue();
+    }
+}
diff --git a/inlong-sort/common/src/test/java/org/apache/inlong/sort/protocol/DeserializationInfoTest.java b/inlong-sort/common/src/test/java/org/apache/inlong/sort/protocol/DeserializationInfoTest.java
new file mode 100644
index 0000000..a09da7f
--- /dev/null
+++ b/inlong-sort/common/src/test/java/org/apache/inlong/sort/protocol/DeserializationInfoTest.java
@@ -0,0 +1,36 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.inlong.sort.protocol.deserialization.CsvDeserializationInfo;
+import java.io.IOException;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.junit.Test;
+
+public class DeserializationInfoTest {
+    @Test
+    public void testCsvDeserializationInfo() throws IOException {
+        ObjectMapper objectMapper = new ObjectMapper();
+        String str = "{\"type\":\"csv\",\"splitter\":\"\\u0026\"}";
+        CsvDeserializationInfo deserializationInfo = objectMapper.readValue(
+                str.getBytes(), CsvDeserializationInfo.class);
+        assertEquals('&', deserializationInfo.getSplitter());
+    }
+}
diff --git a/inlong-sort/common/src/test/java/org/apache/inlong/sort/protocol/FieldInfoTest.java b/inlong-sort/common/src/test/java/org/apache/inlong/sort/protocol/FieldInfoTest.java
new file mode 100644
index 0000000..4c086ec
--- /dev/null
+++ b/inlong-sort/common/src/test/java/org/apache/inlong/sort/protocol/FieldInfoTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.inlong.sort.formats.common.StringFormatInfo;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.junit.Test;
+
+public class FieldInfoTest {
+    @Test
+    public void testSerialize() throws JsonProcessingException {
+        FieldInfo fieldInfo = new FieldInfo("field_name", StringFormatInfo.INSTANCE);
+        ObjectMapper objectMapper = new ObjectMapper();
+        String expected = "{\"name\":\"field_name\",\"format_info\":{\"type\":\"string\"}}";
+        assertEquals(expected, objectMapper.writeValueAsString(fieldInfo));
+    }
+}
diff --git a/inlong-sort/core/pom.xml b/inlong-sort/core/pom.xml
new file mode 100644
index 0000000..f3bc7f8
--- /dev/null
+++ b/inlong-sort/core/pom.xml
@@ -0,0 +1,211 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<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">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.inlong</groupId>
+        <artifactId>inlong-sort</artifactId>
+        <version>1.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>inlong-sort-core</artifactId>
+    <name>Apache InLong - Sort Core</name>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>inlong-sort-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>tubemq-client</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-core</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <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>
+            <version>${flink.version}</version>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-recipes</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-test</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>ru.yandex.clickhouse</groupId>
+            <artifactId>clickhouse-jdbc</artifactId>
+            <version>${clickhouse-jdbc.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>inlong-sort-format-base</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>inlong-sort-format-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>inlong-sort-format-csv</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>inlong-sort-format-kv</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>inlong-sort-format-tdmsg-base</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>inlong-sort-format-tdmsg-csv</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-minicluster</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-metastore</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.logging.log4j</groupId>
+                    <artifactId>log4j-slf4j-impl</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-exec</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>commons-codec</groupId>
+                    <artifactId>commons-codec</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.commons</groupId>
+                    <artifactId>commons-compress</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.logging.log4j</groupId>
+                    <artifactId>log4j-slf4j-impl</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-parquet_2.11</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+        <!--<dependency>
+            <groupId>org.apache.parquet</groupId>
+            <artifactId>parquet-hadoop-bundle</artifactId>
+            <version>1.10.0</version>
+        </dependency>-->
+
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>inlong-sort-api</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.xerial.snappy</groupId>
+            <artifactId>snappy-java</artifactId>
+            <version>${snappy.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.inlong</groupId>
+            <artifactId>inlong-common</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+
+</project>
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/Entrance.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/Entrance.java
new file mode 100644
index 0000000..bc8cab7
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/Entrance.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.flink;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.inlong.sort.configuration.Constants.SINK_TYPE_HIVE;
+
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.inlong.sort.configuration.Configuration;
+import org.apache.inlong.sort.configuration.Constants;
+import org.apache.inlong.sort.flink.clickhouse.ClickHouseMultiSinkFunction;
+import org.apache.inlong.sort.flink.deserialization.DeserializationSchema;
+import org.apache.inlong.sort.flink.hive.HiveMultiTenantCommitter;
+import org.apache.inlong.sort.flink.hive.HiveMultiTenantWriter;
+import org.apache.inlong.sort.flink.tubemq.MultiTopicTubeSourceFunction;
+import org.apache.inlong.sort.util.ParameterTool;
+
+public class Entrance {
+
+    /**
+     * Entrance of a flink job.
+     */
+    public static void main(String[] args) throws Exception {
+        final ParameterTool parameter = ParameterTool.fromArgs(args);
+        final Configuration config = parameter.getConfiguration();
+
+        final String clusterId = checkNotNull(config.getString(Constants.CLUSTER_ID));
+        final String sourceType = checkNotNull(config.getString(Constants.SOURCE_TYPE));
+        final String sinkType = checkNotNull(config.getString(Constants.SINK_TYPE));
+
+        final int sourceParallelism = config.getInteger(Constants.SOURCE_PARALLELISM);
+        final int deserializationParallelism = config.getInteger(Constants.DESERIALIZATION_PARALLELISM);
+        final int sinkParallelism = config.getInteger(Constants.SINK_PARALLELISM);
+
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
+        DataStream<SerializedRecord> sourceStream;
+        if (sourceType.equals(Constants.SOURCE_TYPE_TUBE)) {
+            sourceStream = env
+                    .addSource(new MultiTopicTubeSourceFunction(config))
+                    .setParallelism(sourceParallelism)
+                    .uid(Constants.SOURCE_UID)
+                    .name("TubeMQ source")
+                    .rebalance(); // there might be data lean of source, so re-balance it
+        } else {
+            throw new IllegalArgumentException("Unsupported source type " + sourceType);
+        }
+
+        final SingleOutputStreamOperator<SerializedRecord> deserializationStream = sourceStream
+               .process(new DeserializationSchema(config))
+               .setParallelism(deserializationParallelism)
+               .uid(Constants.DESERIALIZATION_SCHEMA_UID)
+               .name("Deserialization");
+
+        if (sinkType.equals(Constants.SINK_TYPE_CLICKHOUSE)) {
+            deserializationStream
+                    .process(new ClickHouseMultiSinkFunction(config))
+                    .setParallelism(sinkParallelism)
+                    .uid(Constants.SINK_UID)
+                    .name("Clickhouse Sink");
+        } else if (sinkType.equals(SINK_TYPE_HIVE)) {
+            deserializationStream
+                    .process(new HiveMultiTenantWriter(config))
+                    .name("Hive Sink")
+                    .uid(Constants.SINK_UID)
+                    .setParallelism(sinkParallelism)
+                    .process(new HiveMultiTenantCommitter(config))
+                    .name("hive Committer")
+                    .setParallelism(config.getInteger(Constants.COMMITTER_PARALLELISM));
+        } else {
+            throw new IllegalArgumentException("Unsupported sink type " + sinkType);
+        }
+
+        env.execute(clusterId);
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/Record.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/Record.java
new file mode 100644
index 0000000..3b8707c
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/Record.java
@@ -0,0 +1,78 @@
+/*
+ * 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.flink;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.Serializable;
+import java.util.Objects;
+import org.apache.flink.types.Row;
+
+/**
+ * Represents a row with data flow id attached.
+ */
+public class Record implements Serializable {
+
+    private static final long serialVersionUID = 403818036307888751L;
+
+    private long dataflowId;
+
+    private Row row;
+
+    /**
+     * Just to satisfy the requirement of Flink Pojo definition.
+     */
+    public Record() {
+
+    }
+
+    public Record(long dataflowId, Row row) {
+        this.dataflowId = dataflowId;
+        this.row = checkNotNull(row);
+    }
+
+    public long getDataflowId() {
+        return dataflowId;
+    }
+
+    public void setDataflowId(long dataflowId) {
+        this.dataflowId = dataflowId;
+    }
+
+    public Row getRow() {
+        return row;
+    }
+
+    public void setRow(Row row) {
+        this.row = row;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        Record that = (Record) o;
+        return dataflowId == ((Record) o).dataflowId && Objects.equals(row, that.row);
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/SerializedRecord.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/SerializedRecord.java
new file mode 100644
index 0000000..1e8ad87
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/SerializedRecord.java
@@ -0,0 +1,57 @@
+/*
+ * 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.flink;
+
+import java.io.Serializable;
+
+public class SerializedRecord implements Serializable {
+
+    private static final long serialVersionUID = 5418156417016358730L;
+
+    private long dataFlowId;
+
+    private byte[] data;
+
+    /**
+     * Just satisfy requirement of Flink Pojo definition.
+     */
+    public SerializedRecord() {
+
+    }
+
+    public SerializedRecord(long dataFlowId, byte[] data) {
+        this.dataFlowId = dataFlowId;
+        this.data = data;
+    }
+
+    public void setDataFlowId(long dataFlowId) {
+        this.dataFlowId = dataFlowId;
+    }
+
+    public void setData(byte[] data) {
+        this.data = data;
+    }
+
+    public long getDataFlowId() {
+        return dataFlowId;
+    }
+
+    public byte[] getData() {
+        return data;
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/TDMsgSerializedRecord.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/TDMsgSerializedRecord.java
new file mode 100644
index 0000000..3928d76
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/TDMsgSerializedRecord.java
@@ -0,0 +1,50 @@
+/*
+ * 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.flink;
+
+import static org.apache.inlong.sort.configuration.Constants.UNKNOWN_DATAFLOW_ID;
+
+/**
+ * Data flow id might not been got from mixed TDMsg data stream.
+ */
+public class TDMsgSerializedRecord extends SerializedRecord {
+
+    private static final long serialVersionUID = 4075321919886376829L;
+
+    private String topic;
+
+    /**
+     * Just satisfy requirement of Flink Pojo definition.
+     */
+    public TDMsgSerializedRecord() {
+        super();
+    }
+
+    public TDMsgSerializedRecord(String topic, byte[] data) {
+        super(UNKNOWN_DATAFLOW_ID, data);
+        this.topic = topic;
+    }
+
+    public String getTopic() {
+        return topic;
+    }
+
+    public void setTopic(String topic) {
+        this.topic = topic;
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseConnectionProvider.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseConnectionProvider.java
new file mode 100644
index 0000000..5e9e86b
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseConnectionProvider.java
@@ -0,0 +1,172 @@
+/*
+ * 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.flink.clickhouse;
+
+import org.apache.inlong.sort.protocol.sink.ClickHouseSinkInfo;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import ru.yandex.clickhouse.ClickHouseConnection;
+
+import java.io.Serializable;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class ClickHouseConnectionProvider implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(ClickHouseConnectionProvider.class);
+
+    private static final String CLICKHOUSE_DRIVER_NAME = "ru.yandex.clickhouse.ClickHouseDriver";
+
+    private static final Pattern PATTERN = Pattern.compile("You must use port (?<port>[0-9]+) for HTTP.");
+
+    private final ClickHouseSinkInfo clickHouseSinkInfo;
+
+    private transient ClickHouseConnection connection;
+
+    private transient List<ClickHouseConnection> shardConnections;
+
+    public ClickHouseConnectionProvider(ClickHouseSinkInfo clickHouseSinkInfo) {
+        this.clickHouseSinkInfo = clickHouseSinkInfo;
+    }
+
+    public synchronized ClickHouseConnection getConnection() throws Exception {
+        if (connection == null) {
+            connection = createConnection(clickHouseSinkInfo.getUrl(), clickHouseSinkInfo.getDatabaseName());
+        }
+        return connection;
+    }
+
+    private ClickHouseConnection createConnection(String url, String database) throws Exception {
+        LOG.info("Try connect to {}", url);
+
+        ClickHouseConnection conn;
+        Class.forName(CLICKHOUSE_DRIVER_NAME);
+
+        if (clickHouseSinkInfo.getUsername() != null
+                && clickHouseSinkInfo.getPassword() != null) {
+            conn = (ClickHouseConnection) DriverManager.getConnection(
+                    getJdbcUrl(url, database),
+                    clickHouseSinkInfo.getUsername(),
+                    clickHouseSinkInfo.getPassword());
+        } else {
+            conn = (ClickHouseConnection) DriverManager.getConnection(getJdbcUrl(url, database));
+        }
+
+        return conn;
+    }
+
+    /**
+     * Get connection for the specified shard.
+     */
+    public synchronized List<ClickHouseConnection> getShardConnections(
+            String remoteCluster, String remoteDatabase) throws Exception {
+        if (shardConnections == null) {
+            ClickHouseConnection conn = getConnection();
+            PreparedStatement stmt = conn.prepareStatement(
+                    "SELECT shard_num, host_address, port FROM system.clusters WHERE cluster = ?");
+            stmt.setString(1, remoteCluster);
+
+            try (ResultSet rs = stmt.executeQuery()) {
+                shardConnections = new ArrayList<>();
+                while (rs.next()) {
+                    String host = rs.getString("host_address");
+                    int port = getActualHttpPort(host, rs.getInt("port"));
+                    String url = "clickhouse://" + host + ":" + port;
+                    shardConnections.add(createConnection(url, remoteDatabase));
+                }
+            }
+
+            if (shardConnections.isEmpty()) {
+                throw new SQLException("unable to query shards in system.clusters");
+            }
+        }
+        return shardConnections;
+    }
+
+    private int getActualHttpPort(String host, int port) throws SQLException {
+        try (CloseableHttpClient httpclient = HttpClients.createDefault()) {
+            HttpGet request = new HttpGet((new URIBuilder()).setScheme("http").setHost(host).setPort(port).build());
+            CloseableHttpResponse closeableHttpResponse = httpclient.execute(request);
+            int statusCode = closeableHttpResponse.getStatusLine().getStatusCode();
+            if (statusCode == 200) {
+                return port;
+            }
+
+            String raw = EntityUtils.toString(closeableHttpResponse.getEntity());
+            Matcher matcher = PATTERN.matcher(raw);
+            if (matcher.find()) {
+                return Integer.parseInt(matcher.group("port"));
+            }
+
+            throw new SQLException("Cannot query ClickHouse http port");
+        } catch (Exception e) {
+            throw new SQLException("Cannot connect to ClickHouse server using HTTP", e);
+        }
+    }
+
+    public void closeConnections() throws SQLException {
+        if (connection != null) {
+            connection.close();
+        }
+
+        if (shardConnections != null) {
+            for (ClickHouseConnection shardConnection : shardConnections) {
+                shardConnection.close();
+            }
+        }
+    }
+
+    private String getJdbcUrl(String url, String database) throws SQLException {
+        try {
+            return "jdbc:" + (new URIBuilder(url)).setPath("/" + database).build().toString();
+        } catch (Exception e) {
+            throw new SQLException(e);
+        }
+    }
+
+    public String queryTableEngine(String databaseName, String tableName) throws Exception {
+        ClickHouseConnection conn = getConnection();
+        try (PreparedStatement stmt = conn.prepareStatement(
+                "SELECT engine_full FROM system.tables WHERE database = ? AND name = ?")) {
+            stmt.setString(1, databaseName);
+            stmt.setString(2, tableName);
+            try (ResultSet rs = stmt.executeQuery()) {
+                if (rs.next()) {
+                    return rs.getString("engine_full");
+                }
+            }
+        }
+        throw new SQLException("table `" + databaseName + "`.`" + tableName + "` does not exist");
+    }
+}
+
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseMultiSinkFunction.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseMultiSinkFunction.java
new file mode 100644
index 0000000..3077258
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseMultiSinkFunction.java
@@ -0,0 +1,165 @@
+/*
+ * 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.flink.clickhouse;
+
+import com.google.common.base.Preconditions;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.inlong.sort.configuration.Configuration;
+import org.apache.inlong.sort.configuration.Constants;
+import org.apache.inlong.sort.flink.SerializedRecord;
+import org.apache.inlong.sort.flink.transformation.RecordTransformer;
+import org.apache.inlong.sort.meta.MetaManager;
+import org.apache.inlong.sort.meta.MetaManager.DataFlowInfoListener;
+import org.apache.inlong.sort.protocol.DataFlowInfo;
+import org.apache.inlong.sort.protocol.sink.ClickHouseSinkInfo;
+import org.apache.inlong.sort.protocol.sink.SinkInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClickHouseMultiSinkFunction extends ProcessFunction<SerializedRecord, Void>
+        implements CheckpointedFunction {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ClickHouseMultiSinkFunction.class);
+
+    private final Configuration configuration;
+
+    private transient Object lock;
+
+    private transient Map<Long, ClickHouseSinkFunction> clickHouseSinkFunctionMap;
+
+    private transient RecordTransformer recordTransformer;
+
+    public ClickHouseMultiSinkFunction(Configuration configuration) {
+        this.configuration = Preconditions.checkNotNull(configuration);
+    }
+
+    @Override
+    public void open(org.apache.flink.configuration.Configuration parameters) throws Exception {
+        lock = new Object();
+        clickHouseSinkFunctionMap = new HashMap<>();
+        recordTransformer = new RecordTransformer(
+                configuration.getInteger(Constants.ETL_RECORD_SERIALIZATION_BUFFER_SIZE));
+        MetaManager metaManager = MetaManager.getInstance(configuration);
+        metaManager.registerDataFlowInfoListener(new DataFlowInfoListenerImpl());
+    }
+
+    @Override
+    public void close() throws Exception {
+        MetaManager.release();
+
+        synchronized (lock) {
+            if (clickHouseSinkFunctionMap != null) {
+                for (ClickHouseSinkFunction clickHouseSinkFunction : clickHouseSinkFunctionMap.values()) {
+                    clickHouseSinkFunction.close();
+                }
+                clickHouseSinkFunctionMap.clear();
+                clickHouseSinkFunctionMap = null;
+            }
+        }
+    }
+
+    @Override
+    public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throws Exception {
+        synchronized (lock) {
+            for (ClickHouseSinkFunction clickHouseSinkFunction : clickHouseSinkFunctionMap.values()) {
+                clickHouseSinkFunction.snapshotState(functionSnapshotContext);
+            }
+        }
+    }
+
+    @Override
+    public void initializeState(FunctionInitializationContext functionInitializationContext) {
+    }
+
+    @Override
+    public void processElement(SerializedRecord serializedRecord, Context context, Collector<Void> collector)
+            throws Exception {
+        final long dataFlowId = serializedRecord.getDataFlowId();
+
+        synchronized (lock) {
+            ClickHouseSinkFunction clickHouseSinkFunction = clickHouseSinkFunctionMap.get(dataFlowId);
+            if (clickHouseSinkFunction == null) {
+                LOG.warn("Cannot get DataFlowInfo with id {}", dataFlowId);
+                return;
+            }
+
+            clickHouseSinkFunction.invoke(new Tuple2<>(false, recordTransformer.toRecord(serializedRecord).getRow()));
+        }
+    }
+
+    private class DataFlowInfoListenerImpl implements DataFlowInfoListener {
+
+        @Override
+        public void addDataFlow(DataFlowInfo dataFlowInfo) throws Exception {
+            synchronized (lock) {
+                addOrUpdateClickHouseSinkFunction(dataFlowInfo);
+                recordTransformer.addDataFlow(dataFlowInfo);
+            }
+        }
+
+        @Override
+        public void updateDataFlow(DataFlowInfo dataFlowInfo) throws Exception {
+            synchronized (lock) {
+                addOrUpdateClickHouseSinkFunction(dataFlowInfo);
+                recordTransformer.updateDataFlow(dataFlowInfo);
+            }
+        }
+
+        @Override
+        public void removeDataFlow(DataFlowInfo dataFlowInfo) throws Exception {
+            synchronized (lock) {
+                // Close the old one
+                ClickHouseSinkFunction oldClickHouseSinkFunction
+                        = clickHouseSinkFunctionMap.remove(dataFlowInfo.getId());
+                if (oldClickHouseSinkFunction != null) {
+                    oldClickHouseSinkFunction.close();
+                }
+
+                recordTransformer.removeDataFlow(dataFlowInfo);
+            }
+        }
+
+        private void addOrUpdateClickHouseSinkFunction(DataFlowInfo dataFlowInfo) throws Exception {
+            long dataFlowId = dataFlowInfo.getId();
+            SinkInfo sinkInfo = dataFlowInfo.getSinkInfo();
+            if (!(sinkInfo instanceof ClickHouseSinkInfo)) {
+                LOG.error("SinkInfo type {} of dataFlow {} doesn't match application sink type 'clickhouse'!",
+                        sinkInfo.getClass(), dataFlowId);
+                return;
+            }
+            ClickHouseSinkInfo clickHouseSinkInfo = (ClickHouseSinkInfo) sinkInfo;
+            ClickHouseSinkFunction clickHouseSinkFunction = new ClickHouseSinkFunction(clickHouseSinkInfo);
+            clickHouseSinkFunction.setRuntimeContext(getRuntimeContext());
+            clickHouseSinkFunction.open(new org.apache.flink.configuration.Configuration());
+
+            // Close the old one if exist
+            ClickHouseSinkFunction oldClickHouseSinkFunction
+                    = clickHouseSinkFunctionMap.put(dataFlowId, clickHouseSinkFunction);
+            if (oldClickHouseSinkFunction != null) {
+                oldClickHouseSinkFunction.close();
+            }
+        }
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseRowConverter.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseRowConverter.java
new file mode 100644
index 0000000..08f9756
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseRowConverter.java
@@ -0,0 +1,90 @@
+/*
+ * 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.flink.clickhouse;
+
+import org.apache.inlong.sort.formats.common.BooleanTypeInfo;
+import org.apache.inlong.sort.formats.common.ByteTypeInfo;
+import org.apache.inlong.sort.formats.common.DateTypeInfo;
+import org.apache.inlong.sort.formats.common.DecimalTypeInfo;
+import org.apache.inlong.sort.formats.common.DoubleTypeInfo;
+import org.apache.inlong.sort.formats.common.FloatTypeInfo;
+import org.apache.inlong.sort.formats.common.FormatInfo;
+import org.apache.inlong.sort.formats.common.IntTypeInfo;
+import org.apache.inlong.sort.formats.common.LongTypeInfo;
+import org.apache.inlong.sort.formats.common.ShortTypeInfo;
+import org.apache.inlong.sort.formats.common.StringTypeInfo;
+import org.apache.inlong.sort.formats.common.TimeTypeInfo;
+import org.apache.inlong.sort.formats.common.TimestampTypeInfo;
+import org.apache.inlong.sort.formats.common.TypeInfo;
+import org.apache.flink.types.Row;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+
+public class ClickHouseRowConverter {
+
+    public static void setRow(
+            PreparedStatement statement,
+            FormatInfo[] formatInfos,
+            Row row) throws SQLException {
+        for (int i = 0; i < row.getArity(); ++i) {
+            setField(statement, i, formatInfos[i], row.getField(i));
+        }
+    }
+
+    private static void setField(
+            PreparedStatement statement,
+            int index,
+            FormatInfo formatInfo,
+            Object value) throws SQLException {
+        TypeInfo typeInfo = formatInfo.getTypeInfo();
+
+        if (typeInfo instanceof StringTypeInfo) {
+            statement.setString(index + 1, (String) value);
+        } else if (typeInfo instanceof BooleanTypeInfo) {
+            statement.setBoolean(index + 1, (Boolean) value);
+        } else if (typeInfo instanceof ByteTypeInfo) {
+            statement.setByte(index + 1, (Byte) value);
+        } else if (typeInfo instanceof ShortTypeInfo) {
+            statement.setShort(index + 1, (Short) value);
+        } else if (typeInfo instanceof IntTypeInfo) {
+            statement.setInt(index + 1, (Integer) value);
+        } else if (typeInfo instanceof LongTypeInfo) {
+            statement.setLong(index + 1, (Long) value);
+        } else if (typeInfo instanceof FloatTypeInfo) {
+            statement.setFloat(index + 1, (Float) value);
+        } else if (typeInfo instanceof DoubleTypeInfo) {
+            statement.setDouble(index + 1, (Double) value);
+        } else if (typeInfo instanceof DecimalTypeInfo) {
+            statement.setBigDecimal(index + 1, (BigDecimal) value);
+        } else if (typeInfo instanceof TimeTypeInfo) {
+            statement.setTime(index + 1, (Time) value);
+        } else if (typeInfo instanceof DateTypeInfo) {
+            statement.setDate(index + 1, (Date) value);
+        } else if (typeInfo instanceof TimestampTypeInfo) {
+            statement.setTimestamp(index + 1, (Timestamp) value);
+        } else {
+            throw new IllegalArgumentException("Unsupported TypeInfo " + typeInfo.getClass().getName());
+        }
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseSinkFunction.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseSinkFunction.java
new file mode 100644
index 0000000..fbedd7b
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseSinkFunction.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.flink.clickhouse;
+
+import com.google.common.base.Preconditions;
+import org.apache.inlong.sort.formats.common.FormatInfo;
+import org.apache.inlong.sort.flink.clickhouse.output.AbstractClickHouseOutputFormat;
+import org.apache.inlong.sort.protocol.FieldInfo;
+import org.apache.inlong.sort.protocol.sink.ClickHouseSinkInfo;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.types.Row;
+import org.apache.inlong.sort.flink.clickhouse.output.ClickHouseOutputFormatFactory;
+
+public class ClickHouseSinkFunction extends RichSinkFunction<Tuple2<Boolean, Row>> implements CheckpointedFunction {
+
+    private static final long serialVersionUID = 2738357054183678956L;
+
+    private final ClickHouseSinkInfo clickHouseSinkInfo;
+
+    private transient AbstractClickHouseOutputFormat outputFormat;
+
+    public ClickHouseSinkFunction(
+            ClickHouseSinkInfo clickHouseSinkInfo) {
+        this.clickHouseSinkInfo = Preconditions.checkNotNull(clickHouseSinkInfo);
+    }
+
+    @Override
+    public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throws Exception {
+        outputFormat.flush();
+    }
+
+    @Override
+    public void initializeState(FunctionInitializationContext functionInitializationContext) {
+    }
+
+    @Override
+    public void open(Configuration parameters) throws Exception {
+        FieldInfo[] fields = clickHouseSinkInfo.getFields();
+        int fieldsLength = fields.length;
+        String[] fieldNames = new String[fieldsLength];
+        FormatInfo[] formatInfos = new FormatInfo[fieldsLength];
+        for (int i = 0; i < fieldsLength; ++i) {
+            FieldInfo field = fields[i];
+            fieldNames[i] = field.getName();
+            formatInfos[i] = field.getFormatInfo();
+        }
+
+        outputFormat = ClickHouseOutputFormatFactory
+                               .generateClickHouseOutputFormat(fieldNames, formatInfos, clickHouseSinkInfo);
+        RuntimeContext ctx = getRuntimeContext();
+        outputFormat.setRuntimeContext(ctx);
+        outputFormat.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks());
+    }
+
+    @Override
+    public void close() throws Exception {
+        outputFormat.close();
+    }
+
+    @Override
+    public void invoke(Tuple2<Boolean, Row> value) throws Exception {
+        outputFormat.writeRecord(value);
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseStatementFactory.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseStatementFactory.java
new file mode 100644
index 0000000..946e35d
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/ClickHouseStatementFactory.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.flink.clickhouse;
+
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+public class ClickHouseStatementFactory {
+
+    public static String getInsertIntoStatement(String tableName, String[] fieldNames) {
+        String columns = Arrays.stream(fieldNames).map(ClickHouseStatementFactory::quoteIdentifier)
+                .collect(Collectors.joining(", "));
+        String placeholders = Arrays.stream(fieldNames).map(f -> "?").collect(Collectors.joining(", "));
+        return "INSERT INTO " + quoteIdentifier(tableName) + "(" + columns + ") VALUES (" + placeholders + ")";
+    }
+
+    public static String getUpdateStatement(
+            String tableName,
+            String[] fieldNames,
+            String[] conditionFields,
+            Optional<String> clusterName) {
+        String setClause = Arrays.stream(fieldNames).map(f -> quoteIdentifier(f) + "=?")
+                .collect(Collectors.joining(", "));
+        String conditionClause = Arrays.stream(conditionFields).map(f -> quoteIdentifier(f) + "=?")
+                .collect(Collectors.joining(" AND "));
+        String onClusterClause = "";
+        if (clusterName.isPresent()) {
+            onClusterClause = " ON CLUSTER " + quoteIdentifier(clusterName.get());
+        }
+        return "ALTER TABLE " + quoteIdentifier(tableName) + onClusterClause
+                + " UPDATE " + setClause + " WHERE " + conditionClause;
+    }
+
+    public static String getDeleteStatement(
+            String tableName,
+            String[] conditionFields,
+            Optional<String> clusterName) {
+        String conditionClause = Arrays.stream(conditionFields).map(f -> quoteIdentifier(f) + "=?")
+                .collect(Collectors.joining(" AND "));
+        String onClusterClause = "";
+        if (clusterName.isPresent()) {
+            onClusterClause = " ON CLUSTER " + quoteIdentifier(clusterName.get());
+        }
+        return "ALTER TABLE " + quoteIdentifier(tableName) + onClusterClause + " DELETE WHERE " + conditionClause;
+    }
+
+    public static String getRowExistsStatement(String tableName, String[] conditionFields) {
+        String fieldExpressions = Arrays.<String>stream(conditionFields).map(f -> quoteIdentifier(f) + "=?")
+                .collect(Collectors.joining(" AND "));
+        return "SELECT 1 FROM " + quoteIdentifier(tableName) + " WHERE " + fieldExpressions;
+    }
+
+    public static String quoteIdentifier(String identifier) {
+        return "`" + identifier + "`";
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/executor/ClickHouseAppendExecutor.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/executor/ClickHouseAppendExecutor.java
new file mode 100644
index 0000000..19c5b22
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/executor/ClickHouseAppendExecutor.java
@@ -0,0 +1,145 @@
+/*
+ * 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.flink.clickhouse.executor;
+
+import org.apache.inlong.sort.formats.common.FormatInfo;
+import org.apache.inlong.sort.flink.clickhouse.ClickHouseRowConverter;
+import org.apache.inlong.sort.protocol.sink.ClickHouseSinkInfo;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.util.concurrent.AbstractExecutionThreadService;
+import org.apache.flink.types.Row;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import ru.yandex.clickhouse.ClickHouseConnection;
+import ru.yandex.clickhouse.ClickHousePreparedStatement;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class ClickHouseAppendExecutor implements ClickHouseExecutor {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(ClickHouseAppendExecutor.class);
+
+    private final String insertSql;
+
+    private final FormatInfo[] formatInfos;
+
+    private final int flushIntervalSecond;
+
+    private final int maxRetries;
+
+    private transient ClickHousePreparedStatement stmt;
+
+    private transient List<Row> batch = new ArrayList<>();
+
+    private transient ExecuteBatchService executeBatchService;
+
+    public ClickHouseAppendExecutor(
+            String insertSql,
+            FormatInfo[] formatInfos,
+            ClickHouseSinkInfo clickHouseSinkInfo) {
+        this.insertSql = insertSql;
+        this.formatInfos = formatInfos;
+        this.flushIntervalSecond = clickHouseSinkInfo.getFlushInterval();
+        this.maxRetries = clickHouseSinkInfo.getWriteMaxRetryTimes();
+    }
+
+    @Override
+    public void prepareStatement(ClickHouseConnection connection) throws SQLException {
+        if (batch == null) {
+            batch = new ArrayList<>();
+        }
+        stmt = (ClickHousePreparedStatement) connection.prepareStatement(insertSql);
+        executeBatchService = new ExecuteBatchService();
+        executeBatchService.startAsync();
+    }
+
+    @Override
+    public synchronized void addBatch(Tuple2<Boolean, Row> record) {
+        batch.add(record.f1);
+    }
+
+    @Override
+    public synchronized void executeBatch() throws IOException {
+        if (executeBatchService.isRunning()) {
+            this.notifyAll();
+        } else {
+            throw new IOException("executor unexpectedly terminated", executeBatchService.failureCause());
+        }
+    }
+
+    @Override
+    public void closeStatement() throws SQLException {
+        if (executeBatchService != null) {
+            executeBatchService.stopAsync().awaitTerminated();
+        }
+
+        if (stmt != null) {
+            stmt.close();
+            stmt = null;
+        }
+    }
+
+    private class ExecuteBatchService extends AbstractExecutionThreadService {
+
+        private ExecuteBatchService() {
+        }
+
+        protected void run() throws Exception {
+            while (isRunning()) {
+                synchronized (ClickHouseAppendExecutor.this) {
+                    ClickHouseAppendExecutor.this.wait(flushIntervalSecond * 1000L);
+                    if (!batch.isEmpty()) {
+                        for (Row row : batch) {
+                            ClickHouseRowConverter.setRow(stmt, formatInfos, row);
+                            stmt.addBatch();
+                        }
+                        attemptExecuteBatch();
+                    }
+                }
+            }
+        }
+
+        private void attemptExecuteBatch() throws IOException {
+            for (int i = 1; i <= maxRetries; i++) {
+                try {
+                    stmt.executeBatch();
+                    batch.clear();
+                    break;
+                } catch (SQLException e) {
+                    LOG.error("ClickHouse executeBatch error, retry times = {}", i, e);
+                    if (i >= maxRetries) {
+                        throw new IOException(e);
+                    }
+
+                    try {
+                        Thread.sleep((1000L * i));
+                    } catch (InterruptedException ex) {
+                        Thread.currentThread().interrupt();
+                        throw new IOException("unable to flush; interrupted while doing another attempt", e);
+                    }
+                }
+            }
+        }
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/executor/ClickHouseExecutor.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/executor/ClickHouseExecutor.java
new file mode 100644
index 0000000..ba13a0f
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/executor/ClickHouseExecutor.java
@@ -0,0 +1,38 @@
+/*
+ * 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.flink.clickhouse.executor;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.types.Row;
+import ru.yandex.clickhouse.ClickHouseConnection;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.sql.SQLException;
+
+public interface ClickHouseExecutor extends Serializable {
+
+    void prepareStatement(ClickHouseConnection paramClickHouseConnection) throws SQLException;
+
+    void closeStatement() throws SQLException;
+
+    void addBatch(Tuple2<Boolean, Row> rowData);
+
+    void executeBatch() throws IOException;
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/executor/ClickHouseExecutorFactory.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/executor/ClickHouseExecutorFactory.java
new file mode 100644
index 0000000..5f587e3
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/executor/ClickHouseExecutorFactory.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.flink.clickhouse.executor;
+
+import org.apache.inlong.sort.formats.common.FormatInfo;
+import org.apache.inlong.sort.flink.clickhouse.ClickHouseStatementFactory;
+import org.apache.inlong.sort.protocol.sink.ClickHouseSinkInfo;
+import java.util.Optional;
+
+public class ClickHouseExecutorFactory {
+    public static ClickHouseExecutor generateClickHouseExecutor(
+            String tableName,
+            String[] fieldNames,
+            FormatInfo[] formatInfos,
+            ClickHouseSinkInfo clickHouseSinkInfo) {
+        String[] keyFields = clickHouseSinkInfo.getKeyFieldNames();
+        if (keyFields.length > 0) {
+            String insertSql = ClickHouseStatementFactory.getInsertIntoStatement(tableName, fieldNames);
+            String updateSql = ClickHouseStatementFactory.getUpdateStatement(
+                    tableName, fieldNames, keyFields, Optional.empty());
+            String deleteSql = ClickHouseStatementFactory.getDeleteStatement(tableName, keyFields, Optional.empty());
+            return new ClickHouseUpsertExecutor(insertSql, updateSql, deleteSql, formatInfos, clickHouseSinkInfo);
+        }
+
+        String insertSql = ClickHouseStatementFactory.getInsertIntoStatement(tableName, fieldNames);
+        return new ClickHouseAppendExecutor(insertSql, formatInfos, clickHouseSinkInfo);
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/executor/ClickHouseUpsertExecutor.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/executor/ClickHouseUpsertExecutor.java
new file mode 100644
index 0000000..b85e959
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/executor/ClickHouseUpsertExecutor.java
@@ -0,0 +1,182 @@
+/*
+ * 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.flink.clickhouse.executor;
+
+import org.apache.inlong.sort.formats.common.FormatInfo;
+import org.apache.inlong.sort.flink.clickhouse.ClickHouseRowConverter;
+import org.apache.inlong.sort.protocol.sink.ClickHouseSinkInfo;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.util.concurrent.AbstractExecutionThreadService;
+import org.apache.flink.types.Row;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import ru.yandex.clickhouse.ClickHouseConnection;
+import ru.yandex.clickhouse.ClickHousePreparedStatement;
+import ru.yandex.clickhouse.ClickHouseStatement;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class ClickHouseUpsertExecutor implements ClickHouseExecutor {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(ClickHouseUpsertExecutor.class);
+
+    private final String insertSql;
+
+    private final String updateSql;
+
+    private final String deleteSql;
+
+    private final FormatInfo[] formatInfos;
+
+    private final int flushIntervalSecond;
+
+    private final int maxRetries;
+
+    private final List<Row> insertBatch = new ArrayList<>();
+
+    private final List<Row> deleteBatch = new ArrayList<>();
+
+    private transient ClickHousePreparedStatement insertStmt;
+
+    private transient ClickHousePreparedStatement updateStmt;
+
+    private transient ClickHousePreparedStatement deleteStmt;
+
+    private transient ExecuteBatchService executeBatchService;
+
+    public ClickHouseUpsertExecutor(
+            String insertSql,
+            String updateSql,
+            String deleteSql,
+            FormatInfo[] formatInfos,
+            ClickHouseSinkInfo clickHouseSinkInfo) {
+        this.insertSql = insertSql;
+        this.updateSql = updateSql;
+        this.deleteSql = deleteSql;
+        this.formatInfos = formatInfos;
+        this.flushIntervalSecond = clickHouseSinkInfo.getFlushInterval();
+        this.maxRetries = clickHouseSinkInfo.getWriteMaxRetryTimes();
+    }
+
+    @Override
+    public void prepareStatement(ClickHouseConnection connection) throws SQLException {
+        insertStmt = (ClickHousePreparedStatement) connection.prepareStatement(insertSql);
+        updateStmt = (ClickHousePreparedStatement) connection.prepareStatement(updateSql);
+        deleteStmt = (ClickHousePreparedStatement) connection.prepareStatement(deleteSql);
+        executeBatchService = new ExecuteBatchService();
+        executeBatchService.startAsync();
+    }
+
+    @Override
+    public synchronized void addBatch(Tuple2<Boolean, Row> record) {
+        boolean insert = record.f0;
+        if (insert) {
+            insertBatch.add(record.f1);
+        } else {
+            deleteBatch.add(record.f1);
+        }
+    }
+
+    @Override
+    public synchronized void executeBatch() throws IOException {
+        if (executeBatchService.isRunning()) {
+            notifyAll();
+        } else {
+            throw new IOException("executor unexpectedly terminated", executeBatchService.failureCause());
+        }
+    }
+
+    @Override
+    public void closeStatement() throws SQLException {
+        if (executeBatchService != null) {
+            executeBatchService.stopAsync().awaitTerminated();
+        }
+
+        for (ClickHouseStatement stmt : Arrays.asList(insertStmt, updateStmt, deleteStmt)) {
+            if (stmt != null) {
+                stmt.close();
+            }
+        }
+    }
+
+    private class ExecuteBatchService extends AbstractExecutionThreadService {
+
+        private ExecuteBatchService() {
+        }
+
+        protected void run() throws Exception {
+            while (isRunning()) {
+                synchronized (ClickHouseUpsertExecutor.this) {
+                    wait(flushIntervalSecond * 1000L);
+                    processInsertBatch(insertStmt, insertBatch);
+                    processDeleteBatch(deleteStmt, deleteBatch);
+                }
+            }
+        }
+
+        private void processDeleteBatch(ClickHousePreparedStatement stmt, List<Row> batch)
+                throws SQLException {
+            if (!batch.isEmpty()) {
+                for (Row row : batch) {
+                    ClickHouseRowConverter.setRow(stmt, formatInfos, row);
+                    stmt.executeUpdate();
+                }
+            }
+        }
+
+        private void processInsertBatch(ClickHousePreparedStatement stmt, List<Row> batch)
+                throws SQLException, IOException {
+            if (!batch.isEmpty()) {
+                for (Row row : batch) {
+                    ClickHouseRowConverter.setRow(stmt, formatInfos, row);
+                    stmt.addBatch();
+                }
+                attemptExecuteBatch(stmt, batch);
+            }
+        }
+
+        private void attemptExecuteBatch(ClickHousePreparedStatement stmt, List<Row> batch) throws IOException {
+            for (int i = 1; i <= maxRetries; i++) {
+                try {
+                    stmt.executeBatch();
+                    batch.clear();
+                    break;
+                } catch (SQLException e) {
+                    LOG.error("ClickHouse executeBatch error, retry times = {}", i, e);
+                    if (i >= maxRetries) {
+                        throw new IOException(e);
+                    }
+
+                    try {
+                        Thread.sleep((1000L * i));
+                    } catch (InterruptedException ex) {
+                        Thread.currentThread().interrupt();
+                        throw new IOException("unable to flush; interrupted while doing another attempt", e);
+                    }
+                }
+            }
+        }
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/output/AbstractClickHouseOutputFormat.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/output/AbstractClickHouseOutputFormat.java
new file mode 100644
index 0000000..e80902f
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/output/AbstractClickHouseOutputFormat.java
@@ -0,0 +1,36 @@
+/*
+ * 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.flink.clickhouse.output;
+
+import org.apache.flink.api.common.io.RichOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.types.Row;
+
+import java.io.Flushable;
+
+public abstract class AbstractClickHouseOutputFormat
+        extends RichOutputFormat<Tuple2<Boolean, Row>> implements Flushable {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public void configure(Configuration configuration) {
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/output/ClickHouseBatchOutputFormat.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/output/ClickHouseBatchOutputFormat.java
new file mode 100644
index 0000000..12db57d
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/output/ClickHouseBatchOutputFormat.java
@@ -0,0 +1,123 @@
+/*
+ * 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.flink.clickhouse.output;
+
+import org.apache.inlong.sort.formats.common.FormatInfo;
+import org.apache.inlong.sort.flink.clickhouse.ClickHouseConnectionProvider;
+import org.apache.inlong.sort.flink.clickhouse.executor.ClickHouseExecutor;
+import org.apache.inlong.sort.flink.clickhouse.executor.ClickHouseExecutorFactory;
+import org.apache.inlong.sort.protocol.sink.ClickHouseSinkInfo;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import ru.yandex.clickhouse.ClickHouseConnection;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.sql.SQLException;
+
+public class ClickHouseBatchOutputFormat extends AbstractClickHouseOutputFormat {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(ClickHouseBatchOutputFormat.class);
+
+    private final ClickHouseConnectionProvider connectionProvider;
+
+    private final String[] fieldNames;
+
+    private final FormatInfo[] formatInfos;
+
+    private final ClickHouseSinkInfo clickHouseSinkInfo;
+
+    private boolean closed = false;
+
+    private transient ClickHouseExecutor executor;
+
+    private transient int batchCount = 0;
+
+    public ClickHouseBatchOutputFormat(
+            @Nonnull String[] fieldNames,
+            @Nonnull FormatInfo[] formatInfos,
+            @Nonnull ClickHouseSinkInfo clickHouseSinkInfo) {
+        this.connectionProvider = new ClickHouseConnectionProvider(clickHouseSinkInfo);
+        this.fieldNames = Preconditions.checkNotNull(fieldNames);
+        this.formatInfos = Preconditions.checkNotNull(formatInfos);
+        this.clickHouseSinkInfo = Preconditions.checkNotNull(clickHouseSinkInfo);
+    }
+
+    @Override
+    public void open(int taskNumber, int numTasks) throws IOException {
+        try {
+            ClickHouseConnection connection = connectionProvider.getConnection();
+            executor = ClickHouseExecutorFactory.generateClickHouseExecutor(
+                    clickHouseSinkInfo.getTableName(), fieldNames, formatInfos, clickHouseSinkInfo);
+            executor.prepareStatement(connection);
+        } catch (Exception e) {
+            throw new IOException(e);
+        }
+    }
+
+    @Override
+    public void writeRecord(Tuple2<Boolean, Row> record) throws IOException {
+        addBatch(record);
+        batchCount++;
+        if (batchCount >= clickHouseSinkInfo.getFlushRecordNumber()) {
+            flush();
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (!closed) {
+            closed = true;
+            if (batchCount > 0) {
+                try {
+                    flush();
+                } catch (Exception e) {
+                    LOG.warn("Writing records to ClickHouse failed.", e);
+                }
+            }
+            closeConnection();
+        }
+    }
+
+    @Override
+    public void flush() throws IOException {
+        executor.executeBatch();
+        batchCount = 0;
+    }
+
+    private void addBatch(Tuple2<Boolean, Row> record) {
+        executor.addBatch(record);
+    }
+
+    private void closeConnection() {
+        try {
+            executor.closeStatement();
+            connectionProvider.closeConnections();
+        } catch (SQLException se) {
+            LOG.warn("ClickHouse connection could not be closed: {}", se.getMessage());
+        }
+    }
+}
+
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/output/ClickHouseOutputFormatFactory.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/output/ClickHouseOutputFormatFactory.java
new file mode 100644
index 0000000..5526d96
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/output/ClickHouseOutputFormatFactory.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.flink.clickhouse.output;
+
+import org.apache.inlong.sort.formats.common.FormatInfo;
+import org.apache.inlong.sort.flink.clickhouse.partitioner.BalancePartitioner;
+import org.apache.inlong.sort.flink.clickhouse.partitioner.ClickHousePartitioner;
+import org.apache.inlong.sort.flink.clickhouse.partitioner.HashPartitioner;
+import org.apache.inlong.sort.flink.clickhouse.partitioner.RandomPartitioner;
+import org.apache.inlong.sort.protocol.sink.ClickHouseSinkInfo;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClickHouseOutputFormatFactory {
+    private static final Logger LOG = LoggerFactory.getLogger(ClickHouseOutputFormatFactory.class);
+
+    /**
+     * Get ClickHouseOutputFormat.
+     * @param fieldNames names of fields.
+     * @param formatInfos formatInfos of fields.
+     * @param clickHouseSinkInfo {@link ClickHouseSinkInfo}.
+     * @return {@link ClickHouseShardOutputFormat} for distributed table and
+     *         {@link ClickHouseBatchOutputFormat} for normal table.
+     */
+    public static AbstractClickHouseOutputFormat generateClickHouseOutputFormat(
+            String[] fieldNames,
+            FormatInfo[] formatInfos,
+            ClickHouseSinkInfo clickHouseSinkInfo) {
+        if (clickHouseSinkInfo.getKeyFieldNames().length > 0) {
+            LOG.warn("If primary key is specified, connector will be in UPSERT mode. "
+                    + "You will have significant performance loss.");
+        }
+
+        if (clickHouseSinkInfo.isDistributedTable()) {
+            ClickHousePartitioner partitioner;
+            int partitionKeyIndex;
+
+            switch (clickHouseSinkInfo.getPartitionStrategy()) {
+                case BALANCE:
+                    partitioner = new BalancePartitioner();
+                    break;
+                case RANDOM:
+                    partitioner = new RandomPartitioner();
+                    break;
+                case HASH:
+                    partitionKeyIndex = Arrays.asList(fieldNames).indexOf(clickHouseSinkInfo.getPartitionKey());
+                    if (partitionKeyIndex == -1) {
+                        throw new IllegalArgumentException("Partition key `" + clickHouseSinkInfo.getPartitionKey()
+                                + "` not found in table schema");
+                    }
+                    partitioner = new HashPartitioner(partitionKeyIndex);
+                    break;
+                default:
+                    throw new IllegalArgumentException("Unsupported partition strategy `"
+                            + clickHouseSinkInfo.getPartitionStrategy() + "`");
+            }
+
+            return new ClickHouseShardOutputFormat(
+                    fieldNames,
+                    formatInfos,
+                    partitioner,
+                    clickHouseSinkInfo);
+        }
+
+        return new ClickHouseBatchOutputFormat(
+                fieldNames,
+                formatInfos,
+                clickHouseSinkInfo);
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/output/ClickHouseShardOutputFormat.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/output/ClickHouseShardOutputFormat.java
new file mode 100644
index 0000000..4eca329
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/output/ClickHouseShardOutputFormat.java
@@ -0,0 +1,174 @@
+/*
+ * 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.flink.clickhouse.output;
+
+import org.apache.inlong.sort.formats.common.FormatInfo;
+import org.apache.inlong.sort.flink.clickhouse.ClickHouseConnectionProvider;
+import org.apache.inlong.sort.flink.clickhouse.executor.ClickHouseExecutor;
+import org.apache.inlong.sort.flink.clickhouse.executor.ClickHouseExecutorFactory;
+import org.apache.inlong.sort.flink.clickhouse.partitioner.ClickHousePartitioner;
+import org.apache.inlong.sort.protocol.sink.ClickHouseSinkInfo;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import ru.yandex.clickhouse.ClickHouseConnection;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class ClickHouseShardOutputFormat extends AbstractClickHouseOutputFormat {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(ClickHouseShardOutputFormat.class);
+
+    private static final Pattern PATTERN = Pattern.compile("Distributed\\((?<cluster>[a-zA-Z_][0-9a-zA-Z_]*),"
+            + "\\s*(?<database>[a-zA-Z_][0-9a-zA-Z_]*),\\s*(?<table>[a-zA-Z_][0-9a-zA-Z_]*)");
+
+    private final ClickHouseConnectionProvider connectionProvider;
+
+    private final String[] fieldNames;
+
+    private final FormatInfo[] formatInfos;
+
+    private final ClickHousePartitioner partitioner;
+
+    private final ClickHouseSinkInfo clickHouseSinkInfo;
+
+    private final List<ClickHouseExecutor> shardExecutors = new ArrayList<>();
+
+    private boolean closed = false;
+
+    private transient int[] batchCounts;
+
+    public ClickHouseShardOutputFormat(
+            @Nonnull String[] fieldNames,
+            @Nonnull FormatInfo[] formatInfos,
+            @Nonnull ClickHousePartitioner partitioner,
+            @Nonnull ClickHouseSinkInfo clickHouseSinkInfo) {
+        this.connectionProvider = new ClickHouseConnectionProvider(clickHouseSinkInfo);
+        this.fieldNames = Preconditions.checkNotNull(fieldNames);
+        this.formatInfos = Preconditions.checkNotNull(formatInfos);
+        this.partitioner = Preconditions.checkNotNull(partitioner);
+        this.clickHouseSinkInfo = Preconditions.checkNotNull(clickHouseSinkInfo);
+    }
+
+    @Override
+    public void open(int taskNumber, int numTasks) throws IOException {
+        try {
+            initializeExecutors();
+        } catch (Exception e) {
+           throw new IOException(e);
+        }
+    }
+
+    @Override
+    public void writeRecord(Tuple2<Boolean, Row> recordAndMode) throws IOException {
+        boolean rowMode = recordAndMode.f0;
+        if (rowMode) {
+            writeRecordToOneExecutor(recordAndMode);
+        } else {
+            writeRecordToAllExecutors(recordAndMode);
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (!closed) {
+            closed = true;
+            flush();
+            closeConnection();
+        }
+    }
+
+    public void flush() throws IOException {
+        for (int i = 0; i < shardExecutors.size(); i++) {
+            flush(i);
+        }
+    }
+
+    public void flush(int index) throws IOException {
+        shardExecutors.get(index).executeBatch();
+        batchCounts[index] = 0;
+    }
+
+    private void initializeExecutors() throws Exception {
+        String engine = connectionProvider.queryTableEngine(
+                clickHouseSinkInfo.getDatabaseName(), clickHouseSinkInfo.getTableName());
+        Matcher matcher = PATTERN.matcher(engine);
+        List<ClickHouseConnection> shardConnections;
+        String remoteTable;
+
+        if (matcher.find()) {
+            String remoteCluster = matcher.group("cluster");
+            String remoteDatabase = matcher.group("database");
+            remoteTable = matcher.group("table");
+            shardConnections = connectionProvider.getShardConnections(remoteCluster, remoteDatabase);
+            batchCounts = new int[shardConnections.size()];
+        } else {
+            throw new IOException("table `" + clickHouseSinkInfo.getDatabaseName() + "`.`"
+                    + clickHouseSinkInfo.getTableName() + "` is not a Distributed table");
+        }
+
+        for (ClickHouseConnection shardConnection : shardConnections) {
+            ClickHouseExecutor clickHouseExecutor = ClickHouseExecutorFactory.generateClickHouseExecutor(
+                    remoteTable, fieldNames, formatInfos, clickHouseSinkInfo);
+            clickHouseExecutor.prepareStatement(shardConnection);
+            this.shardExecutors.add(clickHouseExecutor);
+        }
+    }
+
+    private void writeRecordToOneExecutor(Tuple2<Boolean, Row> record) throws IOException {
+        int selected = partitioner.select(record, shardExecutors.size());
+        shardExecutors.get(selected).addBatch(record);
+        batchCounts[selected] = batchCounts[selected] + 1;
+        if (batchCounts[selected] >= clickHouseSinkInfo.getFlushRecordNumber()) {
+            flush(selected);
+        }
+    }
+
+    private void writeRecordToAllExecutors(Tuple2<Boolean, Row> record) throws IOException {
+        for (int i = 0; i < shardExecutors.size(); i++) {
+            shardExecutors.get(i).addBatch(record);
+            batchCounts[i] = batchCounts[i] + 1;
+            if (batchCounts[i] >= clickHouseSinkInfo.getFlushRecordNumber()) {
+                flush(i);
+            }
+        }
+    }
+
+    private void closeConnection() {
+        try {
+            for (ClickHouseExecutor shardExecutor : shardExecutors) {
+                shardExecutor.closeStatement();
+            }
+            connectionProvider.closeConnections();
+        } catch (SQLException se) {
+            LOG.warn("ClickHouse connection could not be closed!", se);
+        }
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/partitioner/BalancePartitioner.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/partitioner/BalancePartitioner.java
new file mode 100644
index 0000000..832496f
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/partitioner/BalancePartitioner.java
@@ -0,0 +1,34 @@
+/*
+ * 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.flink.clickhouse.partitioner;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.types.Row;
+
+public class BalancePartitioner implements ClickHousePartitioner {
+
+    private static final long serialVersionUID = 1L;
+
+    private int nextShard = 0;
+
+    public int select(Tuple2<Boolean, Row> record, int numShards) {
+        this.nextShard = (this.nextShard + 1) % numShards;
+        return this.nextShard;
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/partitioner/ClickHousePartitioner.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/partitioner/ClickHousePartitioner.java
new file mode 100644
index 0000000..94ac910
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/partitioner/ClickHousePartitioner.java
@@ -0,0 +1,28 @@
+/*
+ * 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.flink.clickhouse.partitioner;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.types.Row;
+
+import java.io.Serializable;
+
+public interface ClickHousePartitioner extends Serializable {
+    int select(Tuple2<Boolean, Row> paramRowData, int paramInt);
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/partitioner/HashPartitioner.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/partitioner/HashPartitioner.java
new file mode 100644
index 0000000..6f78ced
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/partitioner/HashPartitioner.java
@@ -0,0 +1,40 @@
+/*
+ * 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.flink.clickhouse.partitioner;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.types.Row;
+
+import java.util.Objects;
+
+public class HashPartitioner implements ClickHousePartitioner {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int pos;
+
+    public HashPartitioner(int pos) {
+        this.pos = pos;
+    }
+
+    public int select(Tuple2<Boolean, Row> record, int numShards) {
+        return Objects.hashCode(record.f1.getField(pos)) % numShards;
+    }
+}
+
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/partitioner/RandomPartitioner.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/partitioner/RandomPartitioner.java
new file mode 100644
index 0000000..ed3034e
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/clickhouse/partitioner/RandomPartitioner.java
@@ -0,0 +1,33 @@
+/*
+ * 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.flink.clickhouse.partitioner;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.types.Row;
+
+import java.util.concurrent.ThreadLocalRandom;
+
+public class RandomPartitioner implements ClickHousePartitioner {
+
+    private static final long serialVersionUID = 1L;
+
+    public int select(Tuple2<Boolean, Row> record, int numShards) {
+        return ThreadLocalRandom.current().nextInt(numShards);
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/CallbackCollector.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/CallbackCollector.java
new file mode 100644
index 0000000..43e1265
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/CallbackCollector.java
@@ -0,0 +1,50 @@
+/*
+ * 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.flink.deserialization;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+/**
+ * A collector supporting callback. It also hides the exception thrown in method declaration to make lambda expression
+ * shorter.
+ */
+public class CallbackCollector<T> implements Collector<T> {
+
+    private final ThrowingConsumer<T, Exception> callback;
+
+    public CallbackCollector(ThrowingConsumer<T, Exception> callback) {
+        this.callback = checkNotNull(callback);
+    }
+
+    @Override
+    public void collect(T t) {
+        try {
+            callback.accept(t);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void close() {
+
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/DeserializationSchema.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/DeserializationSchema.java
new file mode 100644
index 0000000..eb8e0e4
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/DeserializationSchema.java
@@ -0,0 +1,132 @@
+/*
+ * 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.flink.deserialization;
+
+import static org.apache.inlong.sort.configuration.Constants.UNKNOWN_DATAFLOW_ID;
+
+import com.google.common.base.Preconditions;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.inlong.sort.configuration.Configuration;
+import org.apache.inlong.sort.configuration.Constants;
+import org.apache.inlong.sort.flink.Record;
+import org.apache.inlong.sort.flink.SerializedRecord;
+import org.apache.inlong.sort.flink.TDMsgSerializedRecord;
+import org.apache.inlong.sort.flink.transformation.FieldMappingTransformer;
+import org.apache.inlong.sort.flink.transformation.RecordTransformer;
+import org.apache.inlong.sort.meta.MetaManager;
+import org.apache.inlong.sort.meta.MetaManager.DataFlowInfoListener;
+import org.apache.inlong.sort.protocol.DataFlowInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DeserializationSchema extends ProcessFunction<SerializedRecord, SerializedRecord> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(DeserializationSchema.class);
+
+    private transient RecordTransformer recordTransformer;
+
+    private transient FieldMappingTransformer fieldMappingTransformer;
+
+    private final Configuration config;
+
+    private transient Object schemaLock;
+
+    private transient MultiTenancyTDMsgMixedDeserializer multiTenancyTdMsgMixedDeserializer;
+
+    private transient MetaManager metaManager;
+
+    public DeserializationSchema(Configuration config) {
+        this.config = Preconditions.checkNotNull(config);
+    }
+
+    @Override
+    public void open(org.apache.flink.configuration.Configuration parameters) throws Exception {
+        schemaLock = new Object();
+        multiTenancyTdMsgMixedDeserializer = new MultiTenancyTDMsgMixedDeserializer();
+        fieldMappingTransformer = new FieldMappingTransformer();
+        recordTransformer = new RecordTransformer(config.getInteger(Constants.ETL_RECORD_SERIALIZATION_BUFFER_SIZE));
+        metaManager = MetaManager.getInstance(config);
+        metaManager.registerDataFlowInfoListener(new DataFlowInfoListenerImpl());
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (metaManager != null) {
+            metaManager.close();
+            metaManager = null;
+        }
+    }
+
+    @Override
+    public void processElement(
+            SerializedRecord serializedRecord,
+            Context context,
+            Collector<SerializedRecord> collector) throws Exception {
+        try {
+            if (serializedRecord instanceof TDMsgSerializedRecord
+                    && serializedRecord.getDataFlowId() == UNKNOWN_DATAFLOW_ID) {
+                final TDMsgSerializedRecord tdmsgRecord = (TDMsgSerializedRecord) serializedRecord;
+                synchronized (schemaLock) {
+                    multiTenancyTdMsgMixedDeserializer.deserialize(
+                            tdmsgRecord,
+                            new CallbackCollector<>(sourceRecord -> {
+                                final Record sinkRecord = fieldMappingTransformer.transform(sourceRecord);
+                                collector.collect(recordTransformer.toSerializedRecord(sinkRecord));
+                            }));
+                }
+            } else {
+                // TODO, support metrics and more data types
+                LOG.warn("Abandon data due to unsupported record {}", serializedRecord);
+            }
+        } catch (Exception e) {
+            // TODO, support metrics
+            LOG.warn("Abandon data", e);
+        }
+    }
+
+    private class DataFlowInfoListenerImpl implements DataFlowInfoListener {
+
+        @Override
+        public void addDataFlow(DataFlowInfo dataFlowInfo) throws Exception {
+            synchronized (schemaLock) {
+                multiTenancyTdMsgMixedDeserializer.addDataFlow(dataFlowInfo);
+                fieldMappingTransformer.addDataFlow(dataFlowInfo);
+                recordTransformer.addDataFlow(dataFlowInfo);
+            }
+        }
+
+        @Override
+        public void updateDataFlow(DataFlowInfo dataFlowInfo) throws Exception {
+            synchronized (schemaLock) {
+                multiTenancyTdMsgMixedDeserializer.updateDataFlow(dataFlowInfo);
+                fieldMappingTransformer.updateDataFlow(dataFlowInfo);
+                recordTransformer.updateDataFlow(dataFlowInfo);
+            }
+        }
+
+        @Override
+        public void removeDataFlow(DataFlowInfo dataFlowInfo) throws Exception {
+            synchronized (schemaLock) {
+                multiTenancyTdMsgMixedDeserializer.removeDataFlow(dataFlowInfo);
+                fieldMappingTransformer.removeDataFlow(dataFlowInfo);
+                recordTransformer.removeDataFlow(dataFlowInfo);
+            }
+        }
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/Deserializer.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/Deserializer.java
new file mode 100644
index 0000000..2dfe264
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/Deserializer.java
@@ -0,0 +1,24 @@
+/*
+ * 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.flink.deserialization;
+
+import org.apache.flink.util.Collector;
+
+public interface Deserializer<IN, OUT> {
+    void deserialize(IN input, Collector<OUT> collector) throws Exception;
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/MultiTenancyTDMsgMixedDeserializer.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/MultiTenancyTDMsgMixedDeserializer.java
new file mode 100644
index 0000000..f3100d8
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/MultiTenancyTDMsgMixedDeserializer.java
@@ -0,0 +1,151 @@
+/*
+ * 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.flink.deserialization;
+
+import static org.apache.inlong.sort.formats.tdmsg.TDMsgUtils.DEFAULT_ATTRIBUTES_FIELD_NAME;
+import static org.apache.inlong.sort.formats.tdmsg.TDMsgUtils.DEFAULT_TIME_FIELD_NAME;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.flink.util.Collector;
+import org.apache.inlong.sort.flink.Record;
+import org.apache.inlong.sort.flink.TDMsgSerializedRecord;
+import org.apache.inlong.sort.formats.common.RowFormatInfo;
+import org.apache.inlong.sort.formats.tdmsg.AbstractTDMsgMixedFormatDeserializer;
+import org.apache.inlong.sort.formats.tdmsg.TDMsgMixedFormatConverter;
+import org.apache.inlong.sort.formats.tdmsgcsv.TDMsgCsvMixedFormatConverter;
+import org.apache.inlong.sort.formats.tdmsgcsv.TDMsgCsvMixedFormatDeserializer;
+import org.apache.inlong.sort.meta.MetaManager.DataFlowInfoListener;
+import org.apache.inlong.sort.protocol.DataFlowInfo;
+import org.apache.inlong.sort.protocol.FieldInfo;
+import org.apache.inlong.sort.protocol.deserialization.DeserializationInfo;
+import org.apache.inlong.sort.protocol.deserialization.TDMsgCsv2DeserializationInfo;
+import org.apache.inlong.sort.protocol.deserialization.TDMsgCsvDeserializationInfo;
+import org.apache.inlong.sort.protocol.deserialization.TDMsgDeserializationInfo;
+import org.apache.inlong.sort.protocol.deserialization.TDMsgKvDeserializationInfo;
+import org.apache.inlong.sort.protocol.deserialization.TDMsgTlogCsvDeserializationInfo;
+import org.apache.inlong.sort.protocol.deserialization.TDMsgTlogKvDeserializationInfo;
+import org.apache.inlong.sort.protocol.source.TubeSourceInfo;
+import org.apache.inlong.sort.util.CommonUtils;
+
+/**
+ * A deserializer to handle mixed TDMsg records.
+ */
+public class MultiTenancyTDMsgMixedDeserializer implements DataFlowInfoListener,
+        Deserializer<TDMsgSerializedRecord, Record> {
+
+    /**
+     * Maps topic to mixed deserializer.
+     */
+    private final Map<String, TDMsgMixedDeserializer> mixedDeserializerMap = new HashMap<>();
+
+    @Override
+    public void addDataFlow(DataFlowInfo dataFlowInfo) throws Exception {
+        updateDataFlow(dataFlowInfo);
+    }
+
+    @Override
+    public void updateDataFlow(DataFlowInfo dataFlowInfo) {
+        if (!isTDMsgDataFlow(dataFlowInfo)) {
+            return;
+        }
+        final TDMsgDeserializationInfo tdMsgDeserializationInfo =
+                (TDMsgDeserializationInfo) dataFlowInfo.getSourceInfo().getDeserializationInfo();
+
+        Pair<AbstractTDMsgMixedFormatDeserializer, TDMsgMixedFormatConverter> allDeserializer = generateDeserializer(
+                dataFlowInfo.getSourceInfo().getFields(), tdMsgDeserializationInfo);
+        final AbstractTDMsgMixedFormatDeserializer preDeserializer = allDeserializer.getLeft();
+        final TDMsgMixedFormatConverter deserializer = allDeserializer.getRight();
+
+        // currently only tubeMQ source supports TDMsg format
+        final TubeSourceInfo tubeSourceInfo = (TubeSourceInfo) dataFlowInfo.getSourceInfo();
+        final TDMsgMixedDeserializer mixedDeserializer = mixedDeserializerMap
+                .computeIfAbsent(tubeSourceInfo.getTopic(), topic -> new TDMsgMixedDeserializer());
+        mixedDeserializer.updateDataFlow(
+                dataFlowInfo.getId(), tdMsgDeserializationInfo.getTid(), preDeserializer, deserializer);
+    }
+
+    @Override
+    public void removeDataFlow(DataFlowInfo dataFlowInfo) {
+        if (!isTDMsgDataFlow(dataFlowInfo)) {
+            return;
+        }
+        final TubeSourceInfo tubeSourceInfo = (TubeSourceInfo) dataFlowInfo.getSourceInfo();
+        final TDMsgDeserializationInfo tdMsgDeserializationInfo = (TDMsgDeserializationInfo) tubeSourceInfo
+                .getDeserializationInfo();
+        final TDMsgMixedDeserializer mixedDeserializer = mixedDeserializerMap.get(tubeSourceInfo.getTopic());
+        if (mixedDeserializer != null) {
+            mixedDeserializer.removeDataFlow(dataFlowInfo.getId(), tdMsgDeserializationInfo.getTid());
+            if (mixedDeserializer.isEmpty()) {
+                mixedDeserializerMap.remove(tubeSourceInfo.getTopic());
+            }
+        }
+    }
+
+    @VisibleForTesting
+    static boolean isTDMsgDataFlow(DataFlowInfo dataFlowInfo) {
+        final DeserializationInfo deserializationInfo = dataFlowInfo.getSourceInfo().getDeserializationInfo();
+        return deserializationInfo instanceof TDMsgDeserializationInfo;
+    }
+
+    public void deserialize(TDMsgSerializedRecord record, Collector<Record> collector) throws Exception {
+        final String topic = record.getTopic();
+        final TDMsgMixedDeserializer mixedDeserializer = mixedDeserializerMap.get(topic);
+        if (mixedDeserializer == null) {
+            throw new Exception("No schema found for topic:" + topic);
+        }
+        mixedDeserializer.deserialize(record, collector);
+    }
+
+    @VisibleForTesting
+    Pair<AbstractTDMsgMixedFormatDeserializer, TDMsgMixedFormatConverter> generateDeserializer(
+            FieldInfo[] fields,
+            TDMsgDeserializationInfo tdMsgDeserializationInfo) {
+
+        final RowFormatInfo rowFormatInfo =
+                CommonUtils.generateRowFormatInfo(fields);
+
+        final AbstractTDMsgMixedFormatDeserializer preDeserializer;
+        final TDMsgMixedFormatConverter deserializer;
+        if (tdMsgDeserializationInfo instanceof TDMsgCsvDeserializationInfo) {
+            final TDMsgCsvDeserializationInfo csvDeserializationInfo =
+                    (TDMsgCsvDeserializationInfo) tdMsgDeserializationInfo;
+            preDeserializer = new TDMsgCsvMixedFormatDeserializer(
+                    StandardCharsets.UTF_8.name(),
+                    csvDeserializationInfo.getDelimiter(),
+                    null,
+                    null,
+                    csvDeserializationInfo.isDeleteHeadDelimiter(),
+                    false);
+            deserializer = new TDMsgCsvMixedFormatConverter(
+                    rowFormatInfo,
+                    DEFAULT_TIME_FIELD_NAME,
+                    DEFAULT_ATTRIBUTES_FIELD_NAME,
+                    null,
+                    false);
+        } else {
+            throw new UnsupportedOperationException(
+                    "Not supported yet " + tdMsgDeserializationInfo.getClass().getSimpleName());
+        }
+
+        return Pair.of(preDeserializer, deserializer);
+    }
+}
\ No newline at end of file
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/TDMsgMixedDeserializer.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/TDMsgMixedDeserializer.java
new file mode 100644
index 0000000..62cf707
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/deserialization/TDMsgMixedDeserializer.java
@@ -0,0 +1,113 @@
+/*
+ * 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.flink.deserialization;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.flink.util.Collector;
+import org.apache.inlong.sort.flink.Record;
+import org.apache.inlong.sort.flink.TDMsgSerializedRecord;
+import org.apache.inlong.sort.formats.tdmsg.AbstractTDMsgFormatDeserializer;
+import org.apache.inlong.sort.formats.tdmsg.TDMsgMixedFormatConverter;
+import org.apache.inlong.sort.formats.tdmsg.TDMsgUtils;
+
+/**
+ * A deserializer to handle mixed TDMsg records of one topic.
+ */
+public class TDMsgMixedDeserializer implements Deserializer<TDMsgSerializedRecord, Record> {
+
+    /**
+     * Each topic should have same preDeserializer, so just keep one.
+     */
+    private AbstractTDMsgFormatDeserializer preDeserializer;
+
+    /**
+     * Tid -> deserializer.
+     */
+    private final Map<String, TDMsgMixedFormatConverter> deserializers = new HashMap<>();
+
+    /**
+     * Tid -> data flow ids.
+     */
+    private final Map<String, Set<Long>> interface2DataFlowsMap = new HashMap<>();
+
+    public TDMsgMixedDeserializer() {
+    }
+
+    public void updateDataFlow(long dataFlowId, String tid, AbstractTDMsgFormatDeserializer preDeserializer,
+            TDMsgMixedFormatConverter deserializer) {
+        // always updates preDeserializer
+        this.preDeserializer = preDeserializer;
+        deserializers.put(tid, deserializer);
+        interface2DataFlowsMap.computeIfAbsent(tid, k -> new HashSet<>()).add(dataFlowId);
+    }
+
+    public void removeDataFlow(long dataFlowId, String tid) {
+        deserializers.remove(tid);
+        final Set<Long> dataFlows = interface2DataFlowsMap.get(tid);
+        if (dataFlows != null) {
+            dataFlows.remove(dataFlowId);
+            if (dataFlows.isEmpty()) {
+                interface2DataFlowsMap.remove(tid);
+            }
+        }
+    }
+
+    public boolean isEmpty() {
+        return interface2DataFlowsMap.isEmpty();
+    }
+
+    @Override
+    public void deserialize(TDMsgSerializedRecord tdMsgRecord, Collector<Record> collector) throws Exception {
+        preDeserializer.flatMap(tdMsgRecord.getData(), new CallbackCollector<>(mixedRow -> {
+            final String tid = TDMsgUtils.getTidFromMixedRow(mixedRow);
+            final Set<Long> dataFlowIds = interface2DataFlowsMap.get(tid);
+            if (dataFlowIds.isEmpty()) {
+                throw new Exception("No data flow found for tid:" + tid);
+            }
+            final TDMsgMixedFormatConverter deserializer = deserializers.get(tid);
+            if (deserializer == null) {
+                throw new Exception("No data flow found for tid:" + tid);
+            }
+            deserializer.flatMap(mixedRow, new CallbackCollector<>((row -> {
+                // each tid might be associated with multiple data flows
+                for (long dataFlowId : dataFlowIds) {
+                    collector.collect(new Record(dataFlowId, row));
+                }
+            })));
+        }));
+    }
+
+    @VisibleForTesting
+    AbstractTDMsgFormatDeserializer getPreDeserializer() {
+        return preDeserializer;
+    }
+
+    @VisibleForTesting
+    Map<String, TDMsgMixedFormatConverter> getDeserializers() {
+        return deserializers;
+    }
+
+    @VisibleForTesting
+    Map<String, Set<Long>> getInterface2DataFlowsMap() {
+        return interface2DataFlowsMap;
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/Bucket.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/Bucket.java
new file mode 100644
index 0000000..a04ed16
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/Bucket.java
@@ -0,0 +1,399 @@
+/*
+ * 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.flink.filesystem;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.fs.RecoverableFsDataOutputStream;
+import org.apache.flink.core.fs.RecoverableWriter;
+import org.apache.flink.core.fs.RecoverableWriter.CommitRecoverable;
+import org.apache.flink.core.fs.RecoverableWriter.ResumeRecoverable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+import java.util.TreeMap;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Copied from Apache Flink project {@link org.apache.flink.streaming.api.functions.sink.filesystem.Bucket}.
+ */
+public class Bucket<IN, BucketID> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(
+            Bucket.class);
+
+    private static final String PART_PREFIX = "part";
+
+    private final BucketID bucketId;
+
+    private final Path bucketPath;
+
+    private final int subtaskIndex;
+
+    private final PartFileWriter.PartFileFactory<IN, BucketID> partFileFactory;
+
+    private final RecoverableWriter fsWriter;
+
+    private final RollingPolicy<IN, BucketID> rollingPolicy;
+
+    private final NavigableMap<Long, ResumeRecoverable> resumablesPerCheckpoint;
+
+    private final NavigableMap<Long, List<CommitRecoverable>> pendingPartsPerCheckpoint;
+
+    private long partCounter;
+
+    @Nullable
+    private PartFileWriter<IN, BucketID> inProgressPart;
+
+    private List<CommitRecoverable> pendingPartsForCurrentCheckpoint;
+
+    /**
+     * Constructor to create a new empty bucket.
+     */
+    private Bucket(
+            final RecoverableWriter fsWriter,
+            final int subtaskIndex,
+            final BucketID bucketId,
+            final Path bucketPath,
+            final long initialPartCounter,
+            final PartFileWriter.PartFileFactory<IN, BucketID> partFileFactory,
+            final RollingPolicy<IN, BucketID> rollingPolicy) {
+
+        this.fsWriter = checkNotNull(fsWriter);
+        this.subtaskIndex = subtaskIndex;
+        this.bucketId = checkNotNull(bucketId);
+        this.bucketPath = checkNotNull(bucketPath);
+        this.partCounter = initialPartCounter;
+        this.partFileFactory = checkNotNull(partFileFactory);
+        this.rollingPolicy = checkNotNull(rollingPolicy);
+
+        this.pendingPartsForCurrentCheckpoint = new ArrayList<>();
+        this.pendingPartsPerCheckpoint = new TreeMap<>();
+        this.resumablesPerCheckpoint = new TreeMap<>();
+    }
+
+    /**
+     * Constructor to restore a bucket from checkpointed state.
+     */
+    private Bucket(
+            final RecoverableWriter fsWriter,
+            final int subtaskIndex,
+            final long initialPartCounter,
+            final PartFileWriter.PartFileFactory<IN, BucketID> partFileFactory,
+            final RollingPolicy<IN, BucketID> rollingPolicy,
+            final BucketState<BucketID> bucketState) throws IOException {
+
+        this(
+                fsWriter,
+                subtaskIndex,
+                bucketState.getBucketId(),
+                bucketState.getBucketPath(),
+                initialPartCounter,
+                partFileFactory,
+                rollingPolicy);
+
+        restoreInProgressFile(bucketState);
+        commitRecoveredPendingFiles(bucketState);
+    }
+
+    private void restoreInProgressFile(final BucketState<BucketID> state) throws IOException {
+        if (!state.hasInProgressResumableFile()) {
+            return;
+        }
+
+        // we try to resume the previous in-progress file
+        final ResumeRecoverable resumable = state.getInProgressResumableFile();
+
+        if (fsWriter.supportsResume()) {
+            final RecoverableFsDataOutputStream stream = fsWriter.recover(resumable);
+            inProgressPart = partFileFactory.resumeFrom(
+                    bucketId, stream, resumable, state.getInProgressFileCreationTime());
+        } else {
+            // if the writer does not support resume, then we close the
+            // in-progress part and commit it, as done in the case of pending files.
+
+            fsWriter.recoverForCommit(resumable).commitAfterRecovery();
+        }
+    }
+
+    private void commitRecoveredPendingFiles(final BucketState<BucketID> state) throws IOException {
+
+        // we commit pending files for checkpoints that precess the last successful one, from which we are recovering
+        for (List<CommitRecoverable> committables : state.getCommittableFilesPerCheckpoint().values()) {
+            for (CommitRecoverable committable : committables) {
+                fsWriter.recoverForCommit(committable).commitAfterRecovery();
+            }
+        }
+    }
+
+    public BucketID getBucketId() {
+        return bucketId;
+    }
+
+    Path getBucketPath() {
+        return bucketPath;
+    }
+
+    long getPartCounter() {
+        return partCounter;
+    }
+
+    boolean isActive() {
+        return inProgressPart != null || !pendingPartsForCurrentCheckpoint.isEmpty() || !pendingPartsPerCheckpoint
+                .isEmpty();
+    }
+
+    void merge(final Bucket<IN, BucketID> bucket) throws IOException {
+        checkNotNull(bucket);
+        checkState(Objects.equals(bucket.bucketPath, bucketPath));
+
+        checkState(bucket.pendingPartsForCurrentCheckpoint.isEmpty());
+        checkState(bucket.pendingPartsPerCheckpoint.isEmpty());
+
+        CommitRecoverable committable = bucket.closePartFile();
+        if (committable != null) {
+            pendingPartsForCurrentCheckpoint.add(committable);
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Subtask {} merging buckets for bucket id={}", subtaskIndex, bucketId);
+        }
+    }
+
+    void write(IN element, long currentTime) throws IOException {
+        if (inProgressPart == null || rollingPolicy.shouldRollOnEvent(inProgressPart, element)) {
+
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Subtask {} closing in-progress part file for bucket id={} due to element {}.",
+                        subtaskIndex, bucketId, element);
+            }
+
+            rollPartFile(currentTime);
+        }
+        inProgressPart.write(element, currentTime);
+    }
+
+    private void rollPartFile(final long currentTime) throws IOException {
+        closePartFile();
+
+        final Path partFilePath = assembleNewPartPath();
+        final RecoverableFsDataOutputStream stream = fsWriter.open(partFilePath);
+        inProgressPart = partFileFactory.openNew(bucketId, stream, partFilePath, currentTime);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Subtask {} opening new part file \"{}\" for bucket id={}.",
+                    subtaskIndex, partFilePath.getName(), bucketId);
+        }
+
+        partCounter++;
+    }
+
+    private Path assembleNewPartPath() {
+        return new Path(bucketPath, PART_PREFIX + '-' + subtaskIndex + '-' + partCounter);
+    }
+
+    private CommitRecoverable closePartFile() throws IOException {
+        CommitRecoverable committable = null;
+        if (inProgressPart != null) {
+            committable = inProgressPart.closeForCommit();
+            pendingPartsForCurrentCheckpoint.add(committable);
+            inProgressPart = null;
+        }
+        return committable;
+    }
+
+    void disposePartFile() {
+        if (inProgressPart != null) {
+            inProgressPart.dispose();
+        }
+    }
+
+    BucketState<BucketID> onReceptionOfCheckpoint(long checkpointId) throws IOException {
+        prepareBucketForCheckpointing(checkpointId);
+
+        ResumeRecoverable inProgressResumable = null;
+        long inProgressFileCreationTime = Long.MAX_VALUE;
+
+        if (inProgressPart != null) {
+            inProgressResumable = inProgressPart.persist();
+            inProgressFileCreationTime = inProgressPart.getCreationTime();
+
+            // the following is an optimization so that writers that do not
+            // require cleanup, they do not have to keep track of resumables
+            // and later iterate over the active buckets.
+            // (see onSuccessfulCompletionOfCheckpoint())
+
+            if (fsWriter.requiresCleanupOfRecoverableState()) {
+                this.resumablesPerCheckpoint.put(checkpointId, inProgressResumable);
+            }
+        }
+
+        return new BucketState<>(bucketId, bucketPath, inProgressFileCreationTime, inProgressResumable,
+                pendingPartsPerCheckpoint);
+    }
+
+    private void prepareBucketForCheckpointing(long checkpointId) throws IOException {
+        if (inProgressPart != null && rollingPolicy.shouldRollOnCheckpoint(inProgressPart)) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Subtask {} closing in-progress part file for bucket id={} on checkpoint.", subtaskIndex,
+                        bucketId);
+            }
+            closePartFile();
+        }
+
+        if (!pendingPartsForCurrentCheckpoint.isEmpty()) {
+            pendingPartsPerCheckpoint.put(checkpointId, pendingPartsForCurrentCheckpoint);
+            pendingPartsForCurrentCheckpoint = new ArrayList<>();
+        }
+    }
+
+    void onSuccessfulCompletionOfCheckpoint(long checkpointId) throws IOException {
+        checkNotNull(fsWriter);
+
+        Iterator<Map.Entry<Long, List<CommitRecoverable>>> it =
+                pendingPartsPerCheckpoint.headMap(checkpointId, true)
+                        .entrySet().iterator();
+
+        while (it.hasNext()) {
+            Map.Entry<Long, List<CommitRecoverable>> entry = it.next();
+
+            for (CommitRecoverable committable : entry.getValue()) {
+                fsWriter.recoverForCommit(committable).commit();
+            }
+            it.remove();
+        }
+
+        cleanupOutdatedResumables(checkpointId);
+    }
+
+    private void cleanupOutdatedResumables(long checkpointId) throws IOException {
+        Iterator<Map.Entry<Long, ResumeRecoverable>> it =
+                resumablesPerCheckpoint.headMap(checkpointId, false)
+                        .entrySet().iterator();
+
+        while (it.hasNext()) {
+            final ResumeRecoverable recoverable = it.next().getValue();
+
+            // this check is redundant, as we only put entries in the resumablesPerCheckpoint map
+            // list when the requiresCleanupOfRecoverableState() returns true, but having it makes
+            // the code more readable.
+
+            if (fsWriter.requiresCleanupOfRecoverableState()) {
+                final boolean successfullyDeleted = fsWriter.cleanupRecoverableState(recoverable);
+
+                if (LOG.isDebugEnabled() && successfullyDeleted) {
+                    LOG.debug("Subtask {} successfully deleted incomplete part for bucket id={}.", subtaskIndex,
+                            bucketId);
+                }
+            }
+            it.remove();
+        }
+    }
+
+    void onProcessingTime(long timestamp) throws IOException {
+        if (inProgressPart != null && rollingPolicy.shouldRollOnProcessingTime(inProgressPart, timestamp)) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Subtask {} closing in-progress part file for bucket id={} "
+                                + "due to processing time rolling policy "
+                                + "(in-progress file created @ {}, last updated @ {} and current time is {}).",
+                        subtaskIndex, bucketId, inProgressPart.getCreationTime(), inProgressPart.getLastUpdateTime(),
+                        timestamp);
+            }
+            closePartFile();
+        }
+    }
+
+    // --------------------------- Testing Methods -----------------------------
+
+    @VisibleForTesting
+    Map<Long, List<CommitRecoverable>> getPendingPartsPerCheckpoint() {
+        return pendingPartsPerCheckpoint;
+    }
+
+    @Nullable
+    @VisibleForTesting
+    PartFileWriter<IN, BucketID> getInProgressPart() {
+        return inProgressPart;
+    }
+
+    @VisibleForTesting
+    List<CommitRecoverable> getPendingPartsForCurrentCheckpoint() {
+        return pendingPartsForCurrentCheckpoint;
+    }
+
+    // --------------------------- Static Factory Methods -----------------------------
+
+    /**
+     * Creates a new empty {@code Bucket}.
+     *
+     * @param fsWriter the filesystem-specific {@link RecoverableWriter}.
+     * @param subtaskIndex the index of the subtask creating the bucket.
+     * @param bucketId the identifier of the bucket, as returned by the {@link BucketAssigner}.
+     * @param bucketPath the path to where the part files for the bucket will be written to.
+     * @param initialPartCounter the initial counter for the part files of the bucket.
+     * @param partFileFactory the {@link PartFileWriter.PartFileFactory} the factory creating part file writers.
+     * @param <I> the type of input elements to the sink.
+     * @param <B> the type of the identifier of the bucket, as returned by the {@link BucketAssigner}
+     * @return The new Bucket.
+     */
+    static <I, B> Bucket<I, B> getNew(
+            final RecoverableWriter fsWriter,
+            final int subtaskIndex,
+            final B bucketId,
+            final Path bucketPath,
+            final long initialPartCounter,
+            final PartFileWriter.PartFileFactory<I, B> partFileFactory,
+            final RollingPolicy<I, B> rollingPolicy) {
+        return new Bucket<>(fsWriter, subtaskIndex, bucketId, bucketPath, initialPartCounter, partFileFactory,
+                rollingPolicy);
+    }
+
+    /**
+     * Restores a {@code Bucket} from the state included in the provided {@link BucketState}.
+     *
+     * @param fsWriter the filesystem-specific {@link RecoverableWriter}.
+     * @param subtaskIndex the index of the subtask creating the bucket.
+     * @param initialPartCounter the initial counter for the part files of the bucket.
+     * @param partFileFactory the {@link PartFileWriter.PartFileFactory} the factory creating part file writers.
+     * @param bucketState the initial state of the restored bucket.
+     * @param <I> the type of input elements to the sink.
+     * @param <B> the type of the identifier of the bucket, as returned by the {@link BucketAssigner}
+     * @return The restored Bucket.
+     */
+    static <I, B> Bucket<I, B> restore(
+            final RecoverableWriter fsWriter,
+            final int subtaskIndex,
+            final long initialPartCounter,
+            final PartFileWriter.PartFileFactory<I, B> partFileFactory,
+            final RollingPolicy<I, B> rollingPolicy,
+            final BucketState<B> bucketState) throws IOException {
+        return new Bucket<>(fsWriter, subtaskIndex, initialPartCounter, partFileFactory, rollingPolicy, bucketState);
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BucketAssigner.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BucketAssigner.java
new file mode 100644
index 0000000..dba3b1f
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BucketAssigner.java
@@ -0,0 +1,77 @@
+/*
+ * 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.flink.filesystem;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+
+/**
+ * Copied from Apache Flink project {@link org.apache.flink.streaming.api.functions.sink.filesystem.BucketAssigner}.
+ */
+public interface BucketAssigner<IN, BucketID> extends Serializable {
+
+    /**
+     * Returns the identifier of the bucket the provided element should be put into.
+     *
+     * @param element The current element being processed.
+     * @param context The {@link SinkFunction.Context context} used by the {@link StreamingFileSink sink}.
+     * @return A string representing the identifier of the bucket the element should be put into.
+     *         The actual path to the bucket will result from the concatenation of the returned string
+     *         and the {@code base path} provided during the initialization of the {@link StreamingFileSink sink}.
+     */
+    BucketID getBucketId(IN element, BucketAssigner.Context context);
+
+    /**
+     * @return A {@link SimpleVersionedSerializer} capable of serializing/deserializing the elements
+     *         of type {@code BucketID}. That is the type of the objects returned by the
+     *         {@link #getBucketId(Object, BucketAssigner.Context)}.
+     */
+    SimpleVersionedSerializer<BucketID> getSerializer();
+
+    /**
+     * Context that the {@link BucketAssigner} can use for getting additional data about
+     * an input record.
+     *
+     * <p>The context is only valid for the duration of a {@link BucketAssigner#getBucketId(Object,
+     * BucketAssigner.Context)} call.
+     * Do not store the context and use afterwards!
+     */
+    interface Context {
+
+        /**
+         * Returns the current processing time.
+         */
+        long currentProcessingTime();
+
+        /**
+         * Returns the current event-time watermark.
+         */
+        long currentWatermark();
+
+        /**
+         * Returns the timestamp of the current input record or
+         * {@code null} if the element does not have an assigned timestamp.
+         */
+        @Nullable
+        Long timestamp();
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BucketFactory.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BucketFactory.java
new file mode 100644
index 0000000..0b288a1
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BucketFactory.java
@@ -0,0 +1,47 @@
+/*
+ * 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.flink.filesystem;
+
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.fs.RecoverableWriter;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * Copied from Apache Flink project, org.apache.flink.streaming.api.functions.sink.filesystem.BucketFactory.
+ */
+public interface BucketFactory<IN, BucketID> extends Serializable {
+
+    Bucket<IN, BucketID> getNewBucket(
+            final RecoverableWriter fsWriter,
+            final int subtaskIndex,
+            final BucketID bucketId,
+            final Path bucketPath,
+            final long initialPartCounter,
+            final PartFileWriter.PartFileFactory<IN, BucketID> partFileWriterFactory,
+            final RollingPolicy<IN, BucketID> rollingPolicy) throws IOException;
+
+    Bucket<IN, BucketID> restoreBucket(
+            final RecoverableWriter fsWriter,
+            final int subtaskIndex,
+            final long initialPartCounter,
+            final PartFileWriter.PartFileFactory<IN, BucketID> partFileWriterFactory,
+            final RollingPolicy<IN, BucketID> rollingPolicy,
+            final BucketState<BucketID> bucketState) throws IOException;
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BucketState.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BucketState.java
new file mode 100644
index 0000000..be308b9
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BucketState.java
@@ -0,0 +1,118 @@
+/*
+ * 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.flink.filesystem;
+
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.fs.RecoverableWriter;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Copied from Apache Flink project, org.apache.flink.streaming.api.functions.sink.filesystem.BucketState.
+ */
+public class BucketState<BucketID> {
+
+    private final BucketID bucketId;
+
+    /** The directory where all the part files of the bucket are stored. */
+    private final Path bucketPath;
+
+    /**
+     * The creation time of the currently open part file,
+     * or {@code Long.MAX_VALUE} if there is no open part file.
+     */
+    private final long inProgressFileCreationTime;
+
+    /**
+     * A {@link RecoverableWriter.ResumeRecoverable} for the currently open
+     * part file, or null if there is no currently open part file.
+     */
+    @Nullable
+    private final RecoverableWriter.ResumeRecoverable inProgressResumableFile;
+
+    /**
+     * The {@link RecoverableWriter.CommitRecoverable files} pending to be
+     * committed, organized by checkpoint id.
+     */
+    private final Map<Long, List<RecoverableWriter.CommitRecoverable>> committableFilesPerCheckpoint;
+
+    BucketState(
+            final BucketID bucketId,
+            final Path bucketPath,
+            final long inProgressFileCreationTime,
+            @Nullable final RecoverableWriter.ResumeRecoverable inProgressResumableFile,
+            final Map<Long, List<RecoverableWriter.CommitRecoverable>> pendingCommittablesPerCheckpoint
+    ) {
+        this.bucketId = Preconditions.checkNotNull(bucketId);
+        this.bucketPath = Preconditions.checkNotNull(bucketPath);
+        this.inProgressFileCreationTime = inProgressFileCreationTime;
+        this.inProgressResumableFile = inProgressResumableFile;
+        this.committableFilesPerCheckpoint = Preconditions.checkNotNull(pendingCommittablesPerCheckpoint);
+    }
+
+    BucketID getBucketId() {
+        return bucketId;
+    }
+
+    Path getBucketPath() {
+        return bucketPath;
+    }
+
+    long getInProgressFileCreationTime() {
+        return inProgressFileCreationTime;
+    }
+
+    boolean hasInProgressResumableFile() {
+        return inProgressResumableFile != null;
+    }
+
+    @Nullable
+    RecoverableWriter.ResumeRecoverable getInProgressResumableFile() {
+        return inProgressResumableFile;
+    }
+
+    Map<Long, List<RecoverableWriter.CommitRecoverable>> getCommittableFilesPerCheckpoint() {
+        return committableFilesPerCheckpoint;
+    }
+
+    @Override
+    public String toString() {
+        final StringBuilder strBuilder = new StringBuilder();
+
+        strBuilder
+                .append("BucketState for bucketId=").append(bucketId)
+                .append(" and bucketPath=").append(bucketPath);
+
+        if (hasInProgressResumableFile()) {
+            strBuilder.append(", has open part file created @ ").append(inProgressFileCreationTime);
+        }
+
+        if (!committableFilesPerCheckpoint.isEmpty()) {
+            strBuilder.append(", has pending files for checkpoints: {");
+            for (long checkpointId: committableFilesPerCheckpoint.keySet()) {
+                strBuilder.append(checkpointId).append(' ');
+            }
+            strBuilder.append('}');
+        }
+        return strBuilder.toString();
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BucketStateSerializer.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BucketStateSerializer.java
new file mode 100644
index 0000000..4d1d0d3
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BucketStateSerializer.java
@@ -0,0 +1,168 @@
+/*
+ * 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.flink.filesystem;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.fs.RecoverableWriter;
+import org.apache.flink.core.io.SimpleVersionedSerialization;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.core.memory.DataInputDeserializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputSerializer;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * Copied from Apache Flink project, org.apache.flink.streaming.api.functions.sink.filesystem.BucketStateSerializer.
+ */
+public class BucketStateSerializer<BucketID> implements SimpleVersionedSerializer<BucketState<BucketID>> {
+
+    private static final int MAGIC_NUMBER = 0x1e764b79;
+
+    private final SimpleVersionedSerializer<RecoverableWriter.ResumeRecoverable> resumableSerializer;
+
+    private final SimpleVersionedSerializer<RecoverableWriter.CommitRecoverable> commitableSerializer;
+
+    private final SimpleVersionedSerializer<BucketID> bucketIdSerializer;
+
+    BucketStateSerializer(
+            final SimpleVersionedSerializer<RecoverableWriter.ResumeRecoverable> resumableSerializer,
+            final SimpleVersionedSerializer<RecoverableWriter.CommitRecoverable> commitableSerializer,
+            final SimpleVersionedSerializer<BucketID> bucketIdSerializer
+    ) {
+        this.resumableSerializer = Preconditions.checkNotNull(resumableSerializer);
+        this.commitableSerializer = Preconditions.checkNotNull(commitableSerializer);
+        this.bucketIdSerializer = Preconditions.checkNotNull(bucketIdSerializer);
+    }
+
+    @Override
+    public int getVersion() {
+        return 1;
+    }
+
+    @Override
+    public byte[] serialize(BucketState<BucketID> state) throws IOException {
+        DataOutputSerializer out = new DataOutputSerializer(256);
+        out.writeInt(MAGIC_NUMBER);
+        serializeV1(state, out);
+        return out.getCopyOfBuffer();
+    }
+
+    @Override
+    public BucketState<BucketID> deserialize(int version, byte[] serialized) throws IOException {
+        switch (version) {
+            case 1:
+                DataInputDeserializer in = new DataInputDeserializer(serialized);
+                validateMagicNumber(in);
+                return deserializeV1(in);
+            default:
+                throw new IOException("Unrecognized version or corrupt state: " + version);
+        }
+    }
+
+    @VisibleForTesting
+    void serializeV1(BucketState<BucketID> state, DataOutputView out) throws IOException {
+        SimpleVersionedSerialization.writeVersionAndSerialize(bucketIdSerializer, state.getBucketId(), out);
+        out.writeUTF(state.getBucketPath().toString());
+        out.writeLong(state.getInProgressFileCreationTime());
+
+        // put the current open part file
+        if (state.hasInProgressResumableFile()) {
+            final RecoverableWriter.ResumeRecoverable resumable = state.getInProgressResumableFile();
+            out.writeBoolean(true);
+            SimpleVersionedSerialization.writeVersionAndSerialize(resumableSerializer, resumable, out);
+        } else {
+            out.writeBoolean(false);
+        }
+
+        // put the map of pending files per checkpoint
+        final Map<Long, List<RecoverableWriter.CommitRecoverable>> pendingCommitters = state
+                .getCommittableFilesPerCheckpoint();
+
+        // manually keep the version here to safe some bytes
+        out.writeInt(commitableSerializer.getVersion());
+
+        out.writeInt(pendingCommitters.size());
+        for (Entry<Long, List<RecoverableWriter.CommitRecoverable>> resumablesForCheckpoint : pendingCommitters
+                .entrySet()) {
+            List<RecoverableWriter.CommitRecoverable> resumables = resumablesForCheckpoint.getValue();
+
+            out.writeLong(resumablesForCheckpoint.getKey());
+            out.writeInt(resumables.size());
+
+            for (RecoverableWriter.CommitRecoverable resumable : resumables) {
+                byte[] serialized = commitableSerializer.serialize(resumable);
+                out.writeInt(serialized.length);
+                out.write(serialized);
+            }
+        }
+    }
+
+    @VisibleForTesting
+    BucketState<BucketID> deserializeV1(DataInputView in) throws IOException {
+        final BucketID bucketId = SimpleVersionedSerialization.readVersionAndDeSerialize(bucketIdSerializer, in);
+        final String bucketPathStr = in.readUTF();
+        final long creationTime = in.readLong();
+
+        // then get the current resumable stream
+        RecoverableWriter.ResumeRecoverable current = null;
+        if (in.readBoolean()) {
+            current = SimpleVersionedSerialization.readVersionAndDeSerialize(resumableSerializer, in);
+        }
+
+        final int committableVersion = in.readInt();
+        final int numCheckpoints = in.readInt();
+        final HashMap<Long, List<RecoverableWriter.CommitRecoverable>> resumablesPerCheckpoint = new HashMap<>(
+                numCheckpoints);
+
+        for (int i = 0; i < numCheckpoints; i++) {
+            final long checkpointId = in.readLong();
+            final int noOfResumables = in.readInt();
+
+            final List<RecoverableWriter.CommitRecoverable> resumables = new ArrayList<>(noOfResumables);
+            for (int j = 0; j < noOfResumables; j++) {
+                final byte[] bytes = new byte[in.readInt()];
+                in.readFully(bytes);
+                resumables.add(commitableSerializer.deserialize(committableVersion, bytes));
+            }
+            resumablesPerCheckpoint.put(checkpointId, resumables);
+        }
+
+        return new BucketState<>(
+                bucketId,
+                new Path(bucketPathStr),
+                creationTime,
+                current,
+                resumablesPerCheckpoint);
+    }
+
+    private static void validateMagicNumber(DataInputView in) throws IOException {
+        final int magicNumber = in.readInt();
+        if (magicNumber != MAGIC_NUMBER) {
+            throw new IOException(String.format("Corrupt data: Unexpected magic number %08X", magicNumber));
+        }
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/Buckets.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/Buckets.java
new file mode 100644
index 0000000..c4a72c7
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/Buckets.java
@@ -0,0 +1,364 @@
+/*
+ * 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.flink.filesystem;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.fs.RecoverableWriter;
+import org.apache.flink.core.io.SimpleVersionedSerialization;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * Copied from Apache Flink project {@link org.apache.flink.streaming.api.functions.sink.filesystem.Buckets}.
+ */
+public class Buckets<IN, BucketID> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(
+            Buckets.class);
+
+    // ------------------------ configuration fields --------------------------
+
+    private final Path basePath;
+
+    private final BucketFactory<IN, BucketID> bucketFactory;
+
+    private final BucketAssigner<IN, BucketID> bucketAssigner;
+
+    private final PartFileWriter.PartFileFactory<IN, BucketID> partFileWriterFactory;
+
+    private final RollingPolicy<IN, BucketID> rollingPolicy;
+
+    // --------------------------- runtime fields -----------------------------
+
+    private final int subtaskIndex;
+
+    private final Buckets.BucketerContext bucketerContext;
+
+    private final Map<BucketID, Bucket<IN, BucketID>> activeBuckets;
+
+    private long maxPartCounter;
+
+    private final RecoverableWriter fsWriter;
+
+    // --------------------------- State Related Fields -----------------------------
+
+    private final BucketStateSerializer<BucketID> bucketStateSerializer;
+
+    /**
+     * A constructor creating a new empty bucket manager.
+     *
+     * @param basePath The base path for our buckets.
+     * @param bucketAssigner The {@link BucketAssigner} provided by the user.
+     * @param bucketFactory The {@link BucketFactory} to be used to create buckets.
+     * @param partFileWriterFactory The {@link PartFileWriter.PartFileFactory} to be used when writing data.
+     * @param rollingPolicy The {@link RollingPolicy} as specified by the user.
+     */
+    Buckets(
+            final Path basePath,
+            final BucketAssigner<IN, BucketID> bucketAssigner,
+            final BucketFactory<IN, BucketID> bucketFactory,
+            final PartFileWriter.PartFileFactory<IN, BucketID> partFileWriterFactory,
+            final RollingPolicy<IN, BucketID> rollingPolicy,
+            final int subtaskIndex) throws IOException {
+
+        this.basePath = Preconditions.checkNotNull(basePath);
+        this.bucketAssigner = Preconditions.checkNotNull(bucketAssigner);
+        this.bucketFactory = Preconditions.checkNotNull(bucketFactory);
+        this.partFileWriterFactory = Preconditions.checkNotNull(partFileWriterFactory);
+        this.rollingPolicy = Preconditions.checkNotNull(rollingPolicy);
+        this.subtaskIndex = subtaskIndex;
+
+        this.activeBuckets = new HashMap<>();
+        this.bucketerContext = new Buckets.BucketerContext();
+
+        try {
+            this.fsWriter = FileSystem.get(basePath.toUri()).createRecoverableWriter();
+        } catch (IOException e) {
+            LOG.error("Unable to create filesystem for path: {}", basePath);
+            throw e;
+        }
+
+        this.bucketStateSerializer = new BucketStateSerializer<>(
+                fsWriter.getResumeRecoverableSerializer(),
+                fsWriter.getCommitRecoverableSerializer(),
+                bucketAssigner.getSerializer()
+        );
+
+        this.maxPartCounter = 0L;
+    }
+
+    /**
+     * Initializes the state after recovery from a failure.
+     *
+     * <p>During this process:
+     * <ol>
+     *     <li>we set the initial value for part counter to the maximum value used before across all tasks and buckets.
+     *     This guarantees that we do not overwrite valid data,</li>
+     *     <li>we commit any pending files for previous checkpoints
+     *     (previous to the last successful one from which we restore),</li>
+     *     <li>we resume writing to the previous in-progress file of each bucket, and</li>
+     *     <li>if we receive multiple states for the same bucket, we merge them.</li>
+     * </ol>
+     *
+     * @param bucketStates the state holding recovered state about active buckets.
+     * @param partCounterState the state holding the max previously used part counters.
+     * @throws Exception if anything goes wrong during retrieving the state or restoring/committing of any
+     *         in-progress/pending part files
+     */
+    void initializeState(final ListState<byte[]> bucketStates, final ListState<Long> partCounterState)
+            throws Exception {
+
+        initializePartCounter(partCounterState);
+
+        LOG.info("Subtask {} initializing its state (max part counter={}).", subtaskIndex, maxPartCounter);
+
+        initializeActiveBuckets(bucketStates);
+    }
+
+    private void initializePartCounter(final ListState<Long> partCounterState) throws Exception {
+        long maxCounter = 0L;
+        for (long partCounter : partCounterState.get()) {
+            maxCounter = Math.max(partCounter, maxCounter);
+        }
+        maxPartCounter = maxCounter;
+    }
+
+    private void initializeActiveBuckets(final ListState<byte[]> bucketStates) throws Exception {
+        for (byte[] serializedRecoveredState : bucketStates.get()) {
+            final BucketState<BucketID> recoveredState =
+                    SimpleVersionedSerialization.readVersionAndDeSerialize(
+                            bucketStateSerializer, serializedRecoveredState);
+            handleRestoredBucketState(recoveredState);
+        }
+    }
+
+    private void handleRestoredBucketState(final BucketState<BucketID> recoveredState) throws Exception {
+        final BucketID bucketId = recoveredState.getBucketId();
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Subtask {} restoring: {}", subtaskIndex, recoveredState);
+        }
+
+        final Bucket<IN, BucketID> restoredBucket = bucketFactory
+                .restoreBucket(
+                        fsWriter,
+                        subtaskIndex,
+                        maxPartCounter,
+                        partFileWriterFactory,
+                        rollingPolicy,
+                        recoveredState
+                );
+
+        updateActiveBucketId(bucketId, restoredBucket);
+    }
+
+    private void updateActiveBucketId(final BucketID bucketId, final Bucket<IN, BucketID> restoredBucket)
+            throws IOException {
+        if (!restoredBucket.isActive()) {
+            return;
+        }
+
+        final Bucket<IN, BucketID> bucket = activeBuckets.get(bucketId);
+        if (bucket != null) {
+            bucket.merge(restoredBucket);
+        } else {
+            activeBuckets.put(bucketId, restoredBucket);
+        }
+    }
+
+    void commitUpToCheckpoint(final long checkpointId) throws IOException {
+        final Iterator<Map.Entry<BucketID, Bucket<IN, BucketID>>> activeBucketIt =
+                activeBuckets.entrySet().iterator();
+
+        LOG.info("Subtask {} received completion notification for checkpoint with id={}.", subtaskIndex, checkpointId);
+
+        while (activeBucketIt.hasNext()) {
+            final Bucket<IN, BucketID> bucket = activeBucketIt.next().getValue();
+            bucket.onSuccessfulCompletionOfCheckpoint(checkpointId);
+
+            if (!bucket.isActive()) {
+                // We've dealt with all the pending files and the writer for this bucket is not currently open.
+                // Therefore this bucket is currently inactive and we can remove it from our state.
+                activeBucketIt.remove();
+            }
+        }
+    }
+
+    void snapshotState(
+            final long checkpointId,
+            final ListState<byte[]> bucketStatesContainer,
+            final ListState<Long> partCounterStateContainer) throws Exception {
+
+        Preconditions.checkState(
+                fsWriter != null && bucketStateSerializer != null,
+                "sink has not been initialized");
+
+        LOG.info("Subtask {} checkpointing for checkpoint with id={} (max part counter={}).",
+                subtaskIndex, checkpointId, maxPartCounter);
+
+        bucketStatesContainer.clear();
+        partCounterStateContainer.clear();
+
+        snapshotActiveBuckets(checkpointId, bucketStatesContainer);
+        partCounterStateContainer.add(maxPartCounter);
+    }
+
+    private void snapshotActiveBuckets(
+            final long checkpointId,
+            final ListState<byte[]> bucketStatesContainer) throws Exception {
+
+        for (Bucket<IN, BucketID> bucket : activeBuckets.values()) {
+            final BucketState<BucketID> bucketState = bucket.onReceptionOfCheckpoint(checkpointId);
+
+            final byte[] serializedBucketState = SimpleVersionedSerialization
+                    .writeVersionAndSerialize(bucketStateSerializer, bucketState);
+
+            bucketStatesContainer.add(serializedBucketState);
+
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Subtask {} checkpointing: {}", subtaskIndex, bucketState);
+            }
+        }
+    }
+
+    Bucket<IN, BucketID> onElement(final IN value, final SinkFunction.Context context) throws Exception {
+        final long currentProcessingTime = context.currentProcessingTime();
+
+        // setting the values in the bucketer context
+        bucketerContext.update(
+                context.timestamp(),
+                context.currentWatermark(),
+                currentProcessingTime);
+
+        final BucketID bucketId = bucketAssigner.getBucketId(value, bucketerContext);
+        final Bucket<IN, BucketID> bucket = getOrCreateBucketForBucketId(bucketId);
+        bucket.write(value, currentProcessingTime);
+
+        // we update the global max counter here because as buckets become inactive and
+        // get removed from the list of active buckets, at the time when we want to create
+        // another part file for the bucket, if we start from 0 we may overwrite previous parts.
+
+        this.maxPartCounter = Math.max(maxPartCounter, bucket.getPartCounter());
+        return bucket;
+    }
+
+    private Bucket<IN, BucketID> getOrCreateBucketForBucketId(final BucketID bucketId) throws IOException {
+        Bucket<IN, BucketID> bucket = activeBuckets.get(bucketId);
+        if (bucket == null) {
+            final Path bucketPath = assembleBucketPath(bucketId);
+            bucket = bucketFactory.getNewBucket(
+                    fsWriter,
+                    subtaskIndex,
+                    bucketId,
+                    bucketPath,
+                    maxPartCounter,
+                    partFileWriterFactory,
+                    rollingPolicy);
+            activeBuckets.put(bucketId, bucket);
+        }
+        return bucket;
+    }
+
+    void onProcessingTime(long timestamp) throws Exception {
+        for (Bucket<IN, BucketID> bucket : activeBuckets.values()) {
+            bucket.onProcessingTime(timestamp);
+        }
+    }
+
+    void close() {
+        if (activeBuckets != null) {
+            activeBuckets.values().forEach(Bucket::disposePartFile);
+        }
+    }
+
+    private Path assembleBucketPath(BucketID bucketId) {
+        final String child = bucketId.toString();
+        if ("".equals(child)) {
+            return basePath;
+        }
+        return new Path(basePath, child);
+    }
+
+    /**
+     * The {@link BucketAssigner.Context} exposed to the
+     * {@link BucketAssigner#getBucketId(Object, BucketAssigner.Context)}
+     * whenever a new incoming element arrives.
+     */
+    private static final class BucketerContext implements BucketAssigner.Context {
+
+        @Nullable
+        private Long elementTimestamp;
+
+        private long currentWatermark;
+
+        private long currentProcessingTime;
+
+        private BucketerContext() {
+            this.elementTimestamp = null;
+            this.currentWatermark = Long.MIN_VALUE;
+            this.currentProcessingTime = Long.MIN_VALUE;
+        }
+
+        void update(@Nullable Long elementTimestamp, long watermark, long processingTime) {
+            this.elementTimestamp = elementTimestamp;
+            this.currentWatermark = watermark;
+            this.currentProcessingTime = processingTime;
+        }
+
+        @Override
+        public long currentProcessingTime() {
+            return currentProcessingTime;
+        }
+
+        @Override
+        public long currentWatermark() {
+            return currentWatermark;
+        }
+
+        @Override
+        @Nullable
+        public Long timestamp() {
+            return elementTimestamp;
+        }
+    }
+
+    // --------------------------- Testing Methods -----------------------------
+
+    @VisibleForTesting
+    public long getMaxPartCounter() {
+        return maxPartCounter;
+    }
+
+    @VisibleForTesting
+    Map<BucketID, Bucket<IN, BucketID>> getActiveBuckets() {
+        return activeBuckets;
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BulkPartWriter.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BulkPartWriter.java
new file mode 100644
index 0000000..a7a7689
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/BulkPartWriter.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.flink.filesystem;
+
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.fs.RecoverableFsDataOutputStream;
+import org.apache.flink.core.fs.RecoverableWriter;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+
+/**
+ * Copied from Apache Flink project, org.apache.flink.streaming.api.functions.sink.filesystem.BulkPartWriter.
+ */
+public class BulkPartWriter<IN, BucketID> extends PartFileWriter<IN, BucketID> {
+
+    private final BulkWriter<IN> writer;
+
+    private BulkPartWriter(
+            final BucketID bucketId,
+            final RecoverableFsDataOutputStream currentPartStream,
+            final BulkWriter<IN> writer,
+            final long creationTime) {
+        super(bucketId, currentPartStream, creationTime);
+        this.writer = Preconditions.checkNotNull(writer);
+    }
+
+    @Override
+    void write(IN element, long currentTime) throws IOException {
+        writer.addElement(element);
+        markWrite(currentTime);
+    }
+
+    @Override
+    RecoverableWriter.ResumeRecoverable persist() {
+        throw new UnsupportedOperationException("Bulk Part Writers do not support \"pause and resume\" operations.");
+    }
+
+    @Override
+    RecoverableWriter.CommitRecoverable closeForCommit() throws IOException {
+        writer.flush();
+        writer.finish();
+        return super.closeForCommit();
+    }
+
+    /**
+     * A factory that creates {@link BulkPartWriter BulkPartWriters}.
+     * @param <IN> The type of input elements.
+     * @param <BucketID> The type of ids for the buckets, as returned by the {@link BucketAssigner}.
+     */
+    static class Factory<IN, BucketID> implements PartFileWriter.PartFileFactory<IN, BucketID> {
+
+        private final BulkWriter.Factory<IN> writerFactory;
+
+        Factory(BulkWriter.Factory<IN> writerFactory) {
+            this.writerFactory = writerFactory;
+        }
+
+        @Override
+        public PartFileWriter<IN, BucketID> resumeFrom(
+                final BucketID bucketId,
+                final RecoverableFsDataOutputStream stream,
+                final RecoverableWriter.ResumeRecoverable resumable,
+                final long creationTime) throws IOException {
+
+            Preconditions.checkNotNull(stream);
+            Preconditions.checkNotNull(resumable);
+
+            final BulkWriter<IN> writer = writerFactory.create(stream);
+            return new BulkPartWriter<>(bucketId, stream, writer, creationTime);
+        }
+
+        @Override
+        public PartFileWriter<IN, BucketID> openNew(
+                final BucketID bucketId,
+                final RecoverableFsDataOutputStream stream,
+                final Path path,
+                final long creationTime) throws IOException {
+
+            Preconditions.checkNotNull(stream);
+            Preconditions.checkNotNull(path);
+
+            final BulkWriter<IN> writer = writerFactory.create(stream);
+            return new BulkPartWriter<>(bucketId, stream, writer, creationTime);
+        }
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/DefaultBucketFactoryImpl.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/DefaultBucketFactoryImpl.java
new file mode 100644
index 0000000..4ca26d8
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/DefaultBucketFactoryImpl.java
@@ -0,0 +1,70 @@
+/*
+ * 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.flink.filesystem;
+
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.fs.RecoverableWriter;
+
+import java.io.IOException;
+
+/**
+ * Copied from Apache Flink project, org.apache.flink.streaming.api.functions.sink.filesystem.DefaultBucketFactoryImpl.
+ */
+public class DefaultBucketFactoryImpl<IN, BucketID> implements
+        BucketFactory<IN, BucketID> {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public Bucket<IN, BucketID> getNewBucket(
+            final RecoverableWriter fsWriter,
+            final int subtaskIndex,
+            final BucketID bucketId,
+            final Path bucketPath,
+            final long initialPartCounter,
+            final PartFileWriter.PartFileFactory<IN, BucketID> partFileWriterFactory,
+            final RollingPolicy<IN, BucketID> rollingPolicy) {
+
+        return Bucket.getNew(
+                fsWriter,
+                subtaskIndex,
+                bucketId,
+                bucketPath,
+                initialPartCounter,
+                partFileWriterFactory,
+                rollingPolicy);
+    }
+
+    @Override
+    public Bucket<IN, BucketID> restoreBucket(
+            final RecoverableWriter fsWriter,
+            final int subtaskIndex,
+            final long initialPartCounter,
+            final PartFileWriter.PartFileFactory<IN, BucketID> partFileWriterFactory,
+            final RollingPolicy<IN, BucketID> rollingPolicy,
+            final BucketState<BucketID> bucketState) throws IOException {
+
+        return Bucket.restore(
+                fsWriter,
+                subtaskIndex,
+                initialPartCounter,
+                partFileWriterFactory,
+                rollingPolicy,
+                bucketState);
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/DefaultRollingPolicy.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/DefaultRollingPolicy.java
new file mode 100644
index 0000000..4c1c2ce
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/DefaultRollingPolicy.java
@@ -0,0 +1,140 @@
+/*
+ * 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.flink.filesystem;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+
+/**
+ * Copied from Apache Flink project,
+ * {@link org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy}.
+ */
+public final class DefaultRollingPolicy<IN, BucketID> implements RollingPolicy<IN, BucketID> {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final long DEFAULT_INACTIVITY_INTERVAL = 60L * 1000L;
+
+    private static final long DEFAULT_ROLLOVER_INTERVAL = 60L * 1000L;
+
+    private static final long DEFAULT_MAX_PART_SIZE = 1024L * 1024L * 128L;
+
+    private final long partSize;
+
+    private final long rolloverInterval;
+
+    private final long inactivityInterval;
+
+    /**
+     * Private constructor to avoid direct instantiation.
+     */
+    private DefaultRollingPolicy(long partSize, long rolloverInterval, long inactivityInterval) {
+        Preconditions.checkArgument(partSize > 0L);
+        Preconditions.checkArgument(rolloverInterval > 0L);
+        Preconditions.checkArgument(inactivityInterval > 0L);
+
+        this.partSize = partSize;
+        this.rolloverInterval = rolloverInterval;
+        this.inactivityInterval = inactivityInterval;
+    }
+
+    @Override
+    public boolean shouldRollOnCheckpoint(PartFileInfo<BucketID> partFileState) throws IOException {
+        return partFileState.getSize() > partSize;
+    }
+
+    @Override
+    public boolean shouldRollOnEvent(PartFileInfo<BucketID> partFileState, IN element) throws IOException {
+        return partFileState.getSize() > partSize;
+    }
+
+    @Override
+    public boolean shouldRollOnProcessingTime(final PartFileInfo<BucketID> partFileState, final long currentTime) {
+        return currentTime - partFileState.getCreationTime() >= rolloverInterval
+                || currentTime - partFileState.getLastUpdateTime() >= inactivityInterval;
+    }
+
+    /**
+     * Initiates the instantiation of a {@code DefaultRollingPolicy}.
+     * To finalize it and have the actual policy, call {@code .create()}.
+     */
+    public static DefaultRollingPolicy.PolicyBuilder create() {
+        return new DefaultRollingPolicy.PolicyBuilder(
+                DEFAULT_MAX_PART_SIZE,
+                DEFAULT_ROLLOVER_INTERVAL,
+                DEFAULT_INACTIVITY_INTERVAL);
+    }
+
+    /**
+     * A helper class that holds the configuration properties for the {@link DefaultRollingPolicy}.
+     */
+    public static final class PolicyBuilder {
+
+        private final long partSize;
+
+        private final long rolloverInterval;
+
+        private final long inactivityInterval;
+
+        private PolicyBuilder(
+                final long partSize,
+                final long rolloverInterval,
+                final long inactivityInterval) {
+            this.partSize = partSize;
+            this.rolloverInterval = rolloverInterval;
+            this.inactivityInterval = inactivityInterval;
+        }
+
+        /**
+         * Sets the part size above which a part file will have to roll.
+         *
+         * @param size the allowed part size.
+         */
+        public DefaultRollingPolicy.PolicyBuilder withMaxPartSize(final long size) {
+            Preconditions.checkState(size > 0L);
+            return new DefaultRollingPolicy.PolicyBuilder(size, rolloverInterval, inactivityInterval);
+        }
+
+        /**
+         * Sets the interval of allowed inactivity after which a part file will have to roll.
+         *
+         * @param interval the allowed inactivity interval.
+         */
+        public DefaultRollingPolicy.PolicyBuilder withInactivityInterval(final long interval) {
+            Preconditions.checkState(interval > 0L);
+            return new DefaultRollingPolicy.PolicyBuilder(partSize, rolloverInterval, interval);
+        }
+
+        /**
+         * Sets the max time a part file can stay open before having to roll.
+         *
+         * @param interval the desired rollover interval.
+         */
+        public DefaultRollingPolicy.PolicyBuilder withRolloverInterval(final long interval) {
+            Preconditions.checkState(interval > 0L);
+            return new DefaultRollingPolicy.PolicyBuilder(partSize, interval, inactivityInterval);
+        }
+
+        /**
+         * Creates the actual policy.
+         */
+        public <I, B> DefaultRollingPolicy<I, B> build() {
+            return new DefaultRollingPolicy<>(partSize, rolloverInterval, inactivityInterval);
+        }
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/PartFileInfo.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/PartFileInfo.java
new file mode 100644
index 0000000..aeaa11d
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/PartFileInfo.java
@@ -0,0 +1,47 @@
+/*
+ * 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.flink.filesystem;
+
+import java.io.IOException;
+
+/**
+ * Copied from Apache Flink project, {@link org.apache.flink.streaming.api.functions.sink.filesystem.PartFileInfo}.
+ */
+public interface PartFileInfo<BucketID> {
+
+    /**
+     * @return The bucket identifier of the current buffer, as returned by the
+     * {@link BucketAssigner#getBucketId(Object, BucketAssigner.Context)}.
+     */
+    BucketID getBucketId();
+
+    /**
+     * @return The creation time (in ms) of the currently open part file.
+     */
+    long getCreationTime();
+
+    /**
+     * @return The size of the currently open part file.
+     */
+    long getSize() throws IOException;
+
+    /**
+     * @return The last time (in ms) the currently open part file was written to.
+     */
+    long getLastUpdateTime();
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/PartFileWriter.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/PartFileWriter.java
new file mode 100644
index 0000000..c6e4b39
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/PartFileWriter.java
@@ -0,0 +1,130 @@
+/*
+ * 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.flink.filesystem;
+
+import com.google.common.base.Preconditions;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.fs.RecoverableFsDataOutputStream;
+import org.apache.flink.core.fs.RecoverableWriter;
+import org.apache.flink.util.IOUtils;
+
+import java.io.IOException;
+
+/**
+ * Copied from Apache Flink project, org.apache.flink.streaming.api.functions.sink.filesystem.PartFileWriter.
+ */
+public abstract class PartFileWriter<IN, BucketID> implements PartFileInfo<BucketID> {
+
+    private final BucketID bucketId;
+
+    private final long creationTime;
+
+    protected final RecoverableFsDataOutputStream currentPartStream;
+
+    private long lastUpdateTime;
+
+    protected PartFileWriter(
+            final BucketID bucketId,
+            final RecoverableFsDataOutputStream currentPartStream,
+            final long creationTime) {
+
+        Preconditions.checkArgument(creationTime >= 0L);
+        this.bucketId = Preconditions.checkNotNull(bucketId);
+        this.currentPartStream = Preconditions.checkNotNull(currentPartStream);
+        this.creationTime = creationTime;
+        this.lastUpdateTime = creationTime;
+    }
+
+    abstract void write(IN element, long currentTime) throws IOException;
+
+    RecoverableWriter.ResumeRecoverable persist() throws IOException {
+        return currentPartStream.persist();
+    }
+
+    RecoverableWriter.CommitRecoverable closeForCommit() throws IOException {
+        return currentPartStream.closeForCommit().getRecoverable();
+    }
+
+    void dispose() {
+        // we can suppress exceptions here, because we do not rely on close() to
+        // flush or persist any data
+        IOUtils.closeQuietly(currentPartStream);
+    }
+
+    void markWrite(long now) {
+        this.lastUpdateTime = now;
+    }
+
+    @Override
+    public BucketID getBucketId() {
+        return bucketId;
+    }
+
+    @Override
+    public long getCreationTime() {
+        return creationTime;
+    }
+
+    @Override
+    public long getSize() throws IOException {
+        return currentPartStream.getPos();
+    }
+
+    @Override
+    public long getLastUpdateTime() {
+        return lastUpdateTime;
+    }
+
+    // ------------------------------------------------------------------------
+
+    /**
+     * An interface for factories that create the different {@link PartFileWriter writers}.
+     */
+    public interface PartFileFactory<IN, BucketID> {
+
+        /**
+         * Used upon recovery from a failure to recover a {@link PartFileWriter writer}.
+         * @param bucketId the id of the bucket this writer is writing to.
+         * @param stream the filesystem-specific output stream to use when writing to the filesystem.
+         * @param resumable the state of the stream we are resurrecting.
+         * @param creationTime the creation time of the stream.
+         * @return the recovered {@link PartFileWriter writer}.
+         * @throws IOException
+         */
+        PartFileWriter<IN, BucketID> resumeFrom(
+                final BucketID bucketId,
+                final RecoverableFsDataOutputStream stream,
+                final RecoverableWriter.ResumeRecoverable resumable,
+                final long creationTime) throws IOException;
+
+        /**
+         * Used to create a new {@link PartFileWriter writer}.
+         * @param bucketId the id of the bucket this writer is writing to.
+         * @param stream the filesystem-specific output stream to use when writing to the filesystem.
+         * @param path the part this writer will write to.
+         * @param creationTime the creation time of the stream.
+         * @return the new {@link PartFileWriter writer}.
+         * @throws IOException
+         */
+        PartFileWriter<IN, BucketID> openNew(
+                final BucketID bucketId,
+                final RecoverableFsDataOutputStream stream,
+                final Path path,
+                final long creationTime) throws IOException;
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/RollingPolicy.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/RollingPolicy.java
new file mode 100644
index 0000000..50b0b4e
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/RollingPolicy.java
@@ -0,0 +1,54 @@
+/*
+ * 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.flink.filesystem;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * Copied from Apache Flink project, {@link org.apache.flink.streaming.api.functions.sink.filesystem.RollingPolicy}.
+ */
+public interface RollingPolicy<IN, BucketID> extends Serializable {
+
+    /**
+     * Determines if the in-progress part file for a bucket should roll on every checkpoint.
+     *
+     * @param partFileState the state of the currently open part file of the bucket.
+     * @return {@code True} if the part file should roll, {@link false} otherwise.
+     */
+    boolean shouldRollOnCheckpoint(final PartFileInfo<BucketID> partFileState) throws IOException;
+
+    /**
+     * Determines if the in-progress part file for a bucket should roll based on its current state, e.g. its size.
+     *
+     * @param element the element being processed.
+     * @param partFileState the state of the currently open part file of the bucket.
+     * @return {@code True} if the part file should roll, {@link false} otherwise.
+     */
+    boolean shouldRollOnEvent(final PartFileInfo<BucketID> partFileState, IN element) throws IOException;
+
+    /**
+     * Determines if the in-progress part file for a bucket should roll based on a time condition.
+     *
+     * @param partFileState the state of the currently open part file of the bucket.
+     * @param currentTime the current processing time.
+     * @return {@code True} if the part file should roll, {@link false} otherwise.
+     */
+    boolean shouldRollOnProcessingTime(final PartFileInfo<BucketID> partFileState, final long currentTime)
+            throws IOException;
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/StreamingFileSink.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/StreamingFileSink.java
new file mode 100644
index 0000000..73db29f
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/filesystem/StreamingFileSink.java
@@ -0,0 +1,220 @@
+/*
+ * 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.flink.filesystem;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.io.Serializable;
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.OperatorStateStore;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+
+/**
+ * Copied from Apache Flink project {@link org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink}.
+ */
+public class StreamingFileSink<IN>
+        extends RichSinkFunction<IN>
+        implements CheckpointedFunction, CheckpointListener, ProcessingTimeCallback {
+
+    private static final long serialVersionUID = 1L;
+
+    // -------------------------- state descriptors ---------------------------
+
+    private static final ListStateDescriptor<byte[]> BUCKET_STATE_DESC =
+            new ListStateDescriptor<>("bucket-states", BytePrimitiveArraySerializer.INSTANCE);
+
+    private static final ListStateDescriptor<Long> MAX_PART_COUNTER_STATE_DESC =
+            new ListStateDescriptor<>("max-part-counter", LongSerializer.INSTANCE);
+
+    // ------------------------ configuration fields --------------------------
+
+    private final long bucketCheckInterval;
+
+    private final StreamingFileSink.BucketsBuilder<IN, ?> bucketsBuilder;
+
+    // --------------------------- runtime fields -----------------------------
+
+    private transient Buckets<IN, ?> buckets;
+
+    private transient ProcessingTimeService processingTimeService;
+
+    // --------------------------- State Related Fields -----------------------------
+
+    private transient ListState<byte[]> bucketStates;
+
+    private transient ListState<Long> maxPartCountersState;
+
+    /**
+     * Creates a new {@code StreamingFileSink} that writes files to the given base directory.
+     */
+    public StreamingFileSink(
+            final StreamingFileSink.BucketsBuilder<IN, ?> bucketsBuilder,
+            final long bucketCheckInterval) {
+
+        Preconditions.checkArgument(bucketCheckInterval > 0L);
+
+        this.bucketsBuilder = Preconditions.checkNotNull(bucketsBuilder);
+        this.bucketCheckInterval = bucketCheckInterval;
+    }
+
+    // ------------------------------------------------------------------------
+
+    // --------------------------- Sink Builders  -----------------------------
+
+    /**
+     * The base abstract class for the {@link StreamingFileSink.BulkFormatBuilder}.
+     */
+    protected abstract static class BucketsBuilder<IN, BucketID> implements Serializable {
+
+        private static final long serialVersionUID = 1L;
+
+        abstract Buckets<IN, BucketID> createBuckets(final int subtaskIndex) throws IOException;
+    }
+
+    /**
+     * A builder for configuring the sink for bulk-encoding formats, e.g. Parquet/ORC.
+     */
+    public static class BulkFormatBuilder<IN, BucketID> extends StreamingFileSink.BucketsBuilder<IN, BucketID> {
+
+        private static final long serialVersionUID = 1L;
+
+        private long bucketCheckInterval = 60L * 1000L;
+
+        private final Path basePath;
+
+        private final BulkWriter.Factory<IN> writerFactory;
+
+        private final BucketAssigner<IN, BucketID> bucketAssigner;
+
+        private BucketFactory<IN, BucketID> bucketFactory = new DefaultBucketFactoryImpl<>();
+
+        private final RollingPolicy<IN, BucketID> rollingPolicy;
+
+        public BulkFormatBuilder(
+                Path basePath,
+                BulkWriter.Factory<IN> writerFactory,
+                BucketAssigner<IN, BucketID> assigner,
+                RollingPolicy<IN, BucketID> rollingPolicy) {
+            this.basePath = Preconditions.checkNotNull(basePath);
+            this.writerFactory = writerFactory;
+            this.bucketAssigner = Preconditions.checkNotNull(assigner);
+            this.rollingPolicy = Preconditions.checkNotNull(rollingPolicy);
+        }
+
+        public StreamingFileSink.BulkFormatBuilder<IN, BucketID> withBucketCheckInterval(long interval) {
+            this.bucketCheckInterval = interval;
+            return this;
+        }
+
+        public StreamingFileSink.BulkFormatBuilder<IN, BucketID> withBucketFactory(
+                final BucketFactory<IN, BucketID> factory) {
+            this.bucketFactory = Preconditions.checkNotNull(factory);
+            return this;
+        }
+
+        /**
+         * Creates the actual sink.
+         */
+        public StreamingFileSink<IN> build() {
+            return new StreamingFileSink<>(this, bucketCheckInterval);
+        }
+
+        @Override
+        Buckets<IN, BucketID> createBuckets(int subtaskIndex) throws IOException {
+            return new Buckets<>(
+                    basePath,
+                    bucketAssigner,
+                    bucketFactory,
+                    new BulkPartWriter.Factory<>(writerFactory),
+                    rollingPolicy,
+                    subtaskIndex);
+        }
+    }
+
+    // --------------------------- Sink Methods -----------------------------
+
+    @Override
+    public void initializeState(FunctionInitializationContext context) throws Exception {
+        final int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
+        this.buckets = bucketsBuilder.createBuckets(subtaskIndex);
+
+        final OperatorStateStore stateStore = context.getOperatorStateStore();
+        bucketStates = stateStore.getListState(BUCKET_STATE_DESC);
+        maxPartCountersState = stateStore.getUnionListState(MAX_PART_COUNTER_STATE_DESC);
+
+        if (context.isRestored()) {
+            buckets.initializeState(bucketStates, maxPartCountersState);
+        }
+    }
+
+    @Override
+    public void notifyCheckpointComplete(long checkpointId) throws Exception {
+        buckets.commitUpToCheckpoint(checkpointId);
+    }
+
+    @Override
+    public void snapshotState(FunctionSnapshotContext context) throws Exception {
+        Preconditions.checkState(bucketStates != null && maxPartCountersState != null, "sink has not been initialized");
+
+        buckets.snapshotState(
+                context.getCheckpointId(),
+                bucketStates,
+                maxPartCountersState);
+    }
+
+    @Override
+    public void open(Configuration parameters) throws Exception {
+        super.open(parameters);
+        this.processingTimeService = ((StreamingRuntimeContext) getRuntimeContext()).getProcessingTimeService();
+        long currentProcessingTime = processingTimeService.getCurrentProcessingTime();
+        processingTimeService.registerTimer(currentProcessingTime + bucketCheckInterval, this);
+    }
+
+    @Override
+    public void onProcessingTime(long timestamp) throws Exception {
+        final long currentTime = processingTimeService.getCurrentProcessingTime();
+        buckets.onProcessingTime(currentTime);
+        processingTimeService.registerTimer(currentTime + bucketCheckInterval, this);
+    }
+
+    @Override
+    public void invoke(IN value, SinkFunction.Context context) throws Exception {
+        buckets.onElement(value, context);
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (buckets != null) {
+            buckets.close();
+        }
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/hive/HiveCommitter.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/hive/HiveCommitter.java
new file mode 100644
index 0000000..3e4770a
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/hive/HiveCommitter.java
@@ -0,0 +1,114 @@
+/*
+ * 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.flink.hive;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.inlong.sort.configuration.Constants.SINK_HIVE_COMMITTED_PARTITIONS_CACHE_SIZE;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.inlong.sort.configuration.Configuration;
+import org.apache.inlong.sort.flink.hive.partition.HivePartition;
+import org.apache.inlong.sort.flink.hive.partition.PartitionCommitInfo;
+import org.apache.inlong.sort.flink.hive.partition.PartitionCommitPolicy;
+import org.apache.inlong.sort.protocol.sink.HiveSinkInfo;
+
+/**
+ * Hive sink committer.
+ */
+public class HiveCommitter extends RichSinkFunction<PartitionCommitInfo> {
+
+    private static final long serialVersionUID = 3070346386139023521L;
+
+    private final Configuration configuration;
+
+    private final HiveSinkInfo hiveSinkInfo;
+
+    private final PartitionCommitPolicy.Factory commitPolicyFactory;
+
+    private transient Map<HivePartition, Boolean> committedPartitions;
+
+    private transient PartitionCommitPolicy policy;
+
+    private transient CommitPolicyContextImpl commitPolicyContext;
+
+    public HiveCommitter(
+            Configuration configuration,
+            HiveSinkInfo hiveSinkInfo,
+            PartitionCommitPolicy.Factory commitPolicyFactory) {
+        this.configuration = configuration;
+        this.hiveSinkInfo = hiveSinkInfo;
+        this.commitPolicyFactory = checkNotNull(commitPolicyFactory);
+    }
+
+    @Override
+    public void open(org.apache.flink.configuration.Configuration parameters) throws Exception {
+        commitPolicyContext = new CommitPolicyContextImpl();
+        policy = commitPolicyFactory.create(configuration, hiveSinkInfo);
+        final int cacheSize = configuration.getInteger(SINK_HIVE_COMMITTED_PARTITIONS_CACHE_SIZE);
+        //noinspection serial
+        committedPartitions = new LinkedHashMap<HivePartition, Boolean>(cacheSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<HivePartition, Boolean> eldest) {
+                return size() > cacheSize;
+            }
+        };
+    }
+
+    @Override
+    public void close() throws Exception {
+        committedPartitions.clear();
+        policy.close();
+    }
+
+    @Override
+    public void invoke(PartitionCommitInfo value, Context context) throws Exception {
+        for (HivePartition partition : value.getPartitions()) {
+            if (committedPartitions.get(partition) == null) {
+                committedPartitions.put(partition, true);
+                policy.commit(commitPolicyContext.setHivePartition(partition));
+            }
+        }
+    }
+
+    private class CommitPolicyContextImpl implements PartitionCommitPolicy.Context {
+
+        private HivePartition hivePartition;
+
+        public CommitPolicyContextImpl setHivePartition(HivePartition hivePartition) {
+            this.hivePartition = hivePartition;
+            return this;
+        }
+
+        @Override
+        public String databaseName() {
+            return hiveSinkInfo.getDatabaseName();
+        }
+
+        @Override
+        public String tableName() {
+            return hiveSinkInfo.getTableName();
+        }
+
+        @Override
+        public HivePartition partition() {
+            return hivePartition;
+        }
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/hive/HiveMultiTenantCommitter.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/hive/HiveMultiTenantCommitter.java
new file mode 100644
index 0000000..7eb9cd5
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/hive/HiveMultiTenantCommitter.java
@@ -0,0 +1,161 @@
+/*
+ * 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.flink.hive;
+
+import com.google.common.base.Preconditions;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.util.Collector;
+import org.apache.inlong.sort.configuration.Configuration;
+import org.apache.inlong.sort.flink.hive.partition.JdbcHivePartitionCommitPolicy;
+import org.apache.inlong.sort.flink.hive.partition.PartitionCommitInfo;
+import org.apache.inlong.sort.flink.hive.partition.PartitionCommitPolicy;
+import org.apache.inlong.sort.meta.MetaManager;
+import org.apache.inlong.sort.meta.MetaManager.DataFlowInfoListener;
+import org.apache.inlong.sort.protocol.DataFlowInfo;
+import org.apache.inlong.sort.protocol.sink.HiveSinkInfo;
+import org.apache.inlong.sort.protocol.sink.SinkInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("SynchronizeOnNonFinalField")
+public class HiveMultiTenantCommitter extends ProcessFunction<PartitionCommitInfo, Void> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(HiveMultiTenantCommitter.class);
+
+    private static final long serialVersionUID = -8084389212282039905L;
+
+    private final Configuration configuration;
+
+    private final PartitionCommitPolicy.Factory commitPolicyFactory;
+
+    private transient Map<Long, HiveCommitter> committers;
+
+    private transient SinkContextProxy contextProxy;
+
+    public HiveMultiTenantCommitter(Configuration configuration) {
+        this(configuration, new JdbcHivePartitionCommitPolicy.Factory());
+    }
+
+    public HiveMultiTenantCommitter(Configuration configuration, PartitionCommitPolicy.Factory commitPolicyFactory) {
+        this.configuration = Preconditions.checkNotNull(configuration);
+        this.commitPolicyFactory = Preconditions.checkNotNull(commitPolicyFactory);
+    }
+
+    @Override
+    public void open(org.apache.flink.configuration.Configuration parameters) throws Exception {
+        committers = new HashMap<>();
+        MetaManager metaManager = MetaManager.getInstance(configuration);
+        metaManager.registerDataFlowInfoListener(new DataFlowInfoListenerImpl());
+        contextProxy = new SinkContextProxy();
+    }
+
+    @Override
+    public void close() throws Exception {
+        MetaManager.release();
+        synchronized (committers) {
+            for (HiveCommitter committer: committers.values()) {
+                committer.close();
+            }
+            committers.clear();
+        }
+    }
+
+    @Override
+    public void processElement(PartitionCommitInfo commitInfo, Context context, Collector<Void> collector)
+            throws Exception {
+        final long dataFlowId = commitInfo.getDataFlowId();
+
+        synchronized (committers) {
+            final HiveCommitter committer = committers.get(dataFlowId);
+            if (committer == null) {
+                LOG.warn("Cannot get DataFlowInfo with id {}", dataFlowId);
+                return;
+            }
+
+            committer.invoke(commitInfo, contextProxy.setContext(context));
+        }
+    }
+
+    private static class SinkContextProxy implements SinkFunction.Context<PartitionCommitInfo> {
+
+        private Context context;
+
+        private SinkContextProxy setContext(Context context) {
+            this.context = context;
+            return this;
+        }
+
+        @Override
+        public long currentProcessingTime() {
+            return context.timerService().currentProcessingTime();
+        }
+
+        @Override
+        public long currentWatermark() {
+            return context.timerService().currentWatermark();
+        }
+
+        @Override
+        public Long timestamp() {
+            return context.timestamp();
+        }
+    }
+
+    private class DataFlowInfoListenerImpl implements DataFlowInfoListener {
+
+        @Override
+        public void addDataFlow(DataFlowInfo dataFlowInfo) throws Exception {
+            synchronized (committers) {
+                long dataFlowId = dataFlowInfo.getId();
+                SinkInfo sinkInfo = dataFlowInfo.getSinkInfo();
+                if (!(sinkInfo instanceof HiveSinkInfo)) {
+                    LOG.error("SinkInfo type {} of dataFlow {} doesn't match application sink type 'hive'!",
+                            sinkInfo.getClass(), dataFlowId);
+                    return;
+                }
+                HiveSinkInfo hiveSinkInfo = (HiveSinkInfo) sinkInfo;
+                HiveCommitter hiveCommitter = new HiveCommitter(configuration, hiveSinkInfo, commitPolicyFactory);
+                hiveCommitter.setRuntimeContext(getRuntimeContext());
+                hiveCommitter.open(new org.apache.flink.configuration.Configuration());
+                committers.put(dataFlowId, hiveCommitter);
+            }
+        }
+
+        @Override
+        public void updateDataFlow(DataFlowInfo dataFlowInfo) throws Exception {
+            synchronized (committers) {
+                removeDataFlow(dataFlowInfo);
+                addDataFlow(dataFlowInfo);
+            }
+        }
+
+        @Override
+        public void removeDataFlow(DataFlowInfo dataFlowInfo) throws Exception {
+            synchronized (committers) {
+                final HiveCommitter existingHiveCommitter = committers.remove(dataFlowInfo.getId());
+                if (existingHiveCommitter != null) {
+                    existingHiveCommitter.close();
+                }
+            }
+        }
+
+    }
+}
diff --git a/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/hive/HiveMultiTenantWriter.java b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/hive/HiveMultiTenantWriter.java
new file mode 100644
index 0000000..fb71fd9
--- /dev/null
+++ b/inlong-sort/core/src/main/java/org/apache/inlong/sort/flink/hive/HiveMultiTenantWriter.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.flink.hive;
+
+import com.google.common.base.Preconditions;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.OutputTag;
+import org.apache.inlong.sort.configuration.Configuration;
+import org.apache.inlong.sort.configuration.Constants;
+import org.apache.inlong.sort.flink.SerializedRecord;
+import org.apache.inlong.sort.flink.hive.partition.PartitionCommitInfo;
+import org.apache.inlong.sort.flink.multitenant.MultiTenantFunctionInitializationContext;
+import org.apache.inlong.sort.flink.transformation.RecordTransformer;
+import org.apache.inlong.sort.meta.MetaManager;
+import org.apache.inlong.sort.meta.MetaManager.DataFlowInfoListener;
+import org.apache.inlong.sort.protocol.DataFlowInfo;
+import org.apache.inlong.sort.protocol.sink.HiveSinkInfo;
+import org.apache.inlong.sort.protocol.sink.SinkInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("SynchronizeOnNonFinalField")
+public class HiveMultiTenantWriter extends ProcessFunction<SerializedRecord, PartitionCommitInfo>
+        implements CheckpointedFunction, CheckpointListener {
+
+    private static final long serialVersionUID = -3254800832674516362L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(HiveMultiTenantWriter.class);
+
+    private final Configuration configuration;
+
+    private transient Map<Long, HiveWriter> hiveWriters;
+
+    private transient RecordTransformer recordTransformer;
+
+    private transient FunctionInitializationContext functionInitializationContext;
+
+    private transient ProcessFunctionContext proxyContext;
+
+    public HiveMultiTenantWriter(Configuration configuration) {
+        this.configuration = Preconditions.checkNotNull(configuration);
+    }
+
... 21012 lines suppressed ...