[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);
+ }
+
+}