From 85da68c92f599f0a289c0e63de919ca417289f69 Mon Sep 17 00:00:00 2001
From: wudongliang <46414265+DongLiang-0@users.noreply.github.com>
Date: Wed, 11 Dec 2024 09:58:45 +0800
Subject: [PATCH] [Improve]Improve doris connector (#539)
* [Improve]Improve doris connector
---
connectors/rocketmq-connect-doris/pom.xml | 68 +-
.../connect/doris/DorisSinkConnector.java | 66 ++
.../rocketmq/connect/doris/DorisSinkTask.java | 66 ++
.../connect/doris/cfg/DorisOptions.java | 387 ++++++++++++
.../doris/cfg/DorisSinkConnectorConfig.java | 134 ++++
.../connect/doris/config/AbstractConfig.java | 132 ----
.../doris/connection/ConnectionProvider.java | 35 ++
.../connection/JdbcConnectionProvider.java | 120 ++++
.../doris/connector/DorisSinkConfig.java | 331 ----------
.../doris/connector/DorisSinkConnector.java | 74 ---
.../doris/connector/DorisSinkTask.java | 107 ----
.../doris/converter/ConverterMode.java | 45 ++
.../doris/converter/RecordDescriptor.java | 258 ++++++++
.../doris/converter/RecordService.java | 334 ++++++++++
.../doris/converter/RecordTypeRegister.java | 122 ++++
.../converter/schema/SchemaChangeManager.java | 189 ++++++
.../converter/schema/SchemaEvolutionMode.java | 44 ++
.../converter/type/AbstractDateType.java | 34 +
.../converter/type/AbstractGeometryType.java | 30 +
.../converter/type/AbstractTemporalType.java | 54 ++
.../converter/type/AbstractTimeType.java | 51 ++
.../converter/type/AbstractTimestampType.java | 46 ++
.../doris/converter/type/AbstractType.java | 70 +++
.../connect/doris/converter/type/Type.java | 58 ++
.../type/connect/AbstractConnectMapType.java | 42 ++
.../connect/AbstractConnectSchemaType.java | 25 +
.../type/connect/ConnectBooleanType.java | 38 ++
.../type/connect/ConnectBytesType.java | 70 +++
.../type/connect/ConnectDateType.java | 49 ++
.../type/connect/ConnectDecimalType.java | 55 ++
.../type/connect/ConnectFloat32Type.java | 43 ++
.../type/connect/ConnectFloat64Type.java | 43 ++
.../type/connect/ConnectInt16Type.java | 43 ++
.../type/connect/ConnectInt32Type.java | 43 ++
.../type/connect/ConnectInt64Type.java | 43 ++
.../type/connect/ConnectInt8Type.java | 43 ++
.../ConnectMapToConnectStringType.java | 42 ++
.../type/connect/ConnectStringType.java | 49 ++
.../type/connect/ConnectTimeType.java | 57 ++
.../type/connect/ConnectTimestampType.java | 50 ++
.../debezium/AbstractDebeziumTimeType.java | 45 ++
.../AbstractDebeziumTimestampType.java | 43 ++
.../converter/type/debezium/ArrayType.java | 107 ++++
.../converter/type/debezium/DateType.java | 50 ++
.../type/debezium/GeographyType.java | 32 +
.../converter/type/debezium/GeometryType.java | 47 ++
.../type/debezium/MicroTimeType.java | 39 ++
.../type/debezium/MicroTimestampType.java | 39 ++
.../converter/type/debezium/NanoTimeType.java | 39 ++
.../type/debezium/NanoTimestampType.java | 43 ++
.../converter/type/debezium/PointType.java | 46 ++
.../converter/type/debezium/TimeType.java | 39 ++
.../type/debezium/TimestampType.java | 40 ++
.../debezium/VariableScaleDecimalType.java | 75 +++
.../type/debezium/ZonedTimeType.java | 59 ++
.../type/debezium/ZonedTimestampType.java | 53 ++
.../doris/converter/type/doris/DorisType.java | 44 ++
.../type/doris/DorisTypeProperties.java | 32 +
.../converter/type/util/DateTimeUtils.java | 121 ++++
.../doris/converter/type/util/GeoUtils.java | 68 ++
.../converter/type/util/SchemaUtils.java | 59 ++
.../doris/exception/ArgumentsException.java | 38 ++
.../doris/exception/ConfigException.java | 36 --
.../exception/ConnectedFailedException.java | 35 ++
.../doris/exception/CopyLoadException.java | 38 ++
.../exception/DataConverterException.java | 47 ++
.../doris/exception/DataFormatException.java | 38 ++
.../doris/exception/DorisException.java | 40 +-
.../exception/SchemaChangeException.java | 49 ++
.../doris/exception/StreamLoadException.java | 47 ++
.../TableAlterOrCreateException.java | 34 -
.../doris/exception/UploadException.java | 38 ++
.../doris/metrics/DorisConnectMonitor.java | 178 ++++++
.../doris/metrics/MetricsJmxReporter.java | 105 ++++
.../connect/doris/metrics/MetricsUtil.java | 49 ++
.../connect/doris/model/BackendV2.java | 89 +++
.../connect/doris/model/BaseResponse.java | 42 ++
.../connect/doris/model/ColumnDescriptor.java | 91 +++
.../connect/doris/model/CopyIntoResp.java | 43 ++
.../connect/doris/model/KafkaRespContent.java | 60 ++
.../connect/doris/model/LoadOperation.java | 32 +
.../connect/doris/model/RespContent.java | 117 ++++
.../connect/doris/model/TableDescriptor.java | 101 +++
.../connect/doris/model/doris/Field.java | 149 +++++
.../connect/doris/model/doris/Schema.java | 110 ++++
.../schema/column/ColumnDefAdjuster.java | 72 ---
.../doris/schema/column/ColumnDefinition.java | 337 ----------
.../connect/doris/schema/column/ColumnId.java | 105 ----
.../column/parser/DateColumnParser.java | 61 --
.../column/parser/TimeColumnParser.java | 63 --
.../column/parser/TimestampColumnParser.java | 54 --
.../connect/doris/schema/db/DbStructure.java | 50 --
.../doris/schema/table/TableDefinition.java | 63 --
.../doris/schema/table/TableDefinitions.java | 79 ---
.../connect/doris/schema/table/TableId.java | 144 -----
.../serialization/ByteArrayDeserializer.java | 25 -
.../serialization/ByteArraySerializer.java | 24 -
.../serialization/ByteBufferDeserializer.java | 30 -
.../serialization/ByteBufferSerializer.java | 41 --
.../doris/serialization/Deserializer.java | 63 --
.../serialization/DoubleDeserializer.java | 38 --
.../doris/serialization/DoubleSerializer.java | 38 --
.../serialization/FloatDeserializer.java | 38 --
.../doris/serialization/FloatSerializer.java | 34 -
.../serialization/IntegerDeserializer.java | 38 --
.../serialization/IntegerSerializer.java | 33 -
.../doris/serialization/JsonDeserializer.java | 44 --
.../doris/serialization/JsonSerializer.java | 50 --
.../doris/serialization/LongDeserializer.java | 38 --
.../doris/serialization/LongSerializer.java | 37 --
.../doris/serialization/Serializer.java | 61 --
.../serialization/ShortDeserializer.java | 39 --
.../doris/serialization/ShortSerializer.java | 31 -
.../serialization/StringDeserializer.java | 49 --
.../doris/serialization/StringSerializer.java | 51 --
.../service/DorisDefaultSinkService.java | 125 ++++
.../doris/service/DorisSinkService.java | 43 ++
.../service/DorisSinkServiceFactory.java | 32 +
.../doris/service/DorisSystemService.java | 116 ++++
.../connect/doris/service/RestService.java | 361 +++++++++++
.../connect/doris/sink/BufferedRecords.java | 234 -------
.../connect/doris/sink/DorisDialect.java | 125 ----
.../connect/doris/sink/DorisStreamLoader.java | 145 -----
.../connect/doris/sink/RecordValidator.java | 101 ---
.../rocketmq/connect/doris/sink/Updater.java | 74 ---
.../doris/sink/metadata/FieldsMetadata.java | 304 ---------
.../doris/sink/metadata/SchemaPair.java | 64 --
.../doris/sink/metadata/SinkRecordField.java | 90 ---
.../connect/doris/util/BytesUtil.java | 30 -
.../doris/util/ConnectorGroupUtils.java | 53 --
.../connect/doris/util/DateTimeUtils.java | 71 ---
.../connect/doris/util/ExpressionBuilder.java | 595 ------------------
.../connect/doris/util/IdentifierRules.java | 186 ------
.../connect/doris/util/JdbcDriverInfo.java | 136 ----
.../connect/doris/util/NumericMapping.java | 54 --
.../connect/doris/util/QuoteMethod.java | 42 --
.../connect/doris/util/TableType.java | 85 ---
.../connect/doris/util/TableUtil.java | 34 -
.../connect/doris/utils/BackendUtils.java | 72 +++
.../doris/utils/BackoffAndRetryUtils.java | 70 +++
.../connect/doris/utils/ConfigCheckUtils.java | 334 ++++++++++
.../doris/utils/ConnectRecordUtil.java | 36 ++
.../connect/doris/utils/FileNameUtils.java | 119 ++++
.../doris/utils/HttpGetWithEntity.java | 37 ++
.../connect/doris/utils/HttpPostBuilder.java | 71 +++
.../connect/doris/utils/HttpPutBuilder.java | 126 ++++
.../connect/doris/utils/HttpUtils.java | 89 +++
.../connect/doris/writer/CopyIntoWriter.java | 159 +++++
.../connect/doris/writer/CopySQLBuilder.java | 72 +++
.../doris/writer/DeliveryGuarantee.java | 44 ++
.../connect/doris/writer/DorisWriter.java | 184 ++++++
.../connect/doris/writer/LabelGenerator.java | 54 ++
.../connect/doris/writer/LoadConstants.java | 34 +
.../connect/doris/writer/LoadStatus.java | 30 +
.../connect/doris/writer/PartitionBuffer.java | 141 +++++
.../connect/doris/writer/RecordBuffer.java | 57 ++
.../connect/doris/writer/ResponseUtil.java | 37 ++
.../doris/writer/StreamLoadWriter.java | 190 ++++++
.../doris/writer/commit/DorisCommittable.java | 93 +++
.../doris/writer/commit/DorisCommitter.java | 139 ++++
.../connect/doris/writer/load/CopyLoad.java | 274 ++++++++
.../connect/doris/writer/load/DataLoad.java | 31 +
.../doris/writer/load/DorisStreamLoad.java | 149 +++++
.../doris/writer/load/GroupCommitMode.java | 28 +
.../connect/doris/writer/load/LoadModel.java | 44 ++
.../connect/doris/DorisSinkTaskTest.java | 156 +++++
166 files changed, 9115 insertions(+), 5001 deletions(-)
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/DorisSinkConnector.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/DorisSinkTask.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/cfg/DorisOptions.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/cfg/DorisSinkConnectorConfig.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/config/AbstractConfig.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connection/ConnectionProvider.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connection/JdbcConnectionProvider.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connector/DorisSinkConfig.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connector/DorisSinkConnector.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/connector/DorisSinkTask.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/ConverterMode.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/RecordDescriptor.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/RecordService.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/RecordTypeRegister.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/schema/SchemaChangeManager.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/schema/SchemaEvolutionMode.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractDateType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractGeometryType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractTemporalType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractTimeType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractTimestampType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/AbstractType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/Type.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/AbstractConnectMapType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/AbstractConnectSchemaType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectBooleanType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectBytesType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectDateType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectDecimalType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectFloat32Type.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectFloat64Type.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt16Type.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt32Type.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt64Type.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectInt8Type.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectMapToConnectStringType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectStringType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectTimeType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/connect/ConnectTimestampType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/AbstractDebeziumTimeType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/AbstractDebeziumTimestampType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/ArrayType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/DateType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/GeographyType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/GeometryType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/MicroTimeType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/MicroTimestampType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/NanoTimeType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/NanoTimestampType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/PointType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/TimeType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/TimestampType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/VariableScaleDecimalType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/ZonedTimeType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/debezium/ZonedTimestampType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/doris/DorisType.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/doris/DorisTypeProperties.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/util/DateTimeUtils.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/util/GeoUtils.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/converter/type/util/SchemaUtils.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/ArgumentsException.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/ConfigException.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/ConnectedFailedException.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/CopyLoadException.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/DataConverterException.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/DataFormatException.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/SchemaChangeException.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/StreamLoadException.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/TableAlterOrCreateException.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/exception/UploadException.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/metrics/DorisConnectMonitor.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/metrics/MetricsJmxReporter.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/metrics/MetricsUtil.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/BackendV2.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/BaseResponse.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/ColumnDescriptor.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/CopyIntoResp.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/KafkaRespContent.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/LoadOperation.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/RespContent.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/TableDescriptor.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/doris/Field.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/model/doris/Schema.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/ColumnDefAdjuster.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/ColumnDefinition.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/ColumnId.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/parser/DateColumnParser.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/parser/TimeColumnParser.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/column/parser/TimestampColumnParser.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/db/DbStructure.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/table/TableDefinition.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/table/TableDefinitions.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/schema/table/TableId.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteArrayDeserializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteArraySerializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteBufferDeserializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ByteBufferSerializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/Deserializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/DoubleDeserializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/DoubleSerializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/FloatDeserializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/FloatSerializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/IntegerDeserializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/IntegerSerializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/JsonDeserializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/JsonSerializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/LongDeserializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/LongSerializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/Serializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ShortDeserializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/ShortSerializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/StringDeserializer.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/serialization/StringSerializer.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisDefaultSinkService.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisSinkService.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisSinkServiceFactory.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/DorisSystemService.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/service/RestService.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/BufferedRecords.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/DorisDialect.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/DorisStreamLoader.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/RecordValidator.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/Updater.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/metadata/FieldsMetadata.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/metadata/SchemaPair.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/sink/metadata/SinkRecordField.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/BytesUtil.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/ConnectorGroupUtils.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/DateTimeUtils.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/ExpressionBuilder.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/IdentifierRules.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/JdbcDriverInfo.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/NumericMapping.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/QuoteMethod.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/TableType.java
delete mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/util/TableUtil.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/BackendUtils.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/BackoffAndRetryUtils.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/ConfigCheckUtils.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/ConnectRecordUtil.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/FileNameUtils.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpGetWithEntity.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpPostBuilder.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpPutBuilder.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/utils/HttpUtils.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/CopyIntoWriter.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/CopySQLBuilder.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/DeliveryGuarantee.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/DorisWriter.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/LabelGenerator.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/LoadConstants.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/LoadStatus.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/PartitionBuffer.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/RecordBuffer.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/ResponseUtil.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/StreamLoadWriter.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/commit/DorisCommittable.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/commit/DorisCommitter.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/CopyLoad.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/DataLoad.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/DorisStreamLoad.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/GroupCommitMode.java
create mode 100644 connectors/rocketmq-connect-doris/src/main/java/org/apache/rocketmq/connect/doris/writer/load/LoadModel.java
create mode 100644 connectors/rocketmq-connect-doris/src/test/java/org/apache/rocketmq/connect/doris/DorisSinkTaskTest.java
diff --git a/connectors/rocketmq-connect-doris/pom.xml b/connectors/rocketmq-connect-doris/pom.xml
index 7ed0e952c..660aa460a 100644
--- a/connectors/rocketmq-connect-doris/pom.xml
+++ b/connectors/rocketmq-connect-doris/pom.xml
@@ -49,10 +49,13 @@
0.1.40.3.1-alpha
-
-
- 1.2.83
-
+ 4.2.25
+ 2.13.2.1
+ 0.0.1-SNAPSHOT
+ 1.9.8.Final
+ 2.2.0
+ 2.3
+ 8.0.33
@@ -94,27 +97,60 @@
test
-
- com.alibaba
- fastjson
- ${fastjson.version}
+ com.fasterxml.jackson.core
+ jackson-databind
+ ${jackson.version}
- ch.qos.logback
- logback-classic
- 1.2.0
-
-
- ch.qos.logback
- logback-core
- 1.2.9
+ org.apache.rocketmq
+ rocketmq-connect-runtime
+ ${rocketmq-connect.version}
+ providedorg.apache.httpcomponentshttpclient4.5.13
+
+ io.dropwizard.metrics
+ metrics-core
+ ${metrics.version}
+
+
+ io.dropwizard.metrics
+ metrics-jmx
+ ${metrics.version}
+
+
+
+ io.debezium
+ debezium-core
+ ${debezium.version}
+
+
+ com.esri.geometry
+ esri-geometry-api
+ ${geometry.version}
+
+
+ com.fasterxml.jackson.core
+ jackson-core
+
+
+
+
+ commons-io
+ commons-io
+ ${commons-io.version}
+
+
+
+ mysql
+ mysql-connector-java
+ ${mysql-connector.version}
+
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 000000000..88244efa3
--- /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 taskConfigs(final int maxTasks) {
+ List 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 000000000..fa14a6a13
--- /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 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 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 000000000..a1164d7c6
--- /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 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 getTopicMap() {
+ return topicMap;
+ }
+
+ public String getQueryUrl() {
+ List queryUrls = getQueryUrls();
+ return queryUrls.get(0);
+ }
+
+ public String getHttpUrl() {
+ List httpUrls = getHttpUrls();
+ return httpUrls.get(0);
+ }
+
+ public List getQueryUrls() {
+ List 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 getHttpUrls() {
+ List 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 getProxyConfig() {
+ return Optional.ofNullable(proxyConfig);
+ }
+
+ /**
+ * parse topic to table map
+ *
+ * @param config connector config file
+ * @return result map
+ */
+ private Map getTopicToTableMap(KeyValue config) {
+ if (config.containsKey(DorisSinkConnectorConfig.TOPICS_TABLES_MAP)) {
+ Map 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 000000000..f52325356
--- /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 5ff2b2aec..000000000
--- 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 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 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 000000000..a77d3c5fb
--- /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 000000000..4207b7f0a
--- /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 5cbe294d2..000000000
--- 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. db1.table01,db1.table02";
-
-
- 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 pkFields;
- private Set fieldsWhitelist;
- private Set tableWhitelist;
- private TimeZone timeZone;
- private EnumSet 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 getPkFields() {
- return pkFields;
- }
-
- public Set getFieldsWhitelist() {
- return fieldsWhitelist;
- }
-
- public Set getTableWhitelist() {
- return tableWhitelist;
- }
-
- public TimeZone getTimeZone() {
- return timeZone;
- }
-
- public EnumSet 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 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 a2e3119f0..000000000
--- 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 taskConfigs(int maxTasks) {
- log.info("Starting task config !!! ");
- List 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 f775b092b..000000000
--- 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 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 000000000..4755166d2
--- /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 000000000..a7d43f040
--- /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 keyFieldNames;
+ private final List nonKeyFieldNames;
+ private final Map fields;
+ private final boolean flattened;
+
+ private RecordDescriptor(
+ ConnectRecord record,
+ String topicName,
+ List keyFieldNames,
+ List nonKeyFieldNames,
+ Map 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 getKeyFieldNames() {
+ return keyFieldNames;
+ }
+
+ public List getNonKeyFieldNames() {
+ return nonKeyFieldNames;
+ }
+
+ public Map 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 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 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 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 typeRegistry;
+
+ // Internal build state
+ private final List keyFieldNames = new ArrayList<>();
+ private final List nonKeyFieldNames = new ArrayList<>();
+ private final Map allFields = new LinkedHashMap<>();
+
+ public Builder withSinkRecord(ConnectRecord record) {
+ this.sinkRecord = record;
+ return this;
+ }
+
+ public Builder withTypeRegistry(Map 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 000000000..c08ff1d92
--- /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 dorisTableDescriptorCache;
+
+ public RecordService() {
+ this.converter = new JsonConverter();
+ Map 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 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 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 resolveMissingFields(
+ RecordDescriptor record, TableDescriptor table) {
+ Set missingFields = new HashSet<>();
+ for (Map.Entry 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 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 fields, boolean isDelete) {
+ Map 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 000000000..ced15e622
--- /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 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 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 000000000..52765d479
--- /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 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 responseMap = handleResponse(httpPost, responseEntity);
+ return handleSchemaChange(responseMap, responseEntity);
+ }
+
+ public HttpPost buildHttpPost(String ddl, String database)
+ throws IllegalArgumentException, IOException {
+ Map 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 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 000000000..fc98b7c90
--- /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 000000000..ef6eea01f
--- /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 000000000..ac6887540
--- /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 000000000..67afcb29f
--- /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 000000000..1100ee060
--- /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 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 000000000..9f13d7cb7
--- /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 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 000000000..4dd9e71a1
--- /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 getSchemaParameter(Schema schema, String parameterName) {
+ if (!Objects.isNull(schema.getParameters())) {
+ return Optional.ofNullable(schema.getParameters().get(parameterName));
+ }
+ return Optional.empty();
+ }
+
+ protected Optional getSourceColumnType(Schema schema) {
+ return SchemaUtils.getSourceColumnType(schema);
+ }
+
+ protected Optional getSourceColumnLength(Schema schema) {
+ return SchemaUtils.getSourceColumnLength(schema);
+ }
+
+ protected Optional 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 000000000..2ada4bca6
--- /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.
+ *
+ *
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 000000000..9de7b72aa
--- /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 000000000..f6560d681
--- /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 000000000..5fc351db1
--- /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 000000000..ab18ea411
--- /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 000000000..26f97d6ae
--- /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 000000000..c33a87dda
--- /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 000000000..c6222339e
--- /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 000000000..a9e017698
--- /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 000000000..57919fd5c
--- /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 000000000..70c2c2c8c
--- /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 000000000..96c220ff7
--- /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 000000000..1839ed36c
--- /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 000000000..e2f32d352
--- /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 000000000..9809601b2
--- /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 000000000..6ccc94480
--- /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 000000000..748ba4420
--- /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 000000000..fa28a7104
--- /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 000000000..8c844eded
--- /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 000000000..8fd97de8e
--- /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