[Improve]Improve doris connector (#539)

* [Improve]Improve doris connector
diff --git a/connectors/rocketmq-connect-doris/pom.xml b/connectors/rocketmq-connect-doris/pom.xml
index 7ed0e95..660aa46 100644
--- a/connectors/rocketmq-connect-doris/pom.xml
+++ b/connectors/rocketmq-connect-doris/pom.xml
@@ -49,10 +49,13 @@
         <!--rocket connect api-->
         <openmessaging-connector.version>0.1.4</openmessaging-connector.version>
         <openmessaging-api.version>0.3.1-alpha</openmessaging-api.version>
-
-        <!--fast json-->
-        <fastjson.version>1.2.83</fastjson.version>
-
+        <metrics.version>4.2.25</metrics.version>
+        <jackson.version>2.13.2.1</jackson.version>
+        <rocketmq-connect.version>0.0.1-SNAPSHOT</rocketmq-connect.version>
+        <debezium.version>1.9.8.Final</debezium.version>
+        <geometry.version>2.2.0</geometry.version>
+        <commons-io.version>2.3</commons-io.version>
+        <mysql-connector.version>8.0.33</mysql-connector.version>
     </properties>
 
     <dependencies>
@@ -94,27 +97,60 @@
             <scope>test</scope>
         </dependency>
 
-        <!--fast json version-->
         <dependency>
-            <groupId>com.alibaba</groupId>
-            <artifactId>fastjson</artifactId>
-            <version>${fastjson.version}</version>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>${jackson.version}</version>
         </dependency>
         <dependency>
-            <groupId>ch.qos.logback</groupId>
-            <artifactId>logback-classic</artifactId>
-            <version>1.2.0</version>
-        </dependency>
-        <dependency>
-            <groupId>ch.qos.logback</groupId>
-            <artifactId>logback-core</artifactId>
-            <version>1.2.9</version>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-connect-runtime</artifactId>
+            <version>${rocketmq-connect.version}</version>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.httpcomponents</groupId>
             <artifactId>httpclient</artifactId>
             <version>4.5.13</version>
         </dependency>
+        <dependency>
+            <groupId>io.dropwizard.metrics</groupId>
+            <artifactId>metrics-core</artifactId>
+            <version>${metrics.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>io.dropwizard.metrics</groupId>
+            <artifactId>metrics-jmx</artifactId>
+            <version>${metrics.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>io.debezium</groupId>
+            <artifactId>debezium-core</artifactId>
+            <version>${debezium.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.esri.geometry</groupId>
+            <artifactId>esri-geometry-api</artifactId>
+            <version>${geometry.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-core</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+            <version>${commons-io.version}</version>
+        </dependency>
+        <!-- https://mvnrepository.com/artifact/mysql/mysql-connector-java -->
+        <dependency>
+            <groupId>mysql</groupId>
+            <artifactId>mysql-connector-java</artifactId>
+            <version>${mysql-connector.version}</version>
+        </dependency>
     </dependencies>
 
 
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/DorisSinkConnector.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/DorisSinkConnector.java
new file mode 100644
index 0000000..88244ef
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/DorisSinkConnector.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.rocketmq.connect.doris;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.task.Task;
+import io.openmessaging.connector.api.component.task.sink.SinkConnector;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.rocketmq.connect.doris.cfg.DorisSinkConnectorConfig;
+import org.apache.rocketmq.connect.doris.utils.ConfigCheckUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DorisSinkConnector extends SinkConnector {
+    private static final Logger LOG = LoggerFactory.getLogger(DorisSinkConnector.class);
+    private KeyValue keyValue;
+
+    @Override
+    public void start(KeyValue keyValue) {
+        this.keyValue = DorisSinkConnectorConfig.convertToLowercase(keyValue);
+        DorisSinkConnectorConfig.setDefaultValues(this.keyValue);
+        ConfigCheckUtils.validateConfig(this.keyValue);
+    }
+
+    /**
+     * stop DorisSinkConnector
+     */
+    @Override
+    public void stop() {
+        LOG.info("doris sink connector stop");
+    }
+
+    @Override
+    public Class<? extends Task> taskClass() {
+        return DorisSinkTask.class;
+    }
+
+    @Override
+    public List<KeyValue> taskConfigs(final int maxTasks) {
+        List<KeyValue> taskConfigs = new ArrayList<>(maxTasks);
+        for (int i = 0; i < maxTasks; i++) {
+            keyValue.put("task_id", i + "");
+            taskConfigs.add(this.keyValue);
+        }
+        return taskConfigs;
+    }
+
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/DorisSinkTask.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/DorisSinkTask.java
new file mode 100644
index 0000000..fa14a6a
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/DorisSinkTask.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.rocketmq.connect.doris;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.task.sink.SinkTask;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.RecordOffset;
+import io.openmessaging.connector.api.data.RecordPartition;
+import io.openmessaging.connector.api.errors.ConnectException;
+import java.util.List;
+import java.util.Map;
+import org.apache.rocketmq.connect.doris.service.DorisSinkService;
+import org.apache.rocketmq.connect.doris.service.DorisSinkServiceFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DorisSinkTask extends SinkTask {
+    private static final Logger LOG = LoggerFactory.getLogger(DorisSinkTask.class);
+    private DorisSinkService sink;
+
+    @Override
+    public void start(KeyValue keyValue) {
+        LOG.info("rocketmq doris sink task start");
+        this.sink = DorisSinkServiceFactory.getDorisSinkService(keyValue);
+    }
+
+    @Override
+    public void put(List<ConnectRecord> sinkRecords) throws ConnectException {
+        LOG.info("Read {} records from Kafka", sinkRecords.size());
+        sink.insert(sinkRecords);
+    }
+
+    /**
+     * Support doris's two-phase commit
+     */
+    @Override
+    public void flush(Map<RecordPartition, RecordOffset> currentOffsets) throws ConnectException {
+        if (sink == null || sink.getDorisWriterSize() == 0) {
+            return;
+        }
+        sink.commit(currentOffsets);
+    }
+
+    @Override
+    public void stop() {
+        LOG.info("rocketmq doris sink task stopped");
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/cfg/DorisOptions.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/cfg/DorisOptions.java
new file mode 100644
index 0000000..a1164d7
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/cfg/DorisOptions.java
@@ -0,0 +1,387 @@
+/*
+ * 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.rocketmq.connect.doris.cfg;
+
+import io.openmessaging.KeyValue;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.stream.Collectors;
+import org.apache.rocketmq.connect.doris.converter.ConverterMode;
+import org.apache.rocketmq.connect.doris.converter.schema.SchemaEvolutionMode;
+import org.apache.rocketmq.connect.doris.exception.DorisException;
+import org.apache.rocketmq.connect.doris.utils.ConfigCheckUtils;
+import org.apache.rocketmq.connect.doris.writer.DeliveryGuarantee;
+import org.apache.rocketmq.connect.doris.writer.load.LoadModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DorisOptions {
+    private static final Logger LOG = LoggerFactory.getLogger(DorisOptions.class);
+    private final String name;
+    private final String urls;
+    private final int queryPort;
+    private final int httpPort;
+    private final String user;
+    private final String password;
+    private final String database;
+    private final Map<String, String> topicMap;
+    private final int fileSize;
+    private final int recordNum;
+    private final long flushTime;
+    private final boolean enableCustomJMX;
+    private final int taskId;
+    private final boolean enableDelete;
+    private boolean enable2PC = true;
+    private boolean force2PC;
+    private boolean autoRedirect = true;
+    private int requestReadTimeoutMs;
+    private int requestConnectTimeoutMs;
+    private final boolean enableGroupCommit;
+    private boolean customCluster;
+    private ProxyConfig proxyConfig;
+    /**
+     * Properties for the StreamLoad.
+     */
+    private final Properties streamLoadProp;
+    private final String databaseTimeZone;
+    private final LoadModel loadModel;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final ConverterMode converterMode;
+    private final SchemaEvolutionMode schemaEvolutionMode;
+
+    public DorisOptions(KeyValue config) {
+        this.name = config.getString(DorisSinkConnectorConfig.NAME);
+        this.urls = config.getString(DorisSinkConnectorConfig.DORIS_URLS);
+        this.queryPort = Integer.parseInt(config.getString(DorisSinkConnectorConfig.DORIS_QUERY_PORT));
+        this.httpPort = Integer.parseInt(config.getString(DorisSinkConnectorConfig.DORIS_HTTP_PORT));
+        this.user = config.getString(DorisSinkConnectorConfig.DORIS_USER);
+        this.password = config.getString(DorisSinkConnectorConfig.DORIS_PASSWORD);
+        this.database = config.getString(DorisSinkConnectorConfig.DORIS_DATABASE);
+        this.taskId = Integer.parseInt(config.getString(ConfigCheckUtils.TASK_ID));
+        this.databaseTimeZone = config.getString(DorisSinkConnectorConfig.DATABASE_TIME_ZONE);
+        this.loadModel = LoadModel.of(config.getString(DorisSinkConnectorConfig.LOAD_MODEL));
+        this.deliveryGuarantee =
+            DeliveryGuarantee.of(config.getString(DorisSinkConnectorConfig.DELIVERY_GUARANTEE));
+        this.converterMode = ConverterMode.of(config.getString(DorisSinkConnectorConfig.CONVERTER_MODE));
+        this.schemaEvolutionMode =
+            SchemaEvolutionMode.of(
+                config.getString(DorisSinkConnectorConfig.DEBEZIUM_SCHEMA_EVOLUTION));
+        this.fileSize = Integer.parseInt(config.getString(DorisSinkConnectorConfig.BUFFER_SIZE_BYTES));
+        this.recordNum =
+            Integer.parseInt(config.getString(DorisSinkConnectorConfig.BUFFER_COUNT_RECORDS));
+
+        this.flushTime = Long.parseLong(config.getString(DorisSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC));
+        this.topicMap = getTopicToTableMap(config);
+
+        if (config.containsKey(DorisSinkConnectorConfig.ENABLE_2PC)) {
+            if (Boolean.parseBoolean(config.getString(DorisSinkConnectorConfig.ENABLE_2PC))) {
+                this.enable2PC = true;
+                this.force2PC = true;
+            } else {
+                this.enable2PC = false;
+            }
+        }
+        this.enableCustomJMX = Boolean.parseBoolean(config.getString(DorisSinkConnectorConfig.JMX_OPT));
+        this.enableDelete =
+            Boolean.parseBoolean(config.getString(DorisSinkConnectorConfig.ENABLE_DELETE));
+        this.requestConnectTimeoutMs =
+            DorisSinkConnectorConfig.DORIS_REQUEST_CONNECT_TIMEOUT_MS_DEFAULT;
+        this.requestReadTimeoutMs = DorisSinkConnectorConfig.DORIS_REQUEST_READ_TIMEOUT_MS_DEFAULT;
+        if (config.containsKey(DorisSinkConnectorConfig.AUTO_REDIRECT)) {
+            this.autoRedirect =
+                Boolean.parseBoolean(config.getString(DorisSinkConnectorConfig.AUTO_REDIRECT));
+        }
+        if (config.containsKey(DorisSinkConnectorConfig.REQUEST_CONNECT_TIMEOUT_MS)) {
+            this.requestConnectTimeoutMs =
+                Integer.parseInt(
+                    config.getString(DorisSinkConnectorConfig.REQUEST_CONNECT_TIMEOUT_MS));
+        }
+        if (config.containsKey(DorisSinkConnectorConfig.REQUEST_READ_TIMEOUT_MS)) {
+            this.requestReadTimeoutMs =
+                Integer.parseInt(config.getString(DorisSinkConnectorConfig.REQUEST_READ_TIMEOUT_MS));
+        }
+        if (config.containsKey(DorisSinkConnectorConfig.DORIS_CUSTOM_CLUSTER)) {
+            this.customCluster = Boolean.parseBoolean(config.getString(DorisSinkConnectorConfig.DORIS_CUSTOM_CLUSTER));
+            parseClusterProxyConfig(config);
+        }
+        this.streamLoadProp = getStreamLoadPropFromConfig(config);
+        this.enableGroupCommit = ConfigCheckUtils.validateGroupCommitMode(this);
+    }
+
+    private void parseClusterProxyConfig(KeyValue config) {
+        if (customCluster) {
+            String socks5Endpoint = config.getString(DorisSinkConnectorConfig.SOCKS5_ENDPOINT);
+            String socks5UserName = config.getString(DorisSinkConnectorConfig.SOCKS5_USERNAME);
+            String socks5Password = config.getString(DorisSinkConnectorConfig.SOCKET5_PASSWORD);
+            if (socks5Endpoint == null || socks5UserName == null || socks5Password == null) {
+                throw new DorisException(
+                    "Currently it is doris custom cluster mode, and socks5Endpoint, socks5UserName, socks5Password need to be provided.");
+            }
+            this.proxyConfig = new ProxyConfig(socks5Endpoint, socks5UserName, socks5Password);
+        }
+    }
+
+    private Properties getStreamLoadPropFromConfig(KeyValue config) {
+        Properties properties = new Properties();
+        properties.putAll(getStreamLoadDefaultValues());
+        for (String key : config.keySet()) {
+            if (key.startsWith(DorisSinkConnectorConfig.STREAM_LOAD_PROP_PREFIX)) {
+                String subKey = key.substring(DorisSinkConnectorConfig.STREAM_LOAD_PROP_PREFIX.length());
+                properties.put(subKey, config.getString(key));
+            }
+        }
+        return properties;
+    }
+
+    private Properties getStreamLoadDefaultValues() {
+        Properties properties = new Properties();
+        properties.setProperty("format", "json");
+        properties.setProperty("read_json_by_line", "true");
+        return properties;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getUrls() {
+        return urls;
+    }
+
+    public int getQueryPort() {
+        return queryPort;
+    }
+
+    public int getHttpPort() {
+        return httpPort;
+    }
+
+    public String getUser() {
+        return user;
+    }
+
+    public String getPassword() {
+        return password;
+    }
+
+    public String getDatabase() {
+        return database;
+    }
+
+    public int getFileSize() {
+        return fileSize;
+    }
+
+    public int getRecordNum() {
+        return recordNum;
+    }
+
+    public long getFlushTime() {
+        return flushTime;
+    }
+
+    public String getTopicMapTable(String topic) {
+        return topicMap.get(topic);
+    }
+
+    public boolean force2PC() {
+        return force2PC;
+    }
+
+    public void setEnable2PC(boolean enable2PC) {
+        this.enable2PC = enable2PC;
+    }
+
+    public boolean enableGroupCommit() {
+        return enableGroupCommit;
+    }
+
+    public boolean enable2PC() {
+        return enable2PC;
+    }
+
+    public Map<String, String> getTopicMap() {
+        return topicMap;
+    }
+
+    public String getQueryUrl() {
+        List<String> queryUrls = getQueryUrls();
+        return queryUrls.get(0);
+    }
+
+    public String getHttpUrl() {
+        List<String> httpUrls = getHttpUrls();
+        return httpUrls.get(0);
+    }
+
+    public List<String> getQueryUrls() {
+        List<String> queryUrls = new ArrayList<>();
+        if (urls.contains(",")) {
+            queryUrls =
+                Arrays.stream(urls.split(","))
+                    .map(
+                        url -> {
+                            return url.trim() + ":" + queryPort;
+                        })
+                    .collect(Collectors.toList());
+            Collections.shuffle(queryUrls);
+            return queryUrls;
+        }
+        queryUrls.add(urls + ":" + queryPort);
+        return queryUrls;
+    }
+
+    public List<String> getHttpUrls() {
+        List<String> httpUrls = new ArrayList<>();
+        if (urls.contains(",")) {
+            httpUrls =
+                Arrays.stream(urls.split(","))
+                    .map(
+                        url -> {
+                            return url.trim() + ":" + httpPort;
+                        })
+                    .collect(Collectors.toList());
+            Collections.shuffle(httpUrls);
+            return httpUrls;
+        }
+        httpUrls.add(urls + ":" + httpPort);
+        return httpUrls;
+    }
+
+    public Integer getRequestReadTimeoutMs() {
+        return this.requestReadTimeoutMs;
+    }
+
+    public Integer getRequestConnectTimeoutMs() {
+        return this.requestConnectTimeoutMs;
+    }
+
+    public Properties getStreamLoadProp() {
+        return streamLoadProp;
+    }
+
+    public boolean isEnableCustomJMX() {
+        return enableCustomJMX;
+    }
+
+    public int getTaskId() {
+        return taskId;
+    }
+
+    public LoadModel getLoadModel() {
+        return loadModel;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return this.deliveryGuarantee;
+    }
+
+    public ConverterMode getConverterMode() {
+        return this.converterMode;
+    }
+
+    public SchemaEvolutionMode getSchemaEvolutionMode() {
+        return this.schemaEvolutionMode;
+    }
+
+    public boolean isAutoRedirect() {
+        return autoRedirect;
+    }
+
+    public String getDatabaseTimeZone() {
+        return databaseTimeZone;
+    }
+
+    public boolean isEnableDelete() {
+        return enableDelete;
+    }
+
+    public boolean customCluster() {
+        return customCluster;
+    }
+
+    public Optional<ProxyConfig> getProxyConfig() {
+        return Optional.ofNullable(proxyConfig);
+    }
+
+    /**
+     * parse topic to table map
+     *
+     * @param config connector config file
+     * @return result map
+     */
+    private Map<String, String> getTopicToTableMap(KeyValue config) {
+        if (config.containsKey(DorisSinkConnectorConfig.TOPICS_TABLES_MAP)) {
+            Map<String, String> result =
+                ConfigCheckUtils.parseTopicToTableMap(
+                    config.getString(DorisSinkConnectorConfig.TOPICS_TABLES_MAP));
+            if (result != null) {
+                return result;
+            }
+            LOG.error("Invalid Input, Topic2Table Map disabled");
+        }
+        return new HashMap<>();
+    }
+
+    public class ProxyConfig {
+        private final String socks5Endpoint;
+        private final String socks5UserName;
+        private final String socks5Password;
+        private final String socks5Host;
+        private final Integer socks5Port;
+
+        public ProxyConfig(String socks5Endpoint, String socks5UserName, String socks5Password) {
+            this.socks5Endpoint = socks5Endpoint;
+            this.socks5UserName = socks5UserName;
+            this.socks5Password = socks5Password;
+            String[] splitEndpoint = socks5Endpoint.split(":");
+            socks5Host = splitEndpoint[0];
+            socks5Port = Integer.parseInt(splitEndpoint[1]);
+            assert socks5Host != null;
+        }
+
+        public String getSocks5Endpoint() {
+            return socks5Endpoint;
+        }
+
+        public String getSocks5Host() {
+            return socks5Host;
+        }
+
+        public String getSocks5Password() {
+            return socks5Password;
+        }
+
+        public Integer getSocks5Port() {
+            return socks5Port;
+        }
+
+        public String getSocks5UserName() {
+            return socks5UserName;
+        }
+    }
+
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/cfg/DorisSinkConnectorConfig.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/cfg/DorisSinkConnectorConfig.java
new file mode 100644
index 0000000..f523253
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/cfg/DorisSinkConnectorConfig.java
@@ -0,0 +1,134 @@
+/*
+ * 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.rocketmq.connect.doris.cfg;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.internal.DefaultKeyValue;
+import java.util.Locale;
+import org.apache.rocketmq.connect.doris.DorisSinkConnector;
+import org.apache.rocketmq.connect.doris.converter.ConverterMode;
+import org.apache.rocketmq.connect.doris.converter.schema.SchemaEvolutionMode;
+import org.apache.rocketmq.connect.doris.writer.DeliveryGuarantee;
+import org.apache.rocketmq.connect.doris.writer.load.LoadModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Doris SinkConnectorConfig
+ */
+public class DorisSinkConnectorConfig {
+    private static final Logger LOG = LoggerFactory.getLogger(DorisSinkConnector.class);
+
+    public static final String NAME = "name";
+    public static final String TOPICS = "topics";
+    public static final String TOPICS_REGEX = "topics.regex";
+
+    // Connector config
+    public static final String BUFFER_COUNT_RECORDS = "buffer.count.records";
+    public static final long BUFFER_COUNT_RECORDS_DEFAULT = 10000;
+    public static final String BUFFER_SIZE_BYTES = "buffer.size.bytes";
+    public static final long BUFFER_SIZE_BYTES_DEFAULT = 5000000;
+    public static final long BUFFER_SIZE_BYTES_MIN = 1;
+    public static final String TOPICS_TABLES_MAP = "doris.topic2table.map";
+    public static final String LABEL_PREFIX = "label.prefix";
+
+    // Time in seconds
+    public static final long BUFFER_FLUSH_TIME_SEC_MIN = 10;
+    public static final long BUFFER_FLUSH_TIME_SEC_DEFAULT = 120;
+    public static final String BUFFER_FLUSH_TIME_SEC = "buffer.flush.time";
+
+    private static final String DORIS_INFO = "Doris Info";
+
+    // doris config
+    public static final String DORIS_URLS = "doris.urls";
+    public static final String DORIS_QUERY_PORT = "doris.query.port";
+    public static final String DORIS_HTTP_PORT = "doris.http.port";
+    public static final String DORIS_USER = "doris.user";
+    public static final String DORIS_PASSWORD = "doris.password";
+    public static final String DORIS_DATABASE = "doris.database";
+    public static final String REQUEST_READ_TIMEOUT_MS = "request.read.timeout.ms";
+    public static final String REQUEST_CONNECT_TIMEOUT_MS = "request.connect.timeout.ms";
+    public static final Integer DORIS_REQUEST_READ_TIMEOUT_MS_DEFAULT = 30 * 1000;
+    public static final Integer DORIS_REQUEST_CONNECT_TIMEOUT_MS_DEFAULT = 30 * 1000;
+    public static final String DATABASE_TIME_ZONE = "database.time_zone";
+    public static final String DATABASE_TIME_ZONE_DEFAULT = "UTC";
+    public static final String LOAD_MODEL = "load.model";
+    public static final String LOAD_MODEL_DEFAULT = LoadModel.STREAM_LOAD.name();
+    public static final String AUTO_REDIRECT = "auto.redirect";
+    public static final String DELIVERY_GUARANTEE = "delivery.guarantee";
+    public static final String DELIVERY_GUARANTEE_DEFAULT = DeliveryGuarantee.AT_LEAST_ONCE.name();
+    public static final String CONVERTER_MODE = "converter.mode";
+    public static final String CONVERT_MODE_DEFAULT = ConverterMode.NORMAL.getName();
+
+    // Prefix for Doris StreamLoad specific properties.
+    public static final String STREAM_LOAD_PROP_PREFIX = "sink.properties.";
+    public static final String DEBEZIUM_SCHEMA_EVOLUTION = "debezium.schema.evolution";
+    public static final String DEBEZIUM_SCHEMA_EVOLUTION_DEFAULT =
+        SchemaEvolutionMode.NONE.getName();
+
+    // custom cluster config
+    public static final String DORIS_CUSTOM_CLUSTER = "doris.custom.cluster";
+    public static final String DORIS_CUSTOM_CLUSTER_DEFAULT = "false";
+    public static final String SOCKS5_ENDPOINT = "socks5Endpoint";
+    public static final String SOCKS5_USERNAME = "socks5UserName";
+    public static final String SOCKET5_PASSWORD = "socks5Password";
+
+    // metrics
+    public static final String JMX_OPT = "jmx";
+    public static final boolean JMX_OPT_DEFAULT = true;
+
+    public static final String ENABLE_DELETE = "enable.delete";
+    public static final boolean ENABLE_DELETE_DEFAULT = false;
+    public static final String ENABLE_2PC = "enable.2pc";
+    public static final boolean ENABLE_2PC_DEFAULT = true;
+
+    public static void setDefaultValues(KeyValue config) {
+        setFieldToDefaultValues(
+            config, BUFFER_COUNT_RECORDS, String.valueOf(BUFFER_COUNT_RECORDS_DEFAULT));
+        setFieldToDefaultValues(
+            config, BUFFER_SIZE_BYTES, String.valueOf(BUFFER_SIZE_BYTES_DEFAULT));
+        setFieldToDefaultValues(
+            config, BUFFER_FLUSH_TIME_SEC, String.valueOf(BUFFER_FLUSH_TIME_SEC_DEFAULT));
+        setFieldToDefaultValues(config, DATABASE_TIME_ZONE, DATABASE_TIME_ZONE_DEFAULT);
+        setFieldToDefaultValues(config, LOAD_MODEL, LOAD_MODEL_DEFAULT);
+        setFieldToDefaultValues(config, DELIVERY_GUARANTEE, DELIVERY_GUARANTEE_DEFAULT);
+        setFieldToDefaultValues(config, CONVERTER_MODE, CONVERT_MODE_DEFAULT);
+        setFieldToDefaultValues(
+            config, DEBEZIUM_SCHEMA_EVOLUTION, DEBEZIUM_SCHEMA_EVOLUTION_DEFAULT);
+        setFieldToDefaultValues(config, JMX_OPT, String.valueOf(JMX_OPT_DEFAULT));
+        setFieldToDefaultValues(config, DORIS_CUSTOM_CLUSTER, DORIS_CUSTOM_CLUSTER_DEFAULT);
+    }
+
+    private static void setFieldToDefaultValues(KeyValue config, String field, String value) {
+        if (!config.containsKey(field)) {
+            config.put(field, value);
+            LOG.info("Set the default value of {} to {}", field, value);
+        }
+    }
+
+    public static KeyValue convertToLowercase(KeyValue config) {
+        KeyValue newConfig = new DefaultKeyValue();
+        for (String key : config.keySet()) {
+            String value = config.getString(key);
+            newConfig.put(key.toLowerCase(Locale.ROOT), value);
+        }
+        return newConfig;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/config/AbstractConfig.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/config/AbstractConfig.java
deleted file mode 100644
index 5ff2b2a..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/config/AbstractConfig.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.config;
-import io.openmessaging.KeyValue;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Objects;
-import java.util.regex.Pattern;
-
-/**
- * abstract config
- */
-public abstract class AbstractConfig {
-
-    private static final Pattern COMMA_WITH_WHITESPACE = Pattern.compile("\\s*,\\s*");
-
-    // connection url
-    public static final String CONNECTION_PREFIX = "connection.";
-    public static final String CONNECTION_URL_CONFIG = CONNECTION_PREFIX + "url";
-    // connection user
-    public static final String CONNECTION_USER_CONFIG = CONNECTION_PREFIX + "user";
-    private static final String CONNECTION_USER_DOC = "JDBC connection user.";
-    // connection password
-    public static final String CONNECTION_PASSWORD_CONFIG = CONNECTION_PREFIX + "password";
-    private static final String CONNECTION_PASSWORD_DOC = "JDBC connection password.";
-    // connection attempts
-    public static final String CONNECTION_ATTEMPTS_CONFIG = CONNECTION_PREFIX + "attempts";
-    public static final String CONNECTION_ATTEMPTS_DOC = "Maximum number of attempts to retrieve a valid JDBC connection.Must be a positive integer.";
-    public static final int CONNECTION_ATTEMPTS_DEFAULT = 3;
-    // backoff ms
-    public static final String CONNECTION_BACKOFF_CONFIG = CONNECTION_PREFIX + "backoff.ms";
-    public static final String CONNECTION_BACKOFF_DOC = "Backoff time in milliseconds between connection attempts.";
-    public static final long CONNECTION_BACKOFF_DEFAULT = 10000L;
-    /**
-     * quote.sql.identifiers
-     */
-    public static final String QUOTE_SQL_IDENTIFIERS_CONFIG = "quote.sql.identifiers";
-    public static final String QUOTE_SQL_IDENTIFIERS_DOC =
-            "When to quote table names, column names, and other identifiers in SQL statements. "
-                    + "For backward compatibility, the default is ``always``.";
-
-
-    private String connectionDbUrl;
-    private String connectionDbUser;
-    private String connectionDbPassword;
-    private Integer attempts;
-    private Long backoffMs;
-    private String quoteSqlIdentifiers;
-
-    public AbstractConfig(KeyValue config) {
-        connectionDbUrl = config.getString(CONNECTION_URL_CONFIG);
-        connectionDbUser = config.getString(CONNECTION_USER_CONFIG);
-        connectionDbPassword = config.getString(CONNECTION_PASSWORD_CONFIG);
-        attempts = config.getInt(CONNECTION_ATTEMPTS_CONFIG, CONNECTION_ATTEMPTS_DEFAULT);
-        backoffMs = config.getLong(CONNECTION_BACKOFF_CONFIG, CONNECTION_BACKOFF_DEFAULT);
-    }
-
-
-    public String getConnectionDbUrl() {
-        return connectionDbUrl;
-    }
-
-    public String getConnectionDbUser() {
-        return connectionDbUser;
-    }
-
-    public String getConnectionDbPassword() {
-        return connectionDbPassword;
-    }
-
-    public Integer getAttempts() {
-        return attempts;
-    }
-
-    public Long getBackoffMs() {
-        return backoffMs;
-    }
-
-    public String getQuoteSqlIdentifiers() {
-        return quoteSqlIdentifiers;
-    }
-
-    /**
-     * get list
-     *
-     * @param config
-     * @param key
-     * @return
-     */
-    protected List<String> getList(KeyValue config, String key) {
-        if (!config.containsKey(key) || Objects.isNull(config.getString(key))) {
-            return new ArrayList<>();
-        }
-        return Arrays.asList(COMMA_WITH_WHITESPACE.split(config.getString(key), -1));
-    }
-
-    /**
-     * get list
-     *
-     * @param config
-     * @param key
-     * @return
-     */
-    protected List<String> getList(KeyValue config, String key, String defaultValue) {
-        if (config.containsKey(key) || Objects.isNull(config.getString(key))) {
-            return Collections.singletonList(defaultValue);
-        }
-        return Arrays.asList(COMMA_WITH_WHITESPACE.split(config.getString(key), -1));
-    }
-
-    protected Boolean getBoolean(KeyValue config, String key, Boolean defaultValue) {
-        return config.containsKey(key) ? Boolean.parseBoolean(config.getString(key)) : defaultValue;
-    }
-
-}
-
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connection/ConnectionProvider.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connection/ConnectionProvider.java
new file mode 100644
index 0000000..a77d3c5
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connection/ConnectionProvider.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.rocketmq.connect.doris.connection;
+
+import java.sql.Connection;
+
+public interface ConnectionProvider {
+
+    /**
+     * Get existing connection or establish an new one if there is none.
+     */
+    Connection getOrEstablishConnection() throws Exception;
+
+    /**
+     * Close possible existing connection.
+     */
+    void closeConnection();
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connection/JdbcConnectionProvider.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connection/JdbcConnectionProvider.java
new file mode 100644
index 0000000..4207b7f
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connection/JdbcConnectionProvider.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.rocketmq.connect.doris.connection;
+
+import java.io.Serializable;
+import java.net.Authenticator;
+import java.net.PasswordAuthentication;
+import java.net.Proxy;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions.ProxyConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcConnectionProvider implements ConnectionProvider, Serializable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(JdbcConnectionProvider.class);
+    protected final String driverName = "com.mysql.jdbc.Driver";
+    protected final String cjDriverName = "com.mysql.cj.jdbc.Driver";
+    private static final String JDBC_URL_TEMPLATE = "jdbc:mysql://%s";
+    private static final String JDBC_URL_PROXY_TEMPLATE = "jdbc:mysql://%s?useSSL=false&proxyHost=%s&proxyPort=%d";
+
+    private static final long serialVersionUID = 1L;
+
+    private final DorisOptions options;
+
+    private transient Connection connection;
+
+    public JdbcConnectionProvider(DorisOptions options) {
+        this.options = options;
+    }
+
+    @Override
+    public Connection getOrEstablishConnection() throws ClassNotFoundException, SQLException {
+        if (connection != null && !connection.isClosed() && connection.isValid(10)) {
+            return connection;
+        }
+        try {
+            Class.forName(cjDriverName);
+        } catch (ClassNotFoundException ex) {
+            LOG.warn(
+                "can not found class com.mysql.cj.jdbc.Driver, use class com.mysql.jdbc.Driver");
+            Class.forName(driverName);
+        }
+        String jdbcUrl;
+        if (options.customCluster()) {
+            ProxyConfig proxyConfig = options.getProxyConfig()
+                .orElseThrow(() -> new NoSuchElementException("Failed to get ProxyConfig."));
+            String proxyHost = proxyConfig.getSocks5Host();
+            int proxyPort = proxyConfig.getSocks5Port();
+            String queryUrl = options.getQueryUrl();
+            jdbcUrl = String.format(JDBC_URL_PROXY_TEMPLATE, queryUrl, proxyHost, proxyPort);
+            configureSocks5ProxyAuthentication();
+        } else {
+            jdbcUrl = String.format(JDBC_URL_TEMPLATE, options.getQueryUrl());
+        }
+
+        if (!Objects.isNull(options.getUser())) {
+            connection =
+                DriverManager.getConnection(jdbcUrl, options.getUser(), options.getPassword());
+        } else {
+            connection = DriverManager.getConnection(jdbcUrl);
+        }
+        return connection;
+    }
+
+    private void configureSocks5ProxyAuthentication() {
+        if (options.customCluster()) {
+            ProxyConfig proxyConfig = options.getProxyConfig()
+                .orElseThrow(() -> new NoSuchElementException("Failed to get ProxyConfig."));
+            String socks5UserName = proxyConfig.getSocks5UserName();
+            String socks5Password = proxyConfig.getSocks5Password();
+            Authenticator.setDefault(new Authenticator() {
+                @Override
+                protected PasswordAuthentication getPasswordAuthentication() {
+                    if (Proxy.Type.SOCKS.name().equalsIgnoreCase(getRequestingProtocol())) {
+                        return new PasswordAuthentication(socks5UserName, socks5Password.toCharArray());
+                    }
+                    return null;
+                }
+            });
+            System.setProperty("java.net.socks.username", socks5UserName);
+            System.setProperty("java.net.socks.password", socks5Password);
+        }
+    }
+
+    @Override
+    public void closeConnection() {
+        if (connection != null) {
+            try {
+                connection.close();
+            } catch (SQLException e) {
+                LOG.warn("JDBC connection close failed.", e);
+            } finally {
+                connection = null;
+            }
+        }
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connector/DorisSinkConfig.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connector/DorisSinkConfig.java
deleted file mode 100644
index 5cbe294..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connector/DorisSinkConfig.java
+++ /dev/null
@@ -1,331 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.connector;
-
-import io.openmessaging.KeyValue;
-import org.apache.rocketmq.connect.doris.config.AbstractConfig;
-import org.apache.rocketmq.connect.doris.exception.ConfigException;
-import org.apache.rocketmq.connect.doris.schema.table.TableId;
-import org.apache.rocketmq.connect.doris.util.TableType;
-import org.apache.rocketmq.connect.doris.util.TableUtil;
-
-import java.time.ZoneId;
-import java.util.EnumSet;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Objects;
-import java.util.Set;
-import java.util.TimeZone;
-import java.util.stream.Collectors;
-
-/**
- * jdbc sink config
- */
-public class DorisSinkConfig extends AbstractConfig {
-
-    public enum InsertMode {
-        INSERT,
-        UPSERT,
-        UPDATE;
-
-    }
-
-    public enum PrimaryKeyMode {
-        NONE,
-        RECORD_KEY,
-        RECORD_VALUE;
-    }
-    public static final String HOST = "host";
-    public static final String PORT = "port";
-    public static final String DATABASE = "database";
-    public static final String USER = "user";
-    public static final String PASSWD = "passwd";
-    public static final String TABLE_NAME_FORMAT = "table.name.format";
-    public static final String TABLE_NAME_FORMAT_DEFAULT = "${topic}";
-    private static final String TABLE_NAME_FORMAT_DISPLAY = "Table Name Format";
-
-    /**
-     * table name from header
-     */
-    public static final String TABLE_NAME_FROM_HEADER = "table.name.from.header";
-    private static final boolean TABLE_NAME_FROM_HEADER_DEFAULT = Boolean.FALSE;
-    private static final String TABLE_NAME_FROM_HEADER_DISPLAY = "Table from header";
-
-    /**
-     * max retries
-     */
-    public static final String MAX_RETRIES = "max.retries";
-    private static final int MAX_RETRIES_DEFAULT = 10;
-    private static final String MAX_RETRIES_DOC =
-            "The maximum number of times to retry on errors before failing the task.";
-
-    public static final String RETRY_BACKOFF_MS = "retry.backoff.ms";
-    private static final int RETRY_BACKOFF_MS_DEFAULT = 3000;
-
-    public static final String BATCH_SIZE = "batch.size";
-    private static final int BATCH_SIZE_DEFAULT = 100;
-
-
-    public static final String DELETE_ENABLED = "delete.enabled";
-    private static final boolean DELETE_ENABLED_DEFAULT = false;
-
-
-    public static final String AUTO_CREATE = "auto.create";
-    private static final boolean AUTO_CREATE_DEFAULT = false;
-
-    public static final String AUTO_EVOLVE = "auto.evolve";
-    private static final boolean AUTO_EVOLVE_DEFAULT = false;
-    private static final String AUTO_EVOLVE_DOC =
-            "Whether to automatically add columns in the table schema when found to be missing relative "
-                    + "to the record schema by issuing ``ALTER``.";
-
-    public static final String INSERT_MODE = "insert.mode";
-    private static final String INSERT_MODE_DEFAULT = "insert";
-    private static final String INSERT_MODE_DOC =
-            "The insertion mode to use. Supported modes are:\n"
-                    + "``insert``\n"
-                    + "    Use standard SQL ``INSERT`` statements.\n"
-                    + "``upsert``\n"
-                    + "    Use the appropriate upsert semantics for the target database if it is supported by "
-                    + "the connector, e.g. ``INSERT OR IGNORE``.\n"
-                    + "``update``\n"
-                    + "    Use the appropriate update semantics for the target database if it is supported by "
-                    + "the connector, e.g. ``UPDATE``.";
-    private static final String INSERT_MODE_DISPLAY = "Insert Mode";
-
-
-    public static final String PK_FIELDS = "pk.fields";
-    private static final String PK_FIELDS_DEFAULT = "";
-    private static final String PK_FIELDS_DOC =
-            "List of comma-separated primary key field names. The runtime interpretation of this config"
-                    + " depends on the ``pk.mode``:\n"
-                    + "``none``\n"
-                    + "    Ignored as no fields are used as primary key in this mode.\n"
-                    + "``record_key``\n"
-                    + "    If empty, all fields from the key struct will be used, otherwise used to extract the"
-                    + " desired fields - for primitive key only a single field name must be configured.\n"
-                    + "``record_value``\n"
-                    + "    If empty, all fields from the value struct will be used, otherwise used to extract "
-                    + "the desired fields.";
-    private static final String PK_FIELDS_DISPLAY = "Primary Key Fields";
-
-    public static final String PK_MODE = "pk.mode";
-    private static final String PK_MODE_DEFAULT = "none";
-    private static final String PK_MODE_DOC =
-            "The primary key mode, also refer to ``" + PK_FIELDS + "`` documentation for interplay. "
-                    + "Supported modes are:\n"
-                    + "``none``\n"
-                    + "    No keys utilized.\n"
-                    + "``record_value``\n"
-                    + "    Field(s) from the record value are used, which must be a struct.";
-    private static final String PK_MODE_DISPLAY = "Primary Key Mode";
-
-    public static final String FIELDS_WHITELIST = "fields.whitelist";
-    private static final String FIELDS_WHITELIST_DEFAULT = "";
-    private static final String FIELDS_WHITELIST_DOC =
-            "List of comma-separated record value field names. If empty, all fields from the record "
-                    + "value are utilized, otherwise used to filter to the desired fields.\n"
-                    + "Note that ``" + PK_FIELDS + "`` is applied independently in the context of which field"
-                    + "(s) form the primary key columns in the destination database,"
-                    + " while this configuration is applicable for the other columns.";
-    private static final String FIELDS_WHITELIST_DISPLAY = "Fields Whitelist";
-
-
-    public static final String DIALECT_NAME_CONFIG = "dialect.name";
-    public static final String DIALECT_NAME_DEFAULT = "";
-
-
-    public static final String DB_TIMEZONE_CONFIG = "db.timezone";
-    public static final String DB_TIMEZONE_DEFAULT = "UTC";
-
-    // table types
-    public static final String TABLE_TYPES_CONFIG = "table.types";
-    public static final String TABLE_TYPES_DEFAULT = TableType.TABLE.toString();
-    private static final String TABLE_TYPES_DOC =
-            "The comma-separated types of database tables to which the sink connector can write. "
-                    + "By default this is ``" + TableType.TABLE + "``, but any combination of ``"
-                    + TableType.TABLE + "`` and ``" + TableType.VIEW + "`` is allowed. Not all databases "
-                    + "support writing to views, and when they do the the sink connector will fail if the "
-                    + "view definition does not match the records' schemas (regardless of ``"
-                    + AUTO_EVOLVE + "``).";
-
-    // white list tables
-    public static final String TABLE_WHITE_LIST_CONFIG = "tables.whitelist";
-    public static final String TABLE_NAME = "tableName";
-    public static final String TABLE_WHITE_LIST_DEFAULT = "";
-    private static final String TABLE_WHITE_LIST_DOC =
-            "Table white list.<br>db1.table01,db1.table02</br>";
-
-
-    private String tableNameFormat;
-    private boolean tableFromHeader;
-    private int maxRetries;
-    private int retryBackoffMs;
-    private int batchSize;
-    private boolean deleteEnabled;
-    private boolean autoCreate;
-    private boolean autoEvolve;
-    private InsertMode insertMode;
-    public final PrimaryKeyMode pkMode;
-    private List<String> pkFields;
-    private Set<String> fieldsWhitelist;
-    private Set<String> tableWhitelist;
-    private TimeZone timeZone;
-    private EnumSet<TableType> tableTypes;
-    public String getHost() {
-        return host;
-    }
-
-    public int getPort() {
-        return port;
-    }
-
-    public String getDatabase() {
-        return database;
-    }
-
-    public String getUser() {
-        return user;
-    }
-
-    public String getPasswd() {
-        return passwd;
-    }
-
-    private String host;
-    private int port;
-    private String database;
-    private String user;
-    private String passwd;
-
-    public DorisSinkConfig(KeyValue config) {
-        super(config);
-        host = config.getString(HOST).trim();
-        port = config.getInt(PORT);
-        database = config.getString(DATABASE).trim();
-        user = config.getString(USER).trim();
-        passwd = config.getString(PASSWD).trim();
-
-        tableNameFormat = config.getString(TABLE_NAME_FORMAT, TABLE_NAME_FORMAT_DEFAULT).trim();
-        tableFromHeader = getBoolean(config, TABLE_NAME_FROM_HEADER, false);
-        batchSize = config.getInt(BATCH_SIZE, BATCH_SIZE_DEFAULT);
-
-        maxRetries = config.getInt(MAX_RETRIES, MAX_RETRIES_DEFAULT);
-        retryBackoffMs = config.getInt(RETRY_BACKOFF_MS, RETRY_BACKOFF_MS_DEFAULT);
-        autoCreate = getBoolean(config, AUTO_CREATE, AUTO_CREATE_DEFAULT);
-        autoEvolve = getBoolean(config, AUTO_EVOLVE, AUTO_EVOLVE_DEFAULT);
-        if (Objects.nonNull(config.getString(INSERT_MODE))) {
-            insertMode = InsertMode.valueOf(config.getString(INSERT_MODE, INSERT_MODE_DEFAULT).toUpperCase());
-        }
-        deleteEnabled = getBoolean(config, DELETE_ENABLED, DELETE_ENABLED_DEFAULT);
-        pkMode = PrimaryKeyMode.valueOf(config.getString(PK_MODE, PK_MODE_DEFAULT).toUpperCase());
-        pkFields = getList(config, PK_FIELDS);
-        if (deleteEnabled && pkMode != PrimaryKeyMode.RECORD_KEY) {
-            throw new ConfigException(
-                    "Primary key mode must be 'record_key' when delete support is enabled");
-        }
-        fieldsWhitelist = new HashSet<>(getList(config, FIELDS_WHITELIST));
-        // table white list
-        tableWhitelist = new HashSet<>(getList(config, TABLE_WHITE_LIST_CONFIG));
-        String dbTimeZone = config.getString(DB_TIMEZONE_CONFIG, DB_TIMEZONE_DEFAULT);
-        timeZone = TimeZone.getTimeZone(ZoneId.of(dbTimeZone));
-        tableTypes = TableType.parse(getList(config, TABLE_TYPES_CONFIG, TABLE_TYPES_DEFAULT));
-    }
-
-    public String getTableNameFormat() {
-        return tableNameFormat;
-    }
-
-    public boolean isTableFromHeader() {
-        return tableFromHeader;
-    }
-
-    public int getMaxRetries() {
-        return maxRetries;
-    }
-
-    public int getRetryBackoffMs() {
-        return retryBackoffMs;
-    }
-
-    public int getBatchSize() {
-        return batchSize;
-    }
-
-    public boolean isDeleteEnabled() {
-        return deleteEnabled;
-    }
-
-    public boolean isAutoCreate() {
-        return autoCreate;
-    }
-
-    public boolean isAutoEvolve() {
-        return autoEvolve;
-    }
-
-    public InsertMode getInsertMode() {
-        return insertMode;
-    }
-
-    public PrimaryKeyMode getPkMode() {
-        return pkMode;
-    }
-
-    public List<String> getPkFields() {
-        return pkFields;
-    }
-
-    public Set<String> getFieldsWhitelist() {
-        return fieldsWhitelist;
-    }
-
-    public Set<String> getTableWhitelist() {
-        return tableWhitelist;
-    }
-
-    public TimeZone getTimeZone() {
-        return timeZone;
-    }
-
-    public EnumSet<TableType> getTableTypes() {
-        return tableTypes;
-    }
-
-    public boolean filterWhiteTable(TableId tableId) {
-        // not filter table
-        if (tableWhitelist.isEmpty()) {
-            return true;
-        }
-        for (String tableName : tableWhitelist) {
-            TableId table = TableUtil.parseToTableId(tableName);
-            if (table.catalogName() != null && table.catalogName().equals(tableId.catalogName())) {
-                return true;
-            }
-            if (table.tableName().equals(tableId.tableName())) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    public Set<String> tableTypeNames() {
-        return tableTypes.stream().map(TableType::toString).collect(Collectors.toSet());
-    }
-
-}
-
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connector/DorisSinkConnector.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connector/DorisSinkConnector.java
deleted file mode 100644
index a2e3119..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connector/DorisSinkConnector.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.connector;
-
-import io.openmessaging.KeyValue;
-import io.openmessaging.connector.api.component.task.Task;
-import io.openmessaging.connector.api.component.task.sink.SinkConnector;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class DorisSinkConnector extends SinkConnector {
-
-    private static final Logger log = LoggerFactory.getLogger(DorisSinkConnector.class);
-    private KeyValue connectConfig;
-
-    public void start(KeyValue config) {
-        this.connectConfig = config;
-    }
-
-    /**
-     * Should invoke before start the connector.
-     *
-     * @param config
-     * @return error message
-     */
-    @Override
-    public void validate(KeyValue config) {
-        // do validate config
-    }
-
-    @Override
-    public void stop() {
-        this.connectConfig = null;
-    }
-
-    /**
-     * Returns a set of configurations for Tasks based on the current configuration,
-     * producing at most count configurations.
-     *
-     * @param maxTasks maximum number of configurations to generate
-     * @return configurations for Tasks
-     */
-    @Override
-    public List<KeyValue> taskConfigs(int maxTasks) {
-        log.info("Starting task config !!! ");
-        List<KeyValue> configs = new ArrayList<>();
-        for (int i = 0; i < maxTasks; i++) {
-            configs.add(this.connectConfig);
-        }
-        return configs;
-    }
-
-    @Override
-    public Class<? extends Task> taskClass() {
-        return DorisSinkTask.class;
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connector/DorisSinkTask.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connector/DorisSinkTask.java
deleted file mode 100644
index f775b09..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connector/DorisSinkTask.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.connector;
-
-import io.openmessaging.KeyValue;
-import io.openmessaging.connector.api.component.task.sink.ErrorRecordReporter;
-import io.openmessaging.connector.api.component.task.sink.SinkTask;
-import io.openmessaging.connector.api.component.task.sink.SinkTaskContext;
-import io.openmessaging.connector.api.data.ConnectRecord;
-import io.openmessaging.connector.api.errors.ConnectException;
-import io.openmessaging.connector.api.errors.RetriableException;
-import org.apache.rocketmq.connect.doris.exception.TableAlterOrCreateException;
-import org.apache.rocketmq.connect.doris.sink.Updater;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.sql.SQLException;
-import java.util.List;
-
-/**
- * jdbc sink task
- */
-public class DorisSinkTask extends SinkTask {
-
-    private static final Logger log = LoggerFactory.getLogger(DorisSinkTask.class);
-    private SinkTaskContext context;
-    private ErrorRecordReporter errorRecordReporter;
-    private KeyValue originalConfig;
-    private DorisSinkConfig config;
-    int remainingRetries;
-    private Updater updater;
-
-
-    /**
-     * Put the records to the sink
-     *
-     * @param records
-     */
-    @Override
-    public void put(List<ConnectRecord> records) throws ConnectException {
-        if (records.isEmpty()) {
-            return;
-        }
-        final int recordsCount = records.size();
-        log.debug("Received {} records.", recordsCount);
-        try {
-            updater.write(records);
-        } catch (TableAlterOrCreateException tace) {
-            throw tace;
-        } catch (SQLException sqle) {
-            SQLException sqlAllMessagesException = getAllMessagesException(sqle);
-            if (remainingRetries > 0) {
-//                updater.closeQuietly();
-                start(originalConfig);
-                remainingRetries--;
-                throw new RetriableException(sqlAllMessagesException);
-            }
-
-        }
-        remainingRetries = config.getMaxRetries();
-    }
-
-    private SQLException getAllMessagesException(SQLException sqle) {
-        String sqleAllMessages = "Exception chain:" + System.lineSeparator();
-        for (Throwable e : sqle) {
-            sqleAllMessages += e + System.lineSeparator();
-        }
-        SQLException sqlAllMessagesException = new SQLException(sqleAllMessages);
-        sqlAllMessagesException.setNextException(sqle);
-        return sqlAllMessagesException;
-    }
-
-    /**
-     * Start the component
-     *
-     * @param keyValue
-     */
-    @Override
-    public void start(KeyValue keyValue) {
-        originalConfig = keyValue;
-        config = new DorisSinkConfig(keyValue);
-        remainingRetries = config.getMaxRetries();
-        log.info("Initializing doris writer");
-        this.updater = new Updater(config);
-    }
-
-    @Override
-    public void stop() {
-        log.info("Stopping task");
-    }
-}
-
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/ConverterMode.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/ConverterMode.java
new file mode 100644
index 0000000..4755166
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/ConverterMode.java
@@ -0,0 +1,45 @@
+/*
+ * 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.rocketmq.connect.doris.converter;
+
+public enum ConverterMode {
+    NORMAL("normal"),
+
+    // kafka upstream data comes from debezium
+    DEBEZIUM_INGESTION("debezium_ingestion");
+
+    private final String name;
+
+    ConverterMode(String name) {
+        this.name = name;
+    }
+
+    public static ConverterMode of(String name) {
+        return ConverterMode.valueOf(name.toUpperCase());
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public static String[] instances() {
+        return new String[] {NORMAL.name, DEBEZIUM_INGESTION.name};
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/RecordDescriptor.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/RecordDescriptor.java
new file mode 100644
index 0000000..a7d43f0
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/RecordDescriptor.java
@@ -0,0 +1,258 @@
+/*
+ * 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.rocketmq.connect.doris.converter;
+
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.Field;
+import io.openmessaging.connector.api.data.Schema;
+import io.openmessaging.connector.api.data.Struct;
+import io.openmessaging.connector.api.errors.ConnectException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.rocketmq.connect.doris.converter.type.Type;
+import org.apache.rocketmq.connect.doris.utils.ConnectRecordUtil;
+
+public class RecordDescriptor {
+    private final ConnectRecord record;
+    private final String topicName;
+    private final List<String> keyFieldNames;
+    private final List<String> nonKeyFieldNames;
+    private final Map<String, FieldDescriptor> fields;
+    private final boolean flattened;
+
+    private RecordDescriptor(
+        ConnectRecord record,
+        String topicName,
+        List<String> keyFieldNames,
+        List<String> nonKeyFieldNames,
+        Map<String, FieldDescriptor> fields,
+        boolean flattened) {
+        this.record = record;
+        this.topicName = topicName;
+        this.keyFieldNames = keyFieldNames;
+        this.nonKeyFieldNames = nonKeyFieldNames;
+        this.fields = fields;
+        this.flattened = flattened;
+    }
+
+    public String getTopicName() {
+        return topicName;
+    }
+
+    public long getQueueOffset() {
+        return ConnectRecordUtil.getQueueOffset(record.getPosition().getOffset());
+    }
+
+    public List<String> getKeyFieldNames() {
+        return keyFieldNames;
+    }
+
+    public List<String> getNonKeyFieldNames() {
+        return nonKeyFieldNames;
+    }
+
+    public Map<String, FieldDescriptor> getFields() {
+        return fields;
+    }
+
+    public boolean isDebeziumSinkRecord() {
+        return !flattened;
+    }
+
+    public boolean isTombstone() {
+        // Debezium TOMBSTONE has both value and valueSchema to null.
+        return record.getData() == null && record.getSchema().getValueSchema() == null;
+    }
+
+    public boolean isDelete() {
+        if (!isDebeziumSinkRecord()) {
+            return record.getData() == null;
+        } else if (record.getData() != null) {
+            final Struct value = (Struct) record.getData();
+            return "d".equals(value.getString("op"));
+        }
+        return false;
+    }
+
+    public Struct getAfterStruct() {
+        if (isDebeziumSinkRecord()) {
+            return ((Struct) record.getData()).getStruct("after");
+        } else {
+            return ((Struct) record.getData());
+        }
+    }
+
+    public Struct getBeforeStruct() {
+        if (isDebeziumSinkRecord()) {
+            return ((Struct) record.getData()).getStruct("before");
+        } else {
+            return ((Struct) record.getData());
+        }
+    }
+
+    public static Builder builder() {
+        return new Builder();
+    }
+
+    public static class FieldDescriptor {
+        private final Schema schema;
+        private final String name;
+        private final Map<String, Type> typeRegistry;
+        private final Type type;
+        private final String typeName;
+        private final String schemaName;
+        private String comment;
+        private String defaultValue;
+
+        public FieldDescriptor(Schema schema, String name, Map<String, Type> typeRegistry) {
+            this.schema = schema;
+            this.name = name;
+            this.typeRegistry = typeRegistry;
+            this.schemaName = schema.getName();
+            this.type =
+                typeRegistry.getOrDefault(
+                    schema.getName(), typeRegistry.get(schema.getFieldType().name()));
+            if (this.type == null) {
+                throw new IllegalArgumentException(
+                    "Type not found in registry for schema: " + schema);
+            }
+            this.typeName = type.getTypeName(schema);
+        }
+
+        public FieldDescriptor(
+            Schema schema,
+            String name,
+            Map<String, Type> typeRegistry,
+            String comment,
+            String defaultValue) {
+            this(schema, name, typeRegistry);
+            this.comment = comment;
+            this.defaultValue = defaultValue;
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public Type getType() {
+            return type;
+        }
+
+        public String getTypeName() {
+            return typeName;
+        }
+
+        public String getSchemaName() {
+            return schemaName;
+        }
+
+        public Schema getSchema() {
+            return schema;
+        }
+
+        public String getComment() {
+            return comment;
+        }
+
+        public String getDefaultValue() {
+            return defaultValue;
+        }
+    }
+
+    public static class Builder {
+
+        private ConnectRecord sinkRecord;
+        private Map<String, Type> typeRegistry;
+
+        // Internal build state
+        private final List<String> keyFieldNames = new ArrayList<>();
+        private final List<String> nonKeyFieldNames = new ArrayList<>();
+        private final Map<String, FieldDescriptor> allFields = new LinkedHashMap<>();
+
+        public Builder withSinkRecord(ConnectRecord record) {
+            this.sinkRecord = record;
+            return this;
+        }
+
+        public Builder withTypeRegistry(Map<String, Type> typeRegistry) {
+            this.typeRegistry = typeRegistry;
+            return this;
+        }
+
+        public RecordDescriptor build() {
+            Objects.requireNonNull(sinkRecord, "The sink record must be provided.");
+
+            final boolean flattened = !isTombstone(sinkRecord) && isFlattened(sinkRecord);
+            readSinkRecordNonKeyData(sinkRecord, flattened);
+
+            return new RecordDescriptor(
+                sinkRecord,
+                ConnectRecordUtil.getTopicName(sinkRecord.getPosition().getPartition()),
+                keyFieldNames,
+                nonKeyFieldNames,
+                allFields,
+                flattened);
+        }
+
+        private boolean isFlattened(ConnectRecord record) {
+            return record.getSchema().getValueSchema().getName() == null || !record.getSchema().getValueSchema()
+                .getName().contains("Envelope");
+        }
+
+        private boolean isTombstone(ConnectRecord record) {
+            return record.getData() == null && record.getSchema() == null;
+        }
+
+        private void readSinkRecordNonKeyData(ConnectRecord record, boolean flattened) {
+            final Schema valueSchema = record.getSchema().getValueSchema();
+            if (valueSchema != null) {
+                if (flattened) {
+                    // In a flattened event type, it's safe to read the field names directly
+                    // from the schema as this isn't a complex Debezium message type.
+                    applyNonKeyFields(valueSchema);
+                } else {
+                    Field after = valueSchema.getField("after");
+                    if (after == null) {
+                        throw new ConnectException(
+                            "Received an unexpected message type that does not have an 'after' Debezium block");
+                    }
+                    applyNonKeyFields(after.getSchema());
+                }
+            }
+        }
+
+        private void applyNonKeyFields(Schema schema) {
+            for (Field field : schema.getFields()) {
+                if (!keyFieldNames.contains(field.getName())) {
+                    applyNonKeyField(field.getName(), field.getSchema());
+                }
+            }
+        }
+
+        private void applyNonKeyField(String name, Schema schema) {
+            FieldDescriptor fieldDescriptor = new FieldDescriptor(schema, name, typeRegistry);
+            nonKeyFieldNames.add(fieldDescriptor.getName());
+            allFields.put(fieldDescriptor.getName(), fieldDescriptor);
+        }
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/RecordService.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/RecordService.java
new file mode 100644
index 0000000..c08ff1d
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/RecordService.java
@@ -0,0 +1,334 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rocketmq.connect.doris.converter;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.Struct;
+import io.openmessaging.connector.api.errors.ConnectException;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.converter.schema.SchemaChangeManager;
+import org.apache.rocketmq.connect.doris.converter.schema.SchemaEvolutionMode;
+import org.apache.rocketmq.connect.doris.converter.type.Type;
+import org.apache.rocketmq.connect.doris.exception.DataFormatException;
+import org.apache.rocketmq.connect.doris.exception.DorisException;
+import org.apache.rocketmq.connect.doris.exception.SchemaChangeException;
+import org.apache.rocketmq.connect.doris.model.ColumnDescriptor;
+import org.apache.rocketmq.connect.doris.model.TableDescriptor;
+import org.apache.rocketmq.connect.doris.model.doris.Schema;
+import org.apache.rocketmq.connect.doris.service.DorisSystemService;
+import org.apache.rocketmq.connect.doris.service.RestService;
+import org.apache.rocketmq.connect.doris.utils.ConnectRecordUtil;
+import org.apache.rocketmq.connect.doris.writer.LoadConstants;
+import org.apache.rocketmq.connect.doris.writer.RecordBuffer;
+import org.apache.rocketmq.connect.runtime.converter.record.json.JsonConverter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RecordService {
+    private static final Logger LOG = LoggerFactory.getLogger(RecordService.class);
+    public static final String SCHEMA_CHANGE_VALUE = "SchemaChangeValue";
+    private static final ObjectMapper MAPPER = new ObjectMapper();
+    private final JsonConverter converter;
+    private DorisSystemService dorisSystemService;
+    private SchemaChangeManager schemaChangeManager;
+    private DorisOptions dorisOptions;
+    private RecordTypeRegister recordTypeRegister;
+    private Map<String, TableDescriptor> dorisTableDescriptorCache;
+
+    public RecordService() {
+        this.converter = new JsonConverter();
+        Map<String, Object> config = new HashMap<>();
+        config.put("converterConfig", "false");
+        this.converter.configure(config);
+    }
+
+    public RecordService(DorisOptions dorisOptions) {
+        this();
+        this.dorisOptions = dorisOptions;
+        this.recordTypeRegister = new RecordTypeRegister(dorisOptions);
+        this.dorisSystemService = new DorisSystemService(dorisOptions);
+        this.schemaChangeManager = new SchemaChangeManager(dorisOptions);
+        this.dorisTableDescriptorCache = new HashMap<>();
+    }
+
+    /**
+     * process struct record from debezium: { "schema": { "type": "struct", "fields": [ ...... ],
+     * "optional": false, "name": "" }, "payload": { "name": "doris", "__deleted": "true" } }
+     */
+    public String processStructRecord(ConnectRecord record) {
+        String processedRecord;
+        String topicName = ConnectRecordUtil.getTopicName(record.getPosition().getPartition());
+        if (ConverterMode.DEBEZIUM_INGESTION == dorisOptions.getConverterMode()) {
+            validate(record);
+            RecordDescriptor recordDescriptor = buildRecordDescriptor(record);
+            if (recordDescriptor.isTombstone()) {
+                return null;
+            }
+            String tableName = dorisOptions.getTopicMapTable(recordDescriptor.getTopicName());
+            checkAndApplyTableChangesIfNeeded(tableName, recordDescriptor);
+
+            List<String> nonKeyFieldNames = recordDescriptor.getNonKeyFieldNames();
+            if (recordDescriptor.isDelete()) {
+                processedRecord =
+                    parseFieldValues(
+                        recordDescriptor,
+                        recordDescriptor.getBeforeStruct(),
+                        nonKeyFieldNames,
+                        true);
+            } else {
+                processedRecord =
+                    parseFieldValues(
+                        recordDescriptor,
+                        recordDescriptor.getAfterStruct(),
+                        nonKeyFieldNames,
+                        false);
+            }
+        } else {
+            byte[] bytes =
+                converter.fromConnectData(topicName, record.getSchema().getValueSchema(), record.getData());
+            processedRecord = new String(bytes, StandardCharsets.UTF_8);
+        }
+        return processedRecord;
+    }
+
+    private void validate(ConnectRecord record) {
+        if (isSchemaChange(record)) {
+            LOG.warn(
+                "Schema change records are not supported by doris-kafka-connector. Adjust `topics` or `topics.regex` to exclude schema change topic.");
+            throw new DorisException(
+                "Schema change records are not supported by doris-kafka-connector. Adjust `topics` or `topics.regex` to exclude schema change topic.");
+        }
+    }
+
+    private static boolean isSchemaChange(final ConnectRecord record) {
+        return record.getSchema().getValueSchema() != null
+            && StringUtils.isNotEmpty(record.getSchema().getValueSchema().getName())
+            && record.getSchema().getValueSchema().getName().contains(SCHEMA_CHANGE_VALUE);
+    }
+
+    private void checkAndApplyTableChangesIfNeeded(
+        String tableName, RecordDescriptor recordDescriptor) {
+        if (!hasTable(tableName)) {
+            // TODO Table does not exist, lets attempt to create it.
+            LOG.warn("The {} table does not exist, please create it manually.", tableName);
+            throw new DorisException(
+                "The " + tableName + " table does not exist, please create it manually.");
+        } else {
+            // Table exists, lets attempt to alter it if necessary.
+            alterTableIfNeeded(tableName, recordDescriptor);
+        }
+    }
+
+    private boolean hasTable(String tableName) {
+        if (!dorisTableDescriptorCache.containsKey(tableName)) {
+            boolean exist = dorisSystemService.tableExists(dorisOptions.getDatabase(), tableName);
+            if (exist) {
+                dorisTableDescriptorCache.put(tableName, null);
+            }
+            return exist;
+        }
+        return true;
+    }
+
+    private void alterTableIfNeeded(String tableName, RecordDescriptor record) {
+        // Resolve table metadata from the database
+        final TableDescriptor table = fetchDorisTableDescriptor(tableName);
+
+        Set<RecordDescriptor.FieldDescriptor> missingFields = resolveMissingFields(record, table);
+        if (missingFields.isEmpty()) {
+            // There are no missing fields, simply return
+            // TODO should we check column type changes or default value changes?
+            return;
+        }
+
+        LOG.info(
+            "Find some miss columns in {} table, try to alter add this columns={}.",
+            tableName,
+            missingFields.stream()
+                .map(RecordDescriptor.FieldDescriptor::getName)
+                .collect(Collectors.toList()));
+        if (SchemaEvolutionMode.NONE.equals(dorisOptions.getSchemaEvolutionMode())) {
+            LOG.warn(
+                "Table '{}' cannot be altered because schema evolution is disabled.",
+                tableName);
+            throw new SchemaChangeException(
+                "Cannot alter table " + table + " because schema evolution is disabled");
+        }
+        for (RecordDescriptor.FieldDescriptor missingField : missingFields) {
+            schemaChangeManager.addColumnDDL(tableName, missingField);
+        }
+        TableDescriptor newTableDescriptor = obtainTableSchema(tableName);
+        dorisTableDescriptorCache.put(tableName, newTableDescriptor);
+    }
+
+    private Set<RecordDescriptor.FieldDescriptor> resolveMissingFields(
+        RecordDescriptor record, TableDescriptor table) {
+        Set<RecordDescriptor.FieldDescriptor> missingFields = new HashSet<>();
+        for (Map.Entry<String, RecordDescriptor.FieldDescriptor> entry :
+            record.getFields().entrySet()) {
+            String filedName = entry.getKey();
+            if (!table.hasColumn(filedName)) {
+                missingFields.add(entry.getValue());
+            }
+        }
+        return missingFields;
+    }
+
+    private TableDescriptor fetchDorisTableDescriptor(String tableName) {
+        if (!dorisTableDescriptorCache.containsKey(tableName)
+            || Objects.isNull(dorisTableDescriptorCache.get(tableName))) {
+            TableDescriptor tableDescriptor = obtainTableSchema(tableName);
+            dorisTableDescriptorCache.put(tableName, tableDescriptor);
+        }
+        return dorisTableDescriptorCache.get(tableName);
+    }
+
+    private TableDescriptor obtainTableSchema(String tableName) {
+        Schema schema =
+            RestService.getSchema(dorisOptions, dorisOptions.getDatabase(), tableName, LOG);
+        List<ColumnDescriptor> columnDescriptors = new ArrayList<>();
+        schema.getProperties()
+            .forEach(
+                column -> {
+                    ColumnDescriptor columnDescriptor =
+                        ColumnDescriptor.builder()
+                            .columnName(column.getName())
+                            .typeName(column.getType())
+                            .comment(column.getComment())
+                            .build();
+                    columnDescriptors.add(columnDescriptor);
+                });
+        return TableDescriptor.builder()
+            .tableName(tableName)
+            .type(schema.getKeysType())
+            .columns(columnDescriptors)
+            .build();
+    }
+
+    /**
+     * process list record from kafka [{"name":"doris1"},{"name":"doris2"}]
+     */
+    public String processListRecord(ConnectRecord record) {
+        try {
+            StringJoiner sj = new StringJoiner(RecordBuffer.LINE_SEPARATOR);
+            List recordList = (List) record.getData();
+            for (Object item : recordList) {
+                sj.add(MAPPER.writeValueAsString(item));
+            }
+            return sj.toString();
+        } catch (IOException e) {
+            LOG.error("process list record failed: {}", record.getData());
+            throw new DataFormatException("process list record failed");
+        }
+    }
+
+    /**
+     * process map record from kafka {"name":"doris"}
+     */
+    public String processMapRecord(ConnectRecord record) {
+        try {
+            return MAPPER.writeValueAsString(record.getData());
+        } catch (IOException e) {
+            LOG.error("process map record failed: {}", record.getData());
+            throw new DataFormatException("process map record failed");
+        }
+    }
+
+    private String parseFieldValues(
+        RecordDescriptor record, Struct source, List<String> fields, boolean isDelete) {
+        Map<String, Object> filedMapping = new LinkedHashMap<>();
+        String filedResult = null;
+        for (String fieldName : fields) {
+            final RecordDescriptor.FieldDescriptor field = record.getFields().get(fieldName);
+            Type type = field.getType();
+            Object value =
+                field.getSchema().isOptional()
+                    ? source.getWithoutDefault(fieldName)
+                    : source.get(fieldName);
+            Object convertValue = type.getValue(value, field.getSchema());
+            if (Objects.nonNull(convertValue) && !type.isNumber()) {
+                filedMapping.put(fieldName, convertValue.toString());
+            } else {
+                filedMapping.put(fieldName, convertValue);
+            }
+        }
+        try {
+            if (isDelete) {
+                filedMapping.put(LoadConstants.DORIS_DELETE_SIGN, LoadConstants.DORIS_DEL_TRUE);
+            } else {
+                filedMapping.put(LoadConstants.DORIS_DELETE_SIGN, LoadConstants.DORIS_DEL_FALSE);
+            }
+            filedResult = MAPPER.writeValueAsString(filedMapping);
+        } catch (JsonProcessingException e) {
+            LOG.error("parse record failed, cause by parse json error: {}", filedMapping);
+        }
+        return filedResult;
+    }
+
+    /**
+     * Given a single Record from put API, process it and convert it into a Json String.
+     *
+     * @param record record from Kafka
+     * @return Json String
+     */
+    public String getProcessedRecord(ConnectRecord record) {
+        String processedRecord;
+        if (record.getData() instanceof Struct) {
+            processedRecord = processStructRecord(record);
+        } else if (record.getData() instanceof List) {
+            processedRecord = processListRecord(record);
+        } else if (record.getData() instanceof Map) {
+            processedRecord = processMapRecord(record);
+        } else {
+            processedRecord = record.getData().toString();
+        }
+        return processedRecord;
+    }
+
+    private RecordDescriptor buildRecordDescriptor(ConnectRecord record) {
+        RecordDescriptor recordDescriptor;
+        try {
+            recordDescriptor =
+                RecordDescriptor.builder()
+                    .withSinkRecord(record)
+                    .withTypeRegistry(recordTypeRegister.getTypeRegistry())
+                    .build();
+        } catch (Exception e) {
+            throw new ConnectException("Failed to process a sink record", e);
+        }
+        return recordDescriptor;
+    }
+
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/RecordTypeRegister.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/RecordTypeRegister.java
new file mode 100644
index 0000000..ced15e6
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/RecordTypeRegister.java
@@ -0,0 +1,122 @@
+/*
+ * 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.rocketmq.connect.doris.converter;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.converter.type.Type;
+import org.apache.rocketmq.connect.doris.converter.type.connect.ConnectBooleanType;
+import org.apache.rocketmq.connect.doris.converter.type.connect.ConnectBytesType;
+import org.apache.rocketmq.connect.doris.converter.type.connect.ConnectDateType;
+import org.apache.rocketmq.connect.doris.converter.type.connect.ConnectDecimalType;
+import org.apache.rocketmq.connect.doris.converter.type.connect.ConnectFloat32Type;
+import org.apache.rocketmq.connect.doris.converter.type.connect.ConnectFloat64Type;
+import org.apache.rocketmq.connect.doris.converter.type.connect.ConnectInt16Type;
+import org.apache.rocketmq.connect.doris.converter.type.connect.ConnectInt32Type;
+import org.apache.rocketmq.connect.doris.converter.type.connect.ConnectInt64Type;
+import org.apache.rocketmq.connect.doris.converter.type.connect.ConnectInt8Type;
+import org.apache.rocketmq.connect.doris.converter.type.connect.ConnectMapToConnectStringType;
+import org.apache.rocketmq.connect.doris.converter.type.connect.ConnectStringType;
+import org.apache.rocketmq.connect.doris.converter.type.connect.ConnectTimeType;
+import org.apache.rocketmq.connect.doris.converter.type.connect.ConnectTimestampType;
+import org.apache.rocketmq.connect.doris.converter.type.debezium.ArrayType;
+import org.apache.rocketmq.connect.doris.converter.type.debezium.DateType;
+import org.apache.rocketmq.connect.doris.converter.type.debezium.GeographyType;
+import org.apache.rocketmq.connect.doris.converter.type.debezium.GeometryType;
+import org.apache.rocketmq.connect.doris.converter.type.debezium.MicroTimeType;
+import org.apache.rocketmq.connect.doris.converter.type.debezium.MicroTimestampType;
+import org.apache.rocketmq.connect.doris.converter.type.debezium.NanoTimeType;
+import org.apache.rocketmq.connect.doris.converter.type.debezium.NanoTimestampType;
+import org.apache.rocketmq.connect.doris.converter.type.debezium.PointType;
+import org.apache.rocketmq.connect.doris.converter.type.debezium.TimeType;
+import org.apache.rocketmq.connect.doris.converter.type.debezium.TimestampType;
+import org.apache.rocketmq.connect.doris.converter.type.debezium.VariableScaleDecimalType;
+import org.apache.rocketmq.connect.doris.converter.type.debezium.ZonedTimeType;
+import org.apache.rocketmq.connect.doris.converter.type.debezium.ZonedTimestampType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RecordTypeRegister {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RecordTypeRegister.class);
+
+    private final Map<String, Type> typeRegistry = new HashMap<>();
+    private final DorisOptions dorisOptions;
+
+    public RecordTypeRegister(DorisOptions dorisOptions) {
+        this.dorisOptions = dorisOptions;
+        registerTypes();
+    }
+
+    protected void registerTypes() {
+        // Supported common Debezium data types
+        registerType(DateType.INSTANCE);
+        registerType(TimeType.INSTANCE);
+        registerType(MicroTimeType.INSTANCE);
+        registerType(TimestampType.INSTANCE);
+        registerType(MicroTimestampType.INSTANCE);
+        registerType(NanoTimeType.INSTANCE);
+        registerType(NanoTimestampType.INSTANCE);
+        registerType(ZonedTimeType.INSTANCE);
+        registerType(ZonedTimestampType.INSTANCE);
+        registerType(VariableScaleDecimalType.INSTANCE);
+        registerType(PointType.INSTANCE);
+        registerType(GeographyType.INSTANCE);
+        registerType(GeometryType.INSTANCE);
+        registerType(ArrayType.INSTANCE);
+
+        // Supported connect data types
+        registerType(ConnectBooleanType.INSTANCE);
+        registerType(ConnectBytesType.INSTANCE);
+        registerType(ConnectDateType.INSTANCE);
+        registerType(ConnectDecimalType.INSTANCE);
+        registerType(ConnectFloat32Type.INSTANCE);
+        registerType(ConnectFloat64Type.INSTANCE);
+        registerType(ConnectInt8Type.INSTANCE);
+        registerType(ConnectInt16Type.INSTANCE);
+        registerType(ConnectInt32Type.INSTANCE);
+        registerType(ConnectInt64Type.INSTANCE);
+        registerType(ConnectStringType.INSTANCE);
+        registerType(ConnectTimestampType.INSTANCE);
+        registerType(ConnectTimeType.INSTANCE);
+        registerType(ConnectMapToConnectStringType.INSTANCE);
+    }
+
+    protected void registerType(Type type) {
+        type.configure(dorisOptions);
+        for (String key : type.getRegistrationKeys()) {
+            final Type existing = typeRegistry.put(key, type);
+            if (existing != null) {
+                LOG.debug(
+                    "Type replaced [{}]: {} -> {}",
+                    key,
+                    existing.getClass().getName(),
+                    type.getClass().getName());
+            } else {
+                LOG.debug("Type registered [{}]: {}", key, type.getClass().getName());
+            }
+        }
+    }
+
+    public Map<String, Type> getTypeRegistry() {
+        return typeRegistry;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/schema/SchemaChangeManager.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/schema/SchemaChangeManager.java
new file mode 100644
index 0000000..52765d4
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/schema/SchemaChangeManager.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rocketmq.connect.doris.converter.schema;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.http.HttpHeaders;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.methods.HttpUriRequest;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.converter.RecordDescriptor;
+import org.apache.rocketmq.connect.doris.exception.SchemaChangeException;
+import org.apache.rocketmq.connect.doris.service.DorisSystemService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.net.HttpURLConnection.HTTP_OK;
+
+public class SchemaChangeManager implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOG = LoggerFactory.getLogger(SchemaChangeManager.class);
+    private static final String ADD_DDL = "ALTER TABLE %s ADD COLUMN %s %s";
+    private static final String SCHEMA_CHANGE_API = "http://%s/api/query/default_cluster/%s";
+    private final ObjectMapper objectMapper = new ObjectMapper();
+    private final DorisOptions dorisOptions;
+    private final DorisSystemService dorisSystemService;
+
+    public SchemaChangeManager(DorisOptions dorisOptions) {
+        this.dorisOptions = dorisOptions;
+        this.dorisSystemService = new DorisSystemService(dorisOptions);
+    }
+
+    private boolean handleSchemaChange(Map<String, Object> responseMap, String responseEntity) {
+        String code = responseMap.getOrDefault("code", "-1").toString();
+        if (code.equals("0")) {
+            return true;
+        } else {
+            throw new SchemaChangeException("Failed to schemaChange, response: " + responseEntity);
+        }
+    }
+
+    public void addColumnDDL(String tableName, RecordDescriptor.FieldDescriptor field) {
+        try {
+            // check the add column whether exist in table.
+            if (dorisSystemService.isColumnExist(
+                dorisOptions.getDatabase(), tableName, field.getName())) {
+                LOG.warn(
+                    "The column {} already exists in table {}, no need to add it again",
+                    field.getName(),
+                    tableName);
+                return;
+            }
+
+            String addColumnDDL = buildAddColumnDDL(dorisOptions.getDatabase(), tableName, field);
+            boolean status = execute(addColumnDDL, dorisOptions.getDatabase());
+            LOG.info(
+                "Add missing column for {} table, ddl={}, status={}",
+                tableName,
+                addColumnDDL,
+                status);
+        } catch (Exception e) {
+            LOG.warn("Failed to add column for {}, cause by: ", tableName, e);
+            throw new SchemaChangeException(
+                "Failed to add column for " + tableName + ", cause by:", e);
+        }
+    }
+
+    public static String buildAddColumnDDL(
+        String database, String tableName, RecordDescriptor.FieldDescriptor field) {
+        String name = field.getName();
+        String typeName = field.getTypeName();
+        String comment = field.getComment();
+        String defaultValue = field.getDefaultValue();
+
+        String addDDL =
+            String.format(
+                ADD_DDL,
+                identifier(database) + "." + identifier(tableName),
+                identifier(name),
+                typeName);
+        if (defaultValue != null) {
+            addDDL = addDDL + " DEFAULT " + quoteDefaultValue(defaultValue);
+        }
+        if (StringUtils.isNotEmpty(comment)) {
+            addDDL = addDDL + " COMMENT '" + quoteComment(comment) + "'";
+        }
+        return addDDL;
+    }
+
+    private static String quoteComment(String comment) {
+        return comment.replaceAll("'", "\\\\'");
+    }
+
+    private static String identifier(String name) {
+        return "`" + name + "`";
+    }
+
+    private static String quoteDefaultValue(String defaultValue) {
+        // DEFAULT current_timestamp not need quote
+        if (defaultValue.equalsIgnoreCase("current_timestamp")) {
+            return defaultValue;
+        }
+        return "'" + defaultValue + "'";
+    }
+
+    /**
+     * execute sql in doris.
+     */
+    public boolean execute(String ddl, String database)
+        throws IOException, IllegalArgumentException {
+        if (StringUtils.isEmpty(ddl)) {
+            return false;
+        }
+        LOG.info("Execute SQL: {}", ddl);
+        HttpPost httpPost = buildHttpPost(ddl, database);
+        String responseEntity = "";
+        Map<String, Object> responseMap = handleResponse(httpPost, responseEntity);
+        return handleSchemaChange(responseMap, responseEntity);
+    }
+
+    public HttpPost buildHttpPost(String ddl, String database)
+        throws IllegalArgumentException, IOException {
+        Map<String, String> param = new HashMap<>();
+        param.put("stmt", ddl);
+        String requestUrl = String.format(SCHEMA_CHANGE_API, dorisOptions.getHttpUrl(), database);
+        HttpPost httpPost = new HttpPost(requestUrl);
+        httpPost.setHeader(HttpHeaders.AUTHORIZATION, authHeader());
+        httpPost.setHeader(HttpHeaders.CONTENT_TYPE, "application/json");
+        httpPost.setEntity(new StringEntity(objectMapper.writeValueAsString(param)));
+        return httpPost;
+    }
+
+    private Map<String, Object> handleResponse(HttpUriRequest request, String responseEntity) {
+        try (CloseableHttpClient httpclient = HttpClients.createDefault()) {
+            CloseableHttpResponse response = httpclient.execute(request);
+            final int statusCode = response.getStatusLine().getStatusCode();
+            final String reasonPhrase = response.getStatusLine().getReasonPhrase();
+            if (statusCode == HTTP_OK && response.getEntity() != null) {
+                responseEntity = EntityUtils.toString(response.getEntity());
+                return objectMapper.readValue(responseEntity, Map.class);
+            } else {
+                throw new SchemaChangeException(
+                    "Failed to schemaChange, status: "
+                        + statusCode
+                        + ", reason: "
+                        + reasonPhrase);
+            }
+        } catch (Exception e) {
+            LOG.error("SchemaChange request error,", e);
+            throw new SchemaChangeException("SchemaChange request error with " + e.getMessage());
+        }
+    }
+
+    private String authHeader() {
+        return "Basic "
+            + new String(
+            Base64.encodeBase64(
+                (dorisOptions.getUser() + ":" + dorisOptions.getPassword())
+                    .getBytes(StandardCharsets.UTF_8)));
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/schema/SchemaEvolutionMode.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/schema/SchemaEvolutionMode.java
new file mode 100644
index 0000000..fc98b7c
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/schema/SchemaEvolutionMode.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.rocketmq.connect.doris.converter.schema;
+
+public enum SchemaEvolutionMode {
+    NONE("none"),
+
+    BASIC("basic");
+
+    private final String name;
+
+    SchemaEvolutionMode(String name) {
+        this.name = name;
+    }
+
+    public static SchemaEvolutionMode of(String name) {
+        return SchemaEvolutionMode.valueOf(name.toUpperCase());
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public static String[] instances() {
+        return new String[] {NONE.name, BASIC.name};
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractDateType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractDateType.java
new file mode 100644
index 0000000..ef6eea0
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractDateType.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.rocketmq.connect.doris.converter.type;
+
+import io.openmessaging.connector.api.data.Schema;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+
+/**
+ * An abstract base class for all temporal date implementations of {@link Type}.
+ */
+public abstract class AbstractDateType extends AbstractTemporalType {
+
+    @Override
+    public String getTypeName(Schema schema) {
+        return DorisType.DATE;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractGeometryType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractGeometryType.java
new file mode 100644
index 0000000..ac68875
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractGeometryType.java
@@ -0,0 +1,30 @@
+/*
+ * 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.rocketmq.connect.doris.converter.type;
+
+import io.openmessaging.connector.api.data.Schema;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+
+public abstract class AbstractGeometryType extends AbstractType {
+    @Override
+    public String getTypeName(Schema schema) {
+        return DorisType.STRING;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractTemporalType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractTemporalType.java
new file mode 100644
index 0000000..67afcb2
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractTemporalType.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.rocketmq.connect.doris.converter.type;
+
+import java.time.ZoneId;
+import java.util.TimeZone;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An abstract base class for all temporal implementations of {@link Type}.
+ */
+public abstract class AbstractTemporalType extends AbstractType {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AbstractTemporalType.class);
+
+    private TimeZone databaseTimeZone;
+
+    @Override
+    public void configure(DorisOptions dorisOptions) {
+        final String databaseTimeZone = dorisOptions.getDatabaseTimeZone();
+        try {
+            this.databaseTimeZone = TimeZone.getTimeZone(ZoneId.of(databaseTimeZone));
+        } catch (Exception e) {
+            LOGGER.error(
+                "Failed to resolve time zone '{}', please specify a correct time zone value",
+                databaseTimeZone,
+                e);
+            throw e;
+        }
+    }
+
+    protected TimeZone getDatabaseTimeZone() {
+        return databaseTimeZone;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractTimeType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractTimeType.java
new file mode 100644
index 0000000..1100ee0
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractTimeType.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.rocketmq.connect.doris.converter.type;
+
+import io.openmessaging.connector.api.data.Schema;
+import java.util.Optional;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisTypeProperties;
+
+/**
+ * An abstract temporal implementation of {@link Type} for {@code TIME} based columns.
+ */
+public abstract class AbstractTimeType extends AbstractTemporalType {
+
+    @Override
+    public String getTypeName(Schema schema) {
+        // NOTE:
+        // The MySQL connector does not use the __debezium.source.column.scale parameter to pass
+        // the time column's precision but instead uses the __debezium.source.column.length key
+        // which differs from all other connector implementations.
+        //
+        final int precision = getTimePrecision(schema);
+        return String.format(
+            "%s(%s)",
+            DorisType.DATETIME,
+            Math.min(precision, DorisTypeProperties.MAX_SUPPORTED_DATE_TIME_PRECISION));
+    }
+
+    protected int getTimePrecision(Schema schema) {
+        final String length = getSourceColumnLength(schema).orElse("0");
+        final Optional<String> scale = getSourceColumnPrecision(schema);
+        return scale.map(Integer::parseInt).orElseGet(() -> Integer.parseInt(length));
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractTimestampType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractTimestampType.java
new file mode 100644
index 0000000..9f13d7c
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractTimestampType.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rocketmq.connect.doris.converter.type;
+
+import io.openmessaging.connector.api.data.Schema;
+import java.util.Optional;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisTypeProperties;
+
+/**
+ * An abstract temporal implementation of {@link Type} for {@code TIMESTAMP} based columns.
+ */
+public abstract class AbstractTimestampType extends AbstractTemporalType {
+
+    @Override
+    public String getTypeName(Schema schema) {
+        final int precision = getTimePrecision(schema);
+        return String.format(
+            "%s(%s)",
+            DorisType.DATETIME,
+            Math.min(precision, DorisTypeProperties.MAX_SUPPORTED_DATE_TIME_PRECISION));
+    }
+
+    protected int getTimePrecision(Schema schema) {
+        final String length = getSourceColumnLength(schema).orElse("0");
+        final Optional<String> scale = getSourceColumnPrecision(schema);
+        return scale.map(Integer::parseInt).orElseGet(() -> Integer.parseInt(length));
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractType.java
new file mode 100644
index 0000000..4dd9e71
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractType.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.rocketmq.connect.doris.converter.type;
+
+import io.openmessaging.connector.api.data.Schema;
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.converter.type.util.SchemaUtils;
+
+/**
+ * An abstract implementation of {@link Type}, which all types should extend.
+ */
+public abstract class AbstractType implements Type {
+
+    @Override
+    public void configure(DorisOptions dorisOptions) {
+    }
+
+    @Override
+    public Object getValue(Object sourceValue) {
+        return sourceValue;
+    }
+
+    @Override
+    public boolean isNumber() {
+        return false;
+    }
+
+    @Override
+    public String toString() {
+        return getClass().getSimpleName();
+    }
+
+    protected Optional<String> getSchemaParameter(Schema schema, String parameterName) {
+        if (!Objects.isNull(schema.getParameters())) {
+            return Optional.ofNullable(schema.getParameters().get(parameterName));
+        }
+        return Optional.empty();
+    }
+
+    protected Optional<String> getSourceColumnType(Schema schema) {
+        return SchemaUtils.getSourceColumnType(schema);
+    }
+
+    protected Optional<String> getSourceColumnLength(Schema schema) {
+        return SchemaUtils.getSourceColumnLength(schema);
+    }
+
+    protected Optional<String> getSourceColumnPrecision(Schema schema) {
+        return SchemaUtils.getSourceColumnPrecision(schema);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/Type.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/Type.java
new file mode 100644
index 0000000..2ada4bc
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/Type.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rocketmq.connect.doris.converter.type;
+
+import io.openmessaging.connector.api.data.Schema;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+
+/**
+ * A type indicates the type of each column of kafka record, including various column types of
+ * debezium and connect.
+ */
+public interface Type {
+
+    /**
+     * Allows a type to perform initialization/configuration tasks based on user configs.
+     */
+    void configure(DorisOptions dorisOptions);
+
+    /**
+     * Returns the names that this type will be mapped as.
+     *
+     * <p>For example, when creating a custom mapping for {@code io.debezium.data.Bits}, a type
+     * could be registered using the {@code LOGICAL_NAME} of the schema if the type is to be used
+     * when a schema name is identified; otherwise it could be registered as the raw column type
+     * when column type propagation is enabled.
+     */
+    String[] getRegistrationKeys();
+
+    /**
+     * Get the actual converted value based on the column type.
+     */
+    Object getValue(Object sourceValue);
+
+    default Object getValue(Object sourceValue, Schema schema) {
+        return getValue(sourceValue);
+    }
+
+    String getTypeName(Schema schema);
+
+    boolean isNumber();
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/AbstractConnectMapType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/AbstractConnectMapType.java
new file mode 100644
index 0000000..9de7b72
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/AbstractConnectMapType.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.rocketmq.connect.doris.converter.type.connect;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.openmessaging.connector.api.errors.ConnectException;
+
+public abstract class AbstractConnectMapType extends AbstractConnectSchemaType {
+
+    private static final ObjectMapper MAPPER = new ObjectMapper();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {"MAP"};
+    }
+
+    protected String mapToJsonString(Object value) {
+        try {
+            return MAPPER.writeValueAsString(value);
+        } catch (JsonProcessingException e) {
+            throw new ConnectException("Failed to deserialize MAP data to JSON", e);
+        }
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/AbstractConnectSchemaType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/AbstractConnectSchemaType.java
new file mode 100644
index 0000000..f6560d6
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/AbstractConnectSchemaType.java
@@ -0,0 +1,25 @@
+/*
+ * 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.rocketmq.connect.doris.converter.type.connect;
+
+import org.apache.rocketmq.connect.doris.converter.type.AbstractType;
+
+public abstract class AbstractConnectSchemaType extends AbstractType {
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectBooleanType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectBooleanType.java
new file mode 100644
index 0000000..5fc351d
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectBooleanType.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.rocketmq.connect.doris.converter.type.connect;
+
+import io.openmessaging.connector.api.data.Schema;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+
+public class ConnectBooleanType extends AbstractConnectSchemaType {
+
+    public static final ConnectBooleanType INSTANCE = new ConnectBooleanType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {"BOOLEAN"};
+    }
+
+    @Override
+    public String getTypeName(Schema schema) {
+        return DorisType.BOOLEAN;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectBytesType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectBytesType.java
new file mode 100644
index 0000000..ab18ea4
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectBytesType.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.rocketmq.connect.doris.converter.type.connect;
+
+import io.openmessaging.connector.api.data.Schema;
+import java.nio.ByteBuffer;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+
+public class ConnectBytesType extends AbstractConnectSchemaType {
+
+    public static final ConnectBytesType INSTANCE = new ConnectBytesType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {"BYTES"};
+    }
+
+    @Override
+    public Object getValue(Object sourceValue) {
+        if (sourceValue == null) {
+            return null;
+        }
+        return bytesToHexString(getByteArrayFromValue(sourceValue));
+    }
+
+    @Override
+    public String getTypeName(Schema schema) {
+        return DorisType.STRING;
+    }
+
+    private byte[] getByteArrayFromValue(Object value) {
+        byte[] byteArray = null;
+        if (value instanceof ByteBuffer) {
+            final ByteBuffer buffer = ((ByteBuffer) value).slice();
+            byteArray = new byte[buffer.remaining()];
+            buffer.get(byteArray);
+        } else if (value instanceof byte[]) {
+            byteArray = (byte[]) value;
+        }
+        return byteArray;
+    }
+
+    /**
+     * Convert hexadecimal byte array to string
+     */
+    private String bytesToHexString(byte[] bytes) {
+        StringBuilder sb = new StringBuilder();
+        for (byte b : bytes) {
+            sb.append(String.format("%02X", b));
+        }
+        return sb.toString();
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectDateType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectDateType.java
new file mode 100644
index 0000000..26f97d6
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectDateType.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rocketmq.connect.doris.converter.type.connect;
+
+import io.openmessaging.connector.api.data.logical.Date;
+import io.openmessaging.connector.api.errors.ConnectException;
+import org.apache.rocketmq.connect.doris.converter.type.AbstractDateType;
+import org.apache.rocketmq.connect.doris.converter.type.util.DateTimeUtils;
+
+public class ConnectDateType extends AbstractDateType {
+
+    public static final ConnectDateType INSTANCE = new ConnectDateType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {Date.LOGICAL_NAME};
+    }
+
+    @Override
+    public Object getValue(Object sourceValue) {
+        if (sourceValue == null) {
+            return null;
+        }
+        if (sourceValue instanceof java.util.Date) {
+            return DateTimeUtils.toLocalDateFromDate((java.util.Date) sourceValue);
+        }
+        throw new ConnectException(
+            String.format(
+                "Unexpected %s value '%s' with type '%s'",
+                getClass().getSimpleName(), sourceValue, sourceValue.getClass().getName()));
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectDecimalType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectDecimalType.java
new file mode 100644
index 0000000..c33a87d
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectDecimalType.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.rocketmq.connect.doris.converter.type.connect;
+
+import io.openmessaging.connector.api.data.Schema;
+import io.openmessaging.connector.api.data.logical.Decimal;
+import org.apache.rocketmq.connect.doris.converter.type.AbstractType;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ConnectDecimalType extends AbstractType {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ConnectDecimalType.class);
+
+    public static final ConnectDecimalType INSTANCE = new ConnectDecimalType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {Decimal.LOGICAL_NAME};
+    }
+
+    @Override
+    public String getTypeName(Schema schema) {
+        int scale = Integer.parseInt(getSchemaParameter(schema, "scale").orElse("0"));
+        int precision =
+            Integer.parseInt(
+                getSchemaParameter(schema, "connect.decimal.precision").orElse("0"));
+        return precision <= 38
+            ? String.format("%s(%s,%s)", DorisType.DECIMAL, precision, Math.max(scale, 0))
+            : DorisType.STRING;
+    }
+
+    @Override
+    public boolean isNumber() {
+        return true;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectFloat32Type.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectFloat32Type.java
new file mode 100644
index 0000000..c622233
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectFloat32Type.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.rocketmq.connect.doris.converter.type.connect;
+
+import io.openmessaging.connector.api.data.Schema;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+
+public class ConnectFloat32Type extends AbstractConnectSchemaType {
+
+    public static final ConnectFloat32Type INSTANCE = new ConnectFloat32Type();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {"FLOAT32"};
+    }
+
+    @Override
+    public String getTypeName(Schema schema) {
+        return DorisType.FLOAT;
+    }
+
+    @Override
+    public boolean isNumber() {
+        return true;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectFloat64Type.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectFloat64Type.java
new file mode 100644
index 0000000..a9e0176
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectFloat64Type.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.rocketmq.connect.doris.converter.type.connect;
+
+import io.openmessaging.connector.api.data.Schema;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+
+public class ConnectFloat64Type extends AbstractConnectSchemaType {
+
+    public static final ConnectFloat64Type INSTANCE = new ConnectFloat64Type();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {"FLOAT64"};
+    }
+
+    @Override
+    public String getTypeName(Schema schema) {
+        return DorisType.DOUBLE;
+    }
+
+    @Override
+    public boolean isNumber() {
+        return true;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt16Type.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt16Type.java
new file mode 100644
index 0000000..57919fd
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt16Type.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.rocketmq.connect.doris.converter.type.connect;
+
+import io.openmessaging.connector.api.data.Schema;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+
+public class ConnectInt16Type extends AbstractConnectSchemaType {
+
+    public static final ConnectInt16Type INSTANCE = new ConnectInt16Type();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {"INT16"};
+    }
+
+    @Override
+    public String getTypeName(Schema schema) {
+        return DorisType.SMALLINT;
+    }
+
+    @Override
+    public boolean isNumber() {
+        return true;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt32Type.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt32Type.java
new file mode 100644
index 0000000..70c2c2c
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt32Type.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.rocketmq.connect.doris.converter.type.connect;
+
+import io.openmessaging.connector.api.data.Schema;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+
+public class ConnectInt32Type extends AbstractConnectSchemaType {
+
+    public static final ConnectInt32Type INSTANCE = new ConnectInt32Type();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {"INT32"};
+    }
+
+    @Override
+    public String getTypeName(Schema schema) {
+        return DorisType.INT;
+    }
+
+    @Override
+    public boolean isNumber() {
+        return true;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt64Type.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt64Type.java
new file mode 100644
index 0000000..96c220f
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt64Type.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.rocketmq.connect.doris.converter.type.connect;
+
+import io.openmessaging.connector.api.data.Schema;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+
+public class ConnectInt64Type extends AbstractConnectSchemaType {
+
+    public static final ConnectInt64Type INSTANCE = new ConnectInt64Type();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {"INT64"};
+    }
+
+    @Override
+    public String getTypeName(Schema schema) {
+        return DorisType.BIGINT;
+    }
+
+    @Override
+    public boolean isNumber() {
+        return true;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt8Type.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt8Type.java
new file mode 100644
index 0000000..1839ed3
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt8Type.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.rocketmq.connect.doris.converter.type.connect;
+
+import io.openmessaging.connector.api.data.Schema;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+
+public class ConnectInt8Type extends AbstractConnectSchemaType {
+
+    public static final ConnectInt8Type INSTANCE = new ConnectInt8Type();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {"INT8"};
+    }
+
+    @Override
+    public String getTypeName(Schema schema) {
+        return DorisType.TINYINT;
+    }
+
+    @Override
+    public boolean isNumber() {
+        return true;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectMapToConnectStringType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectMapToConnectStringType.java
new file mode 100644
index 0000000..e2f32d3
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectMapToConnectStringType.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.rocketmq.connect.doris.converter.type.connect;
+
+import io.openmessaging.connector.api.data.Schema;
+import java.util.Map;
+
+public class ConnectMapToConnectStringType extends AbstractConnectMapType {
+
+    public static final ConnectMapToConnectStringType INSTANCE =
+        new ConnectMapToConnectStringType();
+
+    @Override
+    public String getTypeName(Schema schema) {
+        return ConnectStringType.INSTANCE.getTypeName(schema);
+    }
+
+    @Override
+    public Object getValue(Object sourceValue) {
+        if (sourceValue instanceof Map) {
+            sourceValue = mapToJsonString(sourceValue);
+        }
+        return ConnectStringType.INSTANCE.getValue(sourceValue);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectStringType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectStringType.java
new file mode 100644
index 0000000..9809601
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectStringType.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rocketmq.connect.doris.converter.type.connect;
+
+import io.openmessaging.connector.api.data.Schema;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisTypeProperties;
+
+public class ConnectStringType extends AbstractConnectSchemaType {
+
+    public static final ConnectStringType INSTANCE = new ConnectStringType();
+
+    @Override
+    public String getTypeName(Schema schema) {
+        int columnLength = getColumnLength(schema);
+        if (columnLength > 0) {
+            return columnLength * 3 > DorisTypeProperties.MAX_VARCHAR_SIZE
+                ? DorisType.STRING
+                : String.format("%s(%s)", DorisType.VARCHAR, columnLength * 3);
+        }
+        return DorisType.STRING;
+    }
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {"STRING"};
+    }
+
+    private int getColumnLength(Schema schema) {
+        return Integer.parseInt(getSourceColumnLength(schema).orElse("0"));
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectTimeType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectTimeType.java
new file mode 100644
index 0000000..6ccc944
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectTimeType.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.rocketmq.connect.doris.converter.type.connect;
+
+import io.openmessaging.connector.api.data.logical.Time;
+import io.openmessaging.connector.api.errors.ConnectException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Date;
+import org.apache.rocketmq.connect.doris.converter.type.AbstractTimeType;
+import org.apache.rocketmq.connect.doris.converter.type.util.DateTimeUtils;
+
+public class ConnectTimeType extends AbstractTimeType {
+
+    public static final ConnectTimeType INSTANCE = new ConnectTimeType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {Time.LOGICAL_NAME};
+    }
+
+    @Override
+    public Object getValue(Object sourceValue) {
+        if (sourceValue == null) {
+            return null;
+        }
+        if (sourceValue instanceof Date) {
+
+            final LocalTime localTime = DateTimeUtils.toLocalTimeFromUtcDate((Date) sourceValue);
+            final LocalDateTime localDateTime = localTime.atDate(LocalDate.now());
+            return localDateTime.toLocalTime();
+        }
+
+        throw new ConnectException(
+            String.format(
+                "Unexpected %s value '%s' with type '%s'",
+                getClass().getSimpleName(), sourceValue, sourceValue.getClass().getName()));
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectTimestampType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectTimestampType.java
new file mode 100644
index 0000000..748ba44
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectTimestampType.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.rocketmq.connect.doris.converter.type.connect;
+
+import io.openmessaging.connector.api.data.logical.Timestamp;
+import io.openmessaging.connector.api.errors.ConnectException;
+import org.apache.rocketmq.connect.doris.converter.type.AbstractTimestampType;
+import org.apache.rocketmq.connect.doris.converter.type.util.DateTimeUtils;
+
+public class ConnectTimestampType extends AbstractTimestampType {
+
+    public static final ConnectTimestampType INSTANCE = new ConnectTimestampType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {Timestamp.LOGICAL_NAME};
+    }
+
+    @Override
+    public Object getValue(Object sourceValue) {
+        if (sourceValue == null) {
+            return null;
+        }
+        if (sourceValue instanceof java.util.Date) {
+            return DateTimeUtils.toLocalDateTimeFromDate((java.util.Date) sourceValue);
+        }
+
+        throw new ConnectException(
+            String.format(
+                "Unexpected %s value '%s' with type '%s'",
+                getClass().getSimpleName(), sourceValue, sourceValue.getClass().getName()));
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/AbstractDebeziumTimeType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/AbstractDebeziumTimeType.java
new file mode 100644
index 0000000..fa28a71
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/AbstractDebeziumTimeType.java
@@ -0,0 +1,45 @@
+/*
+ * 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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.openmessaging.connector.api.errors.ConnectException;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import org.apache.rocketmq.connect.doris.converter.type.AbstractTimeType;
+
+public abstract class AbstractDebeziumTimeType extends AbstractTimeType {
+
+    @Override
+    public Object getValue(Object sourceValue) {
+        if (sourceValue == null) {
+            return null;
+        }
+        if (sourceValue instanceof Number) {
+            final LocalTime localTime = getLocalTime((Number) sourceValue);
+            return String.format("%s", DateTimeFormatter.ISO_TIME.format(localTime));
+        }
+        throw new ConnectException(
+            String.format(
+                "Unexpected %s value '%s' with type '%s'",
+                getClass().getSimpleName(), sourceValue, sourceValue.getClass().getName()));
+    }
+
+    protected abstract LocalTime getLocalTime(Number value);
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/AbstractDebeziumTimestampType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/AbstractDebeziumTimestampType.java
new file mode 100644
index 0000000..8c844ed
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/AbstractDebeziumTimestampType.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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.openmessaging.connector.api.errors.ConnectException;
+import java.time.LocalDateTime;
+import org.apache.rocketmq.connect.doris.converter.type.AbstractTimestampType;
+
+public abstract class AbstractDebeziumTimestampType extends AbstractTimestampType {
+
+    @Override
+    public Object getValue(Object sourceValue) {
+        if (sourceValue == null) {
+            return null;
+        }
+        if (sourceValue instanceof Number) {
+            return getLocalDateTime(((Number) sourceValue).longValue());
+        }
+        throw new ConnectException(
+            String.format(
+                "Unexpected %s value '%s' with type '%s'",
+                getClass().getSimpleName(), sourceValue, sourceValue.getClass().getName()));
+    }
+
+    protected abstract LocalDateTime getLocalDateTime(long value);
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/ArrayType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/ArrayType.java
new file mode 100644
index 0000000..8fd97de
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/ArrayType.java
@@ -0,0 +1,107 @@
+/*
+ * 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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.openmessaging.connector.api.data.Schema;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.converter.RecordTypeRegister;
+import org.apache.rocketmq.connect.doris.converter.type.AbstractType;
+import org.apache.rocketmq.connect.doris.converter.type.Type;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+
+public class ArrayType extends AbstractType {
+    private static final String ARRAY_TYPE_TEMPLATE = "%s<%s>";
+    public static final ArrayType INSTANCE = new ArrayType();
+    private DorisOptions dorisOptions;
+    private RecordTypeRegister recordTypeRegister;
+
+    @Override
+    public void configure(DorisOptions dorisOptions) {
+        if (this.dorisOptions == null && this.recordTypeRegister == null) {
+            this.dorisOptions = dorisOptions;
+            registerNestedArrayType();
+        }
+    }
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {"ARRAY"};
+    }
+
+    @Override
+    public String getTypeName(Schema schema) {
+        if (schema.getValueSchema().isOptional()) {
+            Schema valueSchema = schema.getValueSchema();
+            String type =
+                Objects.nonNull(valueSchema.getName())
+                    ? valueSchema.getName()
+                    : valueSchema.getFieldType().name();
+            if (recordTypeRegister == null) {
+                registerNestedArrayType();
+            }
+            Type valueType = recordTypeRegister.getTypeRegistry().get(type);
+            if (valueType == null) {
+                return DorisType.STRING;
+            }
+            String typeName = valueType.getTypeName(schema);
+            return String.format(ARRAY_TYPE_TEMPLATE, DorisType.ARRAY, typeName);
+        }
+        return DorisType.STRING;
+    }
+
+    @Override
+    public Object getValue(Object sourceValue, Schema schema) {
+
+        if (sourceValue == null) {
+            return null;
+        }
+        Schema valueSchema = schema.getValueSchema();
+        String type =
+            Objects.nonNull(valueSchema.getName())
+                ? valueSchema.getName()
+                : valueSchema.getFieldType().name();
+
+        if (sourceValue instanceof List) {
+            List<Object> resultList = new ArrayList<>();
+            ArrayList<?> convertedValue = (ArrayList<?>) sourceValue;
+            if (recordTypeRegister == null) {
+                registerNestedArrayType();
+            }
+            Type valueType = recordTypeRegister.getTypeRegistry().get(type);
+            if (valueType == null) {
+                return sourceValue;
+            }
+
+            for (Object value : convertedValue) {
+                resultList.add(valueType.getValue(value, valueSchema));
+            }
+            return resultList;
+        }
+
+        return sourceValue;
+    }
+
+    private void registerNestedArrayType() {
+        this.recordTypeRegister = new RecordTypeRegister(dorisOptions);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/DateType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/DateType.java
new file mode 100644
index 0000000..f3c6628
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/DateType.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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.debezium.time.Date;
+import io.openmessaging.connector.api.errors.ConnectException;
+import org.apache.rocketmq.connect.doris.converter.type.AbstractDateType;
+import org.apache.rocketmq.connect.doris.converter.type.util.DateTimeUtils;
+
+public class DateType extends AbstractDateType {
+
+    public static final DateType INSTANCE = new DateType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {Date.SCHEMA_NAME};
+    }
+
+    @Override
+    public Object getValue(Object sourceValue) {
+        if (sourceValue == null) {
+            return null;
+        }
+        if (sourceValue instanceof Number) {
+            return DateTimeUtils.toLocalDateOfEpochDays(((Number) sourceValue).longValue());
+        }
+
+        throw new ConnectException(
+            String.format(
+                "Unexpected %s value '%s' with type '%s'",
+                getClass().getSimpleName(), sourceValue, sourceValue.getClass().getName()));
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/GeographyType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/GeographyType.java
new file mode 100644
index 0000000..4af22e7
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/GeographyType.java
@@ -0,0 +1,32 @@
+/*
+ * 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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.debezium.data.geometry.Geography;
+import org.apache.rocketmq.connect.doris.converter.type.AbstractGeometryType;
+
+public class GeographyType extends AbstractGeometryType {
+    public static final GeographyType INSTANCE = new GeographyType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {Geography.LOGICAL_NAME};
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/GeometryType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/GeometryType.java
new file mode 100644
index 0000000..30f6f57
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/GeometryType.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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.debezium.data.geometry.Geometry;
+import io.openmessaging.connector.api.data.Struct;
+import org.apache.rocketmq.connect.doris.converter.type.AbstractGeometryType;
+import org.apache.rocketmq.connect.doris.converter.type.util.GeoUtils;
+
+public class GeometryType extends AbstractGeometryType {
+    public static final GeometryType INSTANCE = new GeometryType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {Geometry.LOGICAL_NAME};
+    }
+
+    @Override
+    public Object getValue(Object sourceValue) {
+        if (sourceValue == null) {
+            return null;
+        }
+
+        if (sourceValue instanceof Struct) {
+            return GeoUtils.handleGeoStructData(sourceValue);
+        }
+
+        return sourceValue;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/MicroTimeType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/MicroTimeType.java
new file mode 100644
index 0000000..28a709e
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/MicroTimeType.java
@@ -0,0 +1,39 @@
+/*
+ * 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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.debezium.time.MicroTime;
+import java.time.LocalTime;
+import org.apache.rocketmq.connect.doris.converter.type.util.DateTimeUtils;
+
+public class MicroTimeType extends AbstractDebeziumTimeType {
+
+    public static final MicroTimeType INSTANCE = new MicroTimeType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {MicroTime.SCHEMA_NAME};
+    }
+
+    @Override
+    protected LocalTime getLocalTime(Number value) {
+        return DateTimeUtils.toLocalTimeFromDurationMicroseconds(value.longValue());
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/MicroTimestampType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/MicroTimestampType.java
new file mode 100644
index 0000000..a76d276
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/MicroTimestampType.java
@@ -0,0 +1,39 @@
+/*
+ * 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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.debezium.time.MicroTimestamp;
+import java.time.LocalDateTime;
+import org.apache.rocketmq.connect.doris.converter.type.util.DateTimeUtils;
+
+public class MicroTimestampType extends AbstractDebeziumTimestampType {
+
+    public static final MicroTimestampType INSTANCE = new MicroTimestampType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {MicroTimestamp.SCHEMA_NAME};
+    }
+
+    @Override
+    protected LocalDateTime getLocalDateTime(long value) {
+        return DateTimeUtils.toLocalDateTimeFromInstantEpochMicros(value);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/NanoTimeType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/NanoTimeType.java
new file mode 100644
index 0000000..2a356e2
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/NanoTimeType.java
@@ -0,0 +1,39 @@
+/*
+ * 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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.debezium.time.NanoTime;
+import java.time.LocalTime;
+import org.apache.rocketmq.connect.doris.converter.type.util.DateTimeUtils;
+
+public class NanoTimeType extends AbstractDebeziumTimeType {
+
+    public static final NanoTimeType INSTANCE = new NanoTimeType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {NanoTime.SCHEMA_NAME};
+    }
+
+    @Override
+    protected LocalTime getLocalTime(Number value) {
+        return DateTimeUtils.toLocalTimeFromDurationNanoseconds(value.longValue());
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/NanoTimestampType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/NanoTimestampType.java
new file mode 100644
index 0000000..3f7e88a
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/NanoTimestampType.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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.debezium.time.NanoTimestamp;
+import java.time.LocalDateTime;
+import org.apache.rocketmq.connect.doris.converter.type.util.DateTimeUtils;
+
+/**
+ * An implementation of {@link org.apache.doris.kafka.connector.converter.type.Type} for {@link
+ * MicroTimestamp} values.
+ */
+public class NanoTimestampType extends AbstractDebeziumTimestampType {
+
+    public static final NanoTimestampType INSTANCE = new NanoTimestampType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {NanoTimestamp.SCHEMA_NAME};
+    }
+
+    @Override
+    protected LocalDateTime getLocalDateTime(long value) {
+        return DateTimeUtils.toLocalDateTimeFromInstantEpochNanos(value);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/PointType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/PointType.java
new file mode 100644
index 0000000..9268acf
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/PointType.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rocketmq.connect.doris.converter.type.debezium;
+
+import io.debezium.data.geometry.Point;
+import io.openmessaging.connector.api.data.Struct;
+import org.apache.rocketmq.connect.doris.converter.type.AbstractGeometryType;
+import org.apache.rocketmq.connect.doris.converter.type.util.GeoUtils;
+
+public class PointType extends AbstractGeometryType {
+    public static final PointType INSTANCE = new PointType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {Point.LOGICAL_NAME};
+    }
+
+    @Override
+    public Object getValue(Object sourceValue) {
+        if (sourceValue == null) {
+            return null;
+        }
+        if (sourceValue instanceof Struct) {
+            return GeoUtils.handleGeoStructData(sourceValue);
+        }
+
+        return sourceValue;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/TimeType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/TimeType.java
new file mode 100644
index 0000000..df099b6
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/TimeType.java
@@ -0,0 +1,39 @@
+/*
+ * 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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.debezium.time.Time;
+import java.time.LocalTime;
+import org.apache.rocketmq.connect.doris.converter.type.util.DateTimeUtils;
+
+public class TimeType extends AbstractDebeziumTimeType {
+
+    public static final TimeType INSTANCE = new TimeType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {Time.SCHEMA_NAME};
+    }
+
+    @Override
+    protected LocalTime getLocalTime(Number value) {
+        return DateTimeUtils.toLocalTimeFromDurationMilliseconds(value.longValue());
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/TimestampType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/TimestampType.java
new file mode 100644
index 0000000..456cb7b
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/TimestampType.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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.debezium.time.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+
+public class TimestampType extends AbstractDebeziumTimestampType {
+
+    public static final TimestampType INSTANCE = new TimestampType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {Timestamp.SCHEMA_NAME};
+    }
+
+    @Override
+    protected LocalDateTime getLocalDateTime(long value) {
+        return LocalDateTime.ofInstant(Instant.ofEpochMilli(value), ZoneOffset.UTC);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/VariableScaleDecimalType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/VariableScaleDecimalType.java
new file mode 100644
index 0000000..062aa84
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/VariableScaleDecimalType.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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.debezium.data.SpecialValueDecimal;
+import io.debezium.data.VariableScaleDecimal;
+import io.openmessaging.connector.api.data.Schema;
+import io.openmessaging.connector.api.data.Struct;
+import io.openmessaging.connector.api.errors.ConnectException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Optional;
+import org.apache.rocketmq.connect.doris.converter.type.AbstractType;
+import org.apache.rocketmq.connect.doris.converter.type.doris.DorisType;
+
+public class VariableScaleDecimalType extends AbstractType {
+
+    public static final VariableScaleDecimalType INSTANCE = new VariableScaleDecimalType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {VariableScaleDecimal.LOGICAL_NAME};
+    }
+
+    @Override
+    public Object getValue(Object sourceValue) {
+        if (sourceValue == null) {
+            return null;
+        }
+        if (sourceValue instanceof Struct) {
+            Optional<BigDecimal> bigDecimalValue = toLogical((Struct) sourceValue).getDecimalValue();
+            return bigDecimalValue.get();
+        }
+
+        throw new ConnectException(
+            String.format(
+                "Unexpected %s value '%s' with type '%s'",
+                getClass().getSimpleName(), sourceValue, sourceValue.getClass().getName()));
+    }
+
+    @Override
+    public String getTypeName(Schema schema) {
+        // The data passed by VariableScaleDecimal data types does not provide adequate information to
+        // resolve the precision and scale for the data type, so instead we're going to default to the
+        // maximum double-based data types for the dialect, using DOUBLE.
+        return DorisType.DOUBLE;
+    }
+
+    @Override
+    public boolean isNumber() {
+        return true;
+    }
+
+    private static SpecialValueDecimal toLogical(final Struct value) {
+        return new SpecialValueDecimal(
+            new BigDecimal(new BigInteger(value.getBytes("value")), value.getInt32("scale")));
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/ZonedTimeType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/ZonedTimeType.java
new file mode 100644
index 0000000..fcd48f6
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/ZonedTimeType.java
@@ -0,0 +1,59 @@
+/*
+ * 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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.debezium.time.ZonedTime;
+import io.openmessaging.connector.api.errors.ConnectException;
+import java.time.OffsetTime;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import org.apache.rocketmq.connect.doris.converter.type.AbstractTimeType;
+
+public class ZonedTimeType extends AbstractTimeType {
+
+    public static final ZonedTimeType INSTANCE = new ZonedTimeType();
+    // The ZonedTime of debezium type only contains three types of hours, minutes and seconds
+    private final DateTimeFormatter TIME_FORMATTER = DateTimeFormatter.ofPattern("HH:mm:ss");
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {ZonedTime.SCHEMA_NAME};
+    }
+
+    @Override
+    public Object getValue(Object sourceValue) {
+        if (sourceValue == null) {
+            return null;
+        }
+        if (sourceValue instanceof String) {
+            OffsetTime offsetTime =
+                OffsetTime.parse((String) sourceValue, ZonedTime.FORMATTER)
+                    .withOffsetSameInstant(
+                        ZonedDateTime.now(getDatabaseTimeZone().toZoneId())
+                            .getOffset());
+            return offsetTime.format(TIME_FORMATTER);
+        }
+
+        throw new ConnectException(
+            String.format(
+                "Unexpected %s value '%s' with type '%s'",
+                getClass().getSimpleName(), sourceValue, sourceValue.getClass().getName()));
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/ZonedTimestampType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/ZonedTimestampType.java
new file mode 100644
index 0000000..8914145
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/ZonedTimestampType.java
@@ -0,0 +1,53 @@
+/*
+ * 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.rocketmq.connect.doris.converter.type.debezium;
+
+import io.debezium.time.ZonedTimestamp;
+import io.openmessaging.connector.api.errors.ConnectException;
+import java.time.ZonedDateTime;
+import org.apache.rocketmq.connect.doris.converter.type.AbstractTimestampType;
+
+public class ZonedTimestampType extends AbstractTimestampType {
+
+    public static final ZonedTimestampType INSTANCE = new ZonedTimestampType();
+
+    @Override
+    public String[] getRegistrationKeys() {
+        return new String[] {ZonedTimestamp.SCHEMA_NAME};
+    }
+
+    @Override
+    public Object getValue(Object sourceValue) {
+        if (sourceValue == null) {
+            return null;
+        }
+        if (sourceValue instanceof String) {
+            final ZonedDateTime zdt =
+                ZonedDateTime.parse((String) sourceValue, ZonedTimestamp.FORMATTER)
+                    .withZoneSameInstant(getDatabaseTimeZone().toZoneId());
+            return zdt.toOffsetDateTime();
+        }
+
+        throw new ConnectException(
+            String.format(
+                "Unexpected %s value '%s' with type '%s'",
+                getClass().getSimpleName(), sourceValue, sourceValue.getClass().getName()));
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/doris/DorisType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/doris/DorisType.java
new file mode 100644
index 0000000..4960919
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/doris/DorisType.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.rocketmq.connect.doris.converter.type.doris;
+
+public class DorisType {
+    public static final String BOOLEAN = "BOOLEAN";
+    public static final String TINYINT = "TINYINT";
+    public static final String SMALLINT = "SMALLINT";
+    public static final String INT = "INT";
+    public static final String BIGINT = "BIGINT";
+    public static final String LARGEINT = "LARGEINT";
+    public static final String FLOAT = "FLOAT";
+    public static final String DOUBLE = "DOUBLE";
+    public static final String DECIMAL = "DECIMAL";
+    public static final String DATE = "DATE";
+    public static final String DATETIME = "DATETIME";
+    public static final String CHAR = "CHAR";
+    public static final String VARCHAR = "VARCHAR";
+    public static final String STRING = "STRING";
+    public static final String HLL = "HLL";
+    public static final String BITMAP = "BITMAP";
+    public static final String ARRAY = "ARRAY";
+    public static final String JSONB = "JSONB";
+    public static final String JSON = "JSON";
+    public static final String MAP = "MAP";
+    public static final String STRUCT = "STRUCT";
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/doris/DorisTypeProperties.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/doris/DorisTypeProperties.java
new file mode 100644
index 0000000..b19fa22
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/doris/DorisTypeProperties.java
@@ -0,0 +1,32 @@
+/*
+ * 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.rocketmq.connect.doris.converter.type.doris;
+
+public class DorisTypeProperties {
+
+    /* Max precision of datetime type of Doris. */
+    public static final int MAX_SUPPORTED_DATE_TIME_PRECISION = 6;
+
+    public static final int TIMESTAMP_TYPE_MAX_PRECISION = 9;
+
+    public static final int MAX_VARCHAR_SIZE = 65533;
+
+    public static final int MAX_CHAR_SIZE = 255;
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/util/DateTimeUtils.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/util/DateTimeUtils.java
new file mode 100644
index 0000000..363841c
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/util/DateTimeUtils.java
@@ -0,0 +1,121 @@
+/*
+ * 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.
+ *
+ * Copied from
+ * https://github.com/debezium/debezium-connector-jdbc/blob/main/src/main/java/io/debezium/connector/jdbc/util/DateTimeUtils.java
+ * modified by doris.
+ */
+
+package org.apache.rocketmq.connect.doris.converter.type.util;
+
+import io.debezium.time.Conversions;
+import java.sql.Timestamp;
+import java.time.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.temporal.ChronoUnit;
+import java.util.Date;
+import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
+
+public class DateTimeUtils {
+
+    private DateTimeUtils() {
+    }
+
+    public static Instant toInstantFromNanos(long epochNanos) {
+        final long epochSeconds = TimeUnit.NANOSECONDS.toSeconds(epochNanos);
+        final long adjustment =
+            TimeUnit.NANOSECONDS.toNanos(epochNanos % TimeUnit.SECONDS.toNanos(1));
+        return Instant.ofEpochSecond(epochSeconds, adjustment);
+    }
+
+    public static ZonedDateTime toZonedDateTimeFromDate(Date date, TimeZone timeZone) {
+        return toZonedDateTimeFromDate(date, timeZone.toZoneId());
+    }
+
+    public static ZonedDateTime toZonedDateTimeFromDate(Date date, ZoneId zoneId) {
+        return date.toInstant().atZone(zoneId);
+    }
+
+    public static ZonedDateTime toZonedDateTimeFromInstantEpochMicros(long epochMicros) {
+        return Conversions.toInstantFromMicros(epochMicros).atZone(ZoneOffset.UTC);
+    }
+
+    public static ZonedDateTime toZonedDateTimeFromInstantEpochNanos(long epochNanos) {
+        return ZonedDateTime.ofInstant(toInstantFromNanos(epochNanos), ZoneOffset.UTC);
+    }
+
+    public static LocalDate toLocalDateOfEpochDays(long epochDays) {
+        return LocalDate.ofEpochDay(epochDays);
+    }
+
+    public static LocalDate toLocalDateFromDate(Date date) {
+        return toLocalDateFromInstantEpochMillis(date.getTime());
+    }
+
+    public static LocalDate toLocalDateFromInstantEpochMillis(long epochMillis) {
+        return LocalDate.ofEpochDay(Duration.ofMillis(epochMillis).toDays());
+    }
+
+    public static LocalTime toLocalTimeFromDurationMilliseconds(long durationMillis) {
+        return LocalTime.ofNanoOfDay(Duration.of(durationMillis, ChronoUnit.MILLIS).toNanos());
+    }
+
+    public static LocalTime toLocalTimeFromDurationMicroseconds(long durationMicros) {
+        return LocalTime.ofNanoOfDay(Duration.of(durationMicros, ChronoUnit.MICROS).toNanos());
+    }
+
+    public static LocalTime toLocalTimeFromDurationNanoseconds(long durationNanos) {
+        return LocalTime.ofNanoOfDay(Duration.of(durationNanos, ChronoUnit.NANOS).toNanos());
+    }
+
+    public static LocalTime toLocalTimeFromUtcDate(Date date) {
+        return date.toInstant().atOffset(ZoneOffset.UTC).toLocalTime();
+    }
+
+    public static LocalDateTime toLocalDateTimeFromDate(Date date) {
+        return toLocalDateTimeFromInstantEpochMillis(date.getTime());
+    }
+
+    public static LocalDateTime toLocalDateTimeFromInstantEpochMillis(long epochMillis) {
+        return LocalDateTime.ofInstant(
+            Conversions.toInstantFromMillis(epochMillis), ZoneOffset.UTC);
+    }
+
+    public static LocalDateTime toLocalDateTimeFromInstantEpochMicros(long epochMicros) {
+        return LocalDateTime.ofInstant(
+            Conversions.toInstantFromMicros(epochMicros), ZoneOffset.UTC);
+    }
+
+    public static LocalDateTime toLocalDateTimeFromInstantEpochNanos(long epochNanos) {
+        return LocalDateTime.ofInstant(toInstantFromNanos(epochNanos), ZoneOffset.UTC);
+    }
+
+    public static Timestamp toTimestampFromMillis(long epochMilliseconds) {
+        final Instant instant = Conversions.toInstantFromMillis(epochMilliseconds);
+        final Timestamp ts = new Timestamp(instant.toEpochMilli());
+        ts.setNanos(instant.getNano());
+        return ts;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/util/GeoUtils.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/util/GeoUtils.java
new file mode 100644
index 0000000..a7e8316
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/util/GeoUtils.java
@@ -0,0 +1,68 @@
+/*
+ * 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.rocketmq.connect.doris.converter.type.util;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.openmessaging.connector.api.data.Struct;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class GeoUtils {
+    private static final Logger LOGGER = LoggerFactory.getLogger(GeoUtils.class);
+
+    private GeoUtils() {
+    }
+
+    public static Object handleGeoStructData(Object sourceValue) {
+        // the Geometry datatype in MySQL will be converted to
+        // a String with Json format
+        final ObjectMapper objectMapper = new ObjectMapper();
+        Struct geometryStruct = (Struct) sourceValue;
+
+        try {
+            byte[] wkb = geometryStruct.getBytes("wkb");
+            String geoJson = OGCGeometry.fromBinary(ByteBuffer.wrap(wkb)).asGeoJson();
+            JsonNode originGeoNode = objectMapper.readTree(geoJson);
+
+            Optional<Integer> srid = Optional.ofNullable(geometryStruct.getInt32("srid"));
+            Map<String, Object> geometryInfo = new HashMap<>();
+            String geometryType = originGeoNode.get("type").asText();
+
+            geometryInfo.put("type", geometryType);
+            if ("GeometryCollection".equals(geometryType)) {
+                geometryInfo.put("geometries", originGeoNode.get("geometries"));
+            } else {
+                geometryInfo.put("coordinates", originGeoNode.get("coordinates"));
+            }
+
+            geometryInfo.put("srid", srid.orElse(0));
+            return geometryInfo;
+        } catch (Exception e) {
+            LOGGER.warn("Failed to parse Geometry datatype, converting the value to null", e);
+            return null;
+        }
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/util/SchemaUtils.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/util/SchemaUtils.java
new file mode 100644
index 0000000..095f90b
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/util/SchemaUtils.java
@@ -0,0 +1,59 @@
+/*
+ * 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.
+ *
+ * Copied from
+ * https://github.com/debezium/debezium-connector-jdbc/blob/main/src/main/java/io/debezium/connector/jdbc/util/SchemaUtils.java
+ * modified by doris.
+ */
+
+package org.apache.rocketmq.connect.doris.converter.type.util;
+
+import io.openmessaging.connector.api.data.Schema;
+import java.util.Objects;
+import java.util.Optional;
+
+public class SchemaUtils {
+    private static final String SCHEMA_PARAMETER_COLUMN_TYPE = "__debezium.source.column.type";
+    private static final String SCHEMA_PARAMETER_COLUMN_LENGTH = "__debezium.source.column.length";
+    private static final String SCHEMA_PARAMETER_COLUMN_PRECISION =
+        "__debezium.source.column.scale";
+    private static final String SCHEMA_PARAMETER_COLUMN_NAME = "__debezium.source.column.name";
+
+    public static Optional<String> getSourceColumnType(Schema schema) {
+        return getSchemaParameter(schema, SCHEMA_PARAMETER_COLUMN_TYPE);
+    }
+
+    public static Optional<String> getSourceColumnLength(Schema schema) {
+        return getSchemaParameter(schema, SCHEMA_PARAMETER_COLUMN_LENGTH);
+    }
+
+    public static Optional<String> getSourceColumnPrecision(Schema schema) {
+        return getSchemaParameter(schema, SCHEMA_PARAMETER_COLUMN_PRECISION);
+    }
+
+    public static Optional<String> getSourceColumnName(Schema schema) {
+        return getSchemaParameter(schema, SCHEMA_PARAMETER_COLUMN_NAME);
+    }
+
+    public static Optional<String> getSchemaParameter(Schema schema, String parameterName) {
+        if (!Objects.isNull(schema.getParameters())) {
+            return Optional.ofNullable(schema.getParameters().get(parameterName));
+        }
+        return Optional.empty();
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/ArgumentsException.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/ArgumentsException.java
new file mode 100644
index 0000000..96dea18
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/ArgumentsException.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.rocketmq.connect.doris.exception;
+
+public class ArgumentsException extends DorisException {
+
+    public ArgumentsException() {
+    }
+
+    public ArgumentsException(String message) {
+        super(message);
+    }
+
+    public ArgumentsException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public ArgumentsException(Throwable cause) {
+        super(cause);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/ConfigException.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/ConfigException.java
deleted file mode 100644
index f74873f..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/ConfigException.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.exception;
-
-/**
- * config exception
- */
-public class ConfigException extends RuntimeException {
-    private static final long serialVersionUID = 1L;
-
-    public ConfigException(String message) {
-        super(message);
-    }
-
-    public ConfigException(String name, Object value) {
-        this(name, value, null);
-    }
-
-    public ConfigException(String name, Object value, String message) {
-        super("Invalid value " + value + " for configuration " + name + (message == null ? "" : ": " + message));
-    }
-}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/ConnectedFailedException.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/ConnectedFailedException.java
new file mode 100644
index 0000000..260b2c6
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/ConnectedFailedException.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.rocketmq.connect.doris.exception;
+
+public class ConnectedFailedException extends DorisException {
+
+    public ConnectedFailedException(String message) {
+        super(message);
+    }
+
+    public ConnectedFailedException(String server, Throwable cause) {
+        super("Connect to " + server + " failed.", cause);
+    }
+
+    public ConnectedFailedException(String server, int statusCode, Throwable cause) {
+        super("Connect to " + server + " failed, status code is " + statusCode + ".", cause);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/CopyLoadException.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/CopyLoadException.java
new file mode 100644
index 0000000..958a7d3
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/CopyLoadException.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.rocketmq.connect.doris.exception;
+
+public class CopyLoadException extends DorisException {
+
+    public CopyLoadException() {
+    }
+
+    public CopyLoadException(String message) {
+        super(message);
+    }
+
+    public CopyLoadException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public CopyLoadException(Throwable cause) {
+        super(cause);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/DataConverterException.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/DataConverterException.java
new file mode 100644
index 0000000..593aa95
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/DataConverterException.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.rocketmq.connect.doris.exception;
+
+public class DataConverterException extends RuntimeException {
+
+    public DataConverterException() {
+        super();
+    }
+
+    public DataConverterException(String message) {
+        super(message);
+    }
+
+    public DataConverterException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public DataConverterException(Throwable cause) {
+        super(cause);
+    }
+
+    protected DataConverterException(
+        String message,
+        Throwable cause,
+        boolean enableSuppression,
+        boolean writableStackTrace) {
+        super(message, cause, enableSuppression, writableStackTrace);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/DataFormatException.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/DataFormatException.java
new file mode 100644
index 0000000..904d78f
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/DataFormatException.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.rocketmq.connect.doris.exception;
+
+public class DataFormatException extends DorisException {
+
+    public DataFormatException() {
+    }
+
+    public DataFormatException(String message) {
+        super(message);
+    }
+
+    public DataFormatException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public DataFormatException(Throwable cause) {
+        super(cause);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/DorisException.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/DorisException.java
index 0c991a9..09f82e7 100644
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/DorisException.java
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/DorisException.java
@@ -1,34 +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
+ * 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
+ *   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.
+ * 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.rocketmq.connect.doris.exception;
 
 public class DorisException extends RuntimeException {
-    private static final long serialVersionUID = 2L;
+
+    public DorisException() {
+    }
 
     public DorisException(String message) {
         super(message);
     }
 
-    public DorisException(String name, Object value) {
-        this(name, value, null);
+    public DorisException(String message, Throwable cause) {
+        super(message, cause);
     }
 
-    public DorisException(String name, Object value, String message) {
-        super("Invalid value " + value + " for configuration " + name + (message == null ? "" : ": " + message));
+    public DorisException(Throwable cause) {
+        super(cause);
     }
-}
\ No newline at end of file
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/SchemaChangeException.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/SchemaChangeException.java
new file mode 100644
index 0000000..d7d7565
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/SchemaChangeException.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rocketmq.connect.doris.exception;
+
+/**
+ * Doris Schema Change run exception.
+ */
+public class SchemaChangeException extends RuntimeException {
+    public SchemaChangeException() {
+        super();
+    }
+
+    public SchemaChangeException(String message) {
+        super(message);
+    }
+
+    public SchemaChangeException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public SchemaChangeException(Throwable cause) {
+        super(cause);
+    }
+
+    protected SchemaChangeException(
+        String message,
+        Throwable cause,
+        boolean enableSuppression,
+        boolean writableStackTrace) {
+        super(message, cause, enableSuppression, writableStackTrace);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/StreamLoadException.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/StreamLoadException.java
new file mode 100644
index 0000000..d2aaf28
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/StreamLoadException.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.rocketmq.connect.doris.exception;
+
+public class StreamLoadException extends RuntimeException {
+
+    public StreamLoadException() {
+        super();
+    }
+
+    public StreamLoadException(String message) {
+        super(message);
+    }
+
+    public StreamLoadException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public StreamLoadException(Throwable cause) {
+        super(cause);
+    }
+
+    protected StreamLoadException(
+        String message,
+        Throwable cause,
+        boolean enableSuppression,
+        boolean writableStackTrace) {
+        super(message, cause, enableSuppression, writableStackTrace);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/TableAlterOrCreateException.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/TableAlterOrCreateException.java
deleted file mode 100644
index 288873d..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/TableAlterOrCreateException.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.exception;
-
-import io.openmessaging.connector.api.errors.ConnectException;
-
-/**
- * table alter or create exception
- */
-public class TableAlterOrCreateException extends ConnectException {
-
-    public TableAlterOrCreateException(String reason) {
-        super(reason);
-    }
-
-    public TableAlterOrCreateException(String reason, Throwable throwable) {
-        super(reason, throwable);
-    }
-}
-
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/UploadException.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/UploadException.java
new file mode 100644
index 0000000..be59946
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/UploadException.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.rocketmq.connect.doris.exception;
+
+public class UploadException extends DorisException {
+
+    public UploadException() {
+    }
+
+    public UploadException(String message) {
+        super(message);
+    }
+
+    public UploadException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public UploadException(Throwable cause) {
+        super(cause);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/metrics/DorisConnectMonitor.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/metrics/DorisConnectMonitor.java
new file mode 100644
index 0000000..743a8a5
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/metrics/DorisConnectMonitor.java
@@ -0,0 +1,178 @@
+/*
+ * 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.rocketmq.connect.doris.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.MetricRegistry;
+import java.util.concurrent.atomic.AtomicLong;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DorisConnectMonitor {
+
+    private static final Logger LOG = LoggerFactory.getLogger(DorisConnectMonitor.class);
+
+    // committed offset in doris
+    private final AtomicLong committedOffset;
+
+    // total record flushed in doris
+    private final AtomicLong totalNumberOfRecord;
+    private final AtomicLong totalSizeOfData;
+    // total number of data successfully imported to doris through stream-load (or the total number
+    // of data files uploaded through copy-into).
+    private final AtomicLong totalLoadCount;
+
+    // buffer metrics, updated everytime when a buffer is flushed
+    private Histogram partitionBufferSizeBytesHistogram; // in Bytes
+    private Histogram partitionBufferCountHistogram;
+    private final AtomicLong buffMemoryUsage;
+    private final int taskId;
+    private final boolean enableCustomJMX;
+
+    public DorisConnectMonitor(
+        final boolean enableCustomJMXConfig,
+        final Integer taskId,
+        final MetricsJmxReporter metricsJmxReporter) {
+        this.committedOffset = new AtomicLong(-1);
+
+        this.totalLoadCount = new AtomicLong(0);
+        this.totalNumberOfRecord = new AtomicLong(0);
+        this.totalSizeOfData = new AtomicLong(0);
+
+        this.buffMemoryUsage = new AtomicLong(0);
+        this.taskId = taskId;
+        this.enableCustomJMX = enableCustomJMXConfig;
+        if (this.enableCustomJMX) {
+            registerJMXMetrics(metricsJmxReporter);
+            LOG.info("init DorisConnectMonitor, taskId={}", taskId);
+        }
+    }
+
+    /**
+     * Registers all the Metrics inside the metricRegistry.
+     *
+     * @param metricsJmxReporter wrapper class for registering all metrics related to above
+     *                           connector
+     */
+    private void registerJMXMetrics(MetricsJmxReporter metricsJmxReporter) {
+        MetricRegistry currentMetricRegistry = metricsJmxReporter.getMetricRegistry();
+
+        // Lazily remove all registered metrics from the registry since this can be invoked during
+        // partition reassignment
+        LOG.debug(
+            "Registering metrics existing:{}",
+            metricsJmxReporter.getMetricRegistry().getMetrics().keySet().toString());
+        metricsJmxReporter.removeMetricsFromRegistry(String.valueOf(taskId));
+
+        try {
+            // Offset JMX
+            currentMetricRegistry.register(
+                MetricsUtil.constructMetricName(
+                    taskId, MetricsUtil.OFFSET_DOMAIN, MetricsUtil.COMMITTED_OFFSET),
+                (Gauge<Long>) committedOffset::get);
+
+            // Total Processed JMX
+            currentMetricRegistry.register(
+                MetricsUtil.constructMetricName(
+                    taskId,
+                    MetricsUtil.TOTAL_PROCESSED_DOMAIN,
+                    MetricsUtil.TOTAL_LOAD_COUNT),
+                (Gauge<Long>) totalLoadCount::get);
+
+            currentMetricRegistry.register(
+                MetricsUtil.constructMetricName(
+                    taskId,
+                    MetricsUtil.TOTAL_PROCESSED_DOMAIN,
+                    MetricsUtil.TOTAL_RECORD_COUNT),
+                (Gauge<Long>) totalNumberOfRecord::get);
+
+            currentMetricRegistry.register(
+                MetricsUtil.constructMetricName(
+                    taskId,
+                    MetricsUtil.TOTAL_PROCESSED_DOMAIN,
+                    MetricsUtil.TOTAL_DATA_SIZE),
+                (Gauge<Long>) totalSizeOfData::get);
+
+            // Buffer histogram JMX
+            partitionBufferCountHistogram =
+                currentMetricRegistry.histogram(
+                    MetricsUtil.constructMetricName(
+                        taskId,
+                        MetricsUtil.BUFFER_DOMAIN,
+                        MetricsUtil.BUFFER_RECORD_COUNT));
+            partitionBufferSizeBytesHistogram =
+                currentMetricRegistry.histogram(
+                    MetricsUtil.constructMetricName(
+                        taskId,
+                        MetricsUtil.BUFFER_DOMAIN,
+                        MetricsUtil.BUFFER_SIZE_BYTES));
+            currentMetricRegistry.register(
+                MetricsUtil.constructMetricName(
+                    taskId, MetricsUtil.BUFFER_DOMAIN, MetricsUtil.BUFFER_MEMORY_USAGE),
+                (Gauge<Long>) buffMemoryUsage::get);
+        } catch (IllegalArgumentException ex) {
+            LOG.warn("Metrics already present:{}", ex.getMessage());
+        }
+    }
+
+    public void setCommittedOffset(long committedOffset) {
+        if (enableCustomJMX) {
+            this.committedOffset.set(committedOffset);
+        }
+    }
+
+    public void addAndGetLoadCount() {
+        if (enableCustomJMX) {
+            this.totalLoadCount.getAndIncrement();
+        }
+    }
+
+    public void addAndGetTotalNumberOfRecord(long totalNumberOfRecord) {
+        if (enableCustomJMX) {
+            this.totalNumberOfRecord.addAndGet(totalNumberOfRecord);
+        }
+    }
+
+    public void addAndGetTotalSizeOfData(long totalSizeOfData) {
+        if (enableCustomJMX) {
+            this.totalSizeOfData.addAndGet(totalSizeOfData);
+        }
+    }
+
+    public void addAndGetBuffMemoryUsage(long memoryUsage) {
+        if (enableCustomJMX) {
+            this.buffMemoryUsage.addAndGet(memoryUsage);
+        }
+    }
+
+    public void resetMemoryUsage() {
+        if (enableCustomJMX) {
+            this.buffMemoryUsage.set(0L);
+        }
+    }
+
+    public void updateBufferMetrics(long bufferSizeBytes, int numOfRecords) {
+        if (enableCustomJMX) {
+            partitionBufferSizeBytesHistogram.update(bufferSizeBytes);
+            partitionBufferCountHistogram.update(numOfRecords);
+        }
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/metrics/MetricsJmxReporter.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/metrics/MetricsJmxReporter.java
new file mode 100644
index 0000000..a4f3844
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/metrics/MetricsJmxReporter.java
@@ -0,0 +1,105 @@
+/*
+ * 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.rocketmq.connect.doris.metrics;
+
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.jmx.JmxReporter;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.concurrent.TimeUnit;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.connect.doris.exception.DorisException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MetricsJmxReporter {
+    static final Logger LOG = LoggerFactory.getLogger(MetricsJmxReporter.class);
+
+    private final MetricRegistry metricRegistry;
+
+    /**
+     * Wrapper on top of listeners and metricRegistry for codehale. This will be useful to start the
+     * jmx metrics when time is appropriate. (Check {@link MetricsJmxReporter#start()}
+     */
+    private final JmxReporter jmxReporter;
+
+    public MetricsJmxReporter(MetricRegistry metricRegistry, final String connectorName) {
+        this.metricRegistry = metricRegistry;
+        this.jmxReporter = createJMXReporter(connectorName);
+    }
+
+    public MetricRegistry getMetricRegistry() {
+        return metricRegistry;
+    }
+
+    /**
+     * This function will internally register all metrics present inside metric registry and will
+     * register mbeans to the mbeanserver
+     */
+    public void start() {
+        jmxReporter.start();
+    }
+
+    private static ObjectName getObjectName(
+        String connectorName, String jmxDomain, String metricName) {
+        try {
+            StringBuilder sb =
+                new StringBuilder(jmxDomain)
+                    .append(":connector=")
+                    .append(connectorName)
+                    .append(',');
+
+            Iterator<String> tokens = Arrays.stream(StringUtils.split(metricName, "/")).iterator();
+            sb.append("task=").append(tokens.next());
+            sb.append(",category=").append(tokens.next());
+            sb.append(",name=").append(tokens.next());
+
+            return new ObjectName(sb.toString());
+        } catch (MalformedObjectNameException e) {
+            LOG.warn("Could not create Object name for MetricName:{}", metricName);
+            throw new DorisException("Object Name is invalid");
+        }
+    }
+
+    public void removeMetricsFromRegistry(final String prefixFilter) {
+        if (!metricRegistry.getMetrics().isEmpty()) {
+            LOG.debug("Unregistering all metrics:{}", prefixFilter);
+            metricRegistry.removeMatching(MetricFilter.startsWith(prefixFilter));
+            LOG.debug(
+                "Metric registry:{}, size is:{}, names:{}",
+                prefixFilter,
+                metricRegistry.getMetrics().size(),
+                metricRegistry.getMetrics().keySet().toString());
+        }
+    }
+
+    private JmxReporter createJMXReporter(final String connectorName) {
+        return JmxReporter.forRegistry(this.metricRegistry)
+            .inDomain(MetricsUtil.JMX_METRIC_PREFIX)
+            .convertDurationsTo(TimeUnit.SECONDS)
+            .createsObjectNamesWith(
+                (ignoreMeterType, jmxDomain, metricName) ->
+                    getObjectName(connectorName, jmxDomain, metricName))
+            .build();
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/metrics/MetricsUtil.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/metrics/MetricsUtil.java
new file mode 100644
index 0000000..34330b8
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/metrics/MetricsUtil.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rocketmq.connect.doris.metrics;
+
+public class MetricsUtil {
+    public static final String JMX_METRIC_PREFIX = "kafka.connector.doris";
+
+    // Offset related constants
+    public static final String TOTAL_PROCESSED_DOMAIN = "total-processed";
+
+    // total number of data successfully imported to doris through stream-load (or the total number
+    // of data files uploaded through copy-into).
+    public static final String TOTAL_LOAD_COUNT = "total-load-count";
+    public static final String TOTAL_RECORD_COUNT = "total-record-count";
+    public static final String TOTAL_DATA_SIZE = "total-data-size";
+
+    // file count related constants
+    public static final String OFFSET_DOMAIN = "offsets";
+    // Successfully submitted data to doris' offset
+    public static final String COMMITTED_OFFSET = "committed-offset";
+
+    // Buffer related constants
+    public static final String BUFFER_DOMAIN = "buffer";
+    public static final String BUFFER_MEMORY_USAGE = "buffer-memory-usage";
+    public static final String BUFFER_SIZE_BYTES = "buffer-size-bytes";
+    public static final String BUFFER_RECORD_COUNT = "buffer-record-count";
+
+    public static String constructMetricName(
+        final Integer taskId, final String domain, final String metricName) {
+        return String.format("%s/%s/%s", taskId, domain, metricName);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/BackendV2.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/BackendV2.java
new file mode 100644
index 0000000..6488308
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/BackendV2.java
@@ -0,0 +1,89 @@
+/*
+ * 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.rocketmq.connect.doris.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.List;
+
+/**
+ * Be response model.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class BackendV2 {
+
+    @JsonProperty(value = "backends")
+    private List<BackendRowV2> backends;
+
+    public List<BackendRowV2> getBackends() {
+        return backends;
+    }
+
+    public void setBackends(List<BackendRowV2> backends) {
+        this.backends = backends;
+    }
+
+    public static class BackendRowV2 {
+        @JsonProperty("ip")
+        public String ip;
+
+        @JsonProperty("http_port")
+        public int httpPort;
+
+        @JsonProperty("is_alive")
+        public boolean isAlive;
+
+        public String getIp() {
+            return ip;
+        }
+
+        public void setIp(String ip) {
+            this.ip = ip;
+        }
+
+        public int getHttpPort() {
+            return httpPort;
+        }
+
+        public void setHttpPort(int httpPort) {
+            this.httpPort = httpPort;
+        }
+
+        public boolean isAlive() {
+            return isAlive;
+        }
+
+        public void setAlive(boolean alive) {
+            isAlive = alive;
+        }
+
+        public String toBackendString() {
+            return ip + ":" + httpPort;
+        }
+
+        public static BackendRowV2 of(String ip, int httpPort, boolean alive) {
+            BackendRowV2 rowV2 = new BackendRowV2();
+            rowV2.setIp(ip);
+            rowV2.setHttpPort(httpPort);
+            rowV2.setAlive(alive);
+            return rowV2;
+        }
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/BaseResponse.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/BaseResponse.java
new file mode 100644
index 0000000..8d29b3d
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/BaseResponse.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.rocketmq.connect.doris.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class BaseResponse<T> {
+    private int code;
+    private String msg;
+    private T data;
+    private int count;
+
+    public int getCode() {
+        return code;
+    }
+
+    public String getMsg() {
+        return msg;
+    }
+
+    public T getData() {
+        return data;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/ColumnDescriptor.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/ColumnDescriptor.java
new file mode 100644
index 0000000..326cf09
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/ColumnDescriptor.java
@@ -0,0 +1,91 @@
+/*
+ * 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.rocketmq.connect.doris.model;
+
+import java.util.Objects;
+
+public class ColumnDescriptor {
+    private final String columnName;
+    private final String typeName;
+    private final String comment;
+    private final String defaultValue;
+
+    private ColumnDescriptor(
+        String columnName, String typeName, String comment, String defaultValue) {
+        this.columnName = columnName;
+        this.typeName = typeName;
+        this.comment = comment;
+        this.defaultValue = defaultValue;
+    }
+
+    public String getColumnName() {
+        return columnName;
+    }
+
+    public String getTypeName() {
+        return typeName;
+    }
+
+    public String getDefaultValue() {
+        return defaultValue;
+    }
+
+    public String getComment() {
+        return comment;
+    }
+
+    public static Builder builder() {
+        return new Builder();
+    }
+
+    public static class Builder {
+        private String columnName;
+        private String typeName;
+        private String comment;
+        private String defaultValue;
+
+        public Builder columnName(String columnName) {
+            this.columnName = columnName;
+            return this;
+        }
+
+        public Builder typeName(String typeName) {
+            this.typeName = typeName;
+            return this;
+        }
+
+        public Builder comment(String comment) {
+            this.comment = comment;
+            return this;
+        }
+
+        public Builder defaultValue(String defaultValue) {
+            this.defaultValue = defaultValue;
+            return this;
+        }
+
+        public ColumnDescriptor build() {
+            Objects.requireNonNull(columnName, "A column name is required");
+            Objects.requireNonNull(typeName, "A type name is required");
+
+            return new ColumnDescriptor(columnName, typeName, comment, defaultValue);
+        }
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/CopyIntoResp.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/CopyIntoResp.java
new file mode 100644
index 0000000..4b270d4
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/CopyIntoResp.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.rocketmq.connect.doris.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import java.util.Map;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class CopyIntoResp extends BaseResponse {
+    private String code;
+    private String exception;
+
+    private Map<String, String> result;
+
+    public String getDataCode() {
+        return code;
+    }
+
+    public String getException() {
+        return exception;
+    }
+
+    public Map<String, String> getResult() {
+        return result;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/KafkaRespContent.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/KafkaRespContent.java
new file mode 100644
index 0000000..b44c78f
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/KafkaRespContent.java
@@ -0,0 +1,60 @@
+/*
+ * 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.rocketmq.connect.doris.model;
+
+public class KafkaRespContent extends RespContent {
+
+    private String topic;
+    private String database;
+    private String table;
+    private long lastOffset;
+
+    public String getTopic() {
+        return topic;
+    }
+
+    public void setTopic(String topic) {
+        this.topic = topic;
+    }
+
+    public String getDatabase() {
+        return database;
+    }
+
+    public void setDatabase(String database) {
+        this.database = database;
+    }
+
+    public String getTable() {
+        return table;
+    }
+
+    public void setTable(String table) {
+        this.table = table;
+    }
+
+    public long getLastOffset() {
+        return lastOffset;
+    }
+
+    public void setLastOffset(long lastOffset) {
+        this.lastOffset = lastOffset;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/LoadOperation.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/LoadOperation.java
new file mode 100644
index 0000000..33f8c2a
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/LoadOperation.java
@@ -0,0 +1,32 @@
+/*
+ * 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.rocketmq.connect.doris.model;
+
+public enum LoadOperation {
+    SEND_REQUEST_TO_DORIS,
+
+    COMMIT_TRANSACTION,
+
+    GET_UPLOAD_ADDRESS,
+
+    UPLOAD_FILE,
+
+    EXECUTE_COPY;
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/RespContent.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/RespContent.java
new file mode 100644
index 0000000..6e5caee
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/RespContent.java
@@ -0,0 +1,117 @@
+/*
+ * 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.rocketmq.connect.doris.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class RespContent {
+
+    @JsonProperty(value = "TxnId")
+    private long txnId;
+
+    @JsonProperty(value = "Label")
+    private String label;
+
+    @JsonProperty(value = "Status")
+    private String status;
+
+    @JsonProperty(value = "TwoPhaseCommit")
+    private String twoPhaseCommit;
+
+    @JsonProperty(value = "ExistingJobStatus")
+    private String existingJobStatus;
+
+    @JsonProperty(value = "Message")
+    private String message;
+
+    @JsonProperty(value = "NumberTotalRows")
+    private long numberTotalRows;
+
+    @JsonProperty(value = "NumberLoadedRows")
+    private long numberLoadedRows;
+
+    @JsonProperty(value = "NumberFilteredRows")
+    private int numberFilteredRows;
+
+    @JsonProperty(value = "NumberUnselectedRows")
+    private int numberUnselectedRows;
+
+    @JsonProperty(value = "LoadBytes")
+    private long loadBytes;
+
+    @JsonProperty(value = "LoadTimeMs")
+    private int loadTimeMs;
+
+    @JsonProperty(value = "BeginTxnTimeMs")
+    private int beginTxnTimeMs;
+
+    @JsonProperty(value = "StreamLoadPutTimeMs")
+    private int streamLoadPutTimeMs;
+
+    @JsonProperty(value = "ReadDataTimeMs")
+    private int readDataTimeMs;
+
+    @JsonProperty(value = "WriteDataTimeMs")
+    private int writeDataTimeMs;
+
+    @JsonProperty(value = "CommitAndPublishTimeMs")
+    private int commitAndPublishTimeMs;
+
+    @JsonProperty(value = "ErrorURL")
+    private String errorURL;
+
+    public long getTxnId() {
+        return txnId;
+    }
+
+    public String getStatus() {
+        return status;
+    }
+
+    public String getTwoPhaseCommit() {
+        return twoPhaseCommit;
+    }
+
+    public String getMessage() {
+        return message;
+    }
+
+    public String getExistingJobStatus() {
+        return existingJobStatus;
+    }
+
+    @Override
+    public String toString() {
+        ObjectMapper mapper = new ObjectMapper();
+        try {
+            return mapper.writeValueAsString(this);
+        } catch (JsonProcessingException e) {
+            return "";
+        }
+    }
+
+    public String getErrorURL() {
+        return errorURL;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/TableDescriptor.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/TableDescriptor.java
new file mode 100644
index 0000000..5f92738
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/TableDescriptor.java
@@ -0,0 +1,101 @@
+/*
+ * 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.rocketmq.connect.doris.model;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+public class TableDescriptor {
+    private final String tableName;
+    private final String tableType;
+    private final Map<String, ColumnDescriptor> columns = new LinkedHashMap<>();
+
+    private TableDescriptor(String tableName, String tableType, List<ColumnDescriptor> columns) {
+        this.tableName = tableName;
+        this.tableType = tableType;
+        columns.forEach(c -> this.columns.put(c.getColumnName(), c));
+    }
+
+    public static Builder builder() {
+        return new Builder();
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public String getTableType() {
+        return tableType;
+    }
+
+    public Collection<ColumnDescriptor> getColumns() {
+        return columns.values();
+    }
+
+    public ColumnDescriptor getColumnByName(String columnName) {
+        return columns.get(columnName);
+    }
+
+    public boolean hasColumn(String columnName) {
+        return columns.containsKey(columnName);
+    }
+
+    public static class Builder {
+        private String schemaName;
+        private String tableName;
+        private String tableType;
+        private final List<ColumnDescriptor> columns = new ArrayList<>();
+
+        private Builder() {
+        }
+
+        public Builder schemaName(String schemaName) {
+            this.schemaName = schemaName;
+            return this;
+        }
+
+        public Builder tableName(String tableName) {
+            this.tableName = tableName;
+            return this;
+        }
+
+        public Builder type(String tableType) {
+            this.tableType = tableType;
+            return this;
+        }
+
+        public Builder column(ColumnDescriptor column) {
+            this.columns.add(column);
+            return this;
+        }
+
+        public Builder columns(List<ColumnDescriptor> columns) {
+            this.columns.addAll(columns);
+            return this;
+        }
+
+        public TableDescriptor build() {
+            return new TableDescriptor(tableName, tableType, columns);
+        }
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/doris/Field.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/doris/Field.java
new file mode 100644
index 0000000..edce95a
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/doris/Field.java
@@ -0,0 +1,149 @@
+/*
+ * 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.rocketmq.connect.doris.model.doris;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.Objects;
+
+public class Field {
+    @JsonProperty(value = "name")
+    private String name;
+
+    @JsonProperty(value = "type")
+    private String type;
+
+    @JsonProperty(value = "comment")
+    private String comment;
+
+    @JsonProperty(value = "precision")
+    private int precision;
+
+    @JsonProperty(value = "scale")
+    private int scale;
+
+    @JsonProperty(value = "aggregation_type")
+    private String aggregationType;
+
+    public Field() {
+    }
+
+    public Field(
+        String name,
+        String type,
+        String comment,
+        int precision,
+        int scale,
+        String aggregationType) {
+        this.name = name;
+        this.type = type;
+        this.comment = comment;
+        this.precision = precision;
+        this.scale = scale;
+        this.aggregationType = aggregationType;
+    }
+
+    public String getAggregationType() {
+        return aggregationType;
+    }
+
+    public void setAggregationType(String aggregationType) {
+        this.aggregationType = aggregationType;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getType() {
+        return type;
+    }
+
+    public void setType(String type) {
+        this.type = type;
+    }
+
+    public String getComment() {
+        return comment;
+    }
+
+    public void setComment(String comment) {
+        this.comment = comment;
+    }
+
+    public int getPrecision() {
+        return precision;
+    }
+
+    public void setPrecision(int precision) {
+        this.precision = precision;
+    }
+
+    public int getScale() {
+        return scale;
+    }
+
+    public void setScale(int scale) {
+        this.scale = scale;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        Field field = (Field) o;
+        return precision == field.precision
+            && scale == field.scale
+            && Objects.equals(name, field.name)
+            && Objects.equals(type, field.type)
+            && Objects.equals(comment, field.comment);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(name, type, comment, precision, scale);
+    }
+
+    @Override
+    public String toString() {
+        return "Field{"
+            + "name='"
+            + name
+            + '\''
+            + ", type='"
+            + type
+            + '\''
+            + ", comment='"
+            + comment
+            + '\''
+            + ", precision="
+            + precision
+            + ", scale="
+            + scale
+            + '}';
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/doris/Schema.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/doris/Schema.java
new file mode 100644
index 0000000..60ffb9d
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/doris/Schema.java
@@ -0,0 +1,110 @@
+/*
+ * 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.rocketmq.connect.doris.model.doris;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+public class Schema {
+    private int status = 0;
+    private String keysType;
+    private List<Field> properties;
+
+    public Schema() {
+        properties = new ArrayList<>();
+    }
+
+    public Schema(int fieldCount) {
+        properties = new ArrayList<>(fieldCount);
+    }
+
+    public int getStatus() {
+        return status;
+    }
+
+    public void setStatus(int status) {
+        this.status = status;
+    }
+
+    public String getKeysType() {
+        return keysType;
+    }
+
+    public void setKeysType(String keysType) {
+        this.keysType = keysType;
+    }
+
+    public List<Field> getProperties() {
+        return properties;
+    }
+
+    public void setProperties(List<Field> properties) {
+        this.properties = properties;
+    }
+
+    public void put(
+        String name,
+        String type,
+        String comment,
+        int scale,
+        int precision,
+        String aggregationType) {
+        properties.add(new Field(name, type, comment, scale, precision, aggregationType));
+    }
+
+    public void put(Field f) {
+        properties.add(f);
+    }
+
+    public Field get(int index) {
+        if (index >= properties.size()) {
+            throw new IndexOutOfBoundsException(
+                "Index: " + index + ", Fields size:" + properties.size());
+        }
+        return properties.get(index);
+    }
+
+    public int size() {
+        return properties.size();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        Schema schema = (Schema) o;
+        return status == schema.status && Objects.equals(properties, schema.properties);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(status, properties);
+    }
+
+    @Override
+    public String toString() {
+        return "Schema{" + "status=" + status + ", properties=" + properties + '}';
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/ColumnDefAdjuster.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/ColumnDefAdjuster.java
deleted file mode 100644
index 12b01b9..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/ColumnDefAdjuster.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.schema.column;
-
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.HashMap;
-import java.util.Map;
-
-public class ColumnDefAdjuster {
-    Map<String, ColumnDefinition.Nullability> nullable = new HashMap<>();
-
-    public ColumnDefAdjuster() {
-    }
-
-    public static ColumnDefAdjuster create(Connection conn,
-                                           String catalogPattern,
-                                           String schemaPattern,
-                                           String tablePattern,
-                                           String columnPattern) {
-        ColumnDefAdjuster adjuster = new ColumnDefAdjuster();
-        try (ResultSet rs = conn.getMetaData().getColumns(
-                catalogPattern, schemaPattern, tablePattern, columnPattern)) {
-            final int rsColumnCount = rs.getMetaData().getColumnCount();
-            while (rs.next()) {
-                final String columnName = rs.getString(4);
-                ColumnDefinition.Nullability nullability;
-                final int nullableValue = rs.getInt(11);
-                switch (nullableValue) {
-                    case DatabaseMetaData.columnNoNulls:
-                        nullability = ColumnDefinition.Nullability.NOT_NULL;
-                        break;
-                    case DatabaseMetaData.columnNullable:
-                        nullability = ColumnDefinition.Nullability.NULL;
-                        break;
-                    case DatabaseMetaData.columnNullableUnknown:
-                    default:
-                        nullability = ColumnDefinition.Nullability.UNKNOWN;
-                        break;
-                }
-                adjuster.nullable.put(columnName, nullability);
-            }
-        } catch (SQLException e) {
-            //pass
-        }
-
-        return adjuster;
-    }
-
-    public ColumnDefinition.Nullability nullable(String columnName) {
-        if (nullable == null || !nullable.containsKey(columnName)) {
-            return null;
-        }
-        return nullable.get(columnName);
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/ColumnDefinition.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/ColumnDefinition.java
deleted file mode 100644
index 3bcd3b8..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/ColumnDefinition.java
+++ /dev/null
@@ -1,337 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.schema.column;
-
-import java.sql.Types;
-import java.util.Objects;
-
-/**
- * column definition
- */
-public class ColumnDefinition {
-
-    /**
-     * The nullability of a column.
-     */
-    public enum Nullability {
-        NULL, NOT_NULL, UNKNOWN
-    }
-
-    /**
-     * The mutability of a column.
-     */
-    public enum Mutability {
-        READ_ONLY, MAYBE_WRITABLE, WRITABLE, UNKNOWN
-    }
-
-    private final ColumnId id;
-    private final String typeName;
-    private final int jdbcType;
-    private final int displaySize;
-    private final int precision;
-    private final int scale;
-    private final boolean autoIncremented;
-    private final boolean caseSensitive;
-    private final boolean searchable;
-    private final boolean currency;
-    private final boolean signedNumbers;
-    private final boolean isPrimaryKey;
-    private final Nullability nullability;
-    private final Mutability mutability;
-    private final String classNameForType;
-
-    public ColumnDefinition(
-            ColumnId id,
-            int jdbcType,
-            String typeName,
-            String classNameForType,
-            Nullability nullability,
-            Mutability mutability,
-            int precision,
-            int scale,
-            boolean signedNumbers,
-            int displaySize,
-            boolean autoIncremented,
-            boolean caseSensitive,
-            boolean searchable,
-            boolean currency,
-            boolean isPrimaryKey
-    ) {
-        this.id = id;
-        this.typeName = typeName;
-        this.jdbcType = jdbcType;
-        this.displaySize = displaySize;
-        this.precision = precision;
-        this.scale = scale;
-        this.autoIncremented = autoIncremented;
-        this.caseSensitive = caseSensitive;
-        this.searchable = searchable;
-        this.currency = currency;
-        this.signedNumbers = signedNumbers;
-        this.nullability = nullability != null ? nullability : Nullability.UNKNOWN;
-        this.mutability = mutability != null ? mutability : Mutability.MAYBE_WRITABLE;
-        this.classNameForType = classNameForType;
-        this.isPrimaryKey = isPrimaryKey;
-    }
-
-
-    /**
-     * Indicates whether the column is automatically numbered.
-     *
-     * @return <code>true</code> if so; <code>false</code> otherwise
-     */
-    public boolean isAutoIncrement() {
-        return autoIncremented;
-    }
-
-    /**
-     * Indicates whether the column's case matters.
-     *
-     * @return <code>true</code> if so; <code>false</code> otherwise
-     */
-    public boolean isCaseSensitive() {
-        return caseSensitive;
-    }
-
-    /**
-     * Indicates whether the column can be used in a where clause.
-     *
-     * @return <code>true</code> if so; <code>false</code> otherwise
-     */
-    public boolean isSearchable() {
-        return searchable;
-    }
-
-    /**
-     * Indicates whether the column is a cash value.
-     *
-     * @return <code>true</code> if so; <code>false</code> otherwise
-     */
-    public boolean isCurrency() {
-        return currency;
-    }
-
-    /**
-     * Indicates whether the column is part of the table's primary key.
-     *
-     * @return <code>true</code> if so; <code>false</code> otherwise
-     */
-    public boolean isPrimaryKey() {
-        return isPrimaryKey;
-    }
-
-    /**
-     * Indicates the nullability of values in the column.
-     *
-     * @return the nullability status of the given column; never null
-     */
-    public Nullability nullability() {
-        return nullability;
-    }
-
-    /**
-     * Indicates whether values in the column are optional. This is equivalent to calling:
-     * <pre>
-     *   nullability() == Nullability.NULL || nullability() == Nullability.UNKNOWN
-     * </pre>
-     *
-     * @return <code>true</code> if so; <code>false</code> otherwise
-     */
-    public boolean isOptional() {
-        return nullability == Nullability.NULL || nullability == Nullability.UNKNOWN;
-    }
-
-    /**
-     * Indicates whether values in the column are signed numbers.
-     *
-     * @return <code>true</code> if so; <code>false</code> otherwise
-     */
-    public boolean isSignedNumber() {
-        return signedNumbers;
-    }
-
-    /**
-     * Indicates the column's normal maximum width in characters.
-     *
-     * @return the normal maximum number of characters allowed as the width of the designated column
-     */
-    public int displaySize() {
-        return displaySize;
-    }
-
-    /**
-     * Get the column's identifier.
-     *
-     * @return column identifier; never null
-     */
-    public ColumnId id() {
-        return id;
-    }
-
-    /**
-     * Get the column's table identifier.
-     *
-     * @return the table identifier; never null
-     */
-//    public TableId tableId() {
-//        return id.tableId();
-//    }
-
-    /**
-     * Get the column's specified column size. For numeric data, this is the maximum precision.  For
-     * character data, this is the length in characters. For datetime datatypes, this is the length in
-     * characters of the String representation (assuming the maximum allowed precision of the
-     * fractional seconds component). For binary data, this is the length in bytes. For the ROWID
-     * datatype, this is the length in bytes. 0 is returned for data types where the column size is
-     * not applicable.
-     *
-     * @return precision
-     */
-    public int precision() {
-        return precision;
-    }
-
-    /**
-     * Gets the column's number of digits to right of the decimal point. 0 is returned for data types
-     * where the scale is not applicable.
-     *
-     * @return scale
-     */
-    public int scale() {
-        return scale;
-    }
-
-    /**
-     * Retrieves the column's JDBC type.
-     *
-     * @return SQL type from java.sql.Types
-     * @see Types
-     */
-    public int type() {
-        return jdbcType;
-    }
-
-    /**
-     * Retrieves the designated column's database-specific type name.
-     *
-     * @return type name used by the database. If the column type is a user-defined type, then a
-     * fully-qualified type name is returned.
-     */
-    public String typeName() {
-        return typeName;
-    }
-
-    /**
-     * Indicates whether the designated column is mutable.
-     *
-     * @return the mutability; never null
-     */
-    public Mutability mutability() {
-        return mutability;
-    }
-
-    /**
-     * Returns the fully-qualified name of the Java class whose instances are manufactured if the
-     * method {@link java.sql.ResultSet#getObject(int)} is called to retrieve a value from the column.
-     * {@link java.sql.ResultSet#getObject(int)} may return a subclass of the class returned by this
-     * method.
-     *
-     * @return the fully-qualified name of the class in the Java programming language that would be
-     * used by the method <code>ResultSet.getObject</code> to retrieve the value in the specified
-     * column. This is the class name used for custom mapping.
-     */
-    public String classNameForType() {
-        return classNameForType;
-    }
-
-
-    @Override
-    public int hashCode() {
-        return id.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == this) {
-            return true;
-        }
-        if (obj instanceof ColumnDefinition) {
-            ColumnDefinition that = (ColumnDefinition) obj;
-            return Objects.equals(this.id, that.id)
-                    && this.jdbcType == that.jdbcType
-                    && this.displaySize == that.displaySize
-                    && this.precision == that.precision
-                    && this.scale == that.scale
-                    && this.autoIncremented == that.autoIncremented
-                    && this.caseSensitive == that.caseSensitive
-                    && this.searchable == that.searchable
-                    && this.currency == that.currency
-                    && this.signedNumbers == that.signedNumbers
-                    && this.nullability == that.nullability
-                    && Objects.equals(this.typeName, that.typeName)
-                    && Objects.equals(this.classNameForType, that.classNameForType)
-                    && Objects.equals(this.isPrimaryKey, that.isPrimaryKey);
-        }
-        return false;
-    }
-
-    @Override
-    public String toString() {
-        return "Column{'" + id.name() + '\'' + ", isPrimaryKey=" + isPrimaryKey + ", allowsNull="
-                + isOptional() + ", sqlType=" + typeName + '}';
-    }
-
-    /**
-     * Obtain a {@link ColumnDefinition} that has all the same characteristics as this column except
-     * that belongs to the specified table.
-     *
-     * @param tableId the new table ID; may not be null
-//     * @return this definition if the specified table ID matches this definition's {@link #tableId()},
-     * or a new definition that is a copy of this definition except with an {@link #id() ID} that
-     * uses the specified table; never null
-     */
-//    public ColumnDefinition forTable(TableId tableId) {
-//        if (tableId().equals(tableId)) {
-//            return this;
-//        }
-//        ColumnId newId = new ColumnId(tableId, this.id().name());
-//        return new ColumnDefinition(newId, jdbcType, typeName, classNameForType, nullability,
-//                mutability, precision, scale, signedNumbers, displaySize,
-//                autoIncremented, caseSensitive, searchable, currency, isPrimaryKey
-//        );
-//    }
-
-    /**
-     * Obtain a {@link ColumnDefinition} that has all the same characteristics as this column except
-     * that it does or does not belong to the table's primary key
-     *
-     * @param isPrimaryKey true if the resulting column definition should be part of the table's
-     *                     primary key, or false otherwise
-     * @return a definition that is the same as this definition except it is or is not part of the
-     * tables primary key, or may be this object if {@link #isPrimaryKey()} already matches the
-     * supplied parameter; never null
-     */
-    public ColumnDefinition asPartOfPrimaryKey(boolean isPrimaryKey) {
-        if (isPrimaryKey == isPrimaryKey()) {
-            return this;
-        }
-        return new ColumnDefinition(id, jdbcType, typeName, classNameForType, nullability, mutability,
-                precision, scale, signedNumbers, displaySize, autoIncremented,
-                caseSensitive, searchable, currency, isPrimaryKey
-        );
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/ColumnId.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/ColumnId.java
deleted file mode 100644
index 39310c9..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/ColumnId.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.schema.column;
-
-import org.apache.rocketmq.connect.doris.schema.table.TableId;
-import org.apache.rocketmq.connect.doris.util.ExpressionBuilder;
-import org.apache.rocketmq.connect.doris.util.QuoteMethod;
-
-import java.util.Objects;
-
-/**
- * column id
- */
-public class ColumnId implements ExpressionBuilder.Expressable {
-
-    private final TableId tableId;
-    private final String name;
-    private final String alias;
-    private final int hash;
-
-    public ColumnId(
-            TableId tableId,
-            String columnName
-    ) {
-        this(tableId, columnName, null);
-    }
-
-    public ColumnId(
-            TableId tableId,
-            String columnName,
-            String alias
-    ) {
-        assert columnName != null;
-        this.tableId = tableId;
-        this.name = columnName;
-        this.alias = alias != null && !alias.trim().isEmpty() ? alias : name;
-        this.hash = Objects.hash(this.tableId, this.name);
-    }
-
-    public TableId tableId() {
-        return tableId;
-    }
-
-    public String name() {
-        return name;
-    }
-
-    public String aliasOrName() {
-        return alias;
-    }
-
-    @Override
-    public void appendTo(ExpressionBuilder builder, boolean useQuotes) {
-        appendTo(builder, useQuotes ? QuoteMethod.ALWAYS : QuoteMethod.NEVER);
-    }
-
-    @Override
-    public void appendTo(
-            ExpressionBuilder builder,
-            QuoteMethod useQuotes
-    ) {
-        if (tableId != null) {
-            builder.append(tableId);
-            builder.appendIdentifierDelimiter();
-        }
-        builder.appendColumnName(this.name, useQuotes);
-    }
-
-    @Override
-    public int hashCode() {
-        return hash;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == this) {
-            return true;
-        }
-        if (obj instanceof ColumnId) {
-            ColumnId that = (ColumnId) obj;
-            return Objects.equals(this.name, that.name) && Objects.equals(this.alias, that.alias)
-                    && Objects.equals(this.tableId, that.tableId);
-        }
-        return false;
-    }
-
-    @Override
-    public String toString() {
-        return ExpressionBuilder.create().append(this).toString();
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/parser/DateColumnParser.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/parser/DateColumnParser.java
deleted file mode 100644
index a089f62..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/parser/DateColumnParser.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.schema.column.parser;
-
-import io.openmessaging.connector.api.data.Schema;
-import io.openmessaging.connector.api.data.SchemaBuilder;
-import io.openmessaging.connector.api.errors.ConnectException;
-
-import java.util.Calendar;
-import java.util.TimeZone;
-
-public class DateColumnParser {
-    public static final String LOGICAL_NAME = "org.apache.rocketmq.connect.doris.schema.column.parser.DateColumnParser";
-    private static final long MILLIS_PER_DAY = 86400000L;
-    private static final TimeZone UTC = TimeZone.getTimeZone("UTC");
-    public static final Schema SCHEMA = builder().build();
-
-    public DateColumnParser() {
-    }
-
-    public static SchemaBuilder builder() {
-        return SchemaBuilder.int32().name("org.apache.kafka.connect.data.Date");
-    }
-
-    public static int fromLogical(Schema schema, java.util.Date value) {
-        if (!LOGICAL_NAME.equals(schema.getName())) {
-            throw new ConnectException("Requested conversion of Date object but the schema does not match.");
-        } else {
-            Calendar calendar = Calendar.getInstance(UTC);
-            calendar.setTime(value);
-            if (calendar.get(11) == 0 && calendar.get(12) == 0 && calendar.get(13) == 0 && calendar.get(14) == 0) {
-                long unixMillis = calendar.getTimeInMillis();
-                return (int) (unixMillis / 86400000L);
-            } else {
-                throw new ConnectException("Kafka Connect Date type should not have any time fields set to non-zero values.");
-            }
-        }
-    }
-
-    public static java.util.Date toLogical(Schema schema, int value) {
-        if (!LOGICAL_NAME.equals(schema.getName())) {
-            throw new ConnectException("Requested conversion of Date object but the schema does not match.");
-        } else {
-            return new java.util.Date((long) value * 86400000L);
-        }
-    }
-}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/parser/TimeColumnParser.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/parser/TimeColumnParser.java
deleted file mode 100644
index d160673..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/parser/TimeColumnParser.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.schema.column.parser;
-
-import io.openmessaging.connector.api.data.Schema;
-import io.openmessaging.connector.api.data.SchemaBuilder;
-import io.openmessaging.connector.api.errors.ConnectException;
-
-import java.util.Calendar;
-import java.util.Date;
-import java.util.TimeZone;
-
-public class TimeColumnParser {
-    public static final String LOGICAL_NAME = "org.apache.rocketmq.connect.doris.schema.column.parser.TimeColumnParser";
-    private static final long MILLIS_PER_DAY = 86400000L;
-    private static final TimeZone UTC = TimeZone.getTimeZone("UTC");
-    public static final Schema SCHEMA = builder().build();
-
-    public static SchemaBuilder builder() {
-        return SchemaBuilder.int32().name("org.apache.rocketmq.connect.doris.schema.column.parser.TimeColumnParser");
-    }
-
-    public static int fromLogical(Schema schema, Date value) {
-        if (!LOGICAL_NAME.equals(schema.getName())) {
-            throw new ConnectException("Requested conversion of Time object but the schema does not match.");
-        } else {
-            Calendar calendar = Calendar.getInstance(UTC);
-            calendar.setTime(value);
-            long unixMillis = calendar.getTimeInMillis();
-            if (unixMillis >= 0L && unixMillis <= 86400000L) {
-                return (int) unixMillis;
-            } else {
-                throw new ConnectException("Kafka Connect Time type should not have any date fields set to non-zero values.");
-            }
-        }
-    }
-
-    public static Date toLogical(Schema schema, int value) {
-        if (!LOGICAL_NAME.equals(schema.getName())) {
-            throw new ConnectException("Requested conversion of Date object but the schema does not match.");
-        } else if (value >= 0 && (long) value <= 86400000L) {
-            return new Date((long) value);
-        } else {
-            throw new ConnectException("Time values must use number of milliseconds greater than 0 and less than 86400000");
-        }
-    }
-
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/parser/TimestampColumnParser.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/parser/TimestampColumnParser.java
deleted file mode 100644
index a095bde..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/parser/TimestampColumnParser.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.schema.column.parser;
-
-import io.openmessaging.connector.api.data.Schema;
-import io.openmessaging.connector.api.data.SchemaBuilder;
-import io.openmessaging.connector.api.errors.ConnectException;
-
-import java.util.Date;
-
-/**
- * time stamp column parser
- */
-public class TimestampColumnParser {
-    public static final String LOGICAL_NAME = "org.apache.rocketmq.connect.doris.schema.column.parser.TimestampColumnParser";
-    public static final Schema SCHEMA = builder().build();
-
-    public TimestampColumnParser() {
-    }
-
-    public static SchemaBuilder builder() {
-        return SchemaBuilder.int64().name("org.apache.kafka.connect.data.Timestamp");
-    }
-
-    public static long fromLogical(Schema schema, Date value) {
-        if (!LOGICAL_NAME.equals(schema.getName())) {
-            throw new ConnectException("Requested conversion of Timestamp object but the schema does not match.");
-        } else {
-            return value.getTime();
-        }
-    }
-
-    public static Date toLogical(Schema schema, long value) {
-        if (!LOGICAL_NAME.equals(schema.getName())) {
-            throw new ConnectException("Requested conversion of Timestamp object but the schema does not match.");
-        } else {
-            return new Date(value);
-        }
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/db/DbStructure.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/db/DbStructure.java
deleted file mode 100644
index bcc1487..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/db/DbStructure.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.schema.db;
-
-import org.apache.rocketmq.connect.doris.connector.DorisSinkConfig;
-import org.apache.rocketmq.connect.doris.schema.table.TableId;
-import org.apache.rocketmq.connect.doris.sink.metadata.FieldsMetadata;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import java.sql.SQLException;
-
-/**
- *
- */
-public class DbStructure {
-    private static final Logger log = LoggerFactory.getLogger(DbStructure.class);
-
-    /**
-     * Create or amend table.
-     *
-     * @param config         the connector configuration
-//     * @param connection     the database connection handle
-     * @param tableId        the table ID
-     * @param fieldsMetadata the fields metadata
-     * @return whether a DDL operation was performed
-     * @throws SQLException if a DDL operation was deemed necessary but failed
-     */
-    public boolean createOrAmendIfNecessary(
-            final DorisSinkConfig config,
-            final TableId tableId,
-            final FieldsMetadata fieldsMetadata
-    ) throws SQLException {
-        // It seems that doris don't support create or amend table via stream load, so do nothing
-        return false;
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/table/TableDefinition.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/table/TableDefinition.java
deleted file mode 100644
index 68ca5aa..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/table/TableDefinition.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.schema.table;
-
-import org.apache.rocketmq.connect.doris.schema.column.ColumnDefinition;
-import org.apache.rocketmq.connect.doris.util.TableType;
-
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Objects;
-
-
-/**
- * A description of a table.
- */
-public class TableDefinition {
-    private final TableId id;
-    private final Map<String, String> pkColumnNames = new LinkedHashMap<>();
-
-    public TableDefinition(TableId id, Iterable<ColumnDefinition> columns, TableType type) {
-        this.id = id;
-    }
-
-    public TableId id() {
-        return id;
-    }
-
-    @Override
-    public int hashCode() {
-        return id.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return true;
-        }
-        if (obj instanceof TableDefinition) {
-            TableDefinition that = (TableDefinition) obj;
-            return Objects.equals(this.id(), that.id());
-        }
-        return false;
-    }
-
-    @Override
-    public String toString() {
-        return String.format("Table{name='%s', type=%s columns=%s}", id);
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/table/TableDefinitions.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/table/TableDefinitions.java
deleted file mode 100644
index 51a0d15..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/table/TableDefinitions.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.schema.table;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.sql.SQLException;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * A simple cache of {@link TableDefinition} keyed.
- */
-public class TableDefinitions {
-
-    private static final Logger log = LoggerFactory.getLogger(TableDefinitions.class);
-
-    private final Map<TableId, TableDefinition> cache = new HashMap<>();
-//    private final DatabaseDialect dialect;
-
-    /**
-     * Create an instance that uses the specified database dialect.
-     *
-     * @param dialect the database dialect; may not be null
-     */
-//    public TableDefinitions(DatabaseDialect dialect) {
-//        this.dialect = dialect;
-//    }
-
-    /**
-     * Get the {@link TableDefinition} for the given table.
-     *
-     * @param tableId    the table identifier; may not be null
-     * @return the cached {@link TableDefinition}, or null if there is no such table
-     * @throws SQLException if there is any problem using the connection
-     */
-    public TableDefinition get(
-            final TableId tableId
-    ) {
-        return cache.get(tableId);
-    }
-
-    public boolean contains(
-            final TableId tableId
-    ) {
-        return cache.containsKey(tableId);
-    }
-
-    /**
-     * Refresh the cached {@link TableDefinition} for the given table.
-     *
-     * @param tableId    the table identifier; may not be null
-     * @return the refreshed {@link TableDefinition}, or null if there is no such table
-     * @throws SQLException if there is any problem using the connection
-     */
-    public TableDefinition refresh(
-            TableId tableId
-    ) throws SQLException {
-        TableDefinition dbTable = null;
-        log.info("Refreshing metadata for table {} to {}", tableId, dbTable);
-        cache.put(dbTable.id(), dbTable);
-        return dbTable;
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/table/TableId.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/table/TableId.java
deleted file mode 100644
index 2b0e821..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/table/TableId.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.schema.table;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.rocketmq.connect.doris.util.ExpressionBuilder;
-import org.apache.rocketmq.connect.doris.util.QuoteMethod;
-
-import java.util.Objects;
-
-public class TableId implements Comparable<TableId>, ExpressionBuilder.Expressable {
-
-    private final String catalogName;
-    private final String schemaName;
-    private final String tableName;
-    private final int hash;
-
-    public TableId(
-            String catalogName,
-            String schemaName,
-            String tableName
-    ) {
-        this.catalogName = catalogName == null || catalogName.isEmpty() ? null : catalogName;
-        this.schemaName = schemaName == null || schemaName.isEmpty() ? null : schemaName;
-        this.tableName = tableName;
-        this.hash = Objects.hash(catalogName, schemaName, tableName);
-    }
-
-    public String catalogName() {
-        return catalogName;
-    }
-
-    public String schemaName() {
-        return schemaName;
-    }
-
-    public String dbName() {
-        if (StringUtils.isNotBlank(catalogName)) {
-            return catalogName;
-        }
-        return schemaName;
-    }
-
-    public String tableName() {
-        return tableName;
-    }
-
-    @Override
-    public void appendTo(ExpressionBuilder builder, boolean useQuotes) {
-        appendTo(builder, useQuotes ? QuoteMethod.ALWAYS : QuoteMethod.NEVER);
-    }
-
-    @Override
-    public void appendTo(
-            ExpressionBuilder builder,
-            QuoteMethod useQuotes
-    ) {
-        if (catalogName != null) {
-            builder.appendIdentifier(catalogName, useQuotes);
-            builder.appendIdentifierDelimiter();
-        }
-        if (schemaName != null) {
-            builder.appendIdentifier(schemaName, useQuotes);
-            builder.appendIdentifierDelimiter();
-        }
-        builder.appendTableName(tableName, useQuotes);
-    }
-
-    @Override
-    public int hashCode() {
-        return hash;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == this) {
-            return true;
-        }
-        if (obj instanceof TableId) {
-            TableId that = (TableId) obj;
-            return Objects.equals(this.catalogName, that.catalogName)
-                    && Objects.equals(this.schemaName, that.schemaName)
-                    && Objects.equals(this.tableName, that.tableName);
-        }
-        return false;
-    }
-
-    @Override
-    public int compareTo(TableId that) {
-        if (that == this) {
-            return 0;
-        }
-        int diff = this.tableName.compareTo(that.tableName);
-        if (diff != 0) {
-            return diff;
-        }
-        if (this.schemaName == null) {
-            if (that.schemaName != null) {
-                return -1;
-            }
-        } else {
-            if (that.schemaName == null) {
-                return 1;
-            }
-            diff = this.schemaName.compareTo(that.schemaName);
-            if (diff != 0) {
-                return diff;
-            }
-        }
-        if (this.catalogName == null) {
-            if (that.catalogName != null) {
-                return -1;
-            }
-        } else {
-            if (that.catalogName == null) {
-                return 1;
-            }
-            diff = this.catalogName.compareTo(that.catalogName);
-            if (diff != 0) {
-                return diff;
-            }
-        }
-        return 0;
-    }
-
-    @Override
-    public String toString() {
-        return ExpressionBuilder.create().append(this).toString();
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteArrayDeserializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteArrayDeserializer.java
deleted file mode 100644
index f0f7921..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteArrayDeserializer.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-public class ByteArrayDeserializer implements Deserializer<byte[]> {
-
-    @Override
-    public byte[] deserialize(String topic, byte[] data) {
-        return data;
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteArraySerializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteArraySerializer.java
deleted file mode 100644
index e786374..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteArraySerializer.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-public class ByteArraySerializer implements Serializer<byte[]> {
-    @Override
-    public byte[] serialize(String topic, byte[] data) {
-        return data;
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteBufferDeserializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteBufferDeserializer.java
deleted file mode 100644
index 3aa5996..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteBufferDeserializer.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-import java.nio.ByteBuffer;
-
-public class ByteBufferDeserializer implements Deserializer<ByteBuffer> {
-    @Override
-    public ByteBuffer deserialize(String topic, byte[] data) {
-        if (data == null) {
-            return null;
-        }
-
-        return ByteBuffer.wrap(data);
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteBufferSerializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteBufferSerializer.java
deleted file mode 100644
index 807eed3..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteBufferSerializer.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-import java.nio.ByteBuffer;
-
-public class ByteBufferSerializer implements Serializer<ByteBuffer> {
-    @Override
-    public byte[] serialize(String topic, ByteBuffer data) {
-        if (data == null) {
-            return null;
-        }
-
-        data.rewind();
-        if (data.hasArray()) {
-            byte[] arr = data.array();
-            if (data.arrayOffset() == 0 && arr.length == data.remaining()) {
-                return arr;
-            }
-        }
-
-        byte[] ret = new byte[data.remaining()];
-        data.get(ret, 0, ret.length);
-        data.rewind();
-        return ret;
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/Deserializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/Deserializer.java
deleted file mode 100644
index 69e9189..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/Deserializer.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-import io.openmessaging.KeyValue;
-
-import java.io.Closeable;
-import java.util.Map;
-
-/**
- * An interface for converting bytes to objects.
- *
- * @param <T> Type to be deserialized into.
- */
-public interface Deserializer<T> extends Closeable {
-
-    /**
-     * Configure this class.
-     * @param configs configs in key/value pairs
-     */
-    default void configure(Map<String, ?> configs) {
-        // intentionally left blank
-    }
-
-    /**
-     * Deserialize a record value from a byte array into a value or object.
-     * @param topic topic associated with the data
-     * @param data serialized bytes; may be null; implementations are recommended to handle null by returning a value or null rather than throwing an exception.
-     * @return deserialized typed data; may be null
-     */
-    T deserialize(String topic, byte[] data);
-
-    /**
-     * Deserialize a record value from a byte array into a value or object.
-     */
-    default T deserialize(String topic, KeyValue extensions, byte[] data) {
-        return deserialize(topic, data);
-    }
-
-    /**
-     * Close this deserializer.
-     * <p>
-     * This method must be idempotent as it may be called multiple times.
-     */
-    @Override
-    default void close() {
-        // intentionally left blank
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/DoubleDeserializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/DoubleDeserializer.java
deleted file mode 100644
index 23a5447..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/DoubleDeserializer.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-import org.apache.commons.lang3.SerializationException;
-
-public class DoubleDeserializer implements Deserializer<Double> {
-
-    @Override
-    public Double deserialize(String topic, byte[] data) {
-        if (data == null)
-            return null;
-        if (data.length != 8) {
-            throw new SerializationException("Size of data received by Deserializer is not 8");
-        }
-
-        long value = 0;
-        for (byte b : data) {
-            value <<= 8;
-            value |= b & 0xFF;
-        }
-        return Double.longBitsToDouble(value);
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/DoubleSerializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/DoubleSerializer.java
deleted file mode 100644
index 86ca81f..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/DoubleSerializer.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-public class DoubleSerializer implements Serializer<Double> {
-    @Override
-    public byte[] serialize(String topic, Double data) {
-        if (data == null) {
-            return null;
-        }
-
-        long bits = Double.doubleToLongBits(data);
-        return new byte[] {
-            (byte) (bits >>> 56),
-            (byte) (bits >>> 48),
-            (byte) (bits >>> 40),
-            (byte) (bits >>> 32),
-            (byte) (bits >>> 24),
-            (byte) (bits >>> 16),
-            (byte) (bits >>> 8),
-            (byte) bits
-        };
-    }
-}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/FloatDeserializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/FloatDeserializer.java
deleted file mode 100644
index b2da4fa..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/FloatDeserializer.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-import org.apache.commons.lang3.SerializationException;
-
-public class FloatDeserializer implements Deserializer<Float> {
-    @Override
-    public Float deserialize(final String topic, final byte[] data) {
-        if (data == null) {
-            return null;
-        }
-        if (data.length != 4) {
-            throw new SerializationException("Size of data received by Deserializer is not 4");
-        }
-
-        int value = 0;
-        for (byte b : data) {
-            value <<= 8;
-            value |= b & 0xFF;
-        }
-        return Float.intBitsToFloat(value);
-    }
-}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/FloatSerializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/FloatSerializer.java
deleted file mode 100644
index fc06f2b..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/FloatSerializer.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-public class FloatSerializer implements Serializer<Float> {
-    @Override
-    public byte[] serialize(final String topic, final Float data) {
-        if (data == null) {
-            return null;
-        }
-
-        long bits = Float.floatToRawIntBits(data);
-        return new byte[] {
-            (byte) (bits >>> 24),
-            (byte) (bits >>> 16),
-            (byte) (bits >>> 8),
-            (byte) bits
-        };
-    }
-}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/IntegerDeserializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/IntegerDeserializer.java
deleted file mode 100644
index b9ab9ee..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/IntegerDeserializer.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-import org.apache.commons.lang3.SerializationException;
-
-public class IntegerDeserializer implements Deserializer<Integer> {
-    @Override
-    public Integer deserialize(String topic, byte[] data) {
-        if (data == null) {
-            return null;
-        }
-        if (data.length != 4) {
-            throw new SerializationException("Size of data received by IntegerDeserializer is not 4");
-        }
-
-        int value = 0;
-        for (byte b : data) {
-            value <<= 8;
-            value |= b & 0xFF;
-        }
-        return value;
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/IntegerSerializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/IntegerSerializer.java
deleted file mode 100644
index 6203dda..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/IntegerSerializer.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-public class IntegerSerializer implements Serializer<Integer> {
-    @Override
-    public byte[] serialize(String topic, Integer data) {
-        if (data == null) {
-            return null;
-        }
-
-        return new byte[] {
-            (byte) (data >>> 24),
-            (byte) (data >>> 16),
-            (byte) (data >>> 8),
-            data.byteValue()
-        };
-    }
-}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/JsonDeserializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/JsonDeserializer.java
deleted file mode 100644
index dd1930c..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/JsonDeserializer.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-import com.alibaba.fastjson.JSON;
-import io.openmessaging.connector.api.errors.ConnectException;
-
-import java.nio.charset.StandardCharsets;
-import java.util.Objects;
-
-/**
- * json deserializer
- */
-public class JsonDeserializer implements Deserializer<Object> {
-
-    @Override
-    public Object deserialize(String topic, byte[] bytes) {
-        if (Objects.isNull(bytes)) {
-            return null;
-        }
-        Object data;
-        try {
-            String json = new String(bytes, StandardCharsets.UTF_8);
-            data = JSON.parse(json);
-        } catch (Exception e) {
-            throw new ConnectException(e);
-        }
-        return data;
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/JsonSerializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/JsonSerializer.java
deleted file mode 100644
index ca794a8..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/JsonSerializer.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-import com.alibaba.fastjson.JSON;
-import com.alibaba.fastjson.serializer.SerializerFeature;
-import io.openmessaging.connector.api.errors.ConnectException;
-
-import java.nio.charset.StandardCharsets;
-import java.util.Objects;
-
-/**
- * json serializer
- */
-public class JsonSerializer implements Serializer<Object> {
-
-
-    /**
-     * Convert {@code data} into a byte array.
-     *
-     * @param topic topic associated with data
-     * @param data  typed data
-     * @return serialized bytes
-     */
-    @Override
-    public byte[] serialize(String topic, Object data) {
-        if (Objects.isNull(data)) {
-            return null;
-        }
-        try {
-            return JSON.toJSONString(data, SerializerFeature.DisableCircularReferenceDetect,  SerializerFeature.WriteMapNullValue).getBytes(StandardCharsets.UTF_8);
-        } catch (Exception e) {
-            throw new ConnectException("Error serializing JSON message", e);
-        }
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/LongDeserializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/LongDeserializer.java
deleted file mode 100644
index 5022b45..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/LongDeserializer.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-import org.apache.commons.lang3.SerializationException;
-
-public class LongDeserializer implements Deserializer<Long> {
-    @Override
-    public Long deserialize(String topic, byte[] data) {
-        if (data == null) {
-            return null;
-        }
-        if (data.length != 8) {
-            throw new SerializationException("Size of data received by LongDeserializer is not 8");
-        }
-
-        long value = 0;
-        for (byte b : data) {
-            value <<= 8;
-            value |= b & 0xFF;
-        }
-        return value;
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/LongSerializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/LongSerializer.java
deleted file mode 100644
index 150e5bd..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/LongSerializer.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-public class LongSerializer implements Serializer<Long> {
-    @Override
-    public byte[] serialize(String topic, Long data) {
-        if (data == null) {
-            return null;
-        }
-
-        return new byte[] {
-            (byte) (data >>> 56),
-            (byte) (data >>> 48),
-            (byte) (data >>> 40),
-            (byte) (data >>> 32),
-            (byte) (data >>> 24),
-            (byte) (data >>> 16),
-            (byte) (data >>> 8),
-            data.byteValue()
-        };
-    }
-}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/Serializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/Serializer.java
deleted file mode 100644
index c96145e..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/Serializer.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-import io.openmessaging.KeyValue;
-
-import java.io.Closeable;
-import java.util.Map;
-
-/**
- * An interface for converting objects to bytes.
- */
-public interface Serializer<T> extends Closeable {
-
-    /**
-     * Configure this class.
-     * @param configs configs in key/value pairs
-     */
-    default void configure(Map<String, ?> configs) {
-        // intentionally left blank
-    }
-
-    /**
-     * Convert {@code data} into a byte array.
-     *
-     * @param topic topic associated with data
-     * @param data typed data
-     * @return serialized bytes
-     */
-    byte[] serialize(String topic, T data);
-
-    /**
-     * Convert data into a byte array.
-     * @return serialized bytes
-     */
-    default byte[] serialize(String topic, KeyValue extensions, T data) {
-        return serialize(topic, data);
-    }
-
-    /**
-     * Close this serializer.
-     */
-    @Override
-    default void close() {
-        // intentionally left blank
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ShortDeserializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ShortDeserializer.java
deleted file mode 100644
index f5d4fd7..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ShortDeserializer.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-import org.apache.commons.lang3.SerializationException;
-
-public class ShortDeserializer implements Deserializer<Short> {
-
-    @Override
-    public Short deserialize(String topic, byte[] data) {
-        if (data == null) {
-            return null;
-        }
-        if (data.length != 2) {
-            throw new SerializationException("Size of data received by ShortDeserializer is not 2");
-        }
-
-        short value = 0;
-        for (byte b : data) {
-            value <<= 8;
-            value |= b & 0xFF;
-        }
-        return value;
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ShortSerializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ShortSerializer.java
deleted file mode 100644
index 1e40b38..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ShortSerializer.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-public class ShortSerializer implements Serializer<Short> {
-    @Override
-    public byte[] serialize(String topic, Short data) {
-        if (data == null) {
-            return null;
-        }
-
-        return new byte[] {
-            (byte) (data >>> 8),
-            data.byteValue()
-        };
-    }
-}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/StringDeserializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/StringDeserializer.java
deleted file mode 100644
index d7a42e6..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/StringDeserializer.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-import java.io.UnsupportedEncodingException;
-import java.util.Map;
-
-/**
- *  String encoding defaults to UTF8 and can be customized by setting the property key.deserializer.encoding,
- *  value.deserializer.encoding or deserializer.encoding. The first two take precedence over the last.
- */
-public class StringDeserializer implements Deserializer<String> {
-    private String encoding = "UTF8";
-
-    @Override
-    public void configure(Map<String, ?> configs) {
-        Object encodingValue = configs.get("deserializer.encoding");
-        if (encodingValue instanceof String) {
-            encoding = (String) encodingValue;
-        }
-    }
-
-    @Override
-    public String deserialize(String topic, byte[] data) {
-        try {
-            if (data == null) {
-                return null;
-            } else {
-                return new String(data, encoding);
-            }
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException("Error when deserializing byte[] to string due to unsupported encoding " + encoding);
-        }
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/StringSerializer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/StringSerializer.java
deleted file mode 100644
index f9b569a..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/StringSerializer.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.serialization;
-
-import java.io.UnsupportedEncodingException;
-import java.util.Map;
-
-/**
- * string serializer
- */
-public class StringSerializer implements Serializer<String> {
-    private String encoding = "UTF8";
-
-    @Override
-    public void configure(Map<String, ?> configs) {
-        String propertyName = "serializer.encoding";
-        if (configs.containsKey(propertyName)) {
-            Object encodingValue = configs.get(propertyName);
-            if (encodingValue instanceof String) {
-                encoding = (String) encodingValue;
-            }
-        }
-    }
-
-    @Override
-    public byte[] serialize(String topic, String data) {
-        try {
-            if (data == null) {
-                return null;
-            } else {
-                return data.getBytes(encoding);
-            }
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException("Error when serializing string to byte[] due to unsupported encoding " + encoding);
-        }
-    }
-}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisDefaultSinkService.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisDefaultSinkService.java
new file mode 100644
index 0000000..34d3466
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisDefaultSinkService.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rocketmq.connect.doris.service;
+
+import com.codahale.metrics.MetricRegistry;
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.RecordOffset;
+import io.openmessaging.connector.api.data.RecordPartition;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.connection.ConnectionProvider;
+import org.apache.rocketmq.connect.doris.connection.JdbcConnectionProvider;
+import org.apache.rocketmq.connect.doris.metrics.DorisConnectMonitor;
+import org.apache.rocketmq.connect.doris.metrics.MetricsJmxReporter;
+import org.apache.rocketmq.connect.doris.utils.ConnectRecordUtil;
+import org.apache.rocketmq.connect.doris.writer.CopyIntoWriter;
+import org.apache.rocketmq.connect.doris.writer.DorisWriter;
+import org.apache.rocketmq.connect.doris.writer.StreamLoadWriter;
+import org.apache.rocketmq.connect.doris.writer.load.LoadModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DorisDefaultSinkService implements DorisSinkService {
+    private static final Logger LOG = LoggerFactory.getLogger(DorisDefaultSinkService.class);
+    private final ConnectionProvider conn;
+    private final Map<String, DorisWriter> writer;
+    private final DorisOptions dorisOptions;
+    private final MetricsJmxReporter metricsJmxReporter;
+    private final DorisConnectMonitor connectMonitor;
+
+    DorisDefaultSinkService(KeyValue config) {
+        this.dorisOptions = new DorisOptions(config);
+        this.writer = new HashMap<>();
+        this.conn = new JdbcConnectionProvider(dorisOptions);
+        MetricRegistry metricRegistry = new MetricRegistry();
+        this.metricsJmxReporter = new MetricsJmxReporter(metricRegistry, dorisOptions.getName());
+        this.connectMonitor =
+            new DorisConnectMonitor(
+                dorisOptions.isEnableCustomJMX(),
+                dorisOptions.getTaskId(),
+                this.metricsJmxReporter);
+    }
+
+    public void startService(String topic) {
+        LoadModel loadModel = dorisOptions.getLoadModel();
+        DorisWriter dorisWriter =
+            LoadModel.COPY_INTO.equals(loadModel)
+                ? new CopyIntoWriter(
+                topic, dorisOptions, conn, connectMonitor)
+                : new StreamLoadWriter(
+                topic, dorisOptions, conn, connectMonitor);
+        writer.put(topic, dorisWriter);
+        metricsJmxReporter.start();
+    }
+
+    @Override
+    public void insert(final List<ConnectRecord> records) {
+        for (ConnectRecord record : records) {
+            if (Objects.isNull(record.getData())) {
+                RecordPartition partition = record.getPosition().getPartition();
+                LOG.debug(
+                    "Null valued record from topic={} brokerName={} queueId={} and offset={} was skipped",
+                    ConnectRecordUtil.getTopicName(partition),
+                    ConnectRecordUtil.getBrokerName(partition),
+                    ConnectRecordUtil.getQueueId(partition),
+                    ConnectRecordUtil.getQueueOffset(record.getPosition().getOffset()));
+                continue;
+            }
+            insert(record);
+        }
+        // check all sink writer to see if they need to be flushed
+        for (DorisWriter writer : writer.values()) {
+            // Time based flushing
+            if (writer.shouldFlush()) {
+                writer.flushBuffer();
+            }
+        }
+    }
+
+    @Override
+    public void insert(final ConnectRecord record) {
+        String topicName = ConnectRecordUtil.getTopicName(record.getPosition().getPartition());
+        if (!writer.containsKey(topicName)) {
+            // todo startTask can be initialized in DorisSinkTask, and SinkTask needs to support the initialization method for each RecordPartition.
+            startService(topicName);
+        }
+        writer.get(topicName).insert(record);
+    }
+
+    @Override
+    public void commit(Map<RecordPartition, RecordOffset> currentOffsets) {
+        currentOffsets.keySet()
+            .forEach(
+                rp -> {
+                    String topicName = ConnectRecordUtil.getTopicName(rp);
+                    writer.get(topicName).commit();
+                });
+    }
+
+    @Override
+    public int getDorisWriterSize() {
+        return writer.size();
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisSinkService.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisSinkService.java
new file mode 100644
index 0000000..8b73c30
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisSinkService.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.rocketmq.connect.doris.service;
+
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.RecordOffset;
+import io.openmessaging.connector.api.data.RecordPartition;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Background service of data sink, responsible to create/drop table and insert/delete files
+ */
+public interface DorisSinkService {
+
+    void startService(String topic);
+
+    void insert(final List<ConnectRecord> records);
+
+    void insert(final ConnectRecord record);
+
+    void commit(Map<RecordPartition, RecordOffset> currentOffsets);
+
+    int getDorisWriterSize();
+
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisSinkServiceFactory.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisSinkServiceFactory.java
new file mode 100644
index 0000000..bb204ef
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisSinkServiceFactory.java
@@ -0,0 +1,32 @@
+/*
+ * 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.rocketmq.connect.doris.service;
+
+import io.openmessaging.KeyValue;
+
+/**
+ * A factory to create {@link DorisSinkService}
+ */
+public class DorisSinkServiceFactory {
+
+    public static DorisSinkService getDorisSinkService(KeyValue connectorConfig) {
+        return new DorisDefaultSinkService(connectorConfig);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisSystemService.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisSystemService.java
new file mode 100644
index 0000000..45f98b2
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisSystemService.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rocketmq.connect.doris.service;
+
+import com.google.common.collect.Lists;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Predicate;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.connection.JdbcConnectionProvider;
+import org.apache.rocketmq.connect.doris.exception.DorisException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DorisSystemService {
+
+    private static final Logger LOG = LoggerFactory.getLogger(DorisSystemService.class);
+    private static final String GET_COLUMN_EXISTS_TEMPLATE =
+        "SELECT COLUMN_NAME FROM information_schema.`COLUMNS` WHERE TABLE_SCHEMA = ? AND TABLE_NAME = ? AND COLUMN_NAME = ?";
+    private final JdbcConnectionProvider jdbcConnectionProvider;
+
+    public DorisSystemService(DorisOptions dorisOptions) {
+        this.jdbcConnectionProvider = new JdbcConnectionProvider(dorisOptions);
+    }
+
+    private static final List<String> builtinDatabases =
+        Collections.singletonList("information_schema");
+
+    public boolean tableExists(String database, String table) {
+        return listTables(database).contains(table);
+    }
+
+    public boolean databaseExists(String database) {
+        return listDatabases().contains(database);
+    }
+
+    public Set<String> listDatabases() {
+        return new HashSet<>(
+            extractColumnValuesBySQL(
+                "SELECT `SCHEMA_NAME` FROM `INFORMATION_SCHEMA`.`SCHEMATA`;",
+                1,
+                dbName -> !builtinDatabases.contains(dbName)));
+    }
+
+    public Set<String> listTables(String databaseName) {
+        if (!databaseExists(databaseName)) {
+            throw new DorisException("database" + databaseName + " is not exists");
+        }
+        return new HashSet<>(
+            extractColumnValuesBySQL(
+                "SELECT TABLE_NAME FROM information_schema.`TABLES` WHERE TABLE_SCHEMA = ?",
+                1,
+                null,
+                databaseName));
+    }
+
+    public boolean isColumnExist(String database, String tableName, String columnName) {
+        List<String> columnList =
+            extractColumnValuesBySQL(
+                GET_COLUMN_EXISTS_TEMPLATE, 1, null, database, tableName, columnName);
+        return !columnList.isEmpty();
+    }
+
+    public List<String> extractColumnValuesBySQL(
+        String sql, int columnIndex, Predicate<String> filterFunc, Object... params) {
+
+        List<String> columnValues = Lists.newArrayList();
+
+        try (Connection connection = jdbcConnectionProvider.getOrEstablishConnection();
+             PreparedStatement ps = connection.prepareStatement(sql)) {
+
+            if (Objects.nonNull(params) && params.length > 0) {
+                for (int i = 0; i < params.length; i++) {
+                    ps.setObject(i + 1, params[i]);
+                }
+            }
+
+            try (ResultSet rs = ps.executeQuery()) {
+                while (rs.next()) {
+                    String columnValue = rs.getString(columnIndex);
+                    if (filterFunc == null || filterFunc.test(columnValue)) {
+                        columnValues.add(columnValue);
+                    }
+                }
+            }
+            return columnValues;
+        } catch (Exception e) {
+            LOG.error("The following SQL query could not be executed: {}", sql, e);
+            throw new DorisException(
+                String.format("The following SQL query could not be executed: %s", sql), e);
+        }
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/RestService.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/RestService.java
new file mode 100644
index 0000000..b01afe5
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/RestService.java
@@ -0,0 +1,361 @@
+/*
+ * 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.rocketmq.connect.doris.service;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PrintWriter;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Scanner;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.io.IOUtils;
+import org.apache.http.HttpHeaders;
+import org.apache.http.HttpStatus;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.methods.HttpRequestBase;
+import org.apache.http.client.methods.HttpUriRequest;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.exception.ConnectedFailedException;
+import org.apache.rocketmq.connect.doris.exception.DorisException;
+import org.apache.rocketmq.connect.doris.exception.SchemaChangeException;
+import org.apache.rocketmq.connect.doris.model.BackendV2;
+import org.apache.rocketmq.connect.doris.model.LoadOperation;
+import org.apache.rocketmq.connect.doris.model.doris.Schema;
+import org.apache.rocketmq.connect.doris.utils.BackoffAndRetryUtils;
+import org.slf4j.Logger;
+
+public class RestService {
+
+    private static final String BACKENDS_V2 = "/api/backends?is_alive=true";
+    private static final String TABLE_SCHEMA_API = "http://%s/api/%s/%s/_schema";
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final String UNIQUE_KEYS_TYPE = "UNIQUE_KEYS";
+
+    /**
+     * get Doris BE nodes to request.
+     *
+     * @param options configuration of request
+     * @param logger  slf4j logger
+     * @return the chosen one Doris BE node
+     * @throws IllegalArgumentException BE nodes is illegal
+     */
+    public static List<BackendV2.BackendRowV2> getBackendsV2(DorisOptions options, Logger logger) {
+        List<String> feNodeList = options.getHttpUrls();
+        if (options.isAutoRedirect()) {
+            return convert(feNodeList);
+        }
+
+        for (String feNode : feNodeList) {
+            try {
+                String beUrl = "http://" + feNode + BACKENDS_V2;
+                HttpGet httpGet = new HttpGet(beUrl);
+                String response = send(options, httpGet, logger);
+                logger.info("Backend Info:{}", response);
+                List<BackendV2.BackendRowV2> backends = parseBackendV2(response, logger);
+                return backends;
+            } catch (ConnectedFailedException e) {
+                logger.info(
+                    "Doris FE node {} is unavailable: {}, Request the next Doris FE node",
+                    feNode,
+                    e.getMessage());
+            }
+        }
+        String errMsg = "No Doris FE is available, please check configuration";
+        logger.error(errMsg);
+        throw new DorisException(errMsg);
+    }
+
+    /**
+     * When the user turns on redirection, there is no need to explicitly obtain the be list, just
+     * treat the fe list as the be list.
+     */
+    private static List<BackendV2.BackendRowV2> convert(List<String> feNodeList) {
+        List<BackendV2.BackendRowV2> nodeList = new ArrayList<>();
+        for (String node : feNodeList) {
+            String[] split = node.split(":");
+            nodeList.add(BackendV2.BackendRowV2.of(split[0], Integer.parseInt(split[1]), true));
+        }
+        return nodeList;
+    }
+
+    public static boolean isUniqueKeyType(
+        DorisOptions dorisOptions, String tableName, Logger logger) {
+        try {
+            return UNIQUE_KEYS_TYPE.equals(
+                getSchema(dorisOptions, dorisOptions.getDatabase(), tableName, logger)
+                    .getKeysType());
+        } catch (Exception e) {
+            logger.error("Failed to match table unique key types", e);
+            throw new DorisException(e);
+        }
+    }
+
+    /**
+     * send request to Doris FE and get response json string.
+     *
+     * @param options configuration of request
+     * @param request {@link HttpRequestBase} real request
+     * @param logger  {@link Logger}
+     * @return Doris FE response in json string
+     * @throws ConnectedFailedException throw when cannot connect to Doris FE
+     */
+    private static String send(DorisOptions options, HttpRequestBase request, Logger logger)
+        throws ConnectedFailedException {
+        int connectTimeout = options.getRequestConnectTimeoutMs();
+        int socketTimeout = options.getRequestReadTimeoutMs();
+        logger.trace(
+            "connect timeout set to '{}'. socket timeout set to '{}'.",
+            connectTimeout,
+            socketTimeout);
+
+        RequestConfig requestConfig =
+            RequestConfig.custom()
+                .setConnectTimeout(connectTimeout)
+                .setSocketTimeout(socketTimeout)
+                .build();
+
+        request.setConfig(requestConfig);
+        logger.info(
+            "Send request to Doris FE '{}' with user '{}'.",
+            request.getURI(),
+            options.getUser());
+        int statusCode = -1;
+        AtomicReference<String> result = new AtomicReference<>();
+        try {
+            BackoffAndRetryUtils.backoffAndRetry(
+                LoadOperation.SEND_REQUEST_TO_DORIS,
+                () -> {
+                    logger.debug("doris request {}.", request.getURI());
+                    try {
+                        String response = null;
+                        if (request instanceof HttpGet) {
+                            response =
+                                getConnectionGet(
+                                    request,
+                                    options.getUser(),
+                                    options.getPassword(),
+                                    logger);
+                        } else {
+                            response =
+                                getConnectionPost(
+                                    request,
+                                    options.getUser(),
+                                    options.getPassword(),
+                                    logger);
+                        }
+                        if (Objects.isNull(response)) {
+                            logger.warn(
+                                "Failed to get response from Doris FE {}, http code is {}",
+                                request.getURI(),
+                                statusCode);
+                            throw new ConnectedFailedException(
+                                "Failed to get response from Doris FE {"
+                                    + request.getURI()
+                                    + "}, http code is {"
+                                    + statusCode
+                                    + "}");
+                        }
+                        logger.trace(
+                            "Success get response from Doris FE: {}, response is: {}.",
+                            request.getURI(),
+                            response);
+                        // Handle the problem of inconsistent data format returned by http v1
+                        // and v2
+                        Map map = OBJECT_MAPPER.readValue(response, Map.class);
+                        if (map.containsKey("code") && map.containsKey("msg")) {
+                            Object data = map.get("data");
+                            result.set(OBJECT_MAPPER.writeValueAsString(data));
+                        } else {
+                            result.set(response);
+                        }
+                        return true;
+                    } catch (IOException e) {
+                        logger.warn(
+                            "Failed to connect doris, requestUri={}", request.getURI(), e);
+                        throw new ConnectedFailedException(
+                            "Failed to connect doris, requestUri=" + request.getURI(), e);
+                    }
+                });
+        } catch (Exception e) {
+            logger.error("Connect to doris {} failed.", request.getURI(), e);
+            throw new ConnectedFailedException(
+                "Failed to connect doris request uri=" + request.getURI(), statusCode, e);
+        }
+        return result.get();
+    }
+
+    private static String getConnectionGet(
+        HttpRequestBase request, String user, String passwd, Logger logger) throws IOException {
+        URL realUrl = new URL(request.getURI().toString());
+        // open connection
+        HttpURLConnection connection = (HttpURLConnection) realUrl.openConnection();
+        String authEncoding =
+            Base64.getEncoder()
+                .encodeToString(
+                    String.format("%s:%s", user, passwd)
+                        .getBytes(StandardCharsets.UTF_8));
+        connection.setRequestProperty("Authorization", "Basic " + authEncoding);
+
+        connection.connect();
+        connection.setConnectTimeout(request.getConfig().getConnectTimeout());
+        connection.setReadTimeout(request.getConfig().getSocketTimeout());
+        return parseResponse(connection, logger);
+    }
+
+    private static String getConnectionPost(
+        HttpRequestBase request, String user, String passwd, Logger logger) throws IOException {
+        URL url = new URL(request.getURI().toString());
+        HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+        conn.setInstanceFollowRedirects(false);
+        conn.setRequestMethod(request.getMethod());
+        String authEncoding =
+            Base64.getEncoder()
+                .encodeToString(
+                    String.format("%s:%s", user, passwd)
+                        .getBytes(StandardCharsets.UTF_8));
+        conn.setRequestProperty("Authorization", "Basic " + authEncoding);
+        InputStream content = ((HttpPost) request).getEntity().getContent();
+        String res = IOUtils.toString(content);
+        conn.setDoOutput(true);
+        conn.setDoInput(true);
+        conn.setConnectTimeout(request.getConfig().getConnectTimeout());
+        conn.setReadTimeout(request.getConfig().getSocketTimeout());
+        PrintWriter out = new PrintWriter(conn.getOutputStream());
+        // send request params
+        out.print(res);
+        // flush
+        out.flush();
+        // read response
+        return parseResponse(conn, logger);
+    }
+
+    private static String parseResponse(HttpURLConnection connection, Logger logger)
+        throws IOException {
+        if (connection.getResponseCode() != HttpStatus.SC_OK) {
+            logger.warn(
+                "Failed to get response from Doris  {}, http code is {}",
+                connection.getURL(),
+                connection.getResponseCode());
+            throw new IOException("Failed to get response from Doris");
+        }
+        StringBuilder result = new StringBuilder();
+        try (Scanner scanner = new Scanner(connection.getInputStream(), "utf-8")) {
+            while (scanner.hasNext()) {
+                result.append(scanner.next());
+            }
+            return result.toString();
+        }
+    }
+
+    private static List<BackendV2.BackendRowV2> parseBackendV2(String response, Logger logger) {
+        ObjectMapper mapper = new ObjectMapper();
+        BackendV2 backend;
+        try {
+            backend = mapper.readValue(response, BackendV2.class);
+        } catch (JsonParseException e) {
+            String errMsg = "Doris BE's response is not a json. res: " + response;
+            logger.error(errMsg, e);
+            throw new DorisException(errMsg, e);
+        } catch (JsonMappingException e) {
+            String errMsg = "Doris BE's response cannot map to schema. res: " + response;
+            logger.error(errMsg, e);
+            throw new DorisException(errMsg, e);
+        } catch (IOException e) {
+            String errMsg = "Parse Doris BE's response to json failed. res: " + response;
+            logger.error(errMsg, e);
+            throw new DorisException(errMsg, e);
+        }
+
+        if (backend == null) {
+            logger.error("Should not come here.");
+            throw new DorisException();
+        }
+        List<BackendV2.BackendRowV2> backendRows = backend.getBackends();
+        logger.debug("Parsing schema result is '{}'.", backendRows);
+        return backendRows;
+    }
+
+    /**
+     * Get table schema from doris.
+     */
+    public static Schema getSchema(
+        DorisOptions dorisOptions, String db, String table, Logger logger) {
+        logger.trace("start get " + db + "." + table + " schema from doris.");
+        Object responseData = null;
+        try {
+            String tableSchemaUri =
+                String.format(TABLE_SCHEMA_API, dorisOptions.getHttpUrl(), db, table);
+            HttpGet httpGet = new HttpGet(tableSchemaUri);
+            httpGet.setHeader(HttpHeaders.AUTHORIZATION, authHeader(dorisOptions));
+            Map<String, Object> responseMap = handleResponse(httpGet, logger);
+            responseData = responseMap.get("data");
+            String schemaStr = OBJECT_MAPPER.writeValueAsString(responseData);
+            return OBJECT_MAPPER.readValue(schemaStr, Schema.class);
+        } catch (JsonProcessingException | IllegalArgumentException e) {
+            throw new SchemaChangeException("can not parse response schema " + responseData, e);
+        }
+    }
+
+    private static String authHeader(DorisOptions dorisOptions) {
+        return "Basic "
+            + new String(
+            org.apache.commons.codec.binary.Base64.encodeBase64(
+                (dorisOptions.getUser() + ":" + dorisOptions.getPassword())
+                    .getBytes(StandardCharsets.UTF_8)));
+    }
+
+    private static Map handleResponse(HttpUriRequest request, Logger logger) {
+        try (CloseableHttpClient httpclient = HttpClients.createDefault()) {
+            CloseableHttpResponse response = httpclient.execute(request);
+            final int statusCode = response.getStatusLine().getStatusCode();
+            final String reasonPhrase = response.getStatusLine().getReasonPhrase();
+            if (statusCode == 200 && response.getEntity() != null) {
+                String responseEntity = EntityUtils.toString(response.getEntity());
+                return OBJECT_MAPPER.readValue(responseEntity, Map.class);
+            } else {
+                throw new SchemaChangeException(
+                    "Failed to schemaChange, status: "
+                        + statusCode
+                        + ", reason: "
+                        + reasonPhrase);
+            }
+        } catch (Exception e) {
+            logger.trace("SchemaChange request error,", e);
+            throw new SchemaChangeException("SchemaChange request error with " + e.getMessage());
+        }
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/BufferedRecords.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/BufferedRecords.java
deleted file mode 100644
index 3058a89..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/BufferedRecords.java
+++ /dev/null
@@ -1,234 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.sink;
-
-import io.openmessaging.connector.api.data.ConnectRecord;
-import io.openmessaging.connector.api.data.Schema;
-import io.openmessaging.connector.api.errors.ConnectException;
-import org.apache.rocketmq.connect.doris.exception.DorisException;
-import org.apache.rocketmq.connect.doris.schema.table.TableId;
-import org.apache.rocketmq.connect.doris.connector.DorisSinkConfig;
-import org.apache.rocketmq.connect.doris.schema.db.DbStructure;
-import org.apache.rocketmq.connect.doris.sink.metadata.FieldsMetadata;
-import org.apache.rocketmq.connect.doris.sink.metadata.SchemaPair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-import java.util.Optional;
-import static java.util.Objects.isNull;
-import static java.util.Objects.nonNull;
-
-/**
- * buffered records
- */
-public class BufferedRecords {
-    private static final Logger log = LoggerFactory.getLogger(BufferedRecords.class);
-
-    private final TableId tableId;
-    private final DorisSinkConfig config;
-    private final DbStructure dbStructure;
-    private List<ConnectRecord> records = new ArrayList<>();
-    private Schema keySchema;
-    private Schema schema;
-    private FieldsMetadata fieldsMetadata;
-    private RecordValidator recordValidator;
-    private List<ConnectRecord> updatePreparedRecords = new ArrayList<>();
-    private List<ConnectRecord> deletePreparedRecords = new ArrayList<>();
-    private boolean deletesInBatch = false;
-    private DorisStreamLoader loader;
-
-    public BufferedRecords(
-            DorisSinkConfig config,
-            TableId tableId,
-            DbStructure dbStructure
-    ) {
-        this.tableId = tableId;
-        this.config = config;
-        this.dbStructure = dbStructure;
-        this.recordValidator = RecordValidator.create(config);
-        this.loader = DorisStreamLoader.create(config);
-    }
-
-    /**
-     * add record
-     *
-     * @param record
-     * @return
-     * @throws SQLException
-     */
-    public List<ConnectRecord> add(ConnectRecord record) throws SQLException {
-        recordValidator.validate(record);
-        final List<ConnectRecord> flushed = new ArrayList<>();
-        boolean schemaChanged = false;
-        if (!Objects.equals(keySchema, record.getKeySchema())) {
-            keySchema = record.getKeySchema();
-            schemaChanged = true;
-        }
-        if (isNull(record.getSchema())) {
-            // For deletes, value and optionally value schema come in as null.
-            // We don't want to treat this as a schema change if key schemas is the same
-            // otherwise we flush unnecessarily.
-            if (config.isDeleteEnabled()) {
-                deletesInBatch = true;
-            }
-        } else if (Objects.equals(schema, record.getSchema())) {
-            if (config.isDeleteEnabled() && deletesInBatch) {
-                // flush so an insert after a delete of same record isn't lost
-                flushed.addAll(flush());
-            }
-        } else {
-            // value schema is not null and has changed. This is a real schema change.
-            schema = record.getSchema();
-            schemaChanged = true;
-        }
-
-        if (schemaChanged) {
-            // Each batch needs to have the same schemas, so get the buffered records out
-            flushed.addAll(flush());
-            // re-initialize everything that depends on the record schema
-            final SchemaPair schemaPair = new SchemaPair(
-                    record.getKeySchema(),
-                    record.getSchema(),
-                    record.getExtensions()
-            );
-            // extract field
-            fieldsMetadata = FieldsMetadata.extract(
-                    tableId.tableName(),
-                    config.pkMode,
-                    config.getPkFields(),
-                    config.getFieldsWhitelist(),
-                    schemaPair
-            );
-        }
-
-        // set deletesInBatch if schema value is not null
-        if (isNull(record.getData()) && config.isDeleteEnabled()) {
-            deletesInBatch = true;
-        }
-
-        records.add(record);
-        if (records.size() >= config.getBatchSize()) {
-            flushed.addAll(flush());
-        }
-        return flushed;
-    }
-
-    public List<ConnectRecord> flush() throws SQLException {
-        if (records.isEmpty()) {
-            log.debug("Records is empty");
-            return new ArrayList<>();
-        }
-        log.debug("Flushing {} buffered records", records.size());
-        for (ConnectRecord record : records) {
-            if (isNull(record.getData())) {
-                deletePreparedRecords.add(record);
-            } else {
-                updatePreparedRecords.add(record);
-            }
-        }
-        Optional<Long> totalUpdateCount = executeUpdates();
-        Optional<Long> totalDeleteCount = executeDeletes();
-        final long expectedCount = updateRecordCount();
-        log.trace("{} records:{} resulting in totalUpdateCount:{} totalDeleteCount:{}",
-                config.getInsertMode(),
-                records.size(),
-                totalUpdateCount,
-                totalDeleteCount
-        );
-        if (totalUpdateCount.filter(total -> total != expectedCount).isPresent()
-                && config.getInsertMode() == DorisSinkConfig.InsertMode.INSERT) {
-            throw new ConnectException(
-                    String.format(
-                            "Update count (%d) did not sum up to total number of records inserted (%d)",
-                            totalUpdateCount.get(),
-                            expectedCount
-                    )
-            );
-        }
-        if (!totalUpdateCount.isPresent()) {
-            log.info(
-                    "{} records:{} , but no count of the number of rows it affected is available",
-                    config.getInsertMode(),
-                    records.size()
-            );
-        }
-
-        final List<ConnectRecord> flushedRecords = records;
-        records = new ArrayList<>();
-        return flushedRecords;
-    }
-
-    /**
-     * @return an optional count of all updated rows or an empty optional if no info is available
-     */
-    private Optional<Long> executeUpdates() throws DorisException {
-        Optional<Long> count = Optional.empty();
-        if (updatePreparedRecords.isEmpty()) {
-            return count;
-        }
-        for (ConnectRecord record : updatePreparedRecords) {
-            String jsonData = DorisDialect.convertToUpdateJsonString(record);
-            try {
-                log.info("[executeUpdates]" + jsonData);
-                loader.loadJson(jsonData, record.getSchema().getName());
-            } catch (DorisException e) {
-                log.error("executeUpdates failed");
-                throw e;
-            } catch (Exception e) {
-                throw new DorisException("doris error");
-            }
-            count = count.isPresent()
-                    ? count.map(total -> total + 1)
-                    : Optional.of(1L);
-        }
-        return count;
-    }
-
-    private Optional<Long> executeDeletes() throws SQLException {
-        Optional<Long> totalDeleteCount = Optional.empty();
-        if (deletePreparedRecords.isEmpty()) {
-            return totalDeleteCount;
-        }
-        for (ConnectRecord record : updatePreparedRecords) {
-            String jsonData = DorisDialect.convertToDeleteJsonString(record);
-            try {
-                log.info("[executeDelete]" + jsonData);
-                loader.loadJson(jsonData, record.getSchema().getName());
-            } catch (DorisException e) {
-                log.error("executeDelete failed");
-                throw e;
-            } catch (Exception e) {
-                throw new DorisException("doris error");
-            }
-            totalDeleteCount = totalDeleteCount.isPresent()
-                    ? totalDeleteCount.map(total -> total + 1)
-                    : Optional.of(1L);
-        }
-        return totalDeleteCount;
-    }
-
-    private long updateRecordCount() {
-        return records
-                .stream()
-                // ignore deletes
-                .filter(record -> nonNull(record.getData()) || !config.isDeleteEnabled())
-                .count();
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/DorisDialect.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/DorisDialect.java
deleted file mode 100644
index 6d17bab..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/DorisDialect.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.sink;
-
-import io.openmessaging.connector.api.data.ConnectRecord;
-import io.openmessaging.connector.api.data.Field;
-import io.openmessaging.connector.api.data.Struct;
-import org.apache.rocketmq.connect.doris.exception.TableAlterOrCreateException;
-import com.alibaba.fastjson.JSON;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-
-public class DorisDialect {
-    public static String convertToUpdateJsonString(ConnectRecord record, boolean isFirst) {
-        try {
-            Struct struct = (Struct) record.getData();
-            Map<String, String> keyValue = new HashMap<>();
-            for (Field field: struct.getSchema().getFields()) {
-                bindValue(keyValue, field, struct.getValues()[field.getIndex()]);
-            }
-            return isFirst ? "" : "," + JSON.toJSON(keyValue).toString();
-        } catch (TableAlterOrCreateException tace) {
-            throw tace;
-        }
-    }
-
-    public static String convertToUpdateJsonString(ConnectRecord record) {
-        try {
-            Struct struct = (Struct) record.getData();
-            Map<String, String> keyValue = new HashMap<>();
-            for (Field field: struct.getSchema().getFields()) {
-                bindValue(keyValue, field, struct.getValues()[field.getIndex()]);
-            }
-            return JSON.toJSON(keyValue).toString();
-        } catch (TableAlterOrCreateException tace) {
-            throw tace;
-        }
-    }
-
-    public static String convertToDeleteJsonString(ConnectRecord record) {
-        try {
-            // it seems that doris doesn't support delete via stream load
-            return "";
-        } catch (TableAlterOrCreateException tace) {
-            throw tace;
-        }
-    }
-    
-    private static void bindValue(Map<String, String> keyValue, Field field, Object value) {
-        switch (field.getSchema().getFieldType()) {
-            case INT8:
-            case BOOLEAN:
-            case FLOAT64:
-            case INT32:
-            case INT64:
-            case FLOAT32:
-                if (value == null) {
-                    keyValue.put(field.getName(), "null");
-                } else {
-                    keyValue.put(field.getName(), value.toString());
-                }
-                break;
-            case STRING:
-                if (value == null) {
-                    keyValue.put(field.getName(), "null");
-                } else {
-                    keyValue.put(field.getName(), (String) value);
-                }
-                break;
-            case BYTES:
-                if (value == null) {
-                    keyValue.put(field.getName(), "null");
-                } else {
-                    final byte[] bytes;
-                    if (value instanceof ByteBuffer) {
-                        final ByteBuffer buffer = ((ByteBuffer) value).slice();
-                        bytes = new byte[buffer.remaining()];
-                        buffer.get(bytes);
-                    } else if (value instanceof BigDecimal) {
-                        keyValue.put(field.getName(), value.toString());
-                        break;
-                    } else {
-                        bytes = (byte[]) value;
-                    }
-                    keyValue.put(field.getName(), Arrays.toString(bytes));
-                }
-                break;
-            case DATETIME:
-                if (value == null) {
-                    keyValue.put(field.getName(), "null");
-                } else {
-                    java.sql.Date date;
-                    if (value instanceof java.util.Date) {
-                        date = new java.sql.Date(((java.util.Date) value).getTime());
-                    } else {
-                        date = new java.sql.Date((int) value);
-                    }
-                    keyValue.put(
-                            field.getName(), date.toString()
-                    );
-                }
-                break;
-            default:
-                throw new TableAlterOrCreateException("Field type not found " + field);
-        }
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/DorisStreamLoader.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/DorisStreamLoader.java
deleted file mode 100644
index 26eb426..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/DorisStreamLoader.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.sink;
-
-import org.apache.commons.codec.binary.Base64;
-import org.apache.http.HttpHeaders;
-import org.apache.http.client.methods.CloseableHttpResponse;
-import org.apache.http.client.methods.HttpPut;
-import org.apache.http.entity.StringEntity;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.impl.client.DefaultRedirectStrategy;
-import org.apache.http.impl.client.HttpClientBuilder;
-import org.apache.http.impl.client.HttpClients;
-import org.apache.http.util.EntityUtils;
-import org.apache.rocketmq.connect.doris.connector.DorisSinkConfig;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.util.UUID;
-
-
-/**
- * This example mainly demonstrates how to use stream load to import data
- * Including file type (CSV) and data in JSON format
- */
-public class DorisStreamLoader {
-    private static final Logger log = LoggerFactory.getLogger(DorisStreamLoader.class);
-    // FE IP Address
-    private String host;
-    // FE port
-    private int port;
-    // db name
-    private String database;
-    // table name
-    private String table;
-    //user name
-    private String user;
-    //user password
-    private String passwd;
-    //http path of stream load task submission
-    private final String loadUrlWithoutTable;
-
-    private DorisStreamLoader(String host, int port, String database, String user, String passwd) {
-        this.host = host;
-        this.port = port;
-        this.database = database;
-        this.user = user;
-        this.passwd = passwd;
-        this.loadUrlWithoutTable = String.format("http://%s:%s/api/%s", host, port, database);
-    }
-
-    public static DorisStreamLoader create(DorisSinkConfig config) {
-        return new DorisStreamLoader(config.getHost(), config.getPort(), config.getDatabase(), config.getUser(), config.getPasswd());
-    }
-
-    //Build http client builder
-    private final HttpClientBuilder httpClientBuilder = HttpClients.custom().setRedirectStrategy(new DefaultRedirectStrategy() {
-        @Override
-        protected boolean isRedirectable(String method) {
-            // If the connection target is FE, you need to deal with 307 redirect
-            return true;
-        }
-    });
-
-    private String getLoadURL(String table) {
-        return String.format("%s/%s/_stream_load", loadUrlWithoutTable, table);
-    }
-
-
-    /**
-     * JSON import
-     *
-     * @param jsonData
-     * @throws Exception
-     */
-    public void loadJson(String jsonData, String table) throws Exception {
-        try (CloseableHttpClient client = httpClientBuilder.build()) {
-            HttpPut put = new HttpPut(getLoadURL(table));
-            put.removeHeaders(HttpHeaders.CONTENT_LENGTH);
-            put.removeHeaders(HttpHeaders.TRANSFER_ENCODING);
-            put.setHeader(HttpHeaders.EXPECT, "100-continue");
-            put.setHeader(HttpHeaders.AUTHORIZATION, basicAuthHeader(user, passwd));
-
-            // You can set stream load related properties in the Header, here we set label and column_separator.
-            put.setHeader("label", UUID.randomUUID().toString());
-            put.setHeader("column_separator", ",");
-            put.setHeader("format", "json");
-
-            // Set up the import file. Here you can also use StringEntity to transfer arbitrary data.
-            StringEntity entity = new StringEntity(jsonData);
-            put.setEntity(entity);
-            log.info(put.toString());
-            try (CloseableHttpResponse response = client.execute(put)) {
-                String loadResult = "";
-                if (response.getEntity() != null) {
-                    loadResult = EntityUtils.toString(response.getEntity());
-                }
-
-                final int statusCode = response.getStatusLine().getStatusCode();
-                if (statusCode != 200) {
-                    throw new IOException(String.format("Stream load failed. status: %s load result: %s", statusCode, loadResult));
-                }
-                log.info("Get load result: " + loadResult);
-            }
-        }
-    }
-
-    /**
-     * Construct authentication information, the authentication method used by doris here is Basic Auth
-     *
-     * @param username
-     * @param password
-     * @return
-     */
-    private String basicAuthHeader(String username, String password) {
-        final String tobeEncode = username + ":" + password;
-        byte[] encoded = Base64.encodeBase64(tobeEncode.getBytes(StandardCharsets.UTF_8));
-        return "Basic " + new String(encoded);
-    }
-
-
-    public static void main(String[] args) throws Exception {
-        DorisStreamLoader loader = new DorisStreamLoader("47.97.179.18", 7030, "db_1", "root", "rocketmq666");
-        //json load
-        String jsonData = "{\"id\":556393582,\"number\":\"123344\",\"price\":\"23.5\",\"skuname\":\"test\",\"skudesc\":\"zhangfeng_test,test\"}";
-        loader.loadJson(jsonData, "doris_test_sink");
-
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/RecordValidator.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/RecordValidator.java
deleted file mode 100644
index 8cf6988..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/RecordValidator.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.sink;
-
-import io.openmessaging.connector.api.data.ConnectRecord;
-import io.openmessaging.connector.api.data.FieldType;
-import io.openmessaging.connector.api.data.Schema;
-import io.openmessaging.connector.api.errors.ConnectException;
-import org.apache.rocketmq.connect.doris.connector.DorisSinkConfig;
-
-@FunctionalInterface
-public interface RecordValidator {
-
-    RecordValidator NO_OP = (record) -> {
-    };
-
-    void validate(ConnectRecord record);
-
-    default RecordValidator and(RecordValidator other) {
-        if (other == null || other == NO_OP || other == this) {
-            return this;
-        }
-        if (this == NO_OP) {
-            return other;
-        }
-        RecordValidator thisValidator = this;
-        return (record) -> {
-            thisValidator.validate(record);
-            other.validate(record);
-        };
-    }
-
-    static RecordValidator create(DorisSinkConfig config) {
-        RecordValidator requiresKey = requiresKey(config);
-        RecordValidator requiresValue = requiresValue(config);
-
-        RecordValidator keyValidator = NO_OP;
-        RecordValidator valueValidator = NO_OP;
-        switch (config.pkMode) {
-            case RECORD_KEY:
-                keyValidator = keyValidator.and(requiresKey);
-                break;
-            case RECORD_VALUE:
-            case NONE:
-                valueValidator = valueValidator.and(requiresValue);
-                break;
-            default:
-                // no primary key is required
-                break;
-        }
-
-        if (config.isDeleteEnabled()) {
-            // When delete is enabled, we need a key
-            keyValidator = keyValidator.and(requiresKey);
-        } else {
-            // When delete is disabled, we need non-tombstone values
-            valueValidator = valueValidator.and(requiresValue);
-        }
-
-        // Compose the validator that may or may be NO_OP
-        return keyValidator.and(valueValidator);
-    }
-
-    static RecordValidator requiresValue(DorisSinkConfig config) {
-        return record -> {
-            Schema valueSchema = record.getSchema();
-            if (record.getData() != null
-                    && valueSchema != null
-                    && valueSchema.getFieldType() == FieldType.STRUCT) {
-                return;
-            }
-            throw new ConnectException(record.toString());
-        };
-    }
-
-    static RecordValidator requiresKey(DorisSinkConfig config) {
-        return record -> {
-            Schema keySchema = record.getKeySchema();
-            if (record.getKey() != null
-                    && keySchema != null
-                    && (keySchema.getFieldType() == FieldType.STRUCT || keySchema.getFieldType().isPrimitive())) {
-                return;
-            }
-            throw new ConnectException(record.toString());
-        };
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/Updater.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/Updater.java
deleted file mode 100644
index 7c3515b..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/Updater.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.sink;
-
-import io.openmessaging.connector.api.data.ConnectRecord;
-import org.apache.rocketmq.connect.doris.connector.DorisSinkConfig;
-import org.apache.rocketmq.connect.doris.exception.TableAlterOrCreateException;
-import org.apache.rocketmq.connect.doris.schema.db.DbStructure;
-import org.apache.rocketmq.connect.doris.schema.table.TableId;
-import org.apache.rocketmq.connect.doris.util.TableUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import java.sql.SQLException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * jdbc db updater
- */
-public class Updater {
-
-    private static final Logger log = LoggerFactory.getLogger(Updater.class);
-    private final DorisSinkConfig config;
-    private final DbStructure dbStructure;
-
-    public Updater(final DorisSinkConfig config) {
-        this.config = config;
-        this.dbStructure = null;
-    }
-
-    public void write(final Collection<ConnectRecord> records)
-            throws SQLException, TableAlterOrCreateException {
-        try {
-            final Map<TableId, BufferedRecords> bufferByTable = new HashMap<>();
-            for (ConnectRecord record : records) {
-                // destination table
-                final TableId tableId = TableUtil.destinationTable(record);
-                if (!config.filterWhiteTable(tableId)) {
-                    continue;
-                }
-                BufferedRecords buffer = bufferByTable.get(tableId);
-                if (buffer == null) {
-                    buffer = new BufferedRecords(config, tableId, dbStructure);
-                    bufferByTable.put(tableId, buffer);
-                }
-                buffer.add(record);
-            }
-            for (Map.Entry<TableId, BufferedRecords> entry : bufferByTable.entrySet()) {
-                TableId tableId = entry.getKey();
-                BufferedRecords buffer = entry.getValue();
-                log.debug("Flushing records in JDBC Writer for table ID: {}", tableId);
-                buffer.flush();
-            }
-        } catch (SQLException | TableAlterOrCreateException e) {
-            log.error(e.toString());
-        }
-    }
-}
-
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/metadata/FieldsMetadata.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/metadata/FieldsMetadata.java
deleted file mode 100644
index e82eba9..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/metadata/FieldsMetadata.java
+++ /dev/null
@@ -1,304 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.sink.metadata;
-
-import io.openmessaging.KeyValue;
-import io.openmessaging.connector.api.data.Field;
-import io.openmessaging.connector.api.data.FieldType;
-import io.openmessaging.connector.api.data.Schema;
-import io.openmessaging.connector.api.errors.ConnectException;
-import org.apache.rocketmq.connect.doris.connector.DorisSinkConfig;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-
-/**
- * fields metadata
- */
-public class FieldsMetadata {
-
-    public final Set<String> keyFieldNames;
-    public final Set<String> nonKeyFieldNames;
-    public final Map<String, SinkRecordField> allFields;
-
-    // visible for testing
-    public FieldsMetadata(
-            Set<String> keyFieldNames,
-            Set<String> nonKeyFieldNames,
-            Map<String, SinkRecordField> allFields
-    ) {
-        boolean fieldCountsMatch = (keyFieldNames.size() + nonKeyFieldNames.size()) == allFields.size();
-        boolean allFieldsContained = allFields.keySet().containsAll(keyFieldNames)
-                && allFields.keySet().containsAll(nonKeyFieldNames);
-        if (!fieldCountsMatch || !allFieldsContained) {
-            throw new IllegalArgumentException(String.format(
-                    "Validation fail -- keyFieldNames:%s nonKeyFieldNames:%s allFields:%s",
-                    keyFieldNames, nonKeyFieldNames, allFields
-            ));
-        }
-        this.keyFieldNames = keyFieldNames;
-        this.nonKeyFieldNames = nonKeyFieldNames;
-        this.allFields = allFields;
-    }
-
-    public static FieldsMetadata extract(
-            final String tableName,
-            final DorisSinkConfig.PrimaryKeyMode pkMode,
-            final List<String> configuredPkFields,
-            final Set<String> fieldsWhitelist,
-            final SchemaPair schemaPair
-    ) {
-        return extract(
-                tableName,
-                pkMode,
-                configuredPkFields,
-                fieldsWhitelist,
-                schemaPair.keySchema,
-                schemaPair.schema,
-                schemaPair.extensions
-        );
-    }
-
-    /**
-     * extract metadata info
-     *
-     * @param tableName
-     * @param pkMode
-     * @param configuredPkFields
-     * @param fieldsWhitelist
-     * @param schema
-     * @param headers
-     * @return
-     */
-    public static FieldsMetadata extract(
-            final String tableName,
-            final DorisSinkConfig.PrimaryKeyMode pkMode,
-            final List<String> configuredPkFields,
-            final Set<String> fieldsWhitelist,
-            final Schema keySchema,
-            final Schema schema,
-            final KeyValue headers
-    ) {
-        if (schema != null && schema.getFieldType() != FieldType.STRUCT) {
-            throw new ConnectException("Value schema must be of type Struct");
-        }
-        final Map<String, SinkRecordField> allFields = new HashMap<>();
-        final Set<String> keyFieldNames = new LinkedHashSet<>();
-        switch (pkMode) {
-            case NONE:
-                break;
-            case RECORD_KEY:
-                extractRecordKeyPk(tableName, configuredPkFields, keySchema, allFields, keyFieldNames);
-                break;
-            case RECORD_VALUE:
-                extractRecordValuePk(tableName, configuredPkFields, schema, headers, allFields, keyFieldNames);
-                break;
-            default:
-                throw new ConnectException("Unknown primary key mode: " + pkMode);
-        }
-        final Set<String> nonKeyFieldNames = new LinkedHashSet<>();
-        if (schema != null) {
-            for (Field field : schema.getFields()) {
-                if (keyFieldNames.contains(field.getName())) {
-                    continue;
-                }
-                if (!fieldsWhitelist.isEmpty() && !fieldsWhitelist.contains(field.getName())) {
-                    continue;
-                }
-                nonKeyFieldNames.add(field.getName());
-                final Schema fieldSchema = field.getSchema();
-                allFields.put(field.getName(), new SinkRecordField(fieldSchema, field.getName(), false));
-            }
-        }
-
-        if (allFields.isEmpty()) {
-            throw new ConnectException(
-                    "No fields found using key and value schemas for table: " + tableName
-            );
-        }
-
-        final Map<String, SinkRecordField> allFieldsOrdered = new LinkedHashMap<>();
-
-        if (schema != null) {
-            for (Field field : schema.getFields()) {
-                String fieldName = field.getName();
-                if (allFields.containsKey(fieldName)) {
-                    allFieldsOrdered.put(fieldName, allFields.get(fieldName));
-                }
-            }
-        }
-
-        if (allFieldsOrdered.size() < allFields.size()) {
-            ArrayList<String> fieldKeys = new ArrayList<>(allFields.keySet());
-            Collections.sort(fieldKeys);
-            for (String fieldName : fieldKeys) {
-                if (!allFieldsOrdered.containsKey(fieldName)) {
-                    allFieldsOrdered.put(fieldName, allFields.get(fieldName));
-                }
-            }
-        }
-
-        return new FieldsMetadata(keyFieldNames, nonKeyFieldNames, allFieldsOrdered);
-    }
-
-    private static void extractRecordKeyPk(
-            final String tableName,
-            final List<String> configuredPkFields,
-            final Schema keySchema,
-            final Map<String, SinkRecordField> allFields,
-            final Set<String> keyFieldNames
-    ) {
-        if (keySchema == null) {
-            throw new ConnectException(String.format(
-                    "PK mode for table '%s' is %s, but record key schema is missing",
-                    tableName,
-                    DorisSinkConfig.PrimaryKeyMode.RECORD_KEY
-            ));
-        }
-        final FieldType keySchemaType = keySchema.getFieldType();
-        switch (keySchemaType) {
-            case STRUCT:
-                if (configuredPkFields.isEmpty()) {
-                    keySchema.getFields().forEach(keyField -> {
-                        keyFieldNames.add(keyField.getName());
-                    });
-                } else {
-                    for (String fieldName : configuredPkFields) {
-                        final Field keyField = keySchema.getField(fieldName);
-                        if (keyField == null) {
-                            throw new ConnectException(String.format(
-                                    "PK mode for table '%s' is %s with configured PK fields %s, but record key "
-                                            + "schema does not contain field: %s",
-                                    tableName, DorisSinkConfig.PrimaryKeyMode.RECORD_KEY, configuredPkFields, fieldName
-                            ));
-                        }
-                    }
-                    keyFieldNames.addAll(configuredPkFields);
-                }
-                for (String fieldName : keyFieldNames) {
-                    final Schema fieldSchema = keySchema.getField(fieldName).getSchema();
-                    allFields.put(fieldName, new SinkRecordField(fieldSchema, fieldName, true));
-                }
-                break;
-            default:
-                // todo
-                if (keySchemaType.isPrimitive()) {
-                    if (configuredPkFields.size() != 1) {
-                        throw new ConnectException(String.format(
-                                "Need exactly one PK column defined since the key schema for records is a "
-                                        + "primitive type, defined columns are: %s",
-                                configuredPkFields
-                        ));
-                    }
-                    final String fieldName = configuredPkFields.get(0);
-                    keyFieldNames.add(fieldName);
-                    allFields.put(fieldName, new SinkRecordField(keySchema, fieldName, true));
-                } else {
-                    throw new ConnectException(
-                            "Key schema must be primitive type or Struct, but is of type: " + keySchemaType
-                    );
-                }
-        }
-    }
-
-    /**
-     * record value
-     *
-     * @param tableName
-     * @param configuredPkFields
-     * @param valueSchema
-     * @param headers
-     * @param allFields
-     * @param keyFieldNames
-     */
-    private static void extractRecordValuePk(
-            final String tableName,
-            final List<String> configuredPkFields,
-            final Schema valueSchema,
-            final KeyValue headers,
-            final Map<String, SinkRecordField> allFields,
-            final Set<String> keyFieldNames
-    ) {
-        if (valueSchema == null) {
-            throw new ConnectException(String.format(
-                    "PK mode for table '%s' is %s, but record value schema is missing",
-                    tableName,
-                    DorisSinkConfig.PrimaryKeyMode.RECORD_VALUE)
-            );
-        }
-        List<String> pkFields = new ArrayList<>(configuredPkFields);
-        if (pkFields.isEmpty()) {
-            for (Field keyField : valueSchema.getFields()) {
-                keyFieldNames.add(keyField.getName());
-            }
-        } else {
-            for (Field keyField : valueSchema.getFields()) {
-                keyFieldNames.add(keyField.getName());
-            }
-            for (String fieldName : pkFields) {
-                if (!keyFieldNames.contains(fieldName)) {
-                    throw new ConnectException(String.format(
-                            "PK mode for table '%s' is %s with configured PK fields %s, but record value "
-                                    + "schema does not contain field: %s",
-                            tableName,
-                            DorisSinkConfig.PrimaryKeyMode.RECORD_VALUE,
-                            pkFields,
-                            fieldName
-                    ));
-                }
-            }
-            keyFieldNames.addAll(pkFields);
-        }
-        for (String fieldName : keyFieldNames) {
-            final Schema fieldSchema = valueSchema.getField(fieldName).getSchema();
-            allFields.put(fieldName, new SinkRecordField(fieldSchema, fieldName, true));
-        }
-
-    }
-
-    public static boolean isPrimitive(FieldType type) {
-        switch (type) {
-            case INT8:
-            case INT32:
-            case INT64:
-            case FLOAT32:
-            case FLOAT64:
-            case BOOLEAN:
-            case STRING:
-            case BYTES:
-                return true;
-            default:
-                return false;
-        }
-    }
-
-    @Override
-    public String toString() {
-        return "FieldsMetadata{"
-                + "keyFieldNames=" + keyFieldNames
-                + ", nonKeyFieldNames=" + nonKeyFieldNames
-                + ", allFields=" + allFields
-                + '}';
-    }
-
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/metadata/SchemaPair.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/metadata/SchemaPair.java
deleted file mode 100644
index 3b5a5f8..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/metadata/SchemaPair.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.sink.metadata;
-
-import io.openmessaging.KeyValue;
-import io.openmessaging.connector.api.data.Schema;
-import java.util.Objects;
-
-/**
- * schema pair
- */
-public class SchemaPair {
-    public final Schema keySchema;
-    public final Schema schema;
-    public final KeyValue extensions;
-
-    public SchemaPair(Schema keySchema, Schema schema) {
-        this.keySchema = keySchema;
-        this.schema = schema;
-        this.extensions = null;
-    }
-
-    public SchemaPair(Schema keySchema, Schema schema, KeyValue extensions) {
-        this.keySchema = keySchema;
-        this.schema = schema;
-        this.extensions = extensions;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-        SchemaPair that = (SchemaPair) o;
-        return Objects.equals(schema, that.schema);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(schema);
-    }
-
-    @Override
-    public String toString() {
-        return String.format("<SchemaPair: %s>", schema);
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/metadata/SinkRecordField.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/metadata/SinkRecordField.java
deleted file mode 100644
index de9f082..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/metadata/SinkRecordField.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.sink.metadata;
-
-import io.openmessaging.connector.api.data.FieldType;
-import io.openmessaging.connector.api.data.Schema;
-
-public class SinkRecordField {
-
-    private final Schema schema;
-    private final String name;
-    private final boolean isPrimaryKey;
-
-    public SinkRecordField(Schema schema, String name, boolean isPrimaryKey) {
-        this.schema = schema;
-        this.name = name;
-        this.isPrimaryKey = isPrimaryKey;
-    }
-
-    public Schema schema() {
-        return schema;
-    }
-
-    public String schemaName() {
-        return schema.getName();
-    }
-
-    public FieldType schemaType() {
-        return schema.getFieldType();
-    }
-
-    public String name() {
-        return name;
-    }
-
-    public boolean isOptional() {
-        return !isPrimaryKey;
-    }
-
-    public Object defaultValue() {
-        return null;
-    }
-
-    public boolean isPrimaryKey() {
-        return isPrimaryKey;
-    }
-
-
-    public Object getDefaultValue(FieldType type) {
-        switch (type) {
-            case BOOLEAN:
-            case INT8:
-                return (byte) 0;
-            case INT32:
-                return 0;
-            case INT64:
-                return 0L;
-            case FLOAT32:
-                return 0.0f;
-            case FLOAT64:
-                return 0.0d;
-            default:
-                return null;
-        }
-
-    }
-
-    @Override
-    public String toString() {
-        return "SinkRecordField{"
-                + "schema=" + schema
-                + ", name='" + name + '\''
-                + ", isPrimaryKey=" + isPrimaryKey
-                + '}';
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/BytesUtil.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/BytesUtil.java
deleted file mode 100644
index 8929407..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/BytesUtil.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.util;
-
-public class BytesUtil {
-
-    private static final char[] HEX_CODE = "0123456789ABCDEF".toCharArray();
-    public static String toHex(byte[] data) {
-        StringBuilder r = new StringBuilder(data.length * 2);
-        for (byte b : data) {
-            r.append(HEX_CODE[(b >> 4) & 0xF]);
-            r.append(HEX_CODE[b & 0xF]);
-        }
-        return r.toString();
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/ConnectorGroupUtils.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/ConnectorGroupUtils.java
deleted file mode 100644
index e3d07b6..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/ConnectorGroupUtils.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.util;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * connector group utils
- */
-public class ConnectorGroupUtils {
-    public ConnectorGroupUtils() {
-    }
-
-    public static <T> List<List<T>> groupPartitions(List<T> elements, int numGroups) {
-        if (numGroups <= 0) {
-            throw new IllegalArgumentException("Number of groups must be positive." + numGroups);
-        } else {
-            List<List<T>> result = new ArrayList(numGroups);
-            int perGroup = elements.size() / numGroups;
-            int leftover = elements.size() - numGroups * perGroup;
-            int assigned = 0;
-
-            for (int group = 0; group < numGroups; ++group) {
-                int numThisGroup = group < leftover ? perGroup + 1 : perGroup;
-                List<T> groupList = new ArrayList(numThisGroup);
-
-                for (int i = 0; i < numThisGroup; ++i) {
-                    groupList.add(elements.get(assigned));
-                    ++assigned;
-                }
-
-                result.add(groupList);
-            }
-
-            return result;
-        }
-    }
-}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/DateTimeUtils.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/DateTimeUtils.java
deleted file mode 100644
index 446aab8..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/DateTimeUtils.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.util;
-
-import java.text.SimpleDateFormat;
-import java.util.Calendar;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.TimeZone;
-import java.util.GregorianCalendar;
-import java.util.Date;
-
-public class DateTimeUtils {
-
-    private static final ThreadLocal<Map<TimeZone, Calendar>> TIMEZONE_CALENDARS =
-            ThreadLocal.withInitial(HashMap::new);
-
-    private static final ThreadLocal<Map<TimeZone, SimpleDateFormat>> TIMEZONE_DATE_FORMATS =
-            ThreadLocal.withInitial(HashMap::new);
-
-    private static final ThreadLocal<Map<TimeZone, SimpleDateFormat>> TIMEZONE_TIME_FORMATS =
-            ThreadLocal.withInitial(HashMap::new);
-
-    private static final ThreadLocal<Map<TimeZone, SimpleDateFormat>> TIMEZONE_TIMESTAMP_FORMATS =
-            ThreadLocal.withInitial(HashMap::new);
-
-    public static Calendar getTimeZoneCalendar(final TimeZone timeZone) {
-        return TIMEZONE_CALENDARS.get().computeIfAbsent(timeZone, GregorianCalendar::new);
-    }
-
-    public static String formatDate(Date date, TimeZone timeZone) {
-        return TIMEZONE_DATE_FORMATS.get().computeIfAbsent(timeZone, aTimeZone -> {
-            SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd");
-            sdf.setTimeZone(aTimeZone);
-            return sdf;
-        }).format(date);
-    }
-
-    public static String formatTime(Date date, TimeZone timeZone) {
-        return TIMEZONE_TIME_FORMATS.get().computeIfAbsent(timeZone, aTimeZone -> {
-            SimpleDateFormat sdf = new SimpleDateFormat("HH:mm:ss.SSS");
-            sdf.setTimeZone(aTimeZone);
-            return sdf;
-        }).format(date);
-    }
-
-    public static String formatTimestamp(Date date, TimeZone timeZone) {
-        return TIMEZONE_TIMESTAMP_FORMATS.get().computeIfAbsent(timeZone, aTimeZone -> {
-            SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
-            sdf.setTimeZone(aTimeZone);
-            return sdf;
-        }).format(date);
-    }
-
-    private DateTimeUtils() {
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/ExpressionBuilder.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/ExpressionBuilder.java
deleted file mode 100644
index 232087f..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/ExpressionBuilder.java
+++ /dev/null
@@ -1,595 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.util;
-
-public class ExpressionBuilder {
-
-    /**
-     * A functional interface for anything that can be appended to an expression builder.
-     * This makes use of double-dispatch to allow implementations to customize the behavior,
-     * yet have callers not care about the differences in behavior.
-     */
-    @FunctionalInterface
-    public interface Expressable {
-
-        /**
-         * Append this object to the specified builder.
-         *
-         * @param builder   the builder to use; may not be null
-         * @param useQuotes whether quotes should be used for this object
-         */
-        void appendTo(
-                ExpressionBuilder builder,
-                boolean useQuotes
-        );
-
-        /**
-         * Append this object to the specified builder.
-         *
-         * @param builder   the builder to use; may not be null
-         * @param useQuotes whether quotes should be used for this object
-         */
-        default void appendTo(
-                ExpressionBuilder builder,
-                QuoteMethod useQuotes
-        ) {
-            switch (useQuotes) {
-                case ALWAYS:
-                    appendTo(builder, true);
-                    break;
-                case NEVER:
-                default:
-                    // do nothing
-                    break;
-            }
-        }
-    }
-
-    /**
-     * A functional interface for a transformation that an expression builder might use when
-     * appending one or more other objects.
-     *
-     * @param <T> the type of object to transform before appending.
-     */
-    @FunctionalInterface
-    public interface Transform<T> {
-        void apply(
-                ExpressionBuilder builder,
-                T input
-        );
-    }
-
-    /**
-     * A fluent API interface returned by the {@link ExpressionBuilder#appendList()} method that
-     * allows a caller to easily define a custom delimiter to be used between items in the list,
-     * an optional transformation that should be applied to each item in the list, and the
-     * items in the list. This is very handle when the number of items is not known a priori.
-     *
-     * @param <T> the type of object to be appended to the expression builder
-     */
-    public interface ListBuilder<T> {
-
-        /**
-         * Define the delimiter to appear between items in the list. If not specified, a comma
-         * is used as the default delimiter.
-         *
-         * @param delimiter the delimiter; may not be null
-         * @return this builder to enable methods to be chained; never null
-         */
-        ListBuilder<T> delimitedBy(String delimiter);
-
-        /**
-         * Define a {@link Transform} that should be applied to every item in the list as it is
-         * appended.
-         *
-         * @param transform the transform; may not be null
-         * @param <R>       the type of item to be transformed
-         * @return this builder to enable methods to be chained; never null
-         */
-        <R> ListBuilder<R> transformedBy(Transform<R> transform);
-
-        /**
-         * Append to this list all of the items in the specified {@link Iterable}.
-         *
-         * @param objects the objects to be appended to the list
-         * @return this builder to enable methods to be chained; never null
-         */
-        ExpressionBuilder of(Iterable<? extends T> objects);
-
-        /**
-         * Append to this list all of the items in the specified {@link Iterable} objects.
-         *
-         * @param objects1 the first collection of objects to be added to the list
-         * @param objects2 a second collection of objects to be added to the list
-         * @return this builder to enable methods to be chained; never null
-         */
-        default ExpressionBuilder of(Iterable<? extends T> objects1, Iterable<? extends T> objects2) {
-            of(objects1);
-            return of(objects2);
-        }
-
-        /**
-         * Append to this list all of the items in the specified {@link Iterable} objects.
-         *
-         * @param objects1 the first collection of objects to be added to the list
-         * @param objects2 a second collection of objects to be added to the list
-         * @param objects3 a third collection of objects to be added to the list
-         * @return this builder to enable methods to be chained; never null
-         */
-        default ExpressionBuilder of(
-                Iterable<? extends T> objects1,
-                Iterable<? extends T> objects2,
-                Iterable<? extends T> objects3
-        ) {
-            of(objects1);
-            of(objects2);
-            return of(objects3);
-        }
-    }
-
-    /**
-     * Get a {@link Transform} that will surround the inputs with quotes.
-     *
-     * @return the transform; never null
-     */
-    public static Transform<String> quote() {
-        return (builder, input) -> builder.appendColumnName(input);
-    }
-
-    /**
-     * Get a {@link Transform} that will quote just the column names.
-     *
-     * @return the transform; never null
-     */
-//    public static Transform<ColumnId> columnNames() {
-//        return (builder, input) -> builder.appendColumnName(input.name());
-//    }
-
-    /**
-     * Get a {@link Transform} that will quote just the column names and append the given string.
-     *
-     * @param appended the string to append after the quoted column names
-     * @return the transform; never null
-     */
-//    public static Transform<ColumnId> columnNamesWith(final String appended) {
-//        return (builder, input) -> {
-//            builder.appendColumnName(input.name());
-//            builder.append(appended);
-//        };
-//    }
-
-    /**
-     * Get a {@link Transform} that will append a placeholder rather than each of the column names.
-     *
-     * @param str the string to output instead the each column name
-     * @return the transform; never null
-     */
-//    public static Transform<ColumnId> placeholderInsteadOfColumnNames(final String str) {
-//        return (builder, input) -> builder.append(str);
-//    }
-
-    /**
-     * Get a {@link Transform} that will append the prefix and then the quoted column name.
-     *
-     * @param prefix the string to output before the quoted column names
-     * @return the transform; never null
-     */
-//    public static Transform<ColumnId> columnNamesWithPrefix(final String prefix) {
-//        return (builder, input) -> {
-//            builder.append(prefix);
-//            builder.appendColumnName(input.name());
-//        };
-//    }
-
-    /**
-     * Create a new ExpressionBuilder using the default {@link IdentifierRules}.
-     *
-     * @return the expression builder
-     */
-    public static ExpressionBuilder create() {
-        return new ExpressionBuilder();
-    }
-
-    protected static final QuoteMethod DEFAULT_QUOTE_METHOD = QuoteMethod.ALWAYS;
-
-    private final IdentifierRules rules;
-    private final StringBuilder sb = new StringBuilder();
-    private QuoteMethod quoteSqlIdentifiers = DEFAULT_QUOTE_METHOD;
-
-    /**
-     * Create a new expression builder with the default {@link IdentifierRules}.
-     */
-    public ExpressionBuilder() {
-        this(null);
-    }
-
-    /**
-     * Create a new expression builder that uses the specified {@link IdentifierRules}.
-     *
-     * @param rules the rules; may be null if the default rules are to be used
-     */
-    public ExpressionBuilder(IdentifierRules rules) {
-        this.rules = rules != null ? rules : IdentifierRules.DEFAULT;
-    }
-
-    /**
-     * Set when this expression builder should quote identifiers, such as table and column names.
-     *
-     * @param method the quoting method; may be null if the default method
-     *               ({@link QuoteMethod#ALWAYS always}) should be used
-     * @return this expression builder; never null
-     */
-    public ExpressionBuilder setQuoteIdentifiers(QuoteMethod method) {
-        this.quoteSqlIdentifiers = method != null ? method : DEFAULT_QUOTE_METHOD;
-        return this;
-    }
-
-    /**
-     * Return a new ExpressionBuilder that escapes quotes with the specified prefix.
-     * This builder remains unaffected.
-     *
-     * @param prefix the prefix
-     * @return the new ExpressionBuilder, or this builder if the prefix is null or empty
-     */
-    public ExpressionBuilder escapeQuotesWith(String prefix) {
-        if (prefix == null || prefix.isEmpty()) {
-            return this;
-        }
-        return new ExpressionBuilder(this.rules.escapeQuotesWith(prefix));
-    }
-
-    /**
-     * Append to this builder's expression the delimiter defined by this builder's
-     * {@link IdentifierRules}.
-     *
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder appendIdentifierDelimiter() {
-        sb.append(rules.identifierDelimiter());
-        return this;
-    }
-
-    /**
-     * Always append to this builder's expression the leading quote character(s) defined by this
-     * builder's {@link IdentifierRules}.
-     *
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder appendLeadingQuote() {
-        return appendLeadingQuote(QuoteMethod.ALWAYS);
-    }
-
-
-    protected ExpressionBuilder appendLeadingQuote(QuoteMethod method) {
-        switch (method) {
-            case ALWAYS:
-                sb.append(rules.leadingQuoteString());
-                break;
-            case NEVER:
-            default:
-                break;
-        }
-        return this;
-    }
-
-    /**
-     * Always append to this builder's expression the trailing quote character(s) defined by this
-     * builder's {@link IdentifierRules}.
-     *
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder appendTrailingQuote() {
-        return appendTrailingQuote(QuoteMethod.ALWAYS);
-    }
-
-    protected ExpressionBuilder appendTrailingQuote(QuoteMethod method) {
-        switch (method) {
-            case ALWAYS:
-                sb.append(rules.trailingQuoteString());
-                break;
-            case NEVER:
-            default:
-                break;
-        }
-        return this;
-    }
-
-    /**
-     * Append to this builder's expression the string quote character ({@code '}).
-     *
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder appendStringQuote() {
-        sb.append("'");
-        return this;
-    }
-
-    /**
-     * Append to this builder's expression a string surrounded by single quote characters ({@code '}).
-     * Use {@link #appendIdentifier(String, QuoteMethod)} for identifiers,
-     * {@link #appendColumnName(String, QuoteMethod)} for column names, or
-     * {@link #appendTableName(String, QuoteMethod)} for table names.
-     *
-     * @param name the object whose string representation is to be appended
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder appendStringQuoted(Object name) {
-        appendStringQuote();
-        sb.append(name);
-        appendStringQuote();
-        return this;
-    }
-
-    /**
-     * Append to this builder's expression the identifier.
-     *
-     * @param name   the name to be appended
-     * @param quoted true if the name should be quoted, or false otherwise
-     * @return this builder to enable methods to be chained; never null
-     * @deprecated use {@link #appendIdentifier(String, QuoteMethod)} instead
-     */
-    @Deprecated
-    public ExpressionBuilder appendIdentifier(
-            String name,
-            boolean quoted
-    ) {
-        return appendIdentifier(name, quoted ? QuoteMethod.ALWAYS : QuoteMethod.NEVER);
-    }
-
-    /**
-     * Append to this builder's expression the identifier.
-     *
-     * @param name   the name to be appended
-     * @param quoted true if the name should be quoted, or false otherwise
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder appendIdentifier(
-            String name,
-            QuoteMethod quoted
-    ) {
-        appendLeadingQuote(quoted);
-        sb.append(name);
-        appendTrailingQuote(quoted);
-        return this;
-    }
-
-    /**
-     * Append to this builder's expression the specified Column identifier, possibly surrounded by
-     * the leading and trailing quotes based upon {@link #setQuoteIdentifiers(QuoteMethod)}.
-     *
-     * @param name the name to be appended
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder appendTableName(String name) {
-        return appendTableName(name, quoteSqlIdentifiers);
-    }
-
-    /**
-     * Append to this builder's expression the specified Column identifier, possibly surrounded by
-     * the leading and trailing quotes based upon {@link #setQuoteIdentifiers(QuoteMethod)}.
-     *
-     * @param name  the name to be appended
-     * @param quote the quote method to be used
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder appendTableName(String name, QuoteMethod quote) {
-        appendLeadingQuote(quote);
-        sb.append(name);
-        appendTrailingQuote(quote);
-        return this;
-    }
-
-    /**
-     * Append to this builder's expression the specified Column identifier, possibly surrounded by
-     * the leading and trailing quotes based upon {@link #setQuoteIdentifiers(QuoteMethod)}.
-     *
-     * @param name the name to be appended
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder appendColumnName(String name) {
-        return appendColumnName(name, quoteSqlIdentifiers);
-    }
-
-    /**
-     * Append to this builder's expression the specified Column identifier, possibly surrounded by
-     * the leading and trailing quotes based upon {@link #setQuoteIdentifiers(QuoteMethod)}.
-     *
-     * @param name  the name to be appended
-     * @param quote whether to quote the column name; may not be null
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder appendColumnName(String name, QuoteMethod quote) {
-        appendLeadingQuote(quote);
-        sb.append(name);
-        appendTrailingQuote(quote);
-        return this;
-    }
-
-    /**
-     * Append to this builder's expression the specified identifier, surrounded by the leading and
-     * trailing quotes.
-     *
-     * @param name the name to be appended
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder appendIdentifierQuoted(String name) {
-        appendLeadingQuote();
-        sb.append(name);
-        appendTrailingQuote();
-        return this;
-    }
-
-    /**
-     * Append to this builder's expression the binary value as a hex string, prefixed and
-     * suffixed by a single quote character.
-     *
-     * @param value the value to be appended
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder appendBinaryLiteral(byte[] value) {
-        return append("x'").append(BytesUtil.toHex(value)).append("'");
-    }
-
-    /**
-     * Append to this builder's expression a new line.
-     *
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder appendNewLine() {
-        sb.append(System.lineSeparator());
-        return this;
-    }
-
-    /**
-     * Append to this builder's expression the specified object. If the object is {@link Expressable},
-     * then this builder delegates to the object's
-     * {@link Expressable#appendTo(ExpressionBuilder, boolean)} method. Otherwise, the string
-     * representation of the object is appended to the expression.
-     *
-     * @param obj       the object to be appended
-     * @param useQuotes true if the object should be surrounded by quotes, or false otherwise
-     * @return this builder to enable methods to be chained; never null
-     * @deprecated use {@link #append(Object, QuoteMethod)} instead
-     */
-    @Deprecated
-    public ExpressionBuilder append(
-            Object obj,
-            boolean useQuotes
-    ) {
-        return append(obj, useQuotes ? QuoteMethod.ALWAYS : QuoteMethod.NEVER);
-    }
-
-    /**
-     * Append to this builder's expression the specified object. If the object is {@link Expressable},
-     * then this builder delegates to the object's
-     * {@link Expressable#appendTo(ExpressionBuilder, boolean)} method. Otherwise, the string
-     * representation of the object is appended to the expression.
-     *
-     * @param obj       the object to be appended
-     * @param useQuotes true if the object should be surrounded by quotes, or false otherwise
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder append(
-            Object obj,
-            QuoteMethod useQuotes
-    ) {
-        if (obj instanceof Expressable) {
-            ((Expressable) obj).appendTo(this, useQuotes);
-        } else if (obj != null) {
-            sb.append(obj);
-        }
-        return this;
-    }
-
-    /**
-     * Append to this builder's expression the specified object surrounded by quotes. If the object
-     * is {@link Expressable}, then this builder delegates to the object's
-     * {@link Expressable#appendTo(ExpressionBuilder, boolean)} method. Otherwise, the string
-     * representation of the object is appended to the expression.
-     *
-     * @param obj the object to be appended
-     * @return this builder to enable methods to be chained; never null
-     */
-    public ExpressionBuilder append(Object obj) {
-        return append(obj, quoteSqlIdentifiers);
-    }
-
-    /**
-     * Append to this builder's expression the specified object surrounded by quotes. If the object
-     * is {@link Expressable}, then this builder delegates to the object's
-     * {@link Expressable#appendTo(ExpressionBuilder, boolean)} method. Otherwise, the string
-     * representation of the object is appended to the expression.
-     *
-     * @param obj       the object to be appended
-     * @param transform the transform that should be used on the supplied object to obtain the
-     *                  representation that is appended to the expression; may be null
-     * @param <T>       the type of object to transform before appending.
-     * @return this builder to enable methods to be chained; never null
-     */
-    public <T> ExpressionBuilder append(
-            T obj,
-            Transform<T> transform
-    ) {
-        if (transform != null) {
-            transform.apply(this, obj);
-        } else {
-            append(obj);
-        }
-        return this;
-    }
-
-    protected class BasicListBuilder<T> implements ListBuilder<T> {
-        private final String delimiter;
-        private final Transform<T> transform;
-        private boolean first = true;
-
-        BasicListBuilder() {
-            this(", ", null);
-        }
-
-        BasicListBuilder(String delimiter, Transform<T> transform) {
-            this.delimiter = delimiter;
-            this.transform = transform != null ? transform : ExpressionBuilder::append;
-        }
-
-        @Override
-        public ListBuilder<T> delimitedBy(String delimiter) {
-            return new BasicListBuilder<T>(delimiter, transform);
-        }
-
-        @Override
-        public <R> ListBuilder<R> transformedBy(Transform<R> transform) {
-            return new BasicListBuilder<>(delimiter, transform);
-        }
-
-        @Override
-        public ExpressionBuilder of(Iterable<? extends T> objects) {
-            for (T obj : objects) {
-                if (first) {
-                    first = false;
-                } else {
-                    append(delimiter);
-                }
-                append(obj, transform);
-            }
-            return ExpressionBuilder.this;
-        }
-    }
-
-    public ListBuilder<Object> appendList() {
-        return new BasicListBuilder<>();
-    }
-
-    public ExpressionBuilder appendMultiple(
-            String delimiter,
-            String expression,
-            int times
-    ) {
-        for (int i = 0; i < times; i++) {
-            if (i > 0) {
-                append(delimiter);
-            }
-            append(expression);
-        }
-        return this;
-    }
-
-    @Override
-    public String toString() {
-        return sb.toString();
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/IdentifierRules.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/IdentifierRules.java
deleted file mode 100644
index 830b4cd..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/IdentifierRules.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.util;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * The rules for how identifiers are parsed and quoted.
- */
-public class IdentifierRules {
-
-    public static final String UNSUPPORTED_QUOTE = " ";
-    private static final String DEFAULT_QUOTE = "\"";
-    private static final String DEFAULT_ID_DELIM = ".";
-
-    public static final IdentifierRules DEFAULT = new IdentifierRules(DEFAULT_ID_DELIM,
-            DEFAULT_QUOTE
-    );
-
-    private final String leadingQuoteString;
-    private final String trailingQuoteString;
-    private final String identifierDelimiter;
-
-    /**
-     * Create new identifier rules using the supplied quote string for both leading and trailing
-     * quotes, and the '{@link #DEFAULT_ID_DELIM}' character for identifier delimiters.
-     *
-     * @param quoteString the string used for leading and trailing quotes; may be null if {@link
-     *                    #DEFAULT_QUOTE} is to be used
-     */
-    public IdentifierRules(String quoteString) {
-        this(DEFAULT_ID_DELIM, quoteString, quoteString);
-    }
-
-    /**
-     * Create new identifier rules using the supplied parameters.
-     *
-     * @param delimiter   the delimiter used within fully qualified names; may be null if {@link
-     *                    #DEFAULT_ID_DELIM} is to be used
-     * @param quoteString the string used for leading and trailing quotes; may be null if {@link
-     *                    #DEFAULT_QUOTE} is to be used
-     */
-    public IdentifierRules(
-            String delimiter,
-            String quoteString
-    ) {
-        this(delimiter, quoteString, quoteString);
-    }
-
-    /**
-     * Create new identifier rules using the supplied parameters.
-     *
-     * @param identifierDelimiter the delimiter used within fully qualified names; may be null if
-     *                            {@link #DEFAULT_ID_DELIM} is to be used
-     * @param leadingQuoteString  the string used for leading quotes; may be null if {@link
-     *                            #DEFAULT_QUOTE} is to be used
-     * @param trailingQuoteString the string used for leading quotes; may be null if {@link
-     *                            #DEFAULT_QUOTE} is to be used
-     */
-    public IdentifierRules(
-            String identifierDelimiter,
-            String leadingQuoteString,
-            String trailingQuoteString
-    ) {
-        this.leadingQuoteString = leadingQuoteString != null ? leadingQuoteString : DEFAULT_QUOTE;
-        this.trailingQuoteString = trailingQuoteString != null ? trailingQuoteString : DEFAULT_QUOTE;
-        this.identifierDelimiter = identifierDelimiter != null ? identifierDelimiter : DEFAULT_ID_DELIM;
-    }
-
-    /**
-     * Get the delimiter that is used to delineate segments within fully-qualified identifiers.
-     *
-     * @return the identifier delimiter; never null
-     */
-    public String identifierDelimiter() {
-        return identifierDelimiter;
-    }
-
-    /**
-     * Get the string used as a leading quote.
-     *
-     * @return the leading quote string; never null
-     */
-    public String leadingQuoteString() {
-        return leadingQuoteString;
-    }
-
-    /**
-     * Get the string used as a trailing quote.
-     *
-     * @return the trailing quote string; never null
-     */
-    public String trailingQuoteString() {
-        return trailingQuoteString;
-    }
-
-    /**
-     * Get an expression builder that uses these identifier rules.
-     *
-     * @return the new expression builder; never null
-     */
-    public ExpressionBuilder expressionBuilder() {
-        return new ExpressionBuilder(this);
-    }
-
-    /**
-     * Parse the unqualified or fully qualified name into its segments.
-     *
-     * @param fqn the unqualified or fully-qualified name; may not be null
-     * @return the segments in the supplied name; never null, but possibly empty
-     */
-    public List<String> parseQualifiedIdentifier(String fqn) {
-        String orig = fqn;
-        String delim = identifierDelimiter();
-        String lead = leadingQuoteString();
-        String trail = trailingQuoteString();
-        List<String> parts = new ArrayList<>();
-        int index;
-        String segment;
-        do {
-            if (!lead.equals(UNSUPPORTED_QUOTE) && fqn.startsWith(lead)) {
-                int end = fqn.indexOf(trail, lead.length());
-                if (end < 0) {
-                    throw new IllegalArgumentException(
-                            "Failure parsing fully qualified identifier; missing trailing quote in " + orig);
-                }
-                segment = fqn.substring(lead.length(), end);
-                fqn = fqn.substring(end + trail.length());
-                if (fqn.startsWith(delim)) {
-                    fqn = fqn.substring(delim.length());
-                    if (fqn.isEmpty()) {
-                        throw new IllegalArgumentException(
-                                "Failure parsing fully qualified identifier; ends in delimiter " + orig);
-                    }
-                }
-            } else {
-                index = fqn.indexOf(delim, 0);
-                if (index == -1) {
-                    segment = fqn;
-                    fqn = "";
-                } else {
-                    segment = fqn.substring(0, index);
-                    fqn = fqn.substring(index + delim.length());
-                    if (fqn.isEmpty()) {
-                        throw new IllegalArgumentException(
-                                "Failure parsing fully qualified identifier; ends in delimiter " + orig);
-                    }
-                }
-            }
-            parts.add(segment);
-        } while (fqn.length() > 0);
-        return parts;
-    }
-
-    /**
-     * Return a new IdentifierRules that escapes quotes with the specified prefix.
-     *
-     * @param prefix the prefix
-     * @return the new IdentifierRules, or this builder if the prefix is null or empty
-     */
-    public IdentifierRules escapeQuotesWith(String prefix) {
-        if (prefix == null || prefix.isEmpty()) {
-            return this;
-        }
-        return new IdentifierRules(
-                identifierDelimiter,
-                prefix + leadingQuoteString,
-                prefix + trailingQuoteString
-        );
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/JdbcDriverInfo.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/JdbcDriverInfo.java
deleted file mode 100644
index b177ea3..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/JdbcDriverInfo.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.util;
-
-/**
- * A summary of the version information about a JDBC driver and the database.
- */
-public class JdbcDriverInfo {
-
-    private final int jdbcMajorVersion;
-    private final int jdbcMinorVersion;
-    private final String jdbcDriverName;
-    private final String productName;
-    private final String productVersion;
-
-    /**
-     * Create the driver information.
-     *
-     * @param jdbcMajorVersion the major version of the JDBC specification supported by the driver
-     * @param jdbcMinorVersion the minor version of the JDBC specification supported by the driver
-     * @param jdbcDriverName   the name of the JDBC driver
-     * @param productName      the name of the database product
-     * @param productVersion   the version of the database product
-     */
-    public JdbcDriverInfo(
-            int jdbcMajorVersion,
-            int jdbcMinorVersion,
-            String jdbcDriverName,
-            String productName,
-            String productVersion
-    ) {
-        this.jdbcMajorVersion = jdbcMajorVersion;
-        this.jdbcMinorVersion = jdbcMinorVersion;
-        this.jdbcDriverName = jdbcDriverName;
-        this.productName = productName;
-        this.productVersion = productVersion;
-    }
-
-    /**
-     * Get the major version of the JDBC specification supported by the driver.
-     *
-     * @return the major version number
-     */
-    public int jdbcMajorVersion() {
-        return jdbcMajorVersion;
-    }
-
-    /**
-     * Get the minor version of the JDBC specification supported by the driver.
-     *
-     * @return the minor version number
-     */
-    public int jdbcMinorVersion() {
-        return jdbcMinorVersion;
-    }
-
-    /**
-     * Get the name of the database product.
-     *
-     * @return the name of the database product
-     */
-    public String productName() {
-        return productName;
-    }
-
-    /**
-     * Get the version of the database product.
-     *
-     * @return the version of the database product
-     */
-    public String productVersion() {
-        return productVersion;
-    }
-
-    /**
-     * Get the name of the JDBC driver.
-     *
-     * @return the name of the JDBC driver
-     */
-    public String jdbcDriverName() {
-        return jdbcDriverName;
-    }
-
-    /**
-     * Determine if the JDBC driver supports at least the specified major and minor version of the
-     * JDBC specifications. This can be used to determine whether or not to call JDBC methods.
-     *
-     * @param jdbcMajorVersion the required major version of the JDBC specification
-     * @param jdbcMinorVersion the required minor version of the JDBC specification
-     * @return true if the driver supports at least the specified version of the JDBC specification,
-     * or false if the driver supports an older version of the JDBC specification
-     */
-    public boolean jdbcVersionAtLeast(
-            int jdbcMajorVersion,
-            int jdbcMinorVersion
-    ) {
-        if (this.jdbcMajorVersion() > jdbcMajorVersion) {
-            return true;
-        }
-        if (jdbcMajorVersion == jdbcMajorVersion() && jdbcMinorVersion() >= jdbcMinorVersion) {
-            return true;
-        }
-        return false;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        if (productName() != null) {
-            sb.append(productName()).append(' ');
-        }
-        if (productVersion() != null) {
-            sb.append(productVersion()).append(' ');
-        }
-        if (jdbcDriverName() != null) {
-            sb.append(" using ").append(jdbcDriverName()).append(' ');
-        }
-        sb.append(jdbcMajorVersion()).append('.').append(jdbcMinorVersion());
-        return sb.toString();
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/NumericMapping.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/NumericMapping.java
deleted file mode 100644
index 97a2d85..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/NumericMapping.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.util;
-
-//import org.apache.rocketmq.connect.doris.connector.DorisSourceConfig;
-
-import java.util.HashMap;
-import java.util.Locale;
-import java.util.Map;
-
-public enum NumericMapping {
-    NONE,
-    PRECISION_ONLY,
-    BEST_FIT,
-    BEST_FIT_EAGER_DOUBLE;
-
-    private static final Map<String, NumericMapping> REVERSE = new HashMap<>(values().length);
-
-    static {
-        for (NumericMapping val : values()) {
-            REVERSE.put(val.name().toLowerCase(Locale.ROOT), val);
-        }
-    }
-
-    public static NumericMapping get(String prop) {
-        // not adding a check for null value because the recommender/validator should catch those.
-        return REVERSE.get(prop.toLowerCase(Locale.ROOT));
-    }
-
-//    public static NumericMapping get(JdbcSourceConfig config) {
-//        // We use 'null' as default to be able to check the old config if the new one is unset.
-//        if (config.getNumericMapping() != null) {
-//            return NumericMapping.valueOf(config.getNumericMapping());
-//        }
-//        if (config.getNumericPrecisionMapping()) {
-//            return NumericMapping.PRECISION_ONLY;
-//        }
-//        return NumericMapping.NONE;
-//    }
-}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/QuoteMethod.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/QuoteMethod.java
deleted file mode 100644
index ace0b7d..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/QuoteMethod.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.util;
-
-public enum QuoteMethod {
-    ALWAYS("always"),
-    NEVER("never");
-
-    public static QuoteMethod get(String name) {
-        for (QuoteMethod method : values()) {
-            if (method.toString().equalsIgnoreCase(name)) {
-                return method;
-            }
-        }
-        throw new IllegalArgumentException("No matching QuoteMethod found for '" + name + "'");
-    }
-
-    private final String name;
-
-    QuoteMethod(String name) {
-        this.name = name;
-    }
-
-    @Override
-    public String toString() {
-        return name;
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/TableType.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/TableType.java
deleted file mode 100644
index 3a49446..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/TableType.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.util;
-
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-/**
- * table type
- */
-public enum TableType {
-
-    TABLE("TABLE", "Table"),
-    VIEW("VIEW", "View");
-
-    private final String value;
-    private final String capitalCase;
-
-    TableType(String value, String capitalCase) {
-        this.value = value.toUpperCase();
-        this.capitalCase = capitalCase;
-    }
-
-    public String capitalized() {
-        return capitalCase;
-    }
-
-    public String jdbcName() {
-        return value;
-    }
-
-    @Override
-    public String toString() {
-        return value;
-    }
-
-    public static TableType get(String name) {
-        if (name != null) {
-            name = name.trim();
-        }
-        for (TableType method : values()) {
-            if (method.toString().equalsIgnoreCase(name)) {
-                return method;
-            }
-        }
-        throw new IllegalArgumentException("No matching QuoteMethod found for '" + name + "'");
-    }
-
-    public static EnumSet<TableType> parse(Collection<String> values) {
-        Set<TableType> types = values.stream().map(TableType::get).collect(Collectors.toSet());
-        return EnumSet.copyOf(types);
-    }
-
-    public static String asJdbcTableTypeNames(EnumSet<TableType> types, String delim) {
-        return types.stream()
-                .map(TableType::jdbcName)
-                .sorted()
-                .collect(Collectors.joining(delim));
-    }
-
-    public static String[] asJdbcTableTypeArray(EnumSet<TableType> types) {
-        return types.stream()
-                .map(TableType::jdbcName)
-                .sorted()
-                .collect(Collectors.toList())
-                .toArray(new String[types.size()]);
-    }
-
-}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/TableUtil.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/TableUtil.java
deleted file mode 100644
index 87bf2b2..0000000
--- a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/TableUtil.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.rocketmq.connect.doris.util;
-
-import io.openmessaging.connector.api.data.ConnectRecord;
-import io.openmessaging.connector.api.data.Struct;
-import org.apache.rocketmq.connect.doris.schema.table.TableId;
-
-public class TableUtil {
-
-    public static TableId parseToTableId(String fqn) {
-        return new TableId(null, null, fqn);
-    }
-    public static TableId destinationTable(ConnectRecord record) {
-        // todo table from header
-        Struct struct = (Struct) record.getData();
-        return new TableId(null, null, struct.getSchema().getName());
-    }
-}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/BackendUtils.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/BackendUtils.java
new file mode 100644
index 0000000..0c7153c
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/BackendUtils.java
@@ -0,0 +1,72 @@
+/*
+ * 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.rocketmq.connect.doris.utils;
+
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.List;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.exception.DorisException;
+import org.apache.rocketmq.connect.doris.model.BackendV2;
+import org.apache.rocketmq.connect.doris.service.RestService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BackendUtils {
+    private static final Logger LOG = LoggerFactory.getLogger(BackendUtils.class);
+    private final List<BackendV2.BackendRowV2> backends;
+    private long pos;
+
+    public BackendUtils(List<BackendV2.BackendRowV2> backends) {
+        this.backends = backends;
+        this.pos = 0;
+    }
+
+    public static BackendUtils getInstance(DorisOptions dorisOptions, Logger logger) {
+        return new BackendUtils(RestService.getBackendsV2(dorisOptions, logger));
+    }
+
+    public String getAvailableBackend() {
+        long tmp = pos + backends.size();
+        while (pos < tmp) {
+            BackendV2.BackendRowV2 backend = backends.get((int) (pos++ % backends.size()));
+            String res = backend.toBackendString();
+            if (tryHttpConnection(res)) {
+                return res;
+            }
+        }
+        throw new DorisException("no available backend.");
+    }
+
+    public static boolean tryHttpConnection(String backend) {
+        try {
+            backend = "http://" + backend;
+            URL url = new URL(backend);
+            HttpURLConnection co = (HttpURLConnection) url.openConnection();
+            co.setConnectTimeout(60000);
+            co.connect();
+            co.disconnect();
+            return true;
+        } catch (Exception ex) {
+            LOG.warn("Failed to connect to backend:{}", backend, ex);
+            return false;
+        }
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/BackoffAndRetryUtils.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/BackoffAndRetryUtils.java
new file mode 100644
index 0000000..fe5e2d0
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/BackoffAndRetryUtils.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.rocketmq.connect.doris.utils;
+
+import org.apache.rocketmq.connect.doris.exception.DorisException;
+import org.apache.rocketmq.connect.doris.model.LoadOperation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BackoffAndRetryUtils {
+
+    private static final Logger LOG = LoggerFactory.getLogger(BackoffAndRetryUtils.class);
+
+    // backoff with 1, 2, 4 seconds
+    private static final int[] backoffSec = {0, 1, 2, 4};
+
+    /**
+     * Interfaces to define the lambda function to be used by backoffAndRetry
+     */
+    public interface backoffFunction {
+        Object apply() throws Exception;
+    }
+
+    /**
+     * Backoff logic
+     *
+     * @param operation Load Operation Type which corresponds to the lambda function runnable
+     * @param runnable  the lambda function itself
+     * @return the object that the function returns
+     * @throws Exception if the runnable function throws exception
+     */
+    public static Object backoffAndRetry(
+        final LoadOperation operation, final backoffFunction runnable) throws Exception {
+        for (final int iteration : backoffSec) {
+            if (iteration != 0) {
+                Thread.sleep(iteration * 1000L);
+                LOG.debug("Retry Count:{} for operation:{}", iteration, operation);
+            }
+            try {
+                return runnable.apply();
+            } catch (Exception e) {
+                LOG.error(
+                    "Retry count:{} caught an exception for operation:{} with message:{}",
+                    iteration,
+                    operation,
+                    e.getMessage());
+            }
+        }
+        String errMsg = "Api retry exceeded the max retry limit, operation = " + operation;
+        LOG.error(errMsg);
+        throw new DorisException(errMsg);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/ConfigCheckUtils.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/ConfigCheckUtils.java
new file mode 100644
index 0000000..21f56c8
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/ConfigCheckUtils.java
@@ -0,0 +1,334 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rocketmq.connect.doris.utils;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.openmessaging.KeyValue;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.regex.Pattern;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.cfg.DorisSinkConnectorConfig;
+import org.apache.rocketmq.connect.doris.converter.ConverterMode;
+import org.apache.rocketmq.connect.doris.converter.schema.SchemaEvolutionMode;
+import org.apache.rocketmq.connect.doris.exception.ArgumentsException;
+import org.apache.rocketmq.connect.doris.exception.DorisException;
+import org.apache.rocketmq.connect.doris.writer.DeliveryGuarantee;
+import org.apache.rocketmq.connect.doris.writer.LoadConstants;
+import org.apache.rocketmq.connect.doris.writer.load.GroupCommitMode;
+import org.apache.rocketmq.connect.doris.writer.load.LoadModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ConfigCheckUtils {
+    private static final Logger LOG = LoggerFactory.getLogger(ConfigCheckUtils.class);
+
+    // task id
+    public static final String TASK_ID = "task_id";
+
+    /**
+     * Validate input configuration
+     *
+     * @param config configuration Map
+     * @return connector name
+     */
+    public static String validateConfig(KeyValue config) {
+        LOG.info("start validate connector config");
+        boolean configIsValid = true; // verify all config
+
+        // unique name of this connector instance
+        String connectorName = config.getString(DorisSinkConnectorConfig.NAME);
+        if (Objects.isNull(connectorName) || connectorName.isEmpty() || !isValidDorisApplicationName(connectorName)) {
+            LOG.error(
+                "{} is empty or invalid. It should match doris object identifier syntax. Please see "
+                    + "the documentation.",
+                DorisSinkConnectorConfig.NAME);
+            configIsValid = false;
+        }
+
+        String topics = config.getString(DorisSinkConnectorConfig.TOPICS);
+        String topicsRegex = config.getString(DorisSinkConnectorConfig.TOPICS_REGEX);
+        if (topics.isEmpty() && topicsRegex.isEmpty()) {
+            LOG.error(
+                "{} or {} cannot be empty.",
+                DorisSinkConnectorConfig.TOPICS,
+                DorisSinkConnectorConfig.TOPICS_REGEX);
+            configIsValid = false;
+        }
+
+        if (!topics.isEmpty() && !topicsRegex.isEmpty()) {
+            LOG.error(
+                "{} and {} cannot be set at the same time.",
+                DorisSinkConnectorConfig.TOPICS,
+                DorisSinkConnectorConfig.TOPICS_REGEX);
+            configIsValid = false;
+        }
+
+        if (config.containsKey(DorisSinkConnectorConfig.TOPICS_TABLES_MAP)
+            && parseTopicToTableMap(config.getString(DorisSinkConnectorConfig.TOPICS_TABLES_MAP))
+            == null) {
+            LOG.error("{} is empty or invalid.", DorisSinkConnectorConfig.TOPICS_TABLES_MAP);
+            configIsValid = false;
+        }
+
+        String dorisUrls = config.getString(DorisSinkConnectorConfig.DORIS_URLS);
+        if (dorisUrls.isEmpty()) {
+            LOG.error("{} cannot be empty.", DorisSinkConnectorConfig.DORIS_URLS);
+            configIsValid = false;
+        }
+
+        String queryPort = config.getString(DorisSinkConnectorConfig.DORIS_QUERY_PORT);
+        if (queryPort.isEmpty()) {
+            LOG.error("{} cannot be empty.", DorisSinkConnectorConfig.DORIS_QUERY_PORT);
+            configIsValid = false;
+        }
+
+        String httpPort = config.getString(DorisSinkConnectorConfig.DORIS_HTTP_PORT);
+        if (httpPort.isEmpty()) {
+            LOG.error("{} cannot be empty.", DorisSinkConnectorConfig.DORIS_HTTP_PORT);
+            configIsValid = false;
+        }
+
+        String dorisUser = config.getString(DorisSinkConnectorConfig.DORIS_USER);
+        if (dorisUser.isEmpty()) {
+            LOG.error("{} cannot be empty.", DorisSinkConnectorConfig.DORIS_USER);
+            configIsValid = false;
+        }
+
+        String autoDirect = config.getString(DorisSinkConnectorConfig.AUTO_REDIRECT);
+        if (!autoDirect.isEmpty()
+            && !("true".equalsIgnoreCase(autoDirect) || "false".equalsIgnoreCase(autoDirect))) {
+            LOG.error("autoDirect non-boolean type, {}", autoDirect);
+            configIsValid = false;
+        }
+
+        String bufferCountRecords = config.getString(DorisSinkConnectorConfig.BUFFER_COUNT_RECORDS);
+        if (!isNumeric(bufferCountRecords)) {
+            LOG.error(
+                "{} cannot be empty or not a number.",
+                DorisSinkConnectorConfig.BUFFER_COUNT_RECORDS);
+            configIsValid = false;
+        }
+
+        String bufferSizeBytes = config.getString(DorisSinkConnectorConfig.BUFFER_SIZE_BYTES);
+        if (!isNumeric(bufferSizeBytes)
+            || isIllegalRange(
+            bufferSizeBytes, DorisSinkConnectorConfig.BUFFER_SIZE_BYTES_MIN)) {
+            LOG.error(
+                "{} cannot be empty or not a number or less than 1.",
+                DorisSinkConnectorConfig.BUFFER_SIZE_BYTES);
+            configIsValid = false;
+        }
+
+        String bufferFlushTime = config.getString(DorisSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC);
+        if (!isNumeric(bufferFlushTime)
+            || isIllegalRange(
+            bufferFlushTime, DorisSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC_MIN)) {
+            LOG.error(
+                "{} cannot be empty or not a number or less than 10.",
+                DorisSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC);
+            configIsValid = false;
+        }
+
+        String loadModel = config.getString(DorisSinkConnectorConfig.LOAD_MODEL);
+        if (!validateEnumInstances(loadModel, LoadModel.instances())) {
+            LOG.error(
+                "The value of {} is an illegal parameter of {}.",
+                loadModel,
+                DorisSinkConnectorConfig.LOAD_MODEL);
+            configIsValid = false;
+        }
+
+        String deliveryGuarantee = config.getString(DorisSinkConnectorConfig.DELIVERY_GUARANTEE);
+        if (!validateEnumInstances(deliveryGuarantee, DeliveryGuarantee.instances())) {
+            LOG.error(
+                "The value of {} is an illegal parameter of {}.",
+                loadModel,
+                DorisSinkConnectorConfig.DELIVERY_GUARANTEE);
+            configIsValid = false;
+        }
+
+        String converterMode = config.getString(DorisSinkConnectorConfig.CONVERTER_MODE);
+        if (!validateEnumInstances(converterMode, ConverterMode.instances())) {
+            LOG.error(
+                "The value of {} is an illegal parameter of {}.",
+                loadModel,
+                DorisSinkConnectorConfig.CONVERTER_MODE);
+            configIsValid = false;
+        }
+
+        String schemaEvolutionMode = config.getString(DorisSinkConnectorConfig.DEBEZIUM_SCHEMA_EVOLUTION);
+        if (!validateEnumInstances(schemaEvolutionMode, SchemaEvolutionMode.instances())) {
+            LOG.error(
+                "The value of {} is an illegal parameter of {}.",
+                loadModel,
+                DorisSinkConnectorConfig.DEBEZIUM_SCHEMA_EVOLUTION);
+            configIsValid = false;
+        }
+
+        if (!configIsValid) {
+            throw new DorisException(
+                "input kafka connector configuration is null, missing required values, or wrong input value");
+        }
+
+        return connectorName;
+    }
+
+    /**
+     * validates that given name is a valid doris application name, support '-'
+     *
+     * @param appName doris application name
+     * @return true if given application name is valid
+     */
+    public static boolean isValidDorisApplicationName(String appName) {
+        return appName.matches("([a-zA-Z0-9_\\-]+)");
+    }
+
+    /**
+     * verify topic name, and generate valid table name
+     *
+     * @param topic       input topic name
+     * @param topic2table topic to table map
+     * @return valid table name
+     */
+    public static String tableName(String topic, Map<String, String> topic2table) {
+        return generateValidName(topic, topic2table);
+    }
+
+    /**
+     * verify topic name, and generate valid table/application name
+     *
+     * @param topic       input topic name
+     * @param topic2table topic to table map
+     * @return valid table/application name
+     */
+    public static String generateValidName(String topic, Map<String, String> topic2table) {
+        if (topic == null || topic.isEmpty()) {
+            throw new DorisException("Topic name is empty String or null");
+        }
+        if (topic2table.containsKey(topic)) {
+            return topic2table.get(topic);
+        }
+        if (isValidTableIdentifier(topic)) {
+            return topic;
+        }
+        // debezium topic default regex name.db.tbl
+        if (topic.contains(".")) {
+            String[] split = topic.split("\\.");
+            return split[split.length - 1];
+        }
+
+        throw new ArgumentsException("Failed get table name from topic");
+    }
+
+    public static Map<String, String> parseTopicToTableMap(String input) {
+        Map<String, String> topic2Table = new HashMap<>();
+        boolean isInvalid = false;
+        for (String str : input.split(",")) {
+            String[] tt = str.split(":");
+
+            if (tt.length != 2 || tt[0].trim().isEmpty() || tt[1].trim().isEmpty()) {
+                LOG.error(
+                    "Invalid {} config format: {}",
+                    DorisSinkConnectorConfig.TOPICS_TABLES_MAP,
+                    input);
+                return null;
+            }
+
+            String topic = tt[0].trim();
+            String table = tt[1].trim();
+
+            if (table.isEmpty()) {
+                LOG.error("tableName is empty");
+                isInvalid = true;
+            }
+
+            if (topic2Table.containsKey(topic)) {
+                LOG.error("topic name {} is duplicated", topic);
+                isInvalid = true;
+            }
+
+            topic2Table.put(tt[0].trim(), tt[1].trim());
+        }
+        if (isInvalid) {
+            throw new DorisException("Failed to parse topic2table map");
+        }
+        return topic2Table;
+    }
+
+    private static boolean isNumeric(String str) {
+        if (str != null && !str.isEmpty()) {
+            Pattern pattern = Pattern.compile("[0-9]*");
+            return pattern.matcher(str).matches();
+        }
+        return false;
+    }
+
+    private static boolean isIllegalRange(String flushTime, long minValue) {
+        long time = Long.parseLong(flushTime);
+        return time < minValue;
+    }
+
+    /**
+     * validates that table name is a valid table identifier
+     */
+    private static boolean isValidTableIdentifier(String tblName) {
+        return tblName.matches("^[a-zA-Z][a-zA-Z0-9_]*$");
+    }
+
+    private static boolean validateEnumInstances(String value, String[] instances) {
+        for (String instance : instances) {
+            if (instance.equalsIgnoreCase(value)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @VisibleForTesting
+    public static boolean validateGroupCommitMode(DorisOptions dorisOptions) {
+        Properties streamLoadProp = dorisOptions.getStreamLoadProp();
+        boolean enable2PC = dorisOptions.enable2PC();
+        boolean force2PC = dorisOptions.force2PC();
+
+        Object value = streamLoadProp.get(LoadConstants.GROUP_COMMIT);
+        String normalizedValue = value.toString().trim().toLowerCase();
+        if (!GroupCommitMode.instances().contains(normalizedValue)) {
+            throw new DorisException(
+                "The value of group commit mode is an illegal parameter, illegal value="
+                    + value);
+        } else if (enable2PC && force2PC) {
+            throw new DorisException(
+                "When group commit is enabled, you should disable two phase commit! Please  set 'enable.2pc':'false'");
+        } else if (streamLoadProp.containsKey(LoadConstants.PARTIAL_COLUMNS)
+            && streamLoadProp.get(LoadConstants.PARTIAL_COLUMNS).equals("true")) {
+            throw new DorisException(
+                "When group commit is enabled,you can not load data with partial column update.");
+        } else if (enable2PC) {
+            // The default enable2PC is true, in the scenario of group commit, it needs to be closed
+            LOG.info(
+                "The Group Commit mode is on, the two phase commit default value should be disabled.");
+            dorisOptions.setEnable2PC(false);
+        }
+        return true;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/ConnectRecordUtil.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/ConnectRecordUtil.java
new file mode 100644
index 0000000..e2e5eb3
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/ConnectRecordUtil.java
@@ -0,0 +1,36 @@
+package org.apache.rocketmq.connect.doris.utils;
+
+import io.openmessaging.connector.api.data.RecordOffset;
+import io.openmessaging.connector.api.data.RecordPartition;
+import java.util.Map;
+
+public class ConnectRecordUtil {
+    public static final String TOPIC = "topic";
+    public static final String BROKER_NAME = "brokerName";
+    public static final String QUEUE_ID = "queueId";
+    public static final String QUEUE_OFFSET = "queueOffset";
+
+    public static long getQueueOffset(RecordOffset recordOffset) {
+        Map<String, ?> offset = recordOffset.getOffset();
+        if (offset.containsKey(QUEUE_OFFSET)) {
+            return Long.parseLong((String) offset.get(QUEUE_OFFSET));
+        }
+        return -1;
+    }
+
+    public static String getTopicName(RecordPartition recordPartition) {
+        Map<String, ?> partition = recordPartition.getPartition();
+        return (String) partition.get(TOPIC);
+    }
+
+    public static String getBrokerName(RecordPartition recordPartition) {
+        Map<String, ?> partition = recordPartition.getPartition();
+        return (String) partition.get(BROKER_NAME);
+    }
+
+    public static String getQueueId(RecordPartition recordPartition) {
+        Map<String, ?> partition = recordPartition.getPartition();
+        return (String) partition.get(QUEUE_ID);
+    }
+
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/FileNameUtils.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/FileNameUtils.java
new file mode 100644
index 0000000..461a22a
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/FileNameUtils.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rocketmq.connect.doris.utils;
+
+import java.util.Objects;
+import org.apache.rocketmq.connect.doris.writer.LoadConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FileNameUtils {
+    private static final Logger LOG = LoggerFactory.getLogger(FileNameUtils.class);
+
+    /**
+     * generate file name
+     *
+     * @param prefix prefix
+     * @param end    end offset
+     * @return file name
+     */
+    public static String fileName(String prefix, long end) {
+        long time = System.currentTimeMillis();
+        String fileName = prefix + end + LoadConstants.FILE_DELIM_DEFAULT + time;
+        LOG.debug("generated file name: {}", fileName);
+        return fileName;
+    }
+
+    /**
+     * generate file prefix
+     *
+     * @param appName   connector name
+     * @param partition partition index
+     * @return file prefix
+     */
+    public static String filePrefix(String appName, String topic) {
+        return appName
+            + LoadConstants.FILE_DELIM_DEFAULT
+            + topic
+            + LoadConstants.FILE_DELIM_DEFAULT;
+    }
+
+    /**
+     * verify file name
+     *
+     * @param fileName file name
+     * @return true if file name format is correct, false otherwise
+     */
+    public static boolean verifyFileName(
+        String name, String topic, String fileName) {
+        String prefix = filePrefix(name, topic);
+        return fileName.startsWith(prefix);
+    }
+
+    /**
+     * read end offset from file name
+     *
+     * @param fileName file name
+     * @return end offset
+     */
+    public static long fileNameToEndOffset(String fileName) {
+        return Long.parseLong(readFromFileName(fileName, 2));
+    }
+
+    public static long labelToEndOffset(String label) {
+        return Long.parseLong(readFromFileName(label, 3));
+    }
+
+    /**
+     * read filename from filepath
+     *
+     * @param filePath name
+     * @return fileName
+     */
+    public static String fileNameFromPath(String filePath) {
+        if (!Objects.isNull(filePath)) {
+            int index = filePath.lastIndexOf("/");
+            return filePath.substring(index + 1);
+        }
+        return null;
+    }
+
+    /**
+     * read a value from file name
+     *
+     * @param fileName file name
+     * @param index    value index
+     * @return string value
+     */
+    private static String readFromFileName(String fileName, int index) {
+        String[] splitFileName = fileName.split(LoadConstants.FILE_DELIM_DEFAULT);
+        if (splitFileName.length == 0) {
+            LOG.warn("The file name does not contain __KC_ and is illegal. fileName={}", fileName);
+            return "-1";
+        }
+        String value = splitFileName[index];
+        // Determine whether a string is a number
+        if (!value.matches("^[0-9]*$")) {
+            LOG.warn("The fileName is not a number. value={}, fileName={}", value, fileName);
+            return "-1";
+        }
+        return value;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpGetWithEntity.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpGetWithEntity.java
new file mode 100644
index 0000000..ce946de
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpGetWithEntity.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rocketmq.connect.doris.utils;
+
+import java.net.URI;
+import org.apache.http.client.methods.HttpEntityEnclosingRequestBase;
+
+public class HttpGetWithEntity extends HttpEntityEnclosingRequestBase {
+    private static final String METHOD_NAME = "GET";
+
+    @Override
+    public String getMethod() {
+        return METHOD_NAME;
+    }
+
+    public HttpGetWithEntity(final String uri) {
+        super();
+        setURI(URI.create(uri));
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpPostBuilder.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpPostBuilder.java
new file mode 100644
index 0000000..02ee524
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpPostBuilder.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.rocketmq.connect.doris.utils;
+
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.http.HttpEntity;
+import org.apache.http.HttpHeaders;
+import org.apache.http.client.methods.HttpPost;
+
+public class HttpPostBuilder {
+    String url;
+    Map<String, String> header;
+    HttpEntity httpEntity;
+
+    public HttpPostBuilder() {
+        header = new HashMap<>();
+    }
+
+    public HttpPostBuilder setUrl(String url) {
+        this.url = url;
+        return this;
+    }
+
+    public HttpPostBuilder addCommonHeader() {
+        header.put(HttpHeaders.EXPECT, "100-continue");
+        return this;
+    }
+
+    public HttpPostBuilder baseAuth(String user, String password) {
+        final String authInfo = user + ":" + password;
+        byte[] encoded = Base64.encodeBase64(authInfo.getBytes(StandardCharsets.UTF_8));
+        header.put(HttpHeaders.AUTHORIZATION, "Basic " + new String(encoded));
+        return this;
+    }
+
+    public HttpPostBuilder setEntity(HttpEntity httpEntity) {
+        this.httpEntity = httpEntity;
+        return this;
+    }
+
+    public HttpPost build() {
+        StringUtils.isNotEmpty(url);
+        Objects.nonNull(httpEntity);
+        HttpPost put = new HttpPost(url);
+        header.forEach(put::setHeader);
+        put.setEntity(httpEntity);
+        return put;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpPutBuilder.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpPutBuilder.java
new file mode 100644
index 0000000..94b5c56
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpPutBuilder.java
@@ -0,0 +1,126 @@
+/*
+ * 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.rocketmq.connect.doris.utils;
+
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.http.HttpEntity;
+import org.apache.http.HttpHeaders;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.entity.StringEntity;
+import org.apache.rocketmq.connect.doris.writer.LoadConstants;
+
+public class HttpPutBuilder {
+    String url;
+    Map<String, String> header;
+    HttpEntity httpEntity;
+
+    public HttpPutBuilder() {
+        header = new HashMap<>();
+    }
+
+    public HttpPutBuilder setUrl(String url) {
+        this.url = url;
+        return this;
+    }
+
+    public HttpPutBuilder addFileName(String fileName) {
+        header.put("fileName", fileName);
+        return this;
+    }
+
+    public HttpPutBuilder setEmptyEntity() {
+        try {
+            this.httpEntity = new StringEntity("");
+        } catch (Exception e) {
+            throw new IllegalArgumentException(e);
+        }
+        return this;
+    }
+
+    public HttpPutBuilder addCommonHeader() {
+        header.put(HttpHeaders.EXPECT, "100-continue");
+        return this;
+    }
+
+    public HttpPutBuilder baseAuth(String user, String password) {
+        final String authInfo = user + ":" + password;
+        byte[] encoded = Base64.encodeBase64(authInfo.getBytes(StandardCharsets.UTF_8));
+        header.put(HttpHeaders.AUTHORIZATION, "Basic " + new String(encoded));
+        return this;
+    }
+
+    public HttpPutBuilder abort() {
+        header.put("txn_operation", "abort");
+        return this;
+    }
+
+    public HttpPutBuilder commit() {
+        header.put("txn_operation", "commit");
+        return this;
+    }
+
+    public HttpPutBuilder addTxnId(long txnID) {
+        header.put("txn_id", String.valueOf(txnID));
+        return this;
+    }
+
+    public HttpPutBuilder setLabel(String label) {
+        header.put("label", label);
+        return this;
+    }
+
+    public HttpPutBuilder setEntity(HttpEntity httpEntity) {
+        this.httpEntity = httpEntity;
+        return this;
+    }
+
+    public HttpPutBuilder addHiddenColumns(boolean add) {
+        if (add) {
+            header.put("hidden_columns", LoadConstants.DORIS_DELETE_SIGN);
+        }
+        return this;
+    }
+
+    public HttpPutBuilder addProperties(Properties properties) {
+        // TODO: check duplicate key.
+        properties.forEach((key, value) -> header.put(String.valueOf(key), String.valueOf(value)));
+        return this;
+    }
+
+    public HttpPutBuilder enable2PC(boolean enable2PC) {
+        header.put("two_phase_commit", String.valueOf(enable2PC));
+        return this;
+    }
+
+    public HttpPut build() {
+        StringUtils.isNotEmpty(url);
+        Objects.nonNull(httpEntity);
+        HttpPut put = new HttpPut(url);
+        header.forEach(put::setHeader);
+        put.setEntity(httpEntity);
+        return put;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpUtils.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpUtils.java
new file mode 100644
index 0000000..01e146a
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpUtils.java
@@ -0,0 +1,89 @@
+/*
+ * 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.rocketmq.connect.doris.utils;
+
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import org.apache.http.HttpHost;
+import org.apache.http.auth.AuthScope;
+import org.apache.http.auth.UsernamePasswordCredentials;
+import org.apache.http.client.CredentialsProvider;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.impl.client.BasicCredentialsProvider;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.DefaultRedirectStrategy;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions.ProxyConfig;
+
+/**
+ * util to build http client.
+ */
+public class HttpUtils {
+    private final DorisOptions dorisOptions;
+    private final ProxyConfig proxyConfig;
+
+    public HttpUtils(DorisOptions dorisOptions) {
+        this.dorisOptions = dorisOptions;
+        this.proxyConfig = dorisOptions.getProxyConfig()
+            .orElseThrow(() -> new NoSuchElementException("Failed to get ProxyConfig."));
+    }
+
+    private final HttpClientBuilder httpClientBuilder =
+        HttpClients.custom()
+            .setRedirectStrategy(
+                new DefaultRedirectStrategy() {
+                    @Override
+                    protected boolean isRedirectable(String method) {
+                        return true;
+                    }
+                })
+            .setDefaultRequestConfig(createRequestConfigWithProxy())
+            .setDefaultCredentialsProvider(createCredentialsProvider());
+
+    private RequestConfig createRequestConfigWithProxy() {
+        if (Objects.requireNonNull(dorisOptions).customCluster()) {
+            String socksProxyHost = proxyConfig.getSocks5Host();
+            int socksProxyPort = proxyConfig.getSocks5Port();           // Socks5 代理端口
+            HttpHost proxy = new HttpHost(socksProxyHost, socksProxyPort);
+            return RequestConfig.custom()
+                .setProxy(proxy)
+                .build();
+        } else {
+            return RequestConfig.custom().build();
+        }
+    }
+
+    private CredentialsProvider createCredentialsProvider() {
+        CredentialsProvider credentialsProvider = new BasicCredentialsProvider();
+        if (Objects.requireNonNull(dorisOptions).customCluster()) {
+            credentialsProvider.setCredentials(
+                new AuthScope(proxyConfig.getSocks5Host(), proxyConfig.getSocks5Port()),
+                new UsernamePasswordCredentials(proxyConfig.getSocks5UserName(), proxyConfig.getSocks5Password())
+            );
+        }
+        return credentialsProvider;
+    }
+
+    public CloseableHttpClient getHttpClient() {
+        return httpClientBuilder.build();
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/CopyIntoWriter.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/CopyIntoWriter.java
new file mode 100644
index 0000000..c9fdf66
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/CopyIntoWriter.java
@@ -0,0 +1,159 @@
+/*
+ * 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.rocketmq.connect.doris.writer;
+
+import io.openmessaging.connector.api.data.ConnectRecord;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.connection.ConnectionProvider;
+import org.apache.rocketmq.connect.doris.exception.CopyLoadException;
+import org.apache.rocketmq.connect.doris.metrics.DorisConnectMonitor;
+import org.apache.rocketmq.connect.doris.utils.FileNameUtils;
+import org.apache.rocketmq.connect.doris.writer.load.CopyLoad;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Import data through copy-into.
+ */
+public class CopyIntoWriter extends DorisWriter {
+    private static final Logger LOG = LoggerFactory.getLogger(CopyIntoWriter.class);
+    private static final int COMMIT_MAX_FILE_NUM = 50;
+    private final CopyLoad copyLoad;
+    private final String prefix;
+
+    public CopyIntoWriter(
+        String topic,
+        DorisOptions dorisOptions,
+        ConnectionProvider connectionProvider,
+        DorisConnectMonitor connectMonitor) {
+        super(topic, dorisOptions, connectionProvider, connectMonitor);
+        this.taskId = dorisOptions.getTaskId();
+        this.prefix = FileNameUtils.filePrefix(dorisOptions.getName(), topic);
+        this.copyLoad = new CopyLoad(dbName, tableName, dorisOptions);
+    }
+
+    public void fetchOffset() {
+        List<String> loadFiles = listLoadFiles();
+        long maxOffset = -1L;
+        for (String filePath : loadFiles) {
+            String name = FileNameUtils.fileNameFromPath(filePath);
+            if (!FileNameUtils.verifyFileName(dorisOptions.getName(), topic, name)) {
+                continue;
+            }
+            long offset = FileNameUtils.fileNameToEndOffset(name);
+            if (offset > maxOffset) {
+                maxOffset = offset;
+            }
+        }
+        this.offsetPersistedInDoris.set(maxOffset);
+        LOG.info("Init {} offset of {} topic.", maxOffset, topic);
+    }
+
+    private List<String> listLoadFiles() {
+        final String SQL_TEMPLATE =
+            "SHOW COPY FROM %s WHERE TABLENAME = '%s' AND STATE = 'FINISHED' AND FILES LIKE '%%%s%%' ORDER BY CREATETIME DESC LIMIT 100";
+        final String filePrefix =
+            FileNameUtils.filePrefix(dorisOptions.getName(), topic);
+        String offsetQuery = String.format(SQL_TEMPLATE, dbName, tableName, filePrefix);
+        LOG.info("query offset by sql: {}", offsetQuery);
+        List<String> loadFileList = new ArrayList<>();
+        try (Connection connection = connectionProvider.getOrEstablishConnection();
+             PreparedStatement ps = connection.prepareStatement(offsetQuery);
+             ResultSet rs = ps.executeQuery()) {
+            while (rs.next()) {
+                String filesStr = rs.getString("Files");
+
+                String[] files = objectMapper.readValue(filesStr, String[].class);
+                loadFileList.addAll(Arrays.asList(files));
+            }
+        } catch (Exception ex) {
+            LOG.warn(
+                "Failed to get copy-into file name, causing the doris kafka connector to not guarantee exactly once.",
+                ex);
+            throw new CopyLoadException(
+                "Failed to get copy-into file name, causing the doris kafka connector to not guarantee exactly once.",
+                ex);
+        }
+        return loadFileList;
+    }
+
+    @Override
+    public void insert(ConnectRecord record) {
+        initRecord();
+        insertRecord(record);
+    }
+
+    protected void flush(final RecordBuffer buff) {
+        super.flush(buff);
+
+        String fileName = null;
+        fileName = FileNameUtils.fileName(prefix, buff.getLastOffset());
+        String content = buff.getData();
+        copyLoad.uploadFile(fileName, content);
+        updateFlushedMetrics(buff);
+
+        fileNames.add(fileName);
+        LOG.info(
+            "flush to table {} with file {}, record {} size {} ",
+            fileName,
+            tableName,
+            buff.getNumOfRecords(),
+            buff.getBufferSizeBytes());
+
+        if (shouldCommit()) {
+            commit();
+            LOG.info("commit by file number {}, with files: {}", fileNames.size(), fileNames);
+            fileNames = new ArrayList<>();
+        }
+    }
+
+    /**
+     * The maximum number of files in a single copy into is 50
+     */
+    public boolean shouldCommit() {
+        return fileNames.size() >= COMMIT_MAX_FILE_NUM;
+    }
+
+    /**
+     * execute copy into sql
+     */
+    public void commit() {
+        // doris commit
+        if (fileNames.isEmpty()) {
+            return;
+        }
+        copyLoad.executeCopy(fileNames);
+        updateCommitOffset();
+    }
+
+    private void updateCommitOffset() {
+        // committedOffset should be updated only when copy load has succeeded.
+        committedOffset.set(flushedOffset.get());
+        connectMonitor.setCommittedOffset(committedOffset.get() - 1);
+        fileNames = new LinkedList<>();
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/CopySQLBuilder.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/CopySQLBuilder.java
new file mode 100644
index 0000000..e6ebc30
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/CopySQLBuilder.java
@@ -0,0 +1,72 @@
+/*
+ * 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.rocketmq.connect.doris.writer;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.StringJoiner;
+
+public class CopySQLBuilder {
+    private static final String COPY_SYNC = "copy.async";
+    private final String database;
+    private final String table;
+    private final List<String> fileList;
+    private final Properties properties;
+
+    private final boolean enableDelete;
+
+    public CopySQLBuilder(
+        String database, String table, List<String> fileList, boolean enableDelete) {
+        this.database = database;
+        this.table = table;
+        this.fileList = fileList;
+        this.enableDelete = enableDelete;
+        this.properties = new Properties();
+    }
+
+    public String buildCopySQL() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("COPY INTO ")
+            .append(database)
+            .append(".")
+            .append(table)
+            .append(" FROM @~('{")
+            .append(String.join(",", fileList))
+            .append("}') ")
+            .append("PROPERTIES (");
+
+        // copy into must be sync
+        properties.put(COPY_SYNC, false);
+        // default json
+        properties.put("file.type", "json");
+        properties.put("file.strip_outer_array", "false");
+        properties.put("copy.use_delete_sign", enableDelete);
+        StringJoiner props = new StringJoiner(",");
+        for (Map.Entry<Object, Object> entry : properties.entrySet()) {
+            String key = String.valueOf(entry.getKey());
+            String value = String.valueOf(entry.getValue());
+            String prop = String.format("'%s'='%s'", key, value);
+            props.add(prop);
+        }
+        sb.append(props).append(")");
+        return sb.toString();
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/DeliveryGuarantee.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/DeliveryGuarantee.java
new file mode 100644
index 0000000..84aecbb
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/DeliveryGuarantee.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.rocketmq.connect.doris.writer;
+
+public enum DeliveryGuarantee {
+    EXACTLY_ONCE("exactly_once"),
+
+    AT_LEAST_ONCE("at_least_once");
+
+    private final String name;
+
+    DeliveryGuarantee(String name) {
+        this.name = name;
+    }
+
+    public static DeliveryGuarantee of(String name) {
+        return DeliveryGuarantee.valueOf(name.toUpperCase());
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public static String[] instances() {
+        return new String[] {EXACTLY_ONCE.name, AT_LEAST_ONCE.name};
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/DorisWriter.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/DorisWriter.java
new file mode 100644
index 0000000..1d4eb77
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/DorisWriter.java
@@ -0,0 +1,184 @@
+/*
+ * 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.rocketmq.connect.doris.writer;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.connection.ConnectionProvider;
+import org.apache.rocketmq.connect.doris.converter.RecordService;
+import org.apache.rocketmq.connect.doris.exception.ArgumentsException;
+import org.apache.rocketmq.connect.doris.metrics.DorisConnectMonitor;
+import org.apache.rocketmq.connect.doris.utils.ConnectRecordUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class DorisWriter {
+    private static final Logger LOG = LoggerFactory.getLogger(DorisWriter.class);
+    protected static final ObjectMapper objectMapper = new ObjectMapper();
+    protected String tableName;
+    protected String dbName;
+    protected final String tableIdentifier;
+    protected List<String> fileNames;
+    private RecordBuffer buffer;
+    protected final AtomicLong committedOffset; // loaded offset + 1
+    protected final AtomicLong flushedOffset; // flushed offset
+    protected final AtomicLong processedOffset; // processed offset
+    protected long previousFlushTimeStamp;
+
+    // make the initialization lazy
+    private boolean hasInitialized = false;
+    protected final AtomicLong offsetPersistedInDoris = new AtomicLong(-1);
+    protected final ConnectionProvider connectionProvider;
+    protected final DorisOptions dorisOptions;
+    protected final String topic;
+    protected RecordService recordService;
+    protected int taskId;
+    protected final DorisConnectMonitor connectMonitor;
+
+    public DorisWriter(
+        String topic,
+        DorisOptions dorisOptions,
+        ConnectionProvider connectionProvider,
+        DorisConnectMonitor connectMonitor) {
+        this.topic = topic;
+        this.tableName = dorisOptions.getTopicMapTable(topic);
+        if (StringUtils.isEmpty(tableName)) {
+            // The mapping of topic and table is not defined
+            this.tableName = this.topic;
+        }
+        if (StringUtils.isNotEmpty(dorisOptions.getDatabase())) {
+            this.dbName = dorisOptions.getDatabase();
+        } else if (tableName.contains(".")) {
+            String[] dbTbl = tableName.split("\\.");
+            this.dbName = dbTbl[0];
+            this.tableName = dbTbl[1];
+        } else {
+            LOG.error("Error params database {}, table {}, topic {}", dbName, tableName, topic);
+            throw new ArgumentsException("Failed to get database and table names");
+        }
+
+        this.tableIdentifier = dbName + "." + tableName;
+        this.fileNames = new ArrayList<>();
+        this.buffer = new RecordBuffer();
+        this.processedOffset = new AtomicLong(-1);
+        this.flushedOffset = new AtomicLong(-1);
+        this.committedOffset = new AtomicLong(0);
+        this.previousFlushTimeStamp = System.currentTimeMillis();
+
+        this.dorisOptions = dorisOptions;
+        this.connectionProvider = connectionProvider;
+        this.recordService = new RecordService(dorisOptions);
+        this.connectMonitor = connectMonitor;
+    }
+
+    /**
+     * read offset from doris
+     */
+    public abstract void fetchOffset();
+
+    public void insert(final ConnectRecord record) {
+    }
+
+    protected void initRecord() {
+        // init offset
+        if (!hasInitialized
+            && DeliveryGuarantee.EXACTLY_ONCE.equals(dorisOptions.getDeliveryGuarantee())) {
+            // This will only be called once at the beginning when an offset arrives for first time
+            // after connector starts/rebalance
+            LOG.info(
+                "Read the offset of {} topic from doris.", topic);
+            fetchOffset();
+            this.hasInitialized = true;
+        }
+    }
+
+    protected void insertRecord(final ConnectRecord record) {
+        // discard the record if the record offset is smaller or equal to server side offset
+        long recordOffset = ConnectRecordUtil.getQueueOffset(record.getPosition().getOffset());
+        if (recordOffset > this.offsetPersistedInDoris.get()
+            && recordOffset > processedOffset.get()) {
+            RecordBuffer tmpBuff = null;
+            processedOffset.set(recordOffset);
+            putBuffer(record);
+            if (buffer.getBufferSizeBytes() >= dorisOptions.getFileSize()
+                || (dorisOptions.getRecordNum() != 0
+                && buffer.getNumOfRecords() >= dorisOptions.getRecordNum())) {
+                tmpBuff = buffer;
+                this.buffer = new RecordBuffer();
+            }
+
+            if (tmpBuff != null) {
+                flush(tmpBuff);
+            }
+        }
+    }
+
+    protected void updateFlushedMetrics(final RecordBuffer buffer) {
+        // compute metrics which will be exported to JMX for now.
+        connectMonitor.updateBufferMetrics(buffer.getBufferSizeBytes(), buffer.getNumOfRecords());
+        this.previousFlushTimeStamp = System.currentTimeMillis();
+        // This is safe and atomic
+        flushedOffset.updateAndGet((value) -> Math.max(buffer.getLastOffset() + 1, value));
+        connectMonitor.resetMemoryUsage();
+        connectMonitor.addAndGetLoadCount();
+    }
+
+    protected void putBuffer(ConnectRecord record) {
+        long offset = ConnectRecordUtil.getQueueOffset(record.getPosition().getOffset());
+        String processedRecord = recordService.getProcessedRecord(record);
+        if (buffer.getBufferSizeBytes() == 0L) {
+            buffer.setFirstOffset(offset);
+        }
+        buffer.insert(processedRecord);
+        buffer.setLastOffset(offset);
+        connectMonitor.addAndGetBuffMemoryUsage(
+            processedRecord.getBytes(StandardCharsets.UTF_8).length);
+    }
+
+    public boolean shouldFlush() {
+        return (System.currentTimeMillis() - this.previousFlushTimeStamp)
+            >= (dorisOptions.getFlushTime() * 1000);
+    }
+
+    public void flushBuffer() {
+        if (buffer.isEmpty()) {
+            return;
+        }
+        RecordBuffer tmpBuff = buffer;
+        this.buffer = new RecordBuffer();
+        flush(tmpBuff);
+    }
+
+    public abstract void commit();
+
+    protected void flush(final RecordBuffer buff) {
+        if (buff == null || buff.isEmpty()) {
+            return;
+        }
+        connectMonitor.addAndGetTotalSizeOfData(buff.getBufferSizeBytes());
+        connectMonitor.addAndGetTotalNumberOfRecord(buff.getNumOfRecords());
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/LabelGenerator.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/LabelGenerator.java
new file mode 100644
index 0000000..ef223a3
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/LabelGenerator.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.rocketmq.connect.doris.writer;
+
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Generator label for stream load.
+ */
+public class LabelGenerator {
+    private final String topic;
+    private final String tableIdentifier;
+    // The label of doris stream load cannot be repeated when loading.
+    // Under special circumstances (usually load failure) when doris-kafka-connector is started,
+    // stream load is performed at the same offset every time, which will cause label duplication.
+    // For this reason, we use labelRandomSuffix to generate a random suffix at startup.
+    private final AtomicLong labelRandomSuffix;
+
+    public LabelGenerator(String topic, String tableIdentifier) {
+        // The label of stream load can not contain `.`
+        this.tableIdentifier = tableIdentifier.replaceAll("\\.", "_");
+        this.topic = topic.replaceAll("\\.", "_");
+        Random random = new Random();
+        labelRandomSuffix = new AtomicLong(random.nextInt(1000));
+    }
+
+    public String generateLabel(long lastOffset) {
+        return topic +
+            LoadConstants.FILE_DELIM_DEFAULT +
+            tableIdentifier +
+            LoadConstants.FILE_DELIM_DEFAULT +
+            lastOffset +
+            LoadConstants.FILE_DELIM_DEFAULT +
+            labelRandomSuffix.getAndIncrement();
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/LoadConstants.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/LoadConstants.java
new file mode 100644
index 0000000..5f6ec2c
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/LoadConstants.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.rocketmq.connect.doris.writer;
+
+public class LoadConstants {
+    public static final String DORIS_DELETE_SIGN = "__DORIS_DELETE_SIGN__";
+    public static final String DELETE_KET = "__deleted";
+    public static final String DORIS_DEL_TRUE = "1";
+    public static final String DORIS_DEL_FALSE = "0";
+
+    // Special identifier, label separator used for kafka-connect sink data
+    public static final String FILE_DELIM_DEFAULT = "__KC_";
+
+    // since apache doris 2.1.0, support stream load with group commit mode.
+    public static final String GROUP_COMMIT = "group_commit";
+    public static final String PARTIAL_COLUMNS = "partial_columns";
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/LoadStatus.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/LoadStatus.java
new file mode 100644
index 0000000..e986481
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/LoadStatus.java
@@ -0,0 +1,30 @@
+/*
+ * 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.rocketmq.connect.doris.writer;
+
+/**
+ * enum of LoadStatus.
+ */
+public class LoadStatus {
+    public static final String SUCCESS = "Success";
+    public static final String PUBLISH_TIMEOUT = "Publish Timeout";
+    public static final String LABEL_ALREADY_EXIST = "Label Already Exists";
+    public static final String FAIL = "Fail";
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/PartitionBuffer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/PartitionBuffer.java
new file mode 100644
index 0000000..046eef3
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/PartitionBuffer.java
@@ -0,0 +1,141 @@
+/*
+ * 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.rocketmq.connect.doris.writer;
+
+/**
+ * Abstract class which holds buffered data per partition including its start offset, end offset,
+ * size.
+ *
+ * <p>The getData() method returns the data specific to the implementation.
+ *
+ * <p>Buffer stores the converted records to Json format.
+ *
+ * <p>Long lived buffer would mean the data in partition would stay across two put APIs since the
+ * buffer thresholds were not met.
+ *
+ * <p>Please check respective implementation class for more details.
+ *
+ * @param <T> Return type of {@link #getData()}
+ */
+public abstract class PartitionBuffer<T> {
+    private int numOfRecords;
+    private long bufferSizeBytes;
+    private long firstOffset;
+    private long lastOffset;
+
+    /**
+     * @return Number of records in this buffer
+     */
+    public int getNumOfRecords() {
+        return numOfRecords;
+    }
+
+    /**
+     * @return Buffer size in bytes
+     */
+    public long getBufferSizeBytes() {
+        return bufferSizeBytes;
+    }
+
+    /**
+     * @return First offset number in this buffer
+     */
+    public long getFirstOffset() {
+        return firstOffset;
+    }
+
+    /**
+     * @return Last offset number in this buffer
+     */
+    public long getLastOffset() {
+        return lastOffset;
+    }
+
+    /**
+     * @param numOfRecords Updates number of records (Usually by 1)
+     */
+    public void setNumOfRecords(int numOfRecords) {
+        this.numOfRecords = numOfRecords;
+    }
+
+    /**
+     * @param bufferSizeBytes Updates sum of size of records present in this buffer (Bytes)
+     */
+    public void setBufferSizeBytes(long bufferSizeBytes) {
+        this.bufferSizeBytes = bufferSizeBytes;
+    }
+
+    /**
+     * @param firstOffset First offset no to set in this buffer
+     */
+    public void setFirstOffset(long firstOffset) {
+        this.firstOffset = firstOffset;
+    }
+
+    /**
+     * @param lastOffset Last offset no to set in this buffer
+     */
+    public void setLastOffset(long lastOffset) {
+        this.lastOffset = lastOffset;
+    }
+
+    /**
+     * @return true if buffer is empty
+     */
+    public boolean isEmpty() {
+        return numOfRecords == 0;
+    }
+
+    /**
+     * Public constructor.
+     */
+    public PartitionBuffer() {
+        numOfRecords = 0;
+        bufferSizeBytes = 0;
+        firstOffset = -1;
+        lastOffset = -1;
+    }
+
+    /**
+     * Inserts the row into Buffer.
+     */
+    public abstract void insert(String record);
+
+    /**
+     * Return the data that was buffered because buffer threshold might have been reached
+     *
+     * @return respective data type implemented by the class.
+     */
+    public abstract T getData();
+
+    @Override
+    public String toString() {
+        return "PartitionBuffer{"
+            + "numOfRecords="
+            + numOfRecords
+            + ", bufferSizeBytes="
+            + bufferSizeBytes
+            + ", firstOffset="
+            + firstOffset
+            + ", lastOffset="
+            + lastOffset
+            + '}';
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/RecordBuffer.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/RecordBuffer.java
new file mode 100644
index 0000000..62f522c
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/RecordBuffer.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.rocketmq.connect.doris.writer;
+
+import java.nio.charset.StandardCharsets;
+import java.util.StringJoiner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * record buffer
+ */
+public class RecordBuffer extends PartitionBuffer<String> {
+    private static final Logger LOG = LoggerFactory.getLogger(RecordBuffer.class);
+    public static final String LINE_SEPARATOR = "\n";
+    private final StringJoiner buffer;
+
+    public RecordBuffer() {
+        super();
+        buffer = new StringJoiner(LINE_SEPARATOR);
+    }
+
+    @Override
+    public void insert(String record) {
+        buffer.add(record);
+        setNumOfRecords(getNumOfRecords() + 1);
+        setBufferSizeBytes(getBufferSizeBytes() + record.getBytes(StandardCharsets.UTF_8).length);
+    }
+
+    public String getData() {
+        String result = buffer.toString();
+        LOG.debug(
+            "flush buffer: {} records, {} bytes, offset {} - {}",
+            getNumOfRecords(),
+            getBufferSizeBytes(),
+            getFirstOffset(),
+            getLastOffset());
+        return result;
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/ResponseUtil.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/ResponseUtil.java
new file mode 100644
index 0000000..43aacd4
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/ResponseUtil.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rocketmq.connect.doris.writer;
+
+import java.util.regex.Pattern;
+
+/**
+ * util for handle response.
+ */
+public class ResponseUtil {
+    public static final Pattern LABEL_EXIST_PATTERN =
+        Pattern.compile("Label \\[(.*)\\] has already been used, relate to txn \\[(\\d+)\\]");
+    public static final Pattern COMMITTED_PATTERN =
+        Pattern.compile(
+            "transaction \\[(\\d+)\\] is already \\b(COMMITTED|committed|VISIBLE|visible)\\b, not pre-committed.");
+
+    public static boolean isCommitted(String msg) {
+        return COMMITTED_PATTERN.matcher(msg).find();
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/StreamLoadWriter.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/StreamLoadWriter.java
new file mode 100644
index 0000000..fa64e2c
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/StreamLoadWriter.java
@@ -0,0 +1,190 @@
+/*
+ * 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.rocketmq.connect.doris.writer;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.connection.ConnectionProvider;
+import org.apache.rocketmq.connect.doris.exception.StreamLoadException;
+import org.apache.rocketmq.connect.doris.metrics.DorisConnectMonitor;
+import org.apache.rocketmq.connect.doris.model.KafkaRespContent;
+import org.apache.rocketmq.connect.doris.service.RestService;
+import org.apache.rocketmq.connect.doris.utils.BackendUtils;
+import org.apache.rocketmq.connect.doris.utils.FileNameUtils;
+import org.apache.rocketmq.connect.doris.writer.commit.DorisCommittable;
+import org.apache.rocketmq.connect.doris.writer.commit.DorisCommitter;
+import org.apache.rocketmq.connect.doris.writer.load.DorisStreamLoad;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Use stream-load to import data into doris.
+ */
+public class StreamLoadWriter extends DorisWriter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(StreamLoadWriter.class);
+    private static final String TRANSACTION_LABEL_PATTEN = "SHOW TRANSACTION FROM %s WHERE LABEL LIKE '";
+    private final LabelGenerator labelGenerator;
+    private final DorisCommitter dorisCommitter;
+    private final DorisStreamLoad dorisStreamLoad;
+    private List<DorisCommittable> committableList = new LinkedList<>();
+
+    public StreamLoadWriter(
+        String topic,
+        DorisOptions dorisOptions,
+        ConnectionProvider connectionProvider,
+        DorisConnectMonitor connectMonitor) {
+        super(topic, dorisOptions, connectionProvider, connectMonitor);
+        this.taskId = dorisOptions.getTaskId();
+        this.labelGenerator = new LabelGenerator(topic, tableIdentifier);
+        BackendUtils backendUtils = BackendUtils.getInstance(dorisOptions, LOG);
+        this.dorisCommitter = new DorisCommitter(dorisOptions, backendUtils);
+        this.dorisStreamLoad = new DorisStreamLoad(backendUtils, dorisOptions, topic);
+        checkDorisTableKey(tableName);
+    }
+
+    /**
+     * The uniq model has 2pc close by default unless 2pc is forced open.
+     */
+    @VisibleForTesting
+    public void checkDorisTableKey(String tableName) {
+        if (dorisOptions.enable2PC()
+            && !dorisOptions.force2PC()
+            && RestService.isUniqueKeyType(dorisOptions, tableName, LOG)) {
+            LOG.info(
+                "The {} table type is unique model, the two phase commit default value should be disabled.",
+                tableName);
+            dorisOptions.setEnable2PC(false);
+        }
+    }
+
+    public void fetchOffset() {
+        Map<String, String> label2Status = fetchLabel2Status();
+        long maxOffset = -1;
+        for (Map.Entry<String, String> entry : label2Status.entrySet()) {
+            String label = entry.getKey();
+            String status = entry.getValue();
+            if (status.equalsIgnoreCase("VISIBLE")) {
+                long offset = FileNameUtils.labelToEndOffset(label);
+                if (offset > maxOffset) {
+                    maxOffset = offset;
+                }
+            }
+        }
+        this.offsetPersistedInDoris.set(maxOffset);
+        LOG.info("Init {} offset of {} topic.", maxOffset, topic);
+    }
+
+    private Map<String, String> fetchLabel2Status() {
+        String queryPatten = String.format(TRANSACTION_LABEL_PATTEN, dorisOptions.getDatabase());
+        String tmpTableIdentifier = tableIdentifier.replaceAll("\\.", "_");
+        String tmpTopic = topic.replaceAll("\\.", "_");
+        String querySQL =
+            queryPatten
+                + tmpTopic
+                + LoadConstants.FILE_DELIM_DEFAULT
+                + tmpTableIdentifier
+                + LoadConstants.FILE_DELIM_DEFAULT
+                + "%'";
+        LOG.info("query doris offset by sql: {}", querySQL);
+        Map<String, String> label2Status = new HashMap<>();
+        try (Connection connection = connectionProvider.getOrEstablishConnection();
+             PreparedStatement ps = connection.prepareStatement(querySQL);
+             ResultSet rs = ps.executeQuery()) {
+            while (rs.next()) {
+                String label = rs.getString("Label");
+                String transactionStatus = rs.getString("TransactionStatus");
+                label2Status.put(label, transactionStatus);
+            }
+        } catch (Exception e) {
+            LOG.warn(
+                "Unable to obtain the label generated when importing data through stream load from doris, "
+                    + "causing the doris kafka connector to not guarantee exactly once.",
+                e);
+            throw new StreamLoadException(
+                "Unable to obtain the label generated when importing data through stream load from doris, "
+                    + "causing the doris kafka connector to not guarantee exactly once.",
+                e);
+        }
+        return label2Status;
+    }
+
+    @Override
+    public void insert(ConnectRecord record) {
+        initRecord();
+        insertRecord(record);
+    }
+
+    protected void flush(final RecordBuffer buff) {
+        super.flush(buff);
+        try {
+            String label = labelGenerator.generateLabel(buff.getLastOffset());
+            dorisStreamLoad.load(label, buff);
+        } catch (IOException e) {
+            LOG.warn(
+                "Failed to load buffer. buffNumOfRecords={}, lastOffset={}",
+                buff.getNumOfRecords(),
+                buff.getLastOffset());
+            throw new StreamLoadException(e);
+        }
+
+        updateFlushedMetrics(buff);
+    }
+
+    @Override
+    public void commit() {
+        // Doris commit
+        Queue<KafkaRespContent> respContents = dorisStreamLoad.getKafkaRespContents();
+        while (!respContents.isEmpty()) {
+            KafkaRespContent respContent = respContents.poll();
+            DorisCommittable dorisCommittable =
+                new DorisCommittable(
+                    dorisStreamLoad.getHostPort(),
+                    respContent.getDatabase(),
+                    respContent.getTxnId(),
+                    respContent.getLastOffset(),
+                    respContent.getTopic(),
+                    respContent.getTable());
+            committableList.add(dorisCommittable);
+        }
+        dorisStreamLoad.setKafkaRespContents(new LinkedList<>());
+        dorisCommitter.commit(committableList);
+        updateCommitOffset();
+    }
+
+    private void updateCommitOffset() {
+        // committedOffset should be updated only when stream load has succeeded.
+        committedOffset.set(flushedOffset.get());
+        connectMonitor.setCommittedOffset(committedOffset.get() - 1);
+
+        committableList = new LinkedList<>();
+    }
+
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/commit/DorisCommittable.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/commit/DorisCommittable.java
new file mode 100644
index 0000000..f60c811
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/commit/DorisCommittable.java
@@ -0,0 +1,93 @@
+/*
+ * 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.rocketmq.connect.doris.writer.commit;
+
+import java.util.Objects;
+
+/**
+ * DorisCommittable hold the info for Committer to commit.
+ */
+public class DorisCommittable {
+    private final String hostPort;
+    private final String db;
+    private final long txnID;
+    private final long lastOffset;
+    private final String topic;
+    private final String table;
+
+    public DorisCommittable(
+        String hostPort,
+        String db,
+        long txnID,
+        long lastOffset,
+        String topic,
+        String table) {
+        this.hostPort = hostPort;
+        this.db = db;
+        this.txnID = txnID;
+        this.lastOffset = lastOffset;
+        this.topic = topic;
+        this.table = table;
+    }
+
+    public String getHostPort() {
+        return hostPort;
+    }
+
+    public String getDb() {
+        return db;
+    }
+
+    public long getTxnID() {
+        return txnID;
+    }
+
+    public long getLastOffset() {
+        return lastOffset;
+    }
+
+    public String getTable() {
+        return table;
+    }
+
+    public String getTopic() {
+        return topic;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        DorisCommittable that = (DorisCommittable) o;
+        return txnID == that.txnID
+            && Objects.equals(hostPort, that.hostPort)
+            && Objects.equals(db, that.db);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(hostPort, db, txnID);
+    }
+
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/commit/DorisCommitter.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/commit/DorisCommitter.java
new file mode 100644
index 0000000..b043a3c
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/commit/DorisCommitter.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rocketmq.connect.doris.writer.commit;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.util.EntityUtils;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.exception.StreamLoadException;
+import org.apache.rocketmq.connect.doris.model.LoadOperation;
+import org.apache.rocketmq.connect.doris.utils.BackendUtils;
+import org.apache.rocketmq.connect.doris.utils.BackoffAndRetryUtils;
+import org.apache.rocketmq.connect.doris.utils.HttpPutBuilder;
+import org.apache.rocketmq.connect.doris.utils.HttpUtils;
+import org.apache.rocketmq.connect.doris.writer.LoadStatus;
+import org.apache.rocketmq.connect.doris.writer.ResponseUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DorisCommitter {
+    private static final Logger LOG = LoggerFactory.getLogger(DorisCommitter.class);
+    private static final String COMMIT_PATTERN = "http://%s/api/%s/_stream_load_2pc";
+    private final ObjectMapper objectMapper = new ObjectMapper();
+    private final CloseableHttpClient httpClient;
+    private final BackendUtils backendUtils;
+    private final DorisOptions dorisOptions;
+
+    public DorisCommitter(DorisOptions dorisOptions, BackendUtils backendUtils) {
+        this.backendUtils = backendUtils;
+        this.dorisOptions = dorisOptions;
+        this.httpClient = new HttpUtils(dorisOptions).getHttpClient();
+    }
+
+    public void commit(List<DorisCommittable> dorisCommittables) {
+        if (!dorisOptions.enable2PC() || dorisCommittables.isEmpty()) {
+            return;
+        }
+        for (DorisCommittable dorisCommittable : dorisCommittables) {
+            try {
+                commitTransaction(dorisCommittable);
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    private void commitTransaction(DorisCommittable committable) throws IOException {
+        // basic params
+        HttpPutBuilder builder =
+            new HttpPutBuilder()
+                .addCommonHeader()
+                .baseAuth(dorisOptions.getUser(), dorisOptions.getPassword())
+                .addTxnId(committable.getTxnID())
+                .commit();
+
+        AtomicReference<String> hostPort = new AtomicReference<>(committable.getHostPort());
+        try {
+            BackoffAndRetryUtils.backoffAndRetry(
+                LoadOperation.COMMIT_TRANSACTION,
+                () -> {
+                    // get latest-url
+                    LOG.info(
+                        "commit txn {} to host {}", committable.getTxnID(), hostPort.get());
+                    String url =
+                        String.format(COMMIT_PATTERN, hostPort.get(), committable.getDb());
+                    HttpPut httpPut = builder.setUrl(url).setEmptyEntity().build();
+
+                    // http execute...
+                    try (CloseableHttpResponse response = httpClient.execute(httpPut)) {
+                        StatusLine statusLine = response.getStatusLine();
+                        if (200 == statusLine.getStatusCode()) {
+                            String loadResult = null;
+                            if (response.getEntity() != null) {
+                                loadResult = EntityUtils.toString(response.getEntity());
+                                Map<String, String> res =
+                                    objectMapper.readValue(
+                                        loadResult,
+                                        new TypeReference<
+                                            HashMap<String, String>>() {
+                                        });
+                                if (!res.get("status").equals(LoadStatus.SUCCESS)
+                                    && !ResponseUtil.isCommitted(res.get("msg"))) {
+                                    throw new StreamLoadException(
+                                        "commit transaction failed " + loadResult);
+                                }
+                            }
+                            LOG.info("load result {}", loadResult);
+                            return true;
+                        }
+                        String reasonPhrase = statusLine.getReasonPhrase();
+                        LOG.error(
+                            "commit failed with {}, reason {}",
+                            hostPort.get(),
+                            reasonPhrase);
+                        hostPort.set(backendUtils.getAvailableBackend());
+                        throw new StreamLoadException(
+                            "commit failed with {"
+                                + hostPort.get()
+                                + "}, reason {"
+                                + reasonPhrase
+                                + "}");
+                    } catch (Exception e) {
+                        LOG.error("commit transaction failed, to retry, {}", e.getMessage());
+                        hostPort.set(backendUtils.getAvailableBackend());
+                        throw new StreamLoadException("commit transaction failed.", e);
+                    }
+                });
+        } catch (Exception e) {
+            LOG.error("commit transaction error:", e);
+            throw new StreamLoadException("commit transaction error: " + e);
+        }
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/CopyLoad.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/CopyLoad.java
new file mode 100644
index 0000000..108961e
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/CopyLoad.java
@@ -0,0 +1,274 @@
+/*
+ * 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.rocketmq.connect.doris.writer.load;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
+import org.apache.http.Header;
+import org.apache.http.HttpEntity;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.entity.ByteArrayEntity;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.util.EntityUtils;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.exception.CopyLoadException;
+import org.apache.rocketmq.connect.doris.exception.UploadException;
+import org.apache.rocketmq.connect.doris.model.BaseResponse;
+import org.apache.rocketmq.connect.doris.model.CopyIntoResp;
+import org.apache.rocketmq.connect.doris.model.LoadOperation;
+import org.apache.rocketmq.connect.doris.utils.BackoffAndRetryUtils;
+import org.apache.rocketmq.connect.doris.utils.HttpPostBuilder;
+import org.apache.rocketmq.connect.doris.utils.HttpPutBuilder;
+import org.apache.rocketmq.connect.doris.utils.HttpUtils;
+import org.apache.rocketmq.connect.doris.writer.CopySQLBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CopyLoad extends DataLoad {
+    private static final long serialVersionUID = 1L;
+    private static final Logger LOG = LoggerFactory.getLogger(CopyLoad.class);
+    public static final int SUCCESS = 0;
+    public static final String FAIL = "1";
+    private static final Pattern COMMITTED_PATTERN =
+        Pattern.compile(
+            "errCode = 2, detailMessage = No files can be copied, matched (\\d+) files, "
+                + "filtered (\\d+) files because files may be loading or loaded");
+    private static final String UPLOAD_URL_PATTERN = "http://%s/copy/upload";
+    private static final String COMMIT_PATTERN = "http://%s/copy/query";
+    private final String loadUrlStr;
+    private final String hostPort;
+    private final DorisOptions dorisOptions;
+    private final CloseableHttpClient httpClient;
+
+    public CopyLoad(String database, String tableName, DorisOptions dorisOptions) {
+        this(
+            database,
+            tableName,
+            dorisOptions,
+            new HttpUtils(dorisOptions).getHttpClient());
+    }
+
+    public CopyLoad(
+        String database,
+        String tableName,
+        DorisOptions dorisOptions,
+        CloseableHttpClient httpClient) {
+        this.database = database;
+        this.table = tableName;
+        this.hostPort = dorisOptions.getUrls() + ":" + dorisOptions.getHttpPort();
+        this.loadUrlStr = String.format(UPLOAD_URL_PATTERN, hostPort);
+        this.dorisOptions = dorisOptions;
+        this.httpClient = httpClient;
+    }
+
+    public void uploadFile(String fileName, String value) {
+        String address = getUploadAddress(fileName);
+        upLoadFile(address, value, fileName);
+    }
+
+    /**
+     * execute copy into
+     */
+    public boolean executeCopy(List<String> fileList) {
+        long start = System.currentTimeMillis();
+        CopySQLBuilder copySQLBuilder =
+            new CopySQLBuilder(database, table, fileList, dorisOptions.isEnableDelete());
+        String copySQL = copySQLBuilder.buildCopySQL();
+        LOG.info("build copy SQL is {}", copySQL);
+        Map<String, String> params = new HashMap<>();
+        params.put("sql", copySQL);
+        try {
+            BackoffAndRetryUtils.backoffAndRetry(
+                LoadOperation.EXECUTE_COPY,
+                () -> {
+                    HttpPostBuilder postBuilder = new HttpPostBuilder();
+                    postBuilder
+                        .setUrl(String.format(COMMIT_PATTERN, hostPort))
+                        .baseAuth(dorisOptions.getUser(), dorisOptions.getPassword())
+                        .setEntity(
+                            new StringEntity(OBJECT_MAPPER.writeValueAsString(params)));
+
+                    try (CloseableHttpResponse response =
+                             httpClient.execute(postBuilder.build())) {
+                        final int statusCode = response.getStatusLine().getStatusCode();
+                        final String reasonPhrase = response.getStatusLine().getReasonPhrase();
+                        String loadResult = "";
+                        if (statusCode != 200) {
+                            LOG.warn(
+                                "commit failed with status {} {}, reason {}",
+                                statusCode,
+                                hostPort,
+                                reasonPhrase);
+                            throw new CopyLoadException(
+                                "commit file failed, cause by: " + reasonPhrase);
+                        } else if (response.getEntity() != null) {
+                            loadResult = EntityUtils.toString(response.getEntity());
+                            boolean success = handleCommitResponse(loadResult);
+                            if (success) {
+                                LOG.info(
+                                    "commit success cost {}ms, response is {}",
+                                    System.currentTimeMillis() - start,
+                                    loadResult);
+                                return true;
+                            }
+                        }
+                        LOG.error("commit failed, cause by: " + loadResult);
+                        throw new CopyLoadException("commit failed, cause by: " + loadResult);
+                    }
+                });
+        } catch (Exception e) {
+            String errMsg = "failed to execute copy, sql=" + copySQL;
+            throw new CopyLoadException(errMsg, e);
+        }
+        return true;
+    }
+
+    public boolean handleCommitResponse(String loadResult) throws IOException {
+        BaseResponse<CopyIntoResp> baseResponse =
+            OBJECT_MAPPER.readValue(
+                loadResult, new TypeReference<BaseResponse<CopyIntoResp>>() {
+                });
+        if (baseResponse.getCode() == SUCCESS) {
+            CopyIntoResp dataResp = baseResponse.getData();
+            if (FAIL.equals(dataResp.getDataCode())) {
+                LOG.error("copy into execute failed, reason:{}", loadResult);
+                return false;
+            } else {
+                Map<String, String> result = dataResp.getResult();
+                if (!result.get("state").equals("FINISHED") && !isCommitted(result.get("msg"))) {
+                    LOG.error("copy into load failed, reason:{}", loadResult);
+                    return false;
+                } else {
+                    return true;
+                }
+            }
+        } else {
+            LOG.error("commit failed, reason:{}", loadResult);
+            return false;
+        }
+    }
+
+    public static boolean isCommitted(String msg) {
+        return COMMITTED_PATTERN.matcher(msg).matches();
+    }
+
+    /**
+     * Upload File
+     */
+    public void upLoadFile(String address, String value, String fileName) {
+        HttpPutBuilder putBuilder = new HttpPutBuilder();
+        putBuilder
+            .setUrl(address)
+            .addCommonHeader()
+            .setEntity(new ByteArrayEntity(value.getBytes(StandardCharsets.UTF_8)));
+        try {
+            BackoffAndRetryUtils.backoffAndRetry(
+                LoadOperation.UPLOAD_FILE,
+                () -> {
+                    long start = System.currentTimeMillis();
+                    try (CloseableHttpResponse response =
+                             httpClient.execute(putBuilder.build())) {
+                        final int statusCode = response.getStatusLine().getStatusCode();
+                        if (statusCode != 200) {
+                            String result =
+                                response.getEntity() == null
+                                    ? null
+                                    : EntityUtils.toString(response.getEntity());
+                            LOG.error("upload file {} error, response {}", fileName, result);
+                            throw new UploadException("upload file error: " + fileName);
+                        }
+                        LOG.info(
+                            "upload file success cost {}ms",
+                            System.currentTimeMillis() - start);
+                        return true;
+                    }
+                });
+        } catch (Exception e) {
+            String errMsg = "Failed to upload file, filename=" + fileName + ", address=" + address;
+            throw new UploadException(errMsg, e);
+        }
+    }
+
+    /**
+     * Get the redirected s3 address
+     */
+    public String getUploadAddress(String fileName) {
+        HttpPutBuilder putBuilder = new HttpPutBuilder();
+        putBuilder
+            .setUrl(loadUrlStr)
+            .addFileName(fileName)
+            .addCommonHeader()
+            .setEmptyEntity()
+            .baseAuth(dorisOptions.getUser(), dorisOptions.getPassword());
+
+        AtomicReference<String> uploadAddress = new AtomicReference<>();
+        try {
+            BackoffAndRetryUtils.backoffAndRetry(
+                LoadOperation.GET_UPLOAD_ADDRESS,
+                () -> {
+                    try (CloseableHttpResponse execute =
+                             httpClient.execute(putBuilder.build())) {
+                        int statusCode = execute.getStatusLine().getStatusCode();
+                        String reason = execute.getStatusLine().getReasonPhrase();
+                        if (statusCode == 307) {
+                            Header location = execute.getFirstHeader("location");
+                            uploadAddress.set(location.getValue());
+                            LOG.info("redirect to s3:{}", uploadAddress.get());
+                            return true;
+                        }
+                        HttpEntity entity = execute.getEntity();
+                        String result = entity == null ? null : EntityUtils.toString(entity);
+                        LOG.error(
+                            "Failed get the redirected address, status {}, reason {}, response {}",
+                            statusCode,
+                            reason,
+                            result);
+                        throw new UploadException("Could not get the redirected address.");
+                    }
+                });
+        } catch (Exception e) {
+            String errMsg =
+                "Failed to get redirected upload address, fileName="
+                    + fileName
+                    + ", loadUrlStr="
+                    + loadUrlStr;
+            throw new UploadException(errMsg, e);
+        }
+        return uploadAddress.get();
+    }
+
+    public void close() throws IOException {
+        if (null != httpClient) {
+            try {
+                httpClient.close();
+            } catch (IOException e) {
+                LOG.error("Closing httpClient failed.", e);
+                throw new RuntimeException("Closing httpClient failed.", e);
+            }
+        }
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/DataLoad.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/DataLoad.java
new file mode 100644
index 0000000..cf9fd71
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/DataLoad.java
@@ -0,0 +1,31 @@
+/*
+ * 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.rocketmq.connect.doris.writer.load;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.Serializable;
+
+public abstract class DataLoad implements Serializable {
+    protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    protected String database;
+    protected String table;
+    protected String user;
+    protected String password;
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/DorisStreamLoad.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/DorisStreamLoad.java
new file mode 100644
index 0000000..5bfce7d
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/DorisStreamLoad.java
@@ -0,0 +1,149 @@
+/*
+ * 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.rocketmq.connect.doris.writer.load;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.entity.ByteArrayEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.util.EntityUtils;
+import org.apache.rocketmq.connect.doris.cfg.DorisOptions;
+import org.apache.rocketmq.connect.doris.exception.StreamLoadException;
+import org.apache.rocketmq.connect.doris.model.KafkaRespContent;
+import org.apache.rocketmq.connect.doris.utils.BackendUtils;
+import org.apache.rocketmq.connect.doris.utils.HttpPutBuilder;
+import org.apache.rocketmq.connect.doris.utils.HttpUtils;
+import org.apache.rocketmq.connect.doris.writer.LoadStatus;
+import org.apache.rocketmq.connect.doris.writer.RecordBuffer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DorisStreamLoad extends DataLoad {
+    private static final Logger LOG = LoggerFactory.getLogger(DorisStreamLoad.class);
+    private static final String LOAD_URL_PATTERN = "http://%s/api/%s/%s/_stream_load";
+    private static final List<String> DORIS_SUCCESS_STATUS =
+        new ArrayList<>(Arrays.asList(LoadStatus.SUCCESS, LoadStatus.PUBLISH_TIMEOUT));
+    private String loadUrl;
+    private final DorisOptions dorisOptions;
+    private final String topic;
+    private String hostPort;
+    private final CloseableHttpClient httpClient;
+    private final BackendUtils backendUtils;
+    private Queue<KafkaRespContent> respContents = new LinkedList<>();
+    private final boolean enableGroupCommit;
+
+    public DorisStreamLoad(BackendUtils backendUtils, DorisOptions dorisOptions, String topic) {
+        this.database = dorisOptions.getDatabase();
+        this.table = dorisOptions.getTopicMapTable(topic);
+        this.user = dorisOptions.getUser();
+        this.password = dorisOptions.getPassword();
+        this.loadUrl = String.format(LOAD_URL_PATTERN, hostPort, database, table);
+        this.dorisOptions = dorisOptions;
+        this.backendUtils = backendUtils;
+        this.topic = topic;
+        this.enableGroupCommit = dorisOptions.enableGroupCommit();
+        this.httpClient = new HttpUtils(dorisOptions).getHttpClient();
+    }
+
+    /**
+     * execute stream load.
+     */
+    public void load(String label, RecordBuffer buffer) throws IOException {
+        if (enableGroupCommit) {
+            label = null;
+        }
+
+        refreshLoadUrl(database, table);
+        String data = buffer.getData();
+        ByteArrayEntity entity = new ByteArrayEntity(data.getBytes(StandardCharsets.UTF_8));
+        HttpPutBuilder putBuilder = new HttpPutBuilder();
+        putBuilder
+            .setUrl(loadUrl)
+            .baseAuth(user, password)
+            .setLabel(label)
+            .addCommonHeader()
+            .setEntity(entity)
+            .addHiddenColumns(dorisOptions.isEnableDelete())
+            .enable2PC(dorisOptions.enable2PC())
+            .addProperties(dorisOptions.getStreamLoadProp());
+
+        if (enableGroupCommit) {
+            LOG.info("stream load started with group commit on host {}", hostPort);
+        } else {
+            LOG.info("stream load started for {} on host {}", label, hostPort);
+        }
+
+        LOG.info("stream load started for {} on host {}", label, hostPort);
+        try (CloseableHttpResponse response = httpClient.execute(putBuilder.build())) {
+            int statusCode = response.getStatusLine().getStatusCode();
+            if (statusCode == 200 && response.getEntity() != null) {
+                String loadResult = EntityUtils.toString(response.getEntity());
+                LOG.info("load Result {}", loadResult);
+                KafkaRespContent respContent =
+                    OBJECT_MAPPER.readValue(loadResult, KafkaRespContent.class);
+                if (!DORIS_SUCCESS_STATUS.contains(respContent.getStatus())) {
+                    String errMsg =
+                        String.format(
+                            "stream load error: %s, see more in %s",
+                            respContent.getMessage(), respContent.getErrorURL());
+                    throw new StreamLoadException(errMsg);
+                }
+                respContent.setDatabase(database);
+                respContent.setTable(table);
+                respContent.setLastOffset(buffer.getLastOffset());
+                respContent.setTopic(topic);
+                respContents.add(respContent);
+                return;
+            }
+        } catch (Exception ex) {
+            String err;
+            if (enableGroupCommit) {
+                err = "failed to stream load data with group commit";
+            } else {
+                err = "failed to stream load data with label: " + label;
+            }
+            LOG.warn(err, ex);
+            throw new StreamLoadException(err, ex);
+        }
+    }
+
+    public Queue<KafkaRespContent> getKafkaRespContents() {
+        return respContents;
+    }
+
+    public void setKafkaRespContents(Queue<KafkaRespContent> respContents) {
+        this.respContents = respContents;
+    }
+
+    public String getHostPort() {
+        return hostPort;
+    }
+
+    private void refreshLoadUrl(String database, String table) {
+        hostPort = backendUtils.getAvailableBackend();
+        loadUrl = String.format(LOAD_URL_PATTERN, hostPort, database, table);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/GroupCommitMode.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/GroupCommitMode.java
new file mode 100644
index 0000000..3a20d31
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/GroupCommitMode.java
@@ -0,0 +1,28 @@
+package org.apache.rocketmq.connect.doris.writer.load;
+
+import java.util.Arrays;
+import java.util.List;
+
+public enum GroupCommitMode {
+    OFF_MODE("off_mode"),
+    SYNC_MODE("sync_mode"),
+    ASYNC_MODE("async_mode");
+
+    private final String name;
+
+    GroupCommitMode(String name) {
+        this.name = name;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public static LoadModel of(String name) {
+        return LoadModel.valueOf(name.toUpperCase());
+    }
+
+    public static List<String> instances() {
+        return Arrays.asList(OFF_MODE.name, SYNC_MODE.name, ASYNC_MODE.name);
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/LoadModel.java b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/LoadModel.java
new file mode 100644
index 0000000..f058b1d
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/LoadModel.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.rocketmq.connect.doris.writer.load;
+
+public enum LoadModel {
+    STREAM_LOAD("stream_load"),
+
+    COPY_INTO("copy_into");
+
+    private final String name;
+
+    LoadModel(String name) {
+        this.name = name;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public static LoadModel of(String name) {
+        return LoadModel.valueOf(name.toUpperCase());
+    }
+
+    public static String[] instances() {
+        return new String[] {STREAM_LOAD.name, COPY_INTO.name};
+    }
+}
diff --git a/connectors/rocketmq-connect-doris/src/test/java/org/apache/rocketmq/connect/doris/DorisSinkTaskTest.java b/connectors/rocketmq-connect-doris/src/test/java/org/apache/rocketmq/connect/doris/DorisSinkTaskTest.java
new file mode 100644
index 0000000..9451600
--- /dev/null
+++ b/connectors/rocketmq-connect-doris/src/test/java/org/apache/rocketmq/connect/doris/DorisSinkTaskTest.java
@@ -0,0 +1,156 @@
+/*
+ * 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.rocketmq.connect.doris;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.RecordOffset;
+import io.openmessaging.connector.api.data.RecordPartition;
+import io.openmessaging.internal.DefaultKeyValue;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.rocketmq.connect.doris.cfg.DorisSinkConnectorConfig;
+import org.apache.rocketmq.connect.doris.utils.ConnectRecordUtil;
+import org.apache.rocketmq.connect.doris.writer.load.LoadModel;
+import org.apache.rocketmq.connect.runtime.converter.record.json.JsonConverter;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class DorisSinkTaskTest {
+
+    private final DorisSinkTask dorisSinkTask = new DorisSinkTask();
+    private final JsonConverter jsonConverter = new JsonConverter();
+    private KeyValue keyValue;
+    private RecordPartition recordPartition;
+    private RecordOffset offset;
+
+    @Before
+    public void init() {
+        keyValue = new DefaultKeyValue();
+        keyValue.put("task_id", "0");
+        keyValue.put(DorisSinkConnectorConfig.DORIS_URLS, "127.0.0.1");
+        keyValue.put(DorisSinkConnectorConfig.DORIS_USER, "root");
+        keyValue.put(DorisSinkConnectorConfig.DORIS_PASSWORD, "");
+        keyValue.put(DorisSinkConnectorConfig.DORIS_HTTP_PORT, "8030");
+        keyValue.put(DorisSinkConnectorConfig.DORIS_QUERY_PORT, "9030");
+        keyValue.put(DorisSinkConnectorConfig.DORIS_DATABASE, "test");
+        keyValue.put(DorisSinkConnectorConfig.TOPICS, "rmq_test");
+        keyValue.put(DorisSinkConnectorConfig.TOPICS_TABLES_MAP, "rmq_test:doris_tab");
+        keyValue.put(DorisSinkConnectorConfig.ENABLE_2PC, "false");
+        keyValue.put(DorisSinkConnectorConfig.BUFFER_COUNT_RECORDS, "1");
+        DorisSinkConnectorConfig.setDefaultValues(keyValue);
+
+        Map<String, String> partition = new HashMap<>();
+        partition.put(ConnectRecordUtil.TOPIC, "rmq_test");
+        partition.put(ConnectRecordUtil.BROKER_NAME, "broker_test");
+        partition.put(ConnectRecordUtil.QUEUE_ID, "111");
+        recordPartition = new RecordPartition(partition);
+
+        Map<String, String> queueOffset = new HashMap<>();
+        queueOffset.put("queueOffset", "1");
+        offset = new RecordOffset(queueOffset);
+        jsonConverter.configure(new HashMap<>());
+    }
+
+    @Test
+    public void testPut() {
+        keyValue.put(DorisSinkConnectorConfig.ENABLE_2PC, "false");
+        dorisSinkTask.start(keyValue);
+        List<ConnectRecord> connectRecordList = new ArrayList<>();
+        ConnectRecord connectRecord = new ConnectRecord(recordPartition, offset, System.currentTimeMillis());
+        connectRecord.setData("{\"id\":1,\"name\":\"lisi\",\"age\":12}");
+        connectRecordList.add(connectRecord);
+        connectRecordList.add(connectRecord);
+        dorisSinkTask.put(connectRecordList);
+    }
+
+    @Test
+    public void testPutAndFlush() {
+        dorisSinkTask.start(keyValue);
+
+        List<ConnectRecord> connectRecordList = new ArrayList<>();
+        ConnectRecord connectRecord = new ConnectRecord(recordPartition, offset, System.currentTimeMillis());
+        connectRecord.setData("{\"id\":2,\"name\":\"zhangsan\",\"age\":13}");
+        connectRecordList.add(connectRecord);
+        connectRecordList.add(connectRecord);
+        dorisSinkTask.put(connectRecordList);
+        Map<RecordPartition, RecordOffset> currentOffsets = new HashMap<>();
+        currentOffsets.put(recordPartition, offset);
+        dorisSinkTask.flush(currentOffsets);
+    }
+
+    @Test
+    public void testCustomClusterProxy() {
+        keyValue.put(DorisSinkConnectorConfig.DORIS_CUSTOM_CLUSTER, "true");
+        keyValue.put(DorisSinkConnectorConfig.SOCKS5_ENDPOINT, "");
+        keyValue.put(DorisSinkConnectorConfig.SOCKS5_USERNAME, "");
+        keyValue.put(DorisSinkConnectorConfig.SOCKET5_PASSWORD, "");
+        dorisSinkTask.start(keyValue);
+
+        List<ConnectRecord> connectRecordList = new ArrayList<>();
+        ConnectRecord connectRecord = new ConnectRecord(recordPartition, offset, System.currentTimeMillis());
+        connectRecord.setData("{\"id\":2,\"name\":\"zhangsan\",\"age\":13}");
+        connectRecordList.add(connectRecord);
+        connectRecordList.add(connectRecord);
+        dorisSinkTask.put(connectRecordList);
+        Map<RecordPartition, RecordOffset> currentOffsets = new HashMap<>();
+        currentOffsets.put(recordPartition, offset);
+        dorisSinkTask.flush(currentOffsets);
+    }
+
+    //    @Test
+    //    public void testDebeziumConverterPut() {
+    //        keyValue.put(DorisSinkConnectorConfig.CONVERTER_MODE, ConverterMode.DEBEZIUM_INGESTION.getName());
+    //        dorisSinkTask.start(keyValue);
+    //
+    //        List<ConnectRecord> connectRecordList = new ArrayList<>();
+    //        ConnectRecord connectRecord = new ConnectRecord(recordPartition, offset, System.currentTimeMillis());
+    //        String msg = "";
+    //        SchemaAndValue schemaAndValue = jsonConverter.toConnectData("a", msg.getBytes(StandardCharsets.UTF_8));
+    //
+    //        connectRecord.setData(schemaAndValue.value());
+    //        connectRecord.setSchema(schemaAndValue.schema());
+    //        connectRecordList.add(connectRecord);
+    //        connectRecordList.add(connectRecord);
+    //        dorisSinkTask.put(connectRecordList);
+    //    }
+
+    @Test
+    public void testCopyIntoLoad() {
+        keyValue.put(DorisSinkConnectorConfig.LOAD_MODEL, LoadModel.COPY_INTO.getName());
+        dorisSinkTask.start(keyValue);
+
+        List<ConnectRecord> connectRecordList = new ArrayList<>();
+        ConnectRecord connectRecord = new ConnectRecord(recordPartition, offset, System.currentTimeMillis());
+        connectRecord.setData("{\"id\":4,\"name\":\"zhaoliu\",\"age\":14}");
+        connectRecordList.add(connectRecord);
+        connectRecordList.add(connectRecord);
+        dorisSinkTask.put(connectRecordList);
+        Map<RecordPartition, RecordOffset> currentOffsets = new HashMap<>();
+        currentOffsets.put(recordPartition, offset);
+        dorisSinkTask.flush(currentOffsets);
+    }
+
+}