diff --git a/docs/en/extending-doris/flink-doris-connector.md b/docs/en/extending-doris/flink-doris-connector.md
new file mode 100644
index 00000000000000..cce7aafb9e3090
--- /dev/null
+++ b/docs/en/extending-doris/flink-doris-connector.md
@@ -0,0 +1,143 @@
+---
+{
+ "title": "Flink Doris Connector",
+ "language": "en"
+}
+---
+
+
+
+# Flink Doris Connector
+
+Flink Doris Connector can support reading data stored in Doris through Flink.
+
+- You can map the `Doris` table to` DataStream` or `Table`.
+
+## Version Compatibility
+
+| Connector | Flink | Doris | Java | Scala |
+| --------- | ----- | ------ | ---- | ----- |
+| 1.0.0 | 1.11.2 | 0.13+ | 8 | 2.12 |
+
+
+## Build and Install
+
+Execute following command in dir `extension/flink-doris-connector/`:
+
+```bash
+sh build.sh
+```
+
+After successful compilation, the file `doris-flink-1.0.0-SNAPSHOT.jar` will be generated in the `output/` directory. Copy this file to `ClassPath` in `Flink` to use `Flink-Doris-Connector`. For example, `Flink` running in `Local` mode, put this file in the `jars/` folder. `Flink` running in `Yarn` cluster mode, put this file in the pre-deployment package.
+
+## How to use
+The purpose of this step is to register the Doris data source on Flink.
+This step is operated on Flink.
+There are two ways to use sql and java. The following are examples to illustrate
+### SQL
+The purpose of this step is to register the Doris data source on Flink.
+This step is operated on Flink
+```sql
+CREATE TABLE flink_doris_source (
+ name STRING,
+ age INT,
+ price DECIMAL(5,2),
+ sale DOUBLE
+ )
+ WITH (
+ 'connector' = 'doris',
+ 'fenodes' = '$YOUR_DORIS_FE_HOSTNAME:$YOUR_DORIS_FE_RESFUL_PORT',
+ 'table.identifier' = '$YOUR_DORIS_DATABASE_NAME.$YOUR_DORIS_TABLE_NAME',
+ 'username' = '$YOUR_DORIS_USERNAME',
+ 'password' = '$YOUR_DORIS_PASSWORD'
+);
+
+CREATE TABLE flink_doris_sink (
+ name STRING,
+ age INT,
+ price DECIMAL(5,2),
+ sale DOUBLE
+ )
+ WITH (
+ 'connector' = 'doris',
+ 'fenodes' = '$YOUR_DORIS_FE_HOSTNAME:$YOUR_DORIS_FE_RESFUL_PORT',
+ 'table.identifier' = '$YOUR_DORIS_DATABASE_NAME.$YOUR_DORIS_TABLE_NAME',
+ 'username' = '$YOUR_DORIS_USERNAME',
+ 'password' = '$YOUR_DORIS_PASSWORD'
+);
+
+INSERT INTO flink_doris_sink select name,age,price,sale from flink_doris_source
+```
+
+### DataStream
+
+```scala
+ Properties properties = new Properties();
+ properties.put("fenodes","FE_IP:8030");
+ properties.put("username","root");
+ properties.put("password","");
+ properties.put("table.identifier","db.table");
+ env.addSource(new DorisSourceFunction(new DorisStreamOptions(properties),new SimpleListDeserializationSchema())).print();
+```
+
+### General
+
+| Key | Default Value | Comment |
+| -------------------------------- | ----------------- | ------------------------------------------------------------ |
+| fenodes | -- | Doris FE http address, support multiple addresses, separated by commas |
+| table.identifier | -- | Doris table identifier, eg, db1.tbl1 |
+| username | -- | Doris username |
+| password | -- | Doris password |
+| doris.request.retries | 3 | Number of retries to send requests to Doris |
+| doris.request.connect.timeout.ms | 30000 | Connection timeout for sending requests to Doris |
+| doris.request.read.timeout.ms | 30000 | Read timeout for sending request to Doris |
+| doris.request.query.timeout.s | 3600 | Query the timeout time of doris, the default is 1 hour, -1 means no timeout limit |
+| doris.request.tablet.size | Integer.MAX_VALUE | The number of Doris Tablets corresponding to an RDD Partition. The smaller this value is set, the more partitions will be generated. This will increase the parallelism on the Spark side, but at the same time will cause greater pressure on Doris. |
+| doris.batch.size | 1024 | The maximum number of rows to read data from BE at one time. Increasing this value can reduce the number of connections between Spark and Doris. Thereby reducing the extra time overhead caused by network delay. |
+| doris.exec.mem.limit | 2147483648 | Memory limit for a single query. The default is 2GB, in bytes. |
+| doris.deserialize.arrow.async | false | Whether to support asynchronous conversion of Arrow format to RowBatch required for spark-doris-connector iteration |
+| doris.deserialize.queue.size | 64 | Asynchronous conversion of the internal processing queue in Arrow format takes effect when doris.deserialize.arrow.async is true |
+| doris.read.field | -- | List of column names in the Doris table, separated by commas |
+| doris.filter.query | -- | Filter expression of the query, which is transparently transmitted to Doris. Doris uses this expression to complete source-side data filtering. |
+| sink.batch.size | 100 | Maximum number of lines in a single write BE |
+| sink.max-retries | 1 | Number of retries after writing BE failed |
+
+
+## Doris & Flink Column Type Mapping
+
+| Doris Type | Flink Type |
+| ---------- | -------------------------------- |
+| NULL_TYPE | NULL |
+| BOOLEAN | BOOLEAN |
+| TINYINT | TINYINT |
+| SMALLINT | SMALLINT |
+| INT | INT |
+| BIGINT | BIGINT |
+| FLOAT | FLOAT |
+| DOUBLE | DOUBLE |
+| DATE | STRING |
+| DATETIME | STRING |
+| DECIMAL | DECIMAL |
+| CHAR | STRING |
+| LARGEINT | STRING |
+| VARCHAR | STRING |
+| DECIMALV2 | DECIMAL |
+| TIME | DOUBLE |
+| HLL | Unsupported datatype |
diff --git a/docs/zh-CN/extending-doris/flink-doris-connector.md b/docs/zh-CN/extending-doris/flink-doris-connector.md
new file mode 100644
index 00000000000000..b3180f64d9ca9b
--- /dev/null
+++ b/docs/zh-CN/extending-doris/flink-doris-connector.md
@@ -0,0 +1,147 @@
+---
+{
+ "title": "Flink Doris Connector",
+ "language": "zh-CN"
+}
+---
+
+
+
+# Flink Doris Connector
+
+Flink Doris Connector 可以支持通过 Flink 读取 Doris 中存储的数据。
+
+- 可以将`Doris`表映射为`DataStream`或者`Table`。
+
+## 版本兼容
+
+| Connector | Flink | Doris | Java | Scala |
+| --------- | ----- | ------ | ---- | ----- |
+| 1.0.0 | 1.11.2 | 0.13+ | 8 | 2.12 |
+
+
+## 编译与安装
+
+在 `extension/flink-doris-connector/` 源码目录下执行:
+
+```bash
+sh build.sh
+```
+
+编译成功后,会在 `output/` 目录下生成文件 `doris-flink-1.0.0-SNAPSHOT.jar`。将此文件复制到 `Flink` 的 `ClassPath` 中即可使用 `Flink-Doris-Connector`。例如,`Local` 模式运行的 `Flink`,将此文件放入 `jars/` 文件夹下。`Yarn`集群模式运行的`Flink`,则将此文件放入预部署包中。
+
+## 使用示例
+此步骤的目的是在Flink上注册Doris数据源。
+此步骤在Flink上进行。
+有两种使用sql和java的方法。 以下是示例说明
+### SQL
+此步骤的目的是在Flink上注册Doris数据源。
+此步骤在Flink上进行。
+```sql
+CREATE TABLE flink_doris_source (
+ name STRING,
+ age INT,
+ price DECIMAL(5,2),
+ sale DOUBLE
+ )
+ WITH (
+ 'connector' = 'doris',
+ 'fenodes' = '$YOUR_DORIS_FE_HOSTNAME:$YOUR_DORIS_FE_RESFUL_PORT',
+ 'table.identifier' = '$YOUR_DORIS_DATABASE_NAME.$YOUR_DORIS_TABLE_NAME',
+ 'username' = '$YOUR_DORIS_USERNAME',
+ 'password' = '$YOUR_DORIS_PASSWORD'
+);
+
+CREATE TABLE flink_doris_sink (
+ name STRING,
+ age INT,
+ price DECIMAL(5,2),
+ sale DOUBLE
+ )
+ WITH (
+ 'connector' = 'doris',
+ 'fenodes' = '$YOUR_DORIS_FE_HOSTNAME:$YOUR_DORIS_FE_RESFUL_PORT',
+ 'table.identifier' = '$YOUR_DORIS_DATABASE_NAME.$YOUR_DORIS_TABLE_NAME',
+ 'username' = '$YOUR_DORIS_USERNAME',
+ 'password' = '$YOUR_DORIS_PASSWORD'
+);
+
+INSERT INTO flink_doris_sink select name,age,price,sale from flink_doris_source
+```
+
+### DataStream
+
+```java
+ Properties properties = new Properties();
+ properties.put("fenodes","FE_IP:8030");
+ properties.put("username","root");
+ properties.put("password","");
+ properties.put("table.identifier","db.table");
+ env.addSource(new DorisSourceFunction(new DorisStreamOptions(properties),new SimpleListDeserializationSchema())).print();
+```
+
+
+## 配置
+
+### 通用配置项
+
+| Key | Default Value | Comment |
+| -------------------------------- | ----------------- | ------------------------------------------------------------ |
+| fenodes | -- | Doris FE http 地址 |
+| table.identifier | -- | Doris 表名,如:db1.tbl1 |
+| username | -- | 访问Doris的用户名 |
+| password | -- | 访问Doris的密码 |
+| doris.request.retries | 3 | 向Doris发送请求的重试次数 |
+| doris.request.connect.timeout.ms | 30000 | 向Doris发送请求的连接超时时间 |
+| doris.request.read.timeout.ms | 30000 | 向Doris发送请求的读取超时时间 |
+| doris.request.query.timeout.s | 3600 | 查询doris的超时时间,默认值为1小时,-1表示无超时限制 |
+| doris.request.tablet.size | Integer.MAX_VALUE | 一个RDD Partition对应的Doris Tablet个数。 此数值设置越小,则会生成越多的Partition。从而提升Spark侧的并行度,但同时会对Doris造成更大的压力。 |
+| doris.batch.size | 1024 | 一次从BE读取数据的最大行数。增大此数值可减少Spark与Doris之间建立连接的次数。 从而减轻网络延迟所带来的的额外时间开销。 |
+| doris.exec.mem.limit | 2147483648 | 单个查询的内存限制。默认为 2GB,单位为字节 |
+| doris.deserialize.arrow.async | false | 是否支持异步转换Arrow格式到spark-doris-connector迭代所需的RowBatch |
+| doris.deserialize.queue.size | 64 | 异步转换Arrow格式的内部处理队列,当doris.deserialize.arrow.async为true时生效 |
+| doris.read.field | -- | 读取Doris表的列名列表,多列之间使用逗号分隔 |
+| doris.filter.query | -- | 过滤读取数据的表达式,此表达式透传给Doris。Doris使用此表达式完成源端数据过滤。 |
+| sink.batch.size | 100 | 单次写BE的最大行数 |
+| sink.max-retries | 1 | 写BE失败之后的重试次数 |
+
+
+
+## Doris 和 Flink 列类型映射关系
+
+| Doris Type | Flink Type |
+| ---------- | -------------------------------- |
+| NULL_TYPE | NULL |
+| BOOLEAN | BOOLEAN |
+| TINYINT | TINYINT |
+| SMALLINT | SMALLINT |
+| INT | INT |
+| BIGINT | BIGINT |
+| FLOAT | FLOAT |
+| DOUBLE | DOUBLE |
+| DATE | STRING |
+| DATETIME | STRING |
+| DECIMAL | DECIMAL |
+| CHAR | STRING |
+| LARGEINT | STRING |
+| VARCHAR | STRING |
+| DECIMALV2 | DECIMAL |
+| TIME | DOUBLE |
+| HLL | Unsupported datatype |
diff --git a/extension/flink-doris-connector/build.sh b/extension/flink-doris-connector/build.sh
new file mode 100644
index 00000000000000..d3e7ca3f758da5
--- /dev/null
+++ b/extension/flink-doris-connector/build.sh
@@ -0,0 +1,60 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+##############################################################
+# This script is used to compile Flink-Doris-Connector
+# Usage:
+# sh build.sh
+#
+##############################################################
+
+set -eo pipefail
+
+ROOT=`dirname "$0"`
+ROOT=`cd "$ROOT"; pwd`
+
+export DORIS_HOME=${ROOT}/../../
+
+# include custom environment variables
+if [[ -f ${DORIS_HOME}/custom_env.sh ]]; then
+ . ${DORIS_HOME}/custom_env.sh
+fi
+
+# check maven
+MVN_CMD=mvn
+if [[ ! -z ${CUSTOM_MVN} ]]; then
+ MVN_CMD=${CUSTOM_MVN}
+fi
+if ! ${MVN_CMD} --version; then
+ echo "Error: mvn is not found"
+ exit 1
+fi
+export MVN_CMD
+
+${MVN_CMD} clean package
+
+
+mkdir -p output/
+cp target/doris-flink-1.0-SNAPSHOT.jar ./output/
+
+echo "*****************************************"
+echo "Successfully build Flink-Doris-Connector"
+echo "*****************************************"
+
+exit 0
+
diff --git a/extension/flink-doris-connector/pom.xml b/extension/flink-doris-connector/pom.xml
new file mode 100644
index 00000000000000..fad5bf9b988829
--- /dev/null
+++ b/extension/flink-doris-connector/pom.xml
@@ -0,0 +1,307 @@
+
+
+ 4.0.0
+
+ org.apache
+ doris-flink
+ 1.0-SNAPSHOT
+
+
+ 2.12
+ 1.11.2
+ 0.9.3
+ 0.15.1
+ UTF-8
+ ${basedir}/../../
+ ${basedir}/../../thirdparty
+
+
+
+
+ thirdparty
+
+
+ env.DORIS_THIRDPARTY
+
+
+
+ ${env.DORIS_THIRDPARTY}
+
+
+
+
+ custom-env
+
+
+ env.CUSTOM_MAVEN_REPO
+
+
+
+
+
+ custom-nexus
+ ${env.CUSTOM_MAVEN_REPO}
+
+
+
+
+
+ custom-nexus
+ ${env.CUSTOM_MAVEN_REPO}
+
+
+
+
+
+
+ general-env
+
+
+ !env.CUSTOM_MAVEN_REPO
+
+
+
+
+
+ central
+ central maven repo https
+ https://repo.maven.apache.org/maven2
+
+
+
+
+
+
+
+ org.apache.flink
+ flink-java
+ ${flink.version}
+ provided
+
+
+ org.apache.flink
+ flink-streaming-java_${scala.version}
+ ${flink.version}
+ provided
+
+
+ org.apache.flink
+ flink-clients_${scala.version}
+ ${flink.version}
+ provided
+
+
+
+ org.apache.flink
+ flink-table-common
+ ${flink.version}
+ provided
+
+
+ org.apache.flink
+ flink-table-api-java-bridge_${scala.version}
+ ${flink.version}
+ provided
+
+
+ org.apache.flink
+ flink-table-planner-blink_${scala.version}
+ ${flink.version}
+ provided
+
+
+ org.apache.thrift
+ libthrift
+ ${libthrift.version}
+
+
+ org.apache.arrow
+ arrow-vector
+ ${arrow.version}
+
+
+ org.slf4j
+ slf4j-api
+ 1.7.25
+
+
+ org.slf4j
+ slf4j-log4j12
+ 1.7.25
+ test
+
+
+ log4j
+ log4j
+ 1.2.17
+
+
+
+ org.hamcrest
+ hamcrest-core
+ 1.3
+ test
+
+
+ org.mockito
+ mockito-scala_${scala.version}
+ 1.4.7
+
+
+ hamcrest-core
+ org.hamcrest
+
+
+ test
+
+
+ junit
+ junit
+ 4.11
+
+
+ hamcrest-core
+ org.hamcrest
+
+
+ test
+
+
+
+
+
+
+ org.apache.thrift.tools
+ maven-thrift-plugin
+ 0.1.11
+
+ ${doris.thirdparty}/installed/bin/thrift
+
+
+
+ thrift-sources
+ generate-sources
+
+ compile
+
+
+
+
+
+ net.alchim31.maven
+ scala-maven-plugin
+ 3.2.1
+
+
+ scala-compile-first
+ process-resources
+
+ compile
+
+
+
+ scala-test-compile
+ process-test-resources
+
+ testCompile
+
+
+
+
+
+ -feature
+
+
+
+
+ org.apache.maven.plugins
+ maven-shade-plugin
+ 3.2.1
+
+
+
+ com.google.code.findbugs:*
+ org.slf4j:*
+
+
+
+
+ org.apache.arrow
+ org.apache.doris.shaded.org.apache.arrow
+
+
+ io.netty
+ org.apache.doris.shaded.io.netty
+
+
+ com.fasterxml.jackson
+ org.apache.doris.shaded.com.fasterxml.jackson
+
+
+ org.apache.commons.codec
+ org.apache.doris.shaded.org.apache.commons.codec
+
+
+ com.google.flatbuffers
+ org.apache.doris.shaded.com.google.flatbuffers
+
+
+ org.apache.thrift
+ org.apache.doris.shaded.org.apache.thrift
+
+
+
+
+
+ package
+
+ shade
+
+
+
+
+
+ org.jacoco
+ jacoco-maven-plugin
+ 0.7.8
+
+
+ **/thrift/**
+
+
+
+ prepare-agent
+
+ prepare-agent
+
+
+
+ check
+
+ check
+
+
+
+ report
+ test
+
+ report
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+ 3.8.1
+
+ 8
+ 8
+
+
+
+
+
+
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/backend/BackendClient.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/backend/BackendClient.java
new file mode 100644
index 00000000000000..93b353c850ac33
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/backend/BackendClient.java
@@ -0,0 +1,217 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.flink.backend;
+
+import org.apache.doris.flink.cfg.ConfigurationOptions;
+import org.apache.doris.flink.cfg.DorisReadOptions;
+import org.apache.doris.flink.exception.ConnectedFailedException;
+import org.apache.doris.flink.exception.DorisException;
+import org.apache.doris.flink.exception.DorisInternalException;
+import org.apache.doris.flink.serialization.Routing;
+import org.apache.doris.flink.util.ErrorMessages;
+import org.apache.doris.thrift.TDorisExternalService;
+import org.apache.doris.thrift.TScanBatchResult;
+import org.apache.doris.thrift.TScanCloseParams;
+import org.apache.doris.thrift.TScanCloseResult;
+import org.apache.doris.thrift.TScanNextBatchParams;
+import org.apache.doris.thrift.TScanOpenParams;
+import org.apache.doris.thrift.TScanOpenResult;
+import org.apache.doris.thrift.TStatusCode;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Client to request Doris BE
+ */
+public class BackendClient {
+ private static Logger logger = LoggerFactory.getLogger(BackendClient.class);
+
+ private Routing routing;
+
+ private TDorisExternalService.Client client;
+ private TTransport transport;
+
+ private boolean isConnected = false;
+ private final int retries;
+ private final int socketTimeout;
+ private final int connectTimeout;
+
+ public BackendClient(Routing routing, DorisReadOptions readOptions) throws ConnectedFailedException {
+ this.routing = routing;
+ this.connectTimeout = readOptions.getRequestConnectTimeoutMs() == null ? ConfigurationOptions.DORIS_REQUEST_CONNECT_TIMEOUT_MS_DEFAULT : readOptions.getRequestConnectTimeoutMs();
+ this.socketTimeout = readOptions.getRequestReadTimeoutMs() == null ? ConfigurationOptions.DORIS_REQUEST_READ_TIMEOUT_MS_DEFAULT : readOptions.getRequestReadTimeoutMs();
+ this.retries = readOptions.getRequestRetries() == null ? ConfigurationOptions.DORIS_REQUEST_RETRIES_DEFAULT : readOptions.getRequestRetries();
+ logger.trace("connect timeout set to '{}'. socket timeout set to '{}'. retries set to '{}'.",
+ this.connectTimeout, this.socketTimeout, this.retries);
+ open();
+ }
+
+ private void open() throws ConnectedFailedException {
+ logger.debug("Open client to Doris BE '{}'.", routing);
+ TException ex = null;
+ for (int attempt = 0; !isConnected && attempt < retries; ++attempt) {
+ logger.debug("Attempt {} to connect {}.", attempt, routing);
+ TBinaryProtocol.Factory factory = new TBinaryProtocol.Factory();
+ transport = new TSocket(routing.getHost(), routing.getPort(), socketTimeout, connectTimeout);
+ TProtocol protocol = factory.getProtocol(transport);
+ client = new TDorisExternalService.Client(protocol);
+ if (isConnected) {
+ logger.info("Success connect to {}.", routing);
+ return;
+ }
+ try {
+ logger.trace("Connect status before open transport to {} is '{}'.", routing, isConnected);
+ if (!transport.isOpen()) {
+ transport.open();
+ isConnected = true;
+ }
+ } catch (TTransportException e) {
+ logger.warn(ErrorMessages.CONNECT_FAILED_MESSAGE, routing, e);
+ ex = e;
+ }
+
+ }
+ if (!isConnected) {
+ logger.error(ErrorMessages.CONNECT_FAILED_MESSAGE, routing);
+ throw new ConnectedFailedException(routing.toString(), ex);
+ }
+ }
+
+ private void close() {
+ logger.trace("Connect status before close with '{}' is '{}'.", routing, isConnected);
+ isConnected = false;
+ if ((transport != null) && transport.isOpen()) {
+ transport.close();
+ logger.info("Closed a connection to {}.", routing);
+ }
+ if (null != client) {
+ client = null;
+ }
+ }
+
+ /**
+ * Open a scanner for reading Doris data.
+ * @param openParams thrift struct to required by request
+ * @return scan open result
+ * @throws ConnectedFailedException throw if cannot connect to Doris BE
+ */
+ public TScanOpenResult openScanner(TScanOpenParams openParams) throws ConnectedFailedException {
+ logger.debug("OpenScanner to '{}', parameter is '{}'.", routing, openParams);
+ if (!isConnected) {
+ open();
+ }
+ TException ex = null;
+ for (int attempt = 0; attempt < retries; ++attempt) {
+ logger.debug("Attempt {} to openScanner {}.", attempt, routing);
+ try {
+ TScanOpenResult result = client.open_scanner(openParams);
+ if (result == null) {
+ logger.warn("Open scanner result from {} is null.", routing);
+ continue;
+ }
+ if (!TStatusCode.OK.equals(result.getStatus().getStatus_code())) {
+ logger.warn("The status of open scanner result from {} is '{}', error message is: {}.",
+ routing, result.getStatus().getStatus_code(), result.getStatus().getError_msgs());
+ continue;
+ }
+ return result;
+ } catch (TException e) {
+ logger.warn("Open scanner from {} failed.", routing, e);
+ ex = e;
+ }
+ }
+ logger.error(ErrorMessages.CONNECT_FAILED_MESSAGE, routing);
+ throw new ConnectedFailedException(routing.toString(), ex);
+ }
+
+ /**
+ * get next row batch from Doris BE
+ * @param nextBatchParams thrift struct to required by request
+ * @return scan batch result
+ * @throws ConnectedFailedException throw if cannot connect to Doris BE
+ */
+ public TScanBatchResult getNext(TScanNextBatchParams nextBatchParams) throws DorisException {
+ logger.debug("GetNext to '{}', parameter is '{}'.", routing, nextBatchParams);
+ if (!isConnected) {
+ open();
+ }
+ TException ex = null;
+ TScanBatchResult result = null;
+ for (int attempt = 0; attempt < retries; ++attempt) {
+ logger.debug("Attempt {} to getNext {}.", attempt, routing);
+ try {
+ result = client.get_next(nextBatchParams);
+ if (result == null) {
+ logger.warn("GetNext result from {} is null.", routing);
+ continue;
+ }
+ if (!TStatusCode.OK.equals(result.getStatus().getStatus_code())) {
+ logger.warn("The status of get next result from {} is '{}', error message is: {}.",
+ routing, result.getStatus().getStatus_code(), result.getStatus().getError_msgs());
+ continue;
+ }
+ return result;
+ } catch (TException e) {
+ logger.warn("Get next from {} failed.", routing, e);
+ ex = e;
+ }
+ }
+ if (result != null && (TStatusCode.OK != (result.getStatus().getStatus_code()))) {
+ logger.error(ErrorMessages.DORIS_INTERNAL_FAIL_MESSAGE, routing, result.getStatus().getStatus_code(),
+ result.getStatus().getError_msgs());
+ throw new DorisInternalException(routing.toString(), result.getStatus().getStatus_code(),
+ result.getStatus().getError_msgs());
+ }
+ logger.error(ErrorMessages.CONNECT_FAILED_MESSAGE, routing);
+ throw new ConnectedFailedException(routing.toString(), ex);
+ }
+
+ /**
+ * close an scanner.
+ * @param closeParams thrift struct to required by request
+ */
+ public void closeScanner(TScanCloseParams closeParams) {
+ logger.debug("CloseScanner to '{}', parameter is '{}'.", routing, closeParams);
+ for (int attempt = 0; attempt < retries; ++attempt) {
+ logger.debug("Attempt {} to closeScanner {}.", attempt, routing);
+ try {
+ TScanCloseResult result = client.close_scanner(closeParams);
+ if (result == null) {
+ logger.warn("CloseScanner result from {} is null.", routing);
+ continue;
+ }
+ if (!TStatusCode.OK.equals(result.getStatus().getStatus_code())) {
+ logger.warn("The status of get next result from {} is '{}', error message is: {}.",
+ routing, result.getStatus().getStatus_code(), result.getStatus().getError_msgs());
+ continue;
+ }
+ break;
+ } catch (TException e) {
+ logger.warn("Close scanner from {} failed.", routing, e);
+ }
+ }
+ logger.info("CloseScanner to Doris BE '{}' success.", routing);
+ close();
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/ConfigurationOptions.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/ConfigurationOptions.java
new file mode 100644
index 00000000000000..47d07b8d17773f
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/ConfigurationOptions.java
@@ -0,0 +1,65 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.flink.cfg;
+
+public interface ConfigurationOptions {
+ // doris fe node address
+ String DORIS_FENODES = "fenodes";
+
+ String DORIS_DEFAULT_CLUSTER = "default_cluster";
+
+ String TABLE_IDENTIFIER = "table.identifier";
+ String DORIS_TABLE_IDENTIFIER = "doris.table.identifier";
+ String DORIS_READ_FIELD = "doris.read.field";
+ String DORIS_FILTER_QUERY = "doris.filter.query";
+ String DORIS_FILTER_QUERY_IN_MAX_COUNT = "doris.filter.query.in.max.count";
+ Integer DORIS_FILTER_QUERY_IN_VALUE_UPPER_LIMIT = 10000;
+
+ String DORIS_USER = "username";
+ String DORIS_PASSWORD = "password";
+
+ String DORIS_REQUEST_AUTH_USER = "doris.request.auth.user";
+ String DORIS_REQUEST_AUTH_PASSWORD = "doris.request.auth.password";
+ String DORIS_REQUEST_RETRIES = "doris.request.retries";
+ String DORIS_REQUEST_CONNECT_TIMEOUT_MS = "doris.request.connect.timeout.ms";
+ String DORIS_REQUEST_READ_TIMEOUT_MS = "doris.request.read.timeout.ms";
+ String DORIS_REQUEST_QUERY_TIMEOUT_S = "doris.request.query.timeout.s";
+ Integer DORIS_REQUEST_RETRIES_DEFAULT = 3;
+ Integer DORIS_REQUEST_CONNECT_TIMEOUT_MS_DEFAULT = 30 * 1000;
+ Integer DORIS_REQUEST_READ_TIMEOUT_MS_DEFAULT = 30 * 1000;
+ Integer DORIS_REQUEST_QUERY_TIMEOUT_S_DEFAULT = 3600;
+
+ String DORIS_TABLET_SIZE = "doris.request.tablet.size";
+ Integer DORIS_TABLET_SIZE_DEFAULT = Integer.MAX_VALUE;
+ Integer DORIS_TABLET_SIZE_MIN = 1;
+
+ String DORIS_BATCH_SIZE = "doris.batch.size";
+ Integer DORIS_BATCH_SIZE_DEFAULT = 1024;
+
+ String DORIS_EXEC_MEM_LIMIT = "doris.exec.mem.limit";
+ Long DORIS_EXEC_MEM_LIMIT_DEFAULT = 2147483648L;
+
+ String DORIS_VALUE_READER_CLASS = "doris.value.reader.class";
+
+ String DORIS_DESERIALIZE_ARROW_ASYNC = "doris.deserialize.arrow.async";
+ Boolean DORIS_DESERIALIZE_ARROW_ASYNC_DEFAULT = false;
+
+ String DORIS_DESERIALIZE_QUEUE_SIZE = "doris.deserialize.queue.size";
+ Integer DORIS_DESERIALIZE_QUEUE_SIZE_DEFAULT = 64;
+
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisConnectionOptions.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisConnectionOptions.java
new file mode 100644
index 00000000000000..619ce74034cfbc
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisConnectionOptions.java
@@ -0,0 +1,80 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package org.apache.doris.flink.cfg;
+
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+
+/**
+ * Doris connection options.
+ */
+public class DorisConnectionOptions implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ protected final String fenodes;
+ protected final String username;
+ protected final String password;
+
+ public DorisConnectionOptions(String fenodes, String username, String password) {
+ this.fenodes = Preconditions.checkNotNull(fenodes, "fenodes is empty");
+ this.username = username;
+ this.password = password;
+ }
+
+ public String getFenodes() {
+ return fenodes;
+ }
+
+ public String getUsername() {
+ return username;
+ }
+
+ public String getPassword() {
+ return password;
+ }
+
+ /**
+ * Builder for {@link DorisConnectionOptions}.
+ */
+ public static class DorisConnectionOptionsBuilder {
+ private String fenodes;
+ private String username;
+ private String password;
+
+ public DorisConnectionOptionsBuilder withFenodes(String fenodes) {
+ this.fenodes = fenodes;
+ return this;
+ }
+
+ public DorisConnectionOptionsBuilder withUsername(String username) {
+ this.username = username;
+ return this;
+ }
+
+ public DorisConnectionOptionsBuilder withPassword(String password) {
+ this.password = password;
+ return this;
+ }
+
+ public DorisConnectionOptions build() {
+ return new DorisConnectionOptions(fenodes, username, password);
+ }
+ }
+
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisExecutionOptions.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisExecutionOptions.java
new file mode 100644
index 00000000000000..ee8b09e4b2ccc7
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisExecutionOptions.java
@@ -0,0 +1,71 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package org.apache.doris.flink.cfg;
+
+
+import java.io.Serializable;
+
+/**
+ * JDBC sink batch options.
+ */
+public class DorisExecutionOptions implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final Integer batchSize;
+ private final Integer maxRetries;
+
+ public DorisExecutionOptions(Integer batchSize, Integer maxRetries) {
+ this.batchSize = batchSize;
+ this.maxRetries = maxRetries;
+ }
+
+ public Integer getBatchSize() {
+ return batchSize;
+ }
+
+ public Integer getMaxRetries() {
+ return maxRetries;
+ }
+
+ public static Builder builder() {
+ return new Builder();
+ }
+
+ /**
+ * Builder of {@link DorisExecutionOptions}.
+ */
+ public static class Builder {
+ private Integer batchSize;
+ private Integer maxRetries;
+
+ public Builder setBatchSize(Integer batchSize) {
+ this.batchSize = batchSize;
+ return this;
+ }
+
+ public Builder setMaxRetries(Integer maxRetries) {
+ this.maxRetries = maxRetries;
+ return this;
+ }
+
+ public DorisExecutionOptions build() {
+ return new DorisExecutionOptions(batchSize,maxRetries);
+ }
+ }
+
+
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisOptions.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisOptions.java
new file mode 100644
index 00000000000000..c542d6b87de2b7
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisOptions.java
@@ -0,0 +1,103 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package org.apache.doris.flink.cfg;
+
+import org.apache.doris.flink.util.IOUtils;
+
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Options for the Doris connector.
+ */
+public class DorisOptions extends DorisConnectionOptions{
+
+ private static final long serialVersionUID = 1L;
+
+ private String tableIdentifier;
+
+
+ public DorisOptions(String fenodes, String username, String password, String tableIdentifier) {
+ super(fenodes, username, password);
+ this.tableIdentifier = tableIdentifier;
+ }
+
+ public String getTableIdentifier() {
+ return tableIdentifier;
+ }
+
+ public String save() throws IllegalArgumentException {
+ Properties copy = new Properties();
+ return IOUtils.propsToString(copy);
+ }
+
+ public static Builder builder() {
+ return new Builder();
+ }
+
+ /**
+ * Builder of {@link DorisOptions}.
+ */
+ public static class Builder {
+ private String fenodes;
+ private String username;
+ private String password;
+ private String tableIdentifier;
+
+ /**
+ * required, tableIdentifier
+ */
+ public Builder setTableIdentifier(String tableIdentifier) {
+ this.tableIdentifier = tableIdentifier;
+ return this;
+ }
+
+ /**
+ * optional, user name.
+ */
+ public Builder setUsername(String username) {
+ this.username = username;
+ return this;
+ }
+
+ /**
+ * optional, password.
+ */
+ public Builder setPassword(String password) {
+ this.password = password;
+ return this;
+ }
+
+ /**
+ * required, JDBC DB url.
+ */
+ public Builder setFenodes(String fenodes) {
+ this.fenodes = fenodes;
+ return this;
+ }
+
+
+ public DorisOptions build() {
+ checkNotNull(fenodes, "No fenodes supplied.");
+ checkNotNull(tableIdentifier, "No tableIdentifier supplied.");
+ return new DorisOptions(fenodes, username, password, tableIdentifier);
+ }
+ }
+
+
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisReadOptions.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisReadOptions.java
new file mode 100644
index 00000000000000..833ecf5bb1258e
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisReadOptions.java
@@ -0,0 +1,185 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package org.apache.doris.flink.cfg;
+
+
+import java.io.Serializable;
+
+/**
+ * Doris read Options
+ */
+public class DorisReadOptions implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private String readFields;
+ private String filterQuery;
+ private Integer requestTabletSize;
+ private Integer requestConnectTimeoutMs;
+ private Integer requestReadTimeoutMs;
+ private Integer requestQueryTimeoutS;
+ private Integer requestRetries;
+ private Integer requestBatchSize;
+ private Long execMemLimit;
+ private Integer deserializeQueueSize;
+ private Boolean deserializeArrowAsync;
+
+ public DorisReadOptions(String readFields, String filterQuery, Integer requestTabletSize, Integer requestConnectTimeoutMs, Integer requestReadTimeoutMs,
+ Integer requestQueryTimeoutS, Integer requestRetries, Integer requestBatchSize, Long execMemLimit,
+ Integer deserializeQueueSize, Boolean deserializeArrowAsync) {
+ this.readFields = readFields;
+ this.filterQuery = filterQuery;
+ this.requestTabletSize = requestTabletSize;
+ this.requestConnectTimeoutMs = requestConnectTimeoutMs;
+ this.requestReadTimeoutMs = requestReadTimeoutMs;
+ this.requestQueryTimeoutS = requestQueryTimeoutS;
+ this.requestRetries = requestRetries;
+ this.requestBatchSize = requestBatchSize;
+ this.execMemLimit = execMemLimit;
+ this.deserializeQueueSize = deserializeQueueSize;
+ this.deserializeArrowAsync = deserializeArrowAsync;
+ }
+
+ public String getReadFields() {
+ return readFields;
+ }
+
+ public String getFilterQuery() {
+ return filterQuery;
+ }
+
+ public Integer getRequestTabletSize() {
+ return requestTabletSize;
+ }
+
+ public Integer getRequestConnectTimeoutMs() {
+ return requestConnectTimeoutMs;
+ }
+
+ public Integer getRequestReadTimeoutMs() {
+ return requestReadTimeoutMs;
+ }
+
+ public Integer getRequestRetries() {
+ return requestRetries;
+ }
+
+ public Integer getRequestBatchSize() {
+ return requestBatchSize;
+ }
+
+ public Integer getRequestQueryTimeoutS() {
+ return requestQueryTimeoutS;
+ }
+
+ public Long getExecMemLimit() {
+ return execMemLimit;
+ }
+
+ public Integer getDeserializeQueueSize() {
+ return deserializeQueueSize;
+ }
+
+ public Boolean getDeserializeArrowAsync() {
+ return deserializeArrowAsync;
+ }
+
+
+ public static Builder builder() {
+ return new Builder();
+ }
+
+ /**
+ * Builder of {@link DorisReadOptions}.
+ */
+ public static class Builder {
+
+ private String readFields;
+ private String filterQuery;
+ private Integer requestTabletSize;
+ private Integer requestConnectTimeoutMs;
+ private Integer requestReadTimeoutMs;
+ private Integer requestQueryTimeoutS;
+ private Integer requestRetries;
+ private Integer requestBatchSize;
+ private Long execMemLimit;
+ private Integer deserializeQueueSize;
+ private Boolean deserializeArrowAsync;
+
+
+ public Builder setReadFields(String readFields) {
+ this.readFields = readFields;
+ return this;
+ }
+
+ public Builder setFilterQuery(String filterQuery) {
+ this.filterQuery = filterQuery;
+ return this;
+ }
+
+ public Builder setRequestTabletSize(Integer requestTabletSize) {
+ this.requestTabletSize = requestTabletSize;
+ return this;
+ }
+
+ public Builder setRequestConnectTimeoutMs(Integer requestConnectTimeoutMs) {
+ this.requestConnectTimeoutMs = requestConnectTimeoutMs;
+ return this;
+ }
+
+ public Builder setRequestReadTimeoutMs(Integer requestReadTimeoutMs) {
+ this.requestReadTimeoutMs = requestReadTimeoutMs;
+ return this;
+ }
+
+ public Builder setRequestQueryTimeoutS(Integer requesQueryTimeoutS) {
+ this.requestQueryTimeoutS = requesQueryTimeoutS;
+ return this;
+ }
+
+ public Builder setRequestRetries(Integer requestRetries) {
+ this.requestRetries = requestRetries;
+ return this;
+ }
+
+ public Builder setRequestBatchSize(Integer requestBatchSize) {
+ this.requestBatchSize = requestBatchSize;
+ return this;
+ }
+
+ public Builder setExecMemLimit(Long execMemLimit) {
+ this.execMemLimit = execMemLimit;
+ return this;
+ }
+
+ public Builder setDeserializeQueueSize(Integer deserializeQueueSize) {
+ this.deserializeQueueSize = deserializeQueueSize;
+ return this;
+ }
+
+ public Builder setDeserializeArrowAsync(Boolean deserializeArrowAsync) {
+ this.deserializeArrowAsync = deserializeArrowAsync;
+ return this;
+ }
+
+ public DorisReadOptions build() {
+ return new DorisReadOptions(readFields,filterQuery,requestTabletSize,requestConnectTimeoutMs,requestReadTimeoutMs,requestQueryTimeoutS,requestRetries,requestBatchSize,execMemLimit,deserializeQueueSize,deserializeArrowAsync);
+ }
+ }
+
+
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisStreamOptions.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisStreamOptions.java
new file mode 100644
index 00000000000000..016b29ae84c75b
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisStreamOptions.java
@@ -0,0 +1,74 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package org.apache.doris.flink.cfg;
+
+
+import java.io.Serializable;
+import java.util.Properties;
+
+
+/**
+ * Options for the Doris stream connector.
+ */
+public class DorisStreamOptions implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private Properties prop;
+ private DorisOptions options;
+ private DorisReadOptions readOptions;
+
+ public DorisStreamOptions(Properties prop) {
+ this.prop = prop;
+ init();
+ }
+
+ /**
+ * convert DorisStreamOptions to DorisOptions and DorisReadOptions
+ */
+ private void init() {
+ DorisOptions.Builder optionsBuilder = DorisOptions.builder()
+ .setFenodes(prop.getProperty(ConfigurationOptions.DORIS_FENODES))
+ .setUsername(prop.getProperty(ConfigurationOptions.DORIS_USER))
+ .setPassword(prop.getProperty(ConfigurationOptions.DORIS_PASSWORD))
+ .setTableIdentifier(prop.getProperty(ConfigurationOptions.TABLE_IDENTIFIER));
+
+ DorisReadOptions.Builder readOptionsBuilder = DorisReadOptions.builder()
+ .setDeserializeArrowAsync(Boolean.valueOf(prop.getProperty(ConfigurationOptions.DORIS_DESERIALIZE_ARROW_ASYNC,ConfigurationOptions.DORIS_DESERIALIZE_ARROW_ASYNC_DEFAULT.toString())))
+ .setDeserializeQueueSize(Integer.valueOf(prop.getProperty(ConfigurationOptions.DORIS_DESERIALIZE_QUEUE_SIZE,ConfigurationOptions.DORIS_DESERIALIZE_QUEUE_SIZE_DEFAULT.toString())))
+ .setExecMemLimit(Long.valueOf(prop.getProperty(ConfigurationOptions.DORIS_EXEC_MEM_LIMIT,ConfigurationOptions.DORIS_EXEC_MEM_LIMIT_DEFAULT.toString())))
+ .setFilterQuery(prop.getProperty(ConfigurationOptions.DORIS_FILTER_QUERY))
+ .setReadFields(prop.getProperty(ConfigurationOptions.DORIS_READ_FIELD))
+ .setRequestQueryTimeoutS(Integer.valueOf(prop.getProperty(ConfigurationOptions.DORIS_REQUEST_QUERY_TIMEOUT_S,ConfigurationOptions.DORIS_REQUEST_QUERY_TIMEOUT_S_DEFAULT.toString())))
+ .setRequestBatchSize(Integer.valueOf(prop.getProperty(ConfigurationOptions.DORIS_BATCH_SIZE,ConfigurationOptions.DORIS_BATCH_SIZE_DEFAULT.toString())))
+ .setRequestConnectTimeoutMs(Integer.valueOf(prop.getProperty(ConfigurationOptions.DORIS_REQUEST_CONNECT_TIMEOUT_MS,ConfigurationOptions.DORIS_REQUEST_CONNECT_TIMEOUT_MS_DEFAULT.toString())))
+ .setRequestReadTimeoutMs(Integer.valueOf(prop.getProperty(ConfigurationOptions.DORIS_REQUEST_READ_TIMEOUT_MS,ConfigurationOptions.DORIS_REQUEST_READ_TIMEOUT_MS_DEFAULT.toString())))
+ .setRequestRetries(Integer.valueOf(prop.getProperty(ConfigurationOptions.DORIS_REQUEST_RETRIES,ConfigurationOptions.DORIS_REQUEST_RETRIES_DEFAULT.toString())))
+ .setRequestTabletSize(Integer.valueOf(prop.getProperty(ConfigurationOptions.DORIS_TABLET_SIZE,ConfigurationOptions.DORIS_TABLET_SIZE_DEFAULT.toString())));
+
+ this.options = optionsBuilder.build();
+ this.readOptions = readOptionsBuilder.build();
+
+ }
+
+ public DorisOptions getOptions() {
+ return options;
+ }
+
+ public DorisReadOptions getReadOptions() {
+ return readOptions;
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/datastream/DorisSourceFunction.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/datastream/DorisSourceFunction.java
new file mode 100644
index 00000000000000..5c580db2bb194c
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/datastream/DorisSourceFunction.java
@@ -0,0 +1,82 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package org.apache.doris.flink.datastream;
+
+import org.apache.doris.flink.cfg.DorisOptions;
+import org.apache.doris.flink.cfg.DorisReadOptions;
+import org.apache.doris.flink.cfg.DorisStreamOptions;
+import org.apache.doris.flink.deserialization.DorisDeserializationSchema;
+import org.apache.doris.flink.rest.PartitionDefinition;
+import org.apache.doris.flink.rest.RestService;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+
+/**
+ * DorisSource
+ **/
+
+public class DorisSourceFunction extends RichSourceFunction implements ResultTypeQueryable {
+
+ private static final Logger logger = LoggerFactory.getLogger(DorisSourceFunction.class);
+
+ private DorisDeserializationSchema deserializer;
+ private DorisOptions options;
+ private DorisReadOptions readOptions;
+ private List dorisPartitions;
+ private ScalaValueReader scalaValueReader;
+
+ public DorisSourceFunction(DorisStreamOptions streamOptions, DorisDeserializationSchema deserializer) {
+ this.deserializer = deserializer;
+ this.options = streamOptions.getOptions();
+ this.readOptions = streamOptions.getReadOptions();
+ }
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ super.open(parameters);
+ this.dorisPartitions = RestService.findPartitions(options,readOptions,logger);
+ }
+
+ @Override
+ public void run(SourceContext sourceContext) throws Exception{
+ for(PartitionDefinition partitions : dorisPartitions){
+ scalaValueReader = new ScalaValueReader(partitions, options,readOptions);
+ while (scalaValueReader.hasNext()){
+ Object next = scalaValueReader.next();
+ sourceContext.collect(next);
+ }
+ }
+ }
+
+ @Override
+ public void cancel() {
+ scalaValueReader.close();
+ }
+
+
+ @Override
+ public TypeInformation getProducedType() {
+ return this.deserializer.getProducedType();
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/deserialization/DorisDeserializationSchema.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/deserialization/DorisDeserializationSchema.java
new file mode 100644
index 00000000000000..ba0921b9aafb02
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/deserialization/DorisDeserializationSchema.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.doris.flink.deserialization;
+
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+
+import java.io.Serializable;
+
+
+public interface DorisDeserializationSchema extends Serializable, ResultTypeQueryable {
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/deserialization/SimpleListDeserializationSchema.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/deserialization/SimpleListDeserializationSchema.java
new file mode 100644
index 00000000000000..6fd68ec4e5ad53
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/deserialization/SimpleListDeserializationSchema.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.doris.flink.deserialization;
+
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import java.util.List;
+
+
+public class SimpleListDeserializationSchema implements DorisDeserializationSchema{
+
+ @Override
+ public TypeInformation getProducedType() {
+ return TypeInformation.of(List.class);
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/ConnectedFailedException.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/ConnectedFailedException.java
new file mode 100644
index 00000000000000..e25d1a592ad604
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/ConnectedFailedException.java
@@ -0,0 +1,28 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.flink.exception;
+
+public class ConnectedFailedException extends DorisException {
+ public ConnectedFailedException(String server, Throwable cause) {
+ super("Connect to " + server + "failed.", cause);
+ }
+
+ public ConnectedFailedException(String server, int statusCode, Throwable cause) {
+ super("Connect to " + server + "failed, status code is " + statusCode + ".", cause);
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisException.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisException.java
new file mode 100644
index 00000000000000..26e11e54702f1b
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisException.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.doris.flink.exception;
+
+public class DorisException extends Exception {
+ public DorisException() {
+ super();
+ }
+ public DorisException(String message) {
+ super(message);
+ }
+ public DorisException(String message, Throwable cause) {
+ super(message, cause);
+ }
+ public DorisException(Throwable cause) {
+ super(cause);
+ }
+ protected DorisException(String message, Throwable cause,
+ boolean enableSuppression,
+ boolean writableStackTrace) {
+ super(message, cause, enableSuppression, writableStackTrace);
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisInternalException.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisInternalException.java
new file mode 100644
index 00000000000000..eadd860de52d56
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/DorisInternalException.java
@@ -0,0 +1,29 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.flink.exception;
+
+import org.apache.doris.thrift.TStatusCode;
+
+import java.util.List;
+
+public class DorisInternalException extends DorisException {
+ public DorisInternalException(String server, TStatusCode statusCode, List errorMsgs) {
+ super("Doris server " + server + " internal failed, status code [" + statusCode + "] error message is " + errorMsgs);
+ }
+
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/IllegalArgumentException.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/IllegalArgumentException.java
new file mode 100644
index 00000000000000..4c0ae0939ae3fa
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/IllegalArgumentException.java
@@ -0,0 +1,28 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.flink.exception;
+
+public class IllegalArgumentException extends DorisException {
+ public IllegalArgumentException(String msg, Throwable cause) {
+ super(msg, cause);
+ }
+
+ public IllegalArgumentException(String arg, String value) {
+ super("argument '" + arg + "' is illegal, value is '" + value + "'.");
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/ShouldNeverHappenException.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/ShouldNeverHappenException.java
new file mode 100644
index 00000000000000..307c3988459f9f
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/ShouldNeverHappenException.java
@@ -0,0 +1,20 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.flink.exception;
+
+public class ShouldNeverHappenException extends DorisException { }
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/StreamLoadException.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/StreamLoadException.java
new file mode 100644
index 00000000000000..6d7a192363bc16
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/exception/StreamLoadException.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.doris.flink.exception;
+
+public class StreamLoadException extends Exception {
+ public StreamLoadException() {
+ super();
+ }
+ public StreamLoadException(String message) {
+ super(message);
+ }
+ public StreamLoadException(String message, Throwable cause) {
+ super(message, cause);
+ }
+ public StreamLoadException(Throwable cause) {
+ super(cause);
+ }
+ protected StreamLoadException(String message, Throwable cause,
+ boolean enableSuppression,
+ boolean writableStackTrace) {
+ super(message, cause, enableSuppression, writableStackTrace);
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/PartitionDefinition.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/PartitionDefinition.java
new file mode 100644
index 00000000000000..19edd21580bcea
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/PartitionDefinition.java
@@ -0,0 +1,150 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.flink.rest;
+
+import org.apache.doris.flink.cfg.DorisOptions;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * Doris partition info.
+ */
+public class PartitionDefinition implements Serializable, Comparable {
+ private final String database;
+ private final String table;
+
+ private final String beAddress;
+ private final Set tabletIds;
+ private final String queryPlan;
+ private final String serializedSettings;
+
+ public PartitionDefinition(String database, String table,
+ DorisOptions settings, String beAddress, Set tabletIds, String queryPlan)
+ throws IllegalArgumentException {
+ if (settings != null) {
+ this.serializedSettings = settings.save();
+ } else {
+ this.serializedSettings = null;
+ }
+ this.database = database;
+ this.table = table;
+ this.beAddress = beAddress;
+ this.tabletIds = tabletIds;
+ this.queryPlan = queryPlan;
+ }
+
+ public String getBeAddress() {
+ return beAddress;
+ }
+
+ public Set getTabletIds() {
+ return tabletIds;
+ }
+
+ public String getDatabase() {
+ return database;
+ }
+
+ public String getTable() {
+ return table;
+ }
+
+ public String getQueryPlan() {
+ return queryPlan;
+ }
+
+
+ @Override
+ public int compareTo(PartitionDefinition o) {
+ int cmp = database.compareTo(o.database);
+ if (cmp != 0) {
+ return cmp;
+ }
+ cmp = table.compareTo(o.table);
+ if (cmp != 0) {
+ return cmp;
+ }
+ cmp = beAddress.compareTo(o.beAddress);
+ if (cmp != 0) {
+ return cmp;
+ }
+ cmp = queryPlan.compareTo(o.queryPlan);
+ if (cmp != 0) {
+ return cmp;
+ }
+
+ cmp = tabletIds.size() - o.tabletIds.size();
+ if (cmp != 0) {
+ return cmp;
+ }
+
+ Set similar = new HashSet<>(tabletIds);
+ Set diffSelf = new HashSet<>(tabletIds);
+ Set diffOther = new HashSet<>(o.tabletIds);
+ similar.retainAll(o.tabletIds);
+ diffSelf.removeAll(similar);
+ diffOther.removeAll(similar);
+ if (diffSelf.size() == 0) {
+ return 0;
+ }
+ long diff = Collections.min(diffSelf) - Collections.min(diffOther);
+ return diff < 0 ? -1 : 1;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ PartitionDefinition that = (PartitionDefinition) o;
+ return Objects.equals(database, that.database) &&
+ Objects.equals(table, that.table) &&
+ Objects.equals(beAddress, that.beAddress) &&
+ Objects.equals(tabletIds, that.tabletIds) &&
+ Objects.equals(queryPlan, that.queryPlan) &&
+ Objects.equals(serializedSettings, that.serializedSettings);
+ }
+
+ @Override
+ public int hashCode() {
+ int result = database.hashCode();
+ result = 31 * result + table.hashCode();
+ result = 31 * result + beAddress.hashCode();
+ result = 31 * result + queryPlan.hashCode();
+ result = 31 * result + tabletIds.hashCode();
+ return result;
+ }
+
+ @Override
+ public String toString() {
+ return "PartitionDefinition{" +
+ ", database='" + database + '\'' +
+ ", table='" + table + '\'' +
+ ", beAddress='" + beAddress + '\'' +
+ ", tabletIds=" + tabletIds +
+ ", queryPlan='" + queryPlan + '\'' +
+ '}';
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/RestService.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/RestService.java
new file mode 100644
index 00000000000000..469f1aa4a46c58
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/RestService.java
@@ -0,0 +1,573 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.flink.rest;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.commons.io.IOUtils;
+import org.apache.doris.flink.cfg.DorisOptions;
+import org.apache.doris.flink.cfg.DorisReadOptions;
+import org.apache.doris.flink.exception.IllegalArgumentException;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.doris.flink.cfg.ConfigurationOptions;
+import org.apache.doris.flink.exception.ConnectedFailedException;
+import org.apache.doris.flink.exception.DorisException;
+import org.apache.doris.flink.exception.ShouldNeverHappenException;
+import org.apache.doris.flink.rest.models.Backend;
+import org.apache.doris.flink.rest.models.BackendRow;
+import org.apache.doris.flink.rest.models.QueryPlan;
+import org.apache.doris.flink.rest.models.Schema;
+import org.apache.doris.flink.rest.models.Tablet;
+import org.apache.flink.shaded.guava18.com.google.common.annotations.VisibleForTesting;
+import org.apache.http.HttpStatus;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.methods.HttpRequestBase;
+import org.apache.http.entity.StringEntity;
+
+
+import org.slf4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+
+
+import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_TABLET_SIZE;
+import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_TABLET_SIZE_DEFAULT;
+import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_TABLET_SIZE_MIN;
+import static org.apache.doris.flink.util.ErrorMessages.CONNECT_FAILED_MESSAGE;
+import static org.apache.doris.flink.util.ErrorMessages.ILLEGAL_ARGUMENT_MESSAGE;
+import static org.apache.doris.flink.util.ErrorMessages.SHOULD_NOT_HAPPEN_MESSAGE;
+
+
+/**
+ * Service for communicate with Doris FE.
+ */
+public class RestService implements Serializable {
+ public final static int REST_RESPONSE_STATUS_OK = 200;
+ public final static int REST_RESPONSE_CODE_OK = 0;
+ private final static String REST_RESPONSE_BE_ROWS_KEY = "rows";
+ private static final String API_PREFIX = "/api";
+ private static final String SCHEMA = "_schema";
+ private static final String QUERY_PLAN = "_query_plan";
+ private static final String BACKENDS = "/rest/v1/system?path=//backends";
+ private static final String FE_LOGIN = "/rest/v1/login";
+
+ /**
+ * send request to Doris FE and get response json string.
+ * @param options configuration of request
+ * @param request {@link HttpRequestBase} real request
+ * @param logger {@link Logger}
+ * @return Doris FE response in json string
+ * @throws ConnectedFailedException throw when cannot connect to Doris FE
+ */
+ private static String send(DorisOptions options,DorisReadOptions readOptions, HttpRequestBase request, Logger logger) throws
+ ConnectedFailedException {
+ int connectTimeout = readOptions.getRequestConnectTimeoutMs() == null ? ConfigurationOptions.DORIS_REQUEST_CONNECT_TIMEOUT_MS_DEFAULT : readOptions.getRequestConnectTimeoutMs();
+ int socketTimeout = readOptions.getRequestReadTimeoutMs() == null ? ConfigurationOptions.DORIS_REQUEST_READ_TIMEOUT_MS_DEFAULT : readOptions.getRequestReadTimeoutMs();
+ int retries = readOptions.getRequestRetries() == null ? ConfigurationOptions.DORIS_REQUEST_RETRIES_DEFAULT : readOptions.getRequestRetries();
+ logger.trace("connect timeout set to '{}'. socket timeout set to '{}'. retries set to '{}'.",
+ connectTimeout, socketTimeout, retries);
+
+ RequestConfig requestConfig = RequestConfig.custom()
+ .setConnectTimeout(connectTimeout)
+ .setSocketTimeout(socketTimeout)
+ .build();
+
+ request.setConfig(requestConfig);
+
+
+ logger.info("Send request to Doris FE '{}' with user '{}'.", request.getURI(), options.getUsername());
+
+ IOException ex = null;
+ int statusCode = -1;
+
+ for (int attempt = 0; attempt < retries; attempt++) {
+ logger.debug("Attempt {} to request {}.", attempt, request.getURI());
+ try {
+ String response;
+ if(request instanceof HttpGet){
+ response = getConnectionGet(request.getURI().toString(), options.getUsername(), options.getPassword(),logger);
+ }else{
+ response = getConnection(request, options.getUsername(), options.getPassword(),logger);
+ }
+ logger.trace("Success get response from Doris FE: {}, response is: {}.",
+ request.getURI(), response);
+ //Handle the problem of inconsistent data format returned by http v1 and v2
+ ObjectMapper mapper = new ObjectMapper();
+ Map map = mapper.readValue(response, Map.class);
+ if(map.containsKey("code") && map.containsKey("msg")) {
+ Object data = map.get("data");
+ return mapper.writeValueAsString(data);
+ } else {
+ return response;
+ }
+ } catch (IOException e) {
+ ex = e;
+ logger.warn(CONNECT_FAILED_MESSAGE, request.getURI(), e);
+ }
+ }
+
+ logger.error(CONNECT_FAILED_MESSAGE, request.getURI(), ex);
+ throw new ConnectedFailedException(request.getURI().toString(), statusCode, ex);
+ }
+
+ private static String getConnection(HttpRequestBase request,String user, String passwd,Logger logger) throws IOException {
+ URL url = new URL(request.getURI().toString());
+ HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+ conn.setInstanceFollowRedirects(false);
+ conn.setRequestMethod("POST");
+ String authEncoding = Base64.getEncoder().encodeToString(String.format("%s:%s", user, passwd).getBytes(StandardCharsets.UTF_8));
+ conn.setRequestProperty("Authorization", "Basic " + authEncoding);
+
+ InputStream content = ((HttpPost)request).getEntity().getContent();
+ String res = IOUtils.toString(content);
+ conn.setDoOutput(true);
+ conn.setDoInput(true);
+ PrintWriter out = new PrintWriter(conn.getOutputStream());
+ // send request params
+ out.print(res);
+ // flush
+ out.flush();
+ // read response
+ return parseResponse(conn,logger);
+ }
+
+ private static String getConnectionGet(String request,String user, String passwd,Logger logger) throws IOException {
+ URL realUrl = new URL(request);
+ // open connection
+ HttpURLConnection connection = (HttpURLConnection)realUrl.openConnection();
+ String authEncoding = Base64.getEncoder().encodeToString(String.format("%s:%s", user, passwd).getBytes(StandardCharsets.UTF_8));
+ connection.setRequestProperty("Authorization", "Basic " + authEncoding);
+
+ connection.connect();
+ return parseResponse(connection,logger);
+ }
+
+ private static String parseResponse(HttpURLConnection connection,Logger logger) throws IOException {
+ if (connection.getResponseCode() != HttpStatus.SC_OK) {
+ logger.warn("Failed to get response from Doris {}, http code is {}",
+ connection.getURL(), connection.getResponseCode());
+ throw new IOException("Failed to get response from Doris");
+ }
+ String result = "";
+ BufferedReader in = new BufferedReader(new InputStreamReader(connection.getInputStream(), "utf-8"));
+ String line;
+ while ((line = in.readLine()) != null) {
+ result += line;
+ }
+ if (in != null) {
+ in.close();
+ }
+ return result;
+ }
+
+ /**
+ * parse table identifier to array.
+ * @param tableIdentifier table identifier string
+ * @param logger {@link Logger}
+ * @return first element is db name, second element is table name
+ * @throws IllegalArgumentException table identifier is illegal
+ */
+ @VisibleForTesting
+ static String[] parseIdentifier(String tableIdentifier, Logger logger) throws IllegalArgumentException {
+ logger.trace("Parse identifier '{}'.", tableIdentifier);
+ if (StringUtils.isEmpty(tableIdentifier)) {
+ logger.error(ILLEGAL_ARGUMENT_MESSAGE, "table.identifier", tableIdentifier);
+ throw new IllegalArgumentException("table.identifier", tableIdentifier);
+ }
+ String[] identifier = tableIdentifier.split("\\.");
+ if (identifier.length != 2) {
+ logger.error(ILLEGAL_ARGUMENT_MESSAGE, "table.identifier", tableIdentifier);
+ throw new IllegalArgumentException("table.identifier", tableIdentifier);
+ }
+ return identifier;
+ }
+
+ /**
+ * choice a Doris FE node to request.
+ * @param feNodes Doris FE node list, separate be comma
+ * @param logger slf4j logger
+ * @return the chosen one Doris FE node
+ * @throws IllegalArgumentException fe nodes is illegal
+ */
+ @VisibleForTesting
+ static String randomEndpoint(String feNodes, Logger logger) throws IllegalArgumentException {
+ logger.trace("Parse fenodes '{}'.", feNodes);
+ if (StringUtils.isEmpty(feNodes)) {
+ logger.error(ILLEGAL_ARGUMENT_MESSAGE, "fenodes", feNodes);
+ throw new IllegalArgumentException("fenodes", feNodes);
+ }
+ List nodes = Arrays.asList(feNodes.split(","));
+ Collections.shuffle(nodes);
+ return nodes.get(0).trim();
+ }
+
+ /**
+ * choice a Doris BE node to request.
+ * @param options configuration of request
+ * @param logger slf4j logger
+ * @return the chosen one Doris BE node
+ * @throws IllegalArgumentException BE nodes is illegal
+ */
+ @VisibleForTesting
+ public static String randomBackend(DorisOptions options,DorisReadOptions readOptions ,Logger logger) throws DorisException, IOException {
+ List backends = getBackends(options,readOptions, logger);
+ logger.trace("Parse beNodes '{}'.", backends);
+ if (backends == null || backends.isEmpty()) {
+ logger.error(ILLEGAL_ARGUMENT_MESSAGE, "beNodes", backends);
+ throw new IllegalArgumentException("beNodes", String.valueOf(backends));
+ }
+ Collections.shuffle(backends);
+ BackendRow backend = backends.get(0);
+ return backend.getIP() + ":" + backend.getHttpPort();
+ }
+
+ /**
+ * get Doris BE nodes to request.
+ * @param options configuration of request
+ * @param logger slf4j logger
+ * @return the chosen one Doris BE node
+ * @throws IllegalArgumentException BE nodes is illegal
+ */
+ @VisibleForTesting
+ static List getBackends(DorisOptions options,DorisReadOptions readOptions, Logger logger) throws DorisException, IOException {
+ String feNodes = options.getFenodes();
+ String feNode = randomEndpoint(feNodes, logger);
+ String beUrl = "http://" + feNode + BACKENDS;
+ HttpGet httpGet = new HttpGet(beUrl);
+ String response = send(options, readOptions,httpGet, logger);
+ logger.info("Backend Info:{}",response);
+ List backends = parseBackend(response, logger);
+ return backends;
+ }
+
+ static List parseBackend(String response, Logger logger) throws DorisException, IOException {
+ ObjectMapper mapper = new ObjectMapper();
+ Backend backend;
+ try {
+ backend = mapper.readValue(response, Backend.class);
+ } catch (JsonParseException e) {
+ String errMsg = "Doris BE's response is not a json. res: " + response;
+ logger.error(errMsg, e);
+ throw new DorisException(errMsg, e);
+ } catch (JsonMappingException e) {
+ String errMsg = "Doris BE's response cannot map to schema. res: " + response;
+ logger.error(errMsg, e);
+ throw new DorisException(errMsg, e);
+ } catch (IOException e) {
+ String errMsg = "Parse Doris BE's response to json failed. res: " + response;
+ logger.error(errMsg, e);
+ throw new DorisException(errMsg, e);
+ }
+
+ if (backend == null) {
+ logger.error(SHOULD_NOT_HAPPEN_MESSAGE);
+ throw new ShouldNeverHappenException();
+ }
+ List backendRows = backend.getRows().stream().filter(v -> v.getAlive()).collect(Collectors.toList());
+ logger.debug("Parsing schema result is '{}'.", backendRows);
+ return backendRows;
+ }
+
+ /**
+ * get a valid URI to connect Doris FE.
+ * @param options configuration of request
+ * @param logger {@link Logger}
+ * @return uri string
+ * @throws IllegalArgumentException throw when configuration is illegal
+ */
+ @VisibleForTesting
+ static String getUriStr(DorisOptions options, Logger logger) throws IllegalArgumentException {
+ String[] identifier = parseIdentifier(options.getTableIdentifier(), logger);
+ return "http://" +
+ randomEndpoint(options.getFenodes(), logger) + API_PREFIX +
+ "/" + identifier[0] +
+ "/" + identifier[1] +
+ "/";
+ }
+
+ /**
+ * discover Doris table schema from Doris FE.
+ * @param options configuration of request
+ * @param logger slf4j logger
+ * @return Doris table schema
+ * @throws DorisException throw when discover failed
+ */
+ public static Schema getSchema(DorisOptions options,DorisReadOptions readOptions, Logger logger)
+ throws DorisException {
+ logger.trace("Finding schema.");
+ HttpGet httpGet = new HttpGet(getUriStr(options, logger) + SCHEMA);
+ String response = send(options, readOptions,httpGet, logger);
+ logger.debug("Find schema response is '{}'.", response);
+ return parseSchema(response, logger);
+ }
+
+ /**
+ * translate Doris FE response to inner {@link Schema} struct.
+ * @param response Doris FE response
+ * @param logger {@link Logger}
+ * @return inner {@link Schema} struct
+ * @throws DorisException throw when translate failed
+ */
+ @VisibleForTesting
+ public static Schema parseSchema(String response, Logger logger) throws DorisException {
+ logger.trace("Parse response '{}' to schema.", response);
+ ObjectMapper mapper = new ObjectMapper();
+ Schema schema;
+ try {
+ schema = mapper.readValue(response, Schema.class);
+ } catch (JsonParseException e) {
+ String errMsg = "Doris FE's response is not a json. res: " + response;
+ logger.error(errMsg, e);
+ throw new DorisException(errMsg, e);
+ } catch (JsonMappingException e) {
+ String errMsg = "Doris FE's response cannot map to schema. res: " + response;
+ logger.error(errMsg, e);
+ throw new DorisException(errMsg, e);
+ } catch (IOException e) {
+ String errMsg = "Parse Doris FE's response to json failed. res: " + response;
+ logger.error(errMsg, e);
+ throw new DorisException(errMsg, e);
+ }
+
+ if (schema == null) {
+ logger.error(SHOULD_NOT_HAPPEN_MESSAGE);
+ throw new ShouldNeverHappenException();
+ }
+
+ if (schema.getStatus() != REST_RESPONSE_STATUS_OK) {
+ String errMsg = "Doris FE's response is not OK, status is " + schema.getStatus();
+ logger.error(errMsg);
+ throw new DorisException(errMsg);
+ }
+ logger.debug("Parsing schema result is '{}'.", schema);
+ return schema;
+ }
+
+ /**
+ * find Doris RDD partitions from Doris FE.
+ * @param options configuration of request
+ * @param logger {@link Logger}
+ * @return an list of Doris RDD partitions
+ * @throws DorisException throw when find partition failed
+ */
+ public static List findPartitions(DorisOptions options, DorisReadOptions readOptions, Logger logger) throws DorisException {
+ String[] tableIdentifiers = parseIdentifier(options.getTableIdentifier(), logger);
+ String readFields = StringUtils.isBlank(readOptions.getReadFields()) ? "*" :readOptions.getReadFields();
+ String sql = "select " + readFields +
+ " from `" + tableIdentifiers[0] + "`.`" + tableIdentifiers[1] + "`";
+ if (!StringUtils.isEmpty(readOptions.getFilterQuery())) {
+ sql += " where " + readOptions.getFilterQuery();
+ }
+ logger.debug("Query SQL Sending to Doris FE is: '{}'.", sql);
+
+ HttpPost httpPost = new HttpPost(getUriStr(options, logger) + QUERY_PLAN);
+ String entity = "{\"sql\": \""+ sql +"\"}";
+ logger.debug("Post body Sending to Doris FE is: '{}'.", entity);
+ StringEntity stringEntity = new StringEntity(entity, StandardCharsets.UTF_8);
+ stringEntity.setContentEncoding("UTF-8");
+ stringEntity.setContentType("application/json");
+ httpPost.setEntity(stringEntity);
+
+ String resStr = send(options, readOptions,httpPost, logger);
+ logger.debug("Find partition response is '{}'.", resStr);
+ QueryPlan queryPlan = getQueryPlan(resStr, logger);
+ Map> be2Tablets = selectBeForTablet(queryPlan, logger);
+ return tabletsMapToPartition(
+ options,
+ readOptions,
+ be2Tablets,
+ queryPlan.getOpaqued_query_plan(),
+ tableIdentifiers[0],
+ tableIdentifiers[1],
+ logger);
+ }
+
+ /**
+ * translate Doris FE response string to inner {@link QueryPlan} struct.
+ * @param response Doris FE response string
+ * @param logger {@link Logger}
+ * @return inner {@link QueryPlan} struct
+ * @throws DorisException throw when translate failed.
+ */
+ @VisibleForTesting
+ static QueryPlan getQueryPlan(String response, Logger logger) throws DorisException {
+ ObjectMapper mapper = new ObjectMapper();
+ QueryPlan queryPlan;
+ try {
+ queryPlan = mapper.readValue(response, QueryPlan.class);
+ } catch (JsonParseException e) {
+ String errMsg = "Doris FE's response is not a json. res: " + response;
+ logger.error(errMsg, e);
+ throw new DorisException(errMsg, e);
+ } catch (JsonMappingException e) {
+ String errMsg = "Doris FE's response cannot map to schema. res: " + response;
+ logger.error(errMsg, e);
+ throw new DorisException(errMsg, e);
+ } catch (IOException e) {
+ String errMsg = "Parse Doris FE's response to json failed. res: " + response;
+ logger.error(errMsg, e);
+ throw new DorisException(errMsg, e);
+ }
+
+ if (queryPlan == null) {
+ logger.error(SHOULD_NOT_HAPPEN_MESSAGE);
+ throw new ShouldNeverHappenException();
+ }
+
+ if (queryPlan.getStatus() != REST_RESPONSE_STATUS_OK) {
+ String errMsg = "Doris FE's response is not OK, status is " + queryPlan.getStatus();
+ logger.error(errMsg);
+ throw new DorisException(errMsg);
+ }
+ logger.debug("Parsing partition result is '{}'.", queryPlan);
+ return queryPlan;
+ }
+
+ /**
+ * select which Doris BE to get tablet data.
+ * @param queryPlan {@link QueryPlan} translated from Doris FE response
+ * @param logger {@link Logger}
+ * @return BE to tablets {@link Map}
+ * @throws DorisException throw when select failed.
+ */
+ @VisibleForTesting
+ static Map> selectBeForTablet(QueryPlan queryPlan, Logger logger) throws DorisException {
+ Map> be2Tablets = new HashMap<>();
+ for (Map.Entry part : queryPlan.getPartitions().entrySet()) {
+ logger.debug("Parse tablet info: '{}'.", part);
+ long tabletId;
+ try {
+ tabletId = Long.parseLong(part.getKey());
+ } catch (NumberFormatException e) {
+ String errMsg = "Parse tablet id '" + part.getKey() + "' to long failed.";
+ logger.error(errMsg, e);
+ throw new DorisException(errMsg, e);
+ }
+ String target = null;
+ int tabletCount = Integer.MAX_VALUE;
+ for (String candidate : part.getValue().getRoutings()) {
+ logger.trace("Evaluate Doris BE '{}' to tablet '{}'.", candidate, tabletId);
+ if (!be2Tablets.containsKey(candidate)) {
+ logger.debug("Choice a new Doris BE '{}' for tablet '{}'.", candidate, tabletId);
+ List tablets = new ArrayList<>();
+ be2Tablets.put(candidate, tablets);
+ target = candidate;
+ break;
+ } else {
+ if (be2Tablets.get(candidate).size() < tabletCount) {
+ target = candidate;
+ tabletCount = be2Tablets.get(candidate).size();
+ logger.debug("Current candidate Doris BE to tablet '{}' is '{}' with tablet count {}.",
+ tabletId, target, tabletCount);
+ }
+ }
+ }
+ if (target == null) {
+ String errMsg = "Cannot choice Doris BE for tablet " + tabletId;
+ logger.error(errMsg);
+ throw new DorisException(errMsg);
+ }
+
+ logger.debug("Choice Doris BE '{}' for tablet '{}'.", target, tabletId);
+ be2Tablets.get(target).add(tabletId);
+ }
+ return be2Tablets;
+ }
+
+ /**
+ * tablet count limit for one Doris RDD partition
+ * @param readOptions configuration of request
+ * @param logger {@link Logger}
+ * @return tablet count limit
+ */
+ @VisibleForTesting
+ static int tabletCountLimitForOnePartition(DorisReadOptions readOptions, Logger logger) {
+ int tabletsSize = DORIS_TABLET_SIZE_DEFAULT;
+ if (readOptions.getRequestTabletSize() != null) {
+ tabletsSize = readOptions.getRequestTabletSize();
+ }
+ if (tabletsSize < DORIS_TABLET_SIZE_MIN) {
+ logger.warn("{} is less than {}, set to default value {}.",
+ DORIS_TABLET_SIZE, DORIS_TABLET_SIZE_MIN, DORIS_TABLET_SIZE_MIN);
+ tabletsSize = DORIS_TABLET_SIZE_MIN;
+ }
+ logger.debug("Tablet size is set to {}.", tabletsSize);
+ return tabletsSize;
+ }
+
+ /**
+ * translate BE tablets map to Doris RDD partition.
+ * @param options configuration of request
+ * @param be2Tablets BE to tablets {@link Map}
+ * @param opaquedQueryPlan Doris BE execute plan getting from Doris FE
+ * @param database database name of Doris table
+ * @param table table name of Doris table
+ * @param logger {@link Logger}
+ * @return Doris RDD partition {@link List}
+ * @throws IllegalArgumentException throw when translate failed
+ */
+ @VisibleForTesting
+ static List tabletsMapToPartition(DorisOptions options,DorisReadOptions readOptions, Map> be2Tablets,
+ String opaquedQueryPlan, String database, String table, Logger logger)
+ throws IllegalArgumentException {
+ int tabletsSize = tabletCountLimitForOnePartition(readOptions, logger);
+ List partitions = new ArrayList<>();
+ for (Map.Entry> beInfo : be2Tablets.entrySet()) {
+ logger.debug("Generate partition with beInfo: '{}'.", beInfo);
+ HashSet tabletSet = new HashSet<>(beInfo.getValue());
+ beInfo.getValue().clear();
+ beInfo.getValue().addAll(tabletSet);
+ int first = 0;
+ while (first < beInfo.getValue().size()) {
+ Set partitionTablets = new HashSet<>(beInfo.getValue().subList(
+ first, Math.min(beInfo.getValue().size(), first + tabletsSize)));
+ first = first + tabletsSize;
+ PartitionDefinition partitionDefinition =
+ new PartitionDefinition(database, table, options,
+ beInfo.getKey(), partitionTablets, opaquedQueryPlan);
+ logger.debug("Generate one PartitionDefinition '{}'.", partitionDefinition);
+ partitions.add(partitionDefinition);
+ }
+ }
+ return partitions;
+ }
+
+
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/SchemaUtils.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/SchemaUtils.java
new file mode 100644
index 00000000000000..46c54c0a2072a4
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/SchemaUtils.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.doris.flink.rest;
+
+import org.apache.doris.flink.rest.models.Field;
+import org.apache.doris.flink.rest.models.Schema;
+import org.apache.doris.thrift.TScanColumnDesc;
+
+import java.util.List;
+
+
+public class SchemaUtils {
+
+ /**
+ * convert Doris return schema to inner schema struct.
+ * @param tscanColumnDescs Doris BE return schema
+ * @return inner schema struct
+ */
+ public static Schema convertToSchema(List tscanColumnDescs ){
+ Schema schema = new Schema(tscanColumnDescs.size());
+ tscanColumnDescs.stream().forEach(desc -> schema.put(new Field(desc.getName(), desc.getType().name(), "", 0, 0)));
+ return schema;
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Backend.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Backend.java
new file mode 100644
index 00000000000000..d74e46f66f345c
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Backend.java
@@ -0,0 +1,41 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.flink.rest.models;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+/**
+ * Be response model
+ **/
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class Backend {
+
+ @JsonProperty(value = "rows")
+ private List rows;
+
+ public List getRows() {
+ return rows;
+ }
+
+ public void setRows(List rows) {
+ this.rows = rows;
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/BackendRow.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/BackendRow.java
new file mode 100644
index 00000000000000..5b7df996ad5f25
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/BackendRow.java
@@ -0,0 +1,67 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.flink.rest.models;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class BackendRow {
+
+ @JsonProperty(value = "HttpPort")
+ private String HttpPort;
+
+ @JsonProperty(value = "IP")
+ private String IP;
+
+ @JsonProperty(value = "Alive")
+ private Boolean Alive;
+
+ public String getHttpPort() {
+ return HttpPort;
+ }
+
+ public void setHttpPort(String httpPort) {
+ HttpPort = httpPort;
+ }
+
+ public String getIP() {
+ return IP;
+ }
+
+ public void setIP(String IP) {
+ this.IP = IP;
+ }
+
+ public Boolean getAlive() {
+ return Alive;
+ }
+
+ public void setAlive(Boolean alive) {
+ Alive = alive;
+ }
+
+ @Override
+ public String toString() {
+ return "BackendRow{" +
+ "HttpPort='" + HttpPort + '\'' +
+ ", IP='" + IP + '\'' +
+ ", Alive=" + Alive +
+ '}';
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Field.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Field.java
new file mode 100644
index 00000000000000..4ac66be3d36b2c
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Field.java
@@ -0,0 +1,110 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.flink.rest.models;
+
+import java.util.Objects;
+
+public class Field {
+ private String name;
+ private String type;
+ private String comment;
+ private int precision;
+ private int scale;
+
+ public Field() { }
+
+ public Field(String name, String type, String comment, int precision, int scale) {
+ this.name = name;
+ this.type = type;
+ this.comment = comment;
+ this.precision = precision;
+ this.scale = scale;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public void setName(String name) {
+ this.name = name;
+ }
+
+ public String getType() {
+ return type;
+ }
+
+ public void setType(String type) {
+ this.type = type;
+ }
+
+ public String getComment() {
+ return comment;
+ }
+
+ public void setComment(String comment) {
+ this.comment = comment;
+ }
+
+ public int getPrecision() {
+ return precision;
+ }
+
+ public void setPrecision(int precision) {
+ this.precision = precision;
+ }
+
+ public int getScale() {
+ return scale;
+ }
+
+ public void setScale(int scale) {
+ this.scale = scale;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ Field field = (Field) o;
+ return precision == field.precision &&
+ scale == field.scale &&
+ Objects.equals(name, field.name) &&
+ Objects.equals(type, field.type) &&
+ Objects.equals(comment, field.comment);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(name, type, comment, precision, scale);
+ }
+
+ @Override
+ public String toString() {
+ return "Field{" +
+ "name='" + name + '\'' +
+ ", type='" + type + '\'' +
+ ", comment='" + comment + '\'' +
+ ", precision=" + precision +
+ ", scale=" + scale +
+ '}';
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/QueryPlan.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/QueryPlan.java
new file mode 100644
index 00000000000000..e65175ca4b87c8
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/QueryPlan.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.doris.flink.rest.models;
+
+import java.util.Map;
+import java.util.Objects;
+
+public class QueryPlan {
+ private int status;
+ private String opaqued_query_plan;
+ private Map partitions;
+
+ public int getStatus() {
+ return status;
+ }
+
+ public void setStatus(int status) {
+ this.status = status;
+ }
+
+ public String getOpaqued_query_plan() {
+ return opaqued_query_plan;
+ }
+
+ public void setOpaqued_query_plan(String opaqued_query_plan) {
+ this.opaqued_query_plan = opaqued_query_plan;
+ }
+
+ public Map getPartitions() {
+ return partitions;
+ }
+
+ public void setPartitions(Map partitions) {
+ this.partitions = partitions;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ QueryPlan queryPlan = (QueryPlan) o;
+ return status == queryPlan.status &&
+ Objects.equals(opaqued_query_plan, queryPlan.opaqued_query_plan) &&
+ Objects.equals(partitions, queryPlan.partitions);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(status, opaqued_query_plan, partitions);
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/RespContent.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/RespContent.java
new file mode 100644
index 00000000000000..b86b3dd3cf2561
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/RespContent.java
@@ -0,0 +1,96 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package org.apache.doris.flink.rest.models;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class RespContent {
+
+ @JsonProperty(value = "TxnId")
+ private int TxnId;
+
+ @JsonProperty(value = "Label")
+ private String Label;
+
+ @JsonProperty(value = "Status")
+ private String Status;
+
+ @JsonProperty(value = "ExistingJobStatus")
+ private String ExistingJobStatus;
+
+ @JsonProperty(value = "Message")
+ private String Message;
+
+ @JsonProperty(value = "NumberTotalRows")
+ private long NumberTotalRows;
+
+ @JsonProperty(value = "NumberLoadedRows")
+ private long NumberLoadedRows;
+
+ @JsonProperty(value = "NumberFilteredRows")
+ private int NumberFilteredRows;
+
+ @JsonProperty(value = "NumberUnselectedRows")
+ private int NumberUnselectedRows;
+
+ @JsonProperty(value = "LoadBytes")
+ private long LoadBytes;
+
+ @JsonProperty(value = "LoadTimeMs")
+ private int LoadTimeMs;
+
+ @JsonProperty(value = "BeginTxnTimeMs")
+ private int BeginTxnTimeMs;
+
+ @JsonProperty(value = "StreamLoadPutTimeMs")
+ private int StreamLoadPutTimeMs;
+
+ @JsonProperty(value = "ReadDataTimeMs")
+ private int ReadDataTimeMs;
+
+ @JsonProperty(value = "WriteDataTimeMs")
+ private int WriteDataTimeMs;
+
+ @JsonProperty(value = "CommitAndPublishTimeMs")
+ private int CommitAndPublishTimeMs;
+
+ @JsonProperty(value = "ErrorURL")
+ private String ErrorURL;
+
+ public String getStatus() {
+ return Status;
+ }
+
+ public String getMessage() {
+ return Message;
+ }
+
+ @Override
+ public String toString() {
+ ObjectMapper mapper = new ObjectMapper();
+ try {
+ return mapper.writeValueAsString(this);
+ } catch (JsonProcessingException e) {
+ return "";
+ }
+
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Schema.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Schema.java
new file mode 100644
index 00000000000000..314aa65785a8d3
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Schema.java
@@ -0,0 +1,96 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.flink.rest.models;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+public class Schema {
+ private int status = 0;
+ private List properties;
+
+ public Schema() {
+ properties = new ArrayList<>();
+ }
+
+ public Schema(int fieldCount) {
+ properties = new ArrayList<>(fieldCount);
+ }
+
+ public int getStatus() {
+ return status;
+ }
+
+ public void setStatus(int status) {
+ this.status = status;
+ }
+
+ public List getProperties() {
+ return properties;
+ }
+
+ public void setProperties(List properties) {
+ this.properties = properties;
+ }
+
+ public void put(String name, String type, String comment, int scale, int precision) {
+ properties.add(new Field(name, type, comment, scale, precision));
+ }
+
+ public void put(Field f) {
+ properties.add(f);
+ }
+
+ public Field get(int index) {
+ if (index >= properties.size()) {
+ throw new IndexOutOfBoundsException("Index: " + index + ", Fields size:" + properties.size());
+ }
+ return properties.get(index);
+ }
+
+ public int size() {
+ return properties.size();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ Schema schema = (Schema) o;
+ return status == schema.status &&
+ Objects.equals(properties, schema.properties);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(status, properties);
+ }
+
+ @Override
+ public String toString() {
+ return "Schema{" +
+ "status=" + status +
+ ", properties=" + properties +
+ '}';
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Tablet.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Tablet.java
new file mode 100644
index 00000000000000..70b0f13959f45e
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/models/Tablet.java
@@ -0,0 +1,80 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.flink.rest.models;
+
+import java.util.List;
+import java.util.Objects;
+
+public class Tablet {
+ private List routings;
+ private int version;
+ private long versionHash;
+ private long schemaHash;
+
+ public List getRoutings() {
+ return routings;
+ }
+
+ public void setRoutings(List routings) {
+ this.routings = routings;
+ }
+
+ public int getVersion() {
+ return version;
+ }
+
+ public void setVersion(int version) {
+ this.version = version;
+ }
+
+ public long getVersionHash() {
+ return versionHash;
+ }
+
+ public void setVersionHash(long versionHash) {
+ this.versionHash = versionHash;
+ }
+
+ public long getSchemaHash() {
+ return schemaHash;
+ }
+
+ public void setSchemaHash(long schemaHash) {
+ this.schemaHash = schemaHash;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ Tablet tablet = (Tablet) o;
+ return version == tablet.version &&
+ versionHash == tablet.versionHash &&
+ schemaHash == tablet.schemaHash &&
+ Objects.equals(routings, tablet.routings);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(routings, version, versionHash, schemaHash);
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/Routing.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/Routing.java
new file mode 100644
index 00000000000000..25fdbe50e79b29
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/Routing.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.doris.flink.serialization;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.doris.flink.exception.IllegalArgumentException;
+
+import static org.apache.doris.flink.util.ErrorMessages.PARSE_NUMBER_FAILED_MESSAGE;
+
+/**
+ * present an Doris BE address.
+ */
+public class Routing {
+ private static Logger logger = LoggerFactory.getLogger(Routing.class);
+
+ private String host;
+ private int port;
+
+ public Routing(String routing) throws IllegalArgumentException {
+ parseRouting(routing);
+ }
+
+ private void parseRouting(String routing) throws IllegalArgumentException {
+ logger.debug("Parse Doris BE address: '{}'.", routing);
+ String[] hostPort = routing.split(":");
+ if (hostPort.length != 2) {
+ logger.error("Format of Doris BE address '{}' is illegal.", routing);
+ throw new IllegalArgumentException("Doris BE", routing);
+ }
+ this.host = hostPort[0];
+ try {
+ this.port = Integer.parseInt(hostPort[1]);
+ } catch (NumberFormatException e) {
+ logger.error(PARSE_NUMBER_FAILED_MESSAGE, "Doris BE's port", hostPort[1]);
+ throw new IllegalArgumentException("Doris BE", routing);
+ }
+ }
+
+ public String getHost() {
+ return host;
+ }
+
+ public int getPort() {
+ return port;
+ }
+
+ @Override
+ public String toString() {
+ return "Doris BE{" +
+ "host='" + host + '\'' +
+ ", port=" + port +
+ '}';
+ }
+}
diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/RowBatch.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/RowBatch.java
new file mode 100644
index 00000000000000..8d81a37dcae265
--- /dev/null
+++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/RowBatch.java
@@ -0,0 +1,308 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.flink.serialization;
+
+import org.apache.arrow.memory.RootAllocator;
+
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.SmallIntVector;
+import org.apache.arrow.vector.TinyIntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.ipc.ArrowStreamReader;
+import org.apache.arrow.vector.types.Types;
+import org.apache.doris.flink.exception.DorisException;
+import org.apache.doris.flink.rest.models.Schema;
+import org.apache.doris.thrift.TScanBatchResult;
+
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+/**
+ * row batch data container.
+ */
+public class RowBatch {
+ private static Logger logger = LoggerFactory.getLogger(RowBatch.class);
+
+ public static class Row {
+ private List