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 cols; + + Row(int colCount) { + this.cols = new ArrayList<>(colCount); + } + + public List getCols() { + return cols; + } + + public void put(Object o) { + cols.add(o); + } + } + + // offset for iterate the rowBatch + private int offsetInRowBatch = 0; + private int rowCountInOneBatch = 0; + private int readRowCount = 0; + private List rowBatch = new ArrayList<>(); + private final ArrowStreamReader arrowStreamReader; + private VectorSchemaRoot root; + private List fieldVectors; + private RootAllocator rootAllocator; + private final Schema schema; + + public List getRowBatch() { + return rowBatch; + } + + public RowBatch(TScanBatchResult nextResult, Schema schema){ + this.schema = schema; + this.rootAllocator = new RootAllocator(Integer.MAX_VALUE); + this.arrowStreamReader = new ArrowStreamReader( + new ByteArrayInputStream(nextResult.getRows()), + rootAllocator + ); + this.offsetInRowBatch = 0; + } + + public RowBatch readArrow() throws DorisException { + try { + this.root = arrowStreamReader.getVectorSchemaRoot(); + while (arrowStreamReader.loadNextBatch()) { + fieldVectors = root.getFieldVectors(); + if (fieldVectors.size() != schema.size()) { + logger.error("Schema size '{}' is not equal to arrow field size '{}'.", + fieldVectors.size(), schema.size()); + throw new DorisException("Load Doris data failed, schema size of fetch data is wrong."); + } + if (fieldVectors.size() == 0 || root.getRowCount() == 0) { + logger.debug("One batch in arrow has no data."); + continue; + } + rowCountInOneBatch = root.getRowCount(); + // init the rowBatch + for (int i = 0; i < rowCountInOneBatch; ++i) { + rowBatch.add(new Row(fieldVectors.size())); + } + convertArrowToRowBatch(); + readRowCount += root.getRowCount(); + } + return this; + } catch (Exception e) { + logger.error("Read Doris Data failed because: ", e); + throw new DorisException(e.getMessage()); + } finally { + close(); + } + } + + public boolean hasNext() { + if (offsetInRowBatch < readRowCount) { + return true; + } + return false; + } + + private void addValueToRow(int rowIndex, Object obj) { + if (rowIndex > rowCountInOneBatch) { + String errMsg = "Get row offset: " + rowIndex + " larger than row size: " + + rowCountInOneBatch; + logger.error(errMsg); + throw new NoSuchElementException(errMsg); + } + rowBatch.get(readRowCount + rowIndex).put(obj); + } + + public void convertArrowToRowBatch() throws DorisException { + try { + for (int col = 0; col < fieldVectors.size(); col++) { + FieldVector curFieldVector = fieldVectors.get(col); + Types.MinorType mt = curFieldVector.getMinorType(); + + final String currentType = schema.get(col).getType(); + switch (currentType) { + case "NULL_TYPE": + for (int rowIndex = 0; rowIndex < rowCountInOneBatch; rowIndex++) { + addValueToRow(rowIndex, null); + } + break; + case "BOOLEAN": + Preconditions.checkArgument(mt.equals(Types.MinorType.BIT), + typeMismatchMessage(currentType, mt)); + BitVector bitVector = (BitVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCountInOneBatch; rowIndex++) { + Object fieldValue = bitVector.isNull(rowIndex) ? null : bitVector.get(rowIndex) != 0; + addValueToRow(rowIndex, fieldValue); + } + break; + case "TINYINT": + Preconditions.checkArgument(mt.equals(Types.MinorType.TINYINT), + typeMismatchMessage(currentType, mt)); + TinyIntVector tinyIntVector = (TinyIntVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCountInOneBatch; rowIndex++) { + Object fieldValue = tinyIntVector.isNull(rowIndex) ? null : tinyIntVector.get(rowIndex); + addValueToRow(rowIndex, fieldValue); + } + break; + case "SMALLINT": + Preconditions.checkArgument(mt.equals(Types.MinorType.SMALLINT), + typeMismatchMessage(currentType, mt)); + SmallIntVector smallIntVector = (SmallIntVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCountInOneBatch; rowIndex++) { + Object fieldValue = smallIntVector.isNull(rowIndex) ? null : smallIntVector.get(rowIndex); + addValueToRow(rowIndex, fieldValue); + } + break; + case "INT": + Preconditions.checkArgument(mt.equals(Types.MinorType.INT), + typeMismatchMessage(currentType, mt)); + IntVector intVector = (IntVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCountInOneBatch; rowIndex++) { + Object fieldValue = intVector.isNull(rowIndex) ? null : intVector.get(rowIndex); + addValueToRow(rowIndex, fieldValue); + } + break; + case "BIGINT": + + Preconditions.checkArgument(mt.equals(Types.MinorType.BIGINT), + typeMismatchMessage(currentType, mt)); + BigIntVector bigIntVector = (BigIntVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCountInOneBatch; rowIndex++) { + Object fieldValue = bigIntVector.isNull(rowIndex) ? null : bigIntVector.get(rowIndex); + addValueToRow(rowIndex, fieldValue); + } + break; + case "FLOAT": + Preconditions.checkArgument(mt.equals(Types.MinorType.FLOAT4), + typeMismatchMessage(currentType, mt)); + Float4Vector float4Vector = (Float4Vector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCountInOneBatch; rowIndex++) { + Object fieldValue = float4Vector.isNull(rowIndex) ? null : float4Vector.get(rowIndex); + addValueToRow(rowIndex, fieldValue); + } + break; + case "TIME": + case "DOUBLE": + Preconditions.checkArgument(mt.equals(Types.MinorType.FLOAT8), + typeMismatchMessage(currentType, mt)); + Float8Vector float8Vector = (Float8Vector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCountInOneBatch; rowIndex++) { + Object fieldValue = float8Vector.isNull(rowIndex) ? null : float8Vector.get(rowIndex); + addValueToRow(rowIndex, fieldValue); + } + break; + case "BINARY": + Preconditions.checkArgument(mt.equals(Types.MinorType.VARBINARY), + typeMismatchMessage(currentType, mt)); + VarBinaryVector varBinaryVector = (VarBinaryVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCountInOneBatch; rowIndex++) { + Object fieldValue = varBinaryVector.isNull(rowIndex) ? null : varBinaryVector.get(rowIndex); + addValueToRow(rowIndex, fieldValue); + } + break; + case "DECIMAL": + case "DECIMALV2": + Preconditions.checkArgument(mt.equals(Types.MinorType.DECIMAL), + typeMismatchMessage(currentType, mt)); + DecimalVector decimalVector = (DecimalVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCountInOneBatch; rowIndex++) { + if (decimalVector.isNull(rowIndex)) { + addValueToRow(rowIndex, null); + continue; + } + BigDecimal value = decimalVector.getObject(rowIndex).stripTrailingZeros(); + addValueToRow(rowIndex, DecimalData.fromBigDecimal(value,value.precision(),value.scale())); + } + break; + case "DATE": + case "LARGEINT": + case "DATETIME": + case "CHAR": + case "VARCHAR": + Preconditions.checkArgument(mt.equals(Types.MinorType.VARCHAR), + typeMismatchMessage(currentType, mt)); + VarCharVector varCharVector = (VarCharVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCountInOneBatch; rowIndex++) { + if (varCharVector.isNull(rowIndex)) { + addValueToRow(rowIndex, null); + continue; + } + String value = new String(varCharVector.get(rowIndex)); + addValueToRow(rowIndex, StringData.fromString(value)); + } + break; + default: + String errMsg = "Unsupported type " + schema.get(col).getType(); + logger.error(errMsg); + throw new DorisException(errMsg); + } + } + } catch (Exception e) { + close(); + throw e; + } + } + + public List next() throws DorisException { + if (!hasNext()) { + String errMsg = "Get row offset:" + offsetInRowBatch + " larger than row size: " + readRowCount; + logger.error(errMsg); + throw new NoSuchElementException(errMsg); + } + return rowBatch.get(offsetInRowBatch++).getCols(); + } + + private String typeMismatchMessage(final String sparkType, final Types.MinorType arrowType) { + final String messageTemplate = "FLINK type is %1$s, but arrow type is %2$s."; + return String.format(messageTemplate, sparkType, arrowType.name()); + } + + public int getReadRowCount() { + return readRowCount; + } + + public void close() { + try { + if (arrowStreamReader != null) { + arrowStreamReader.close(); + } + if (rootAllocator != null) { + rootAllocator.close(); + } + } catch (IOException ioe) { + // do nothing + } + } +} diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicOutputFormat.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicOutputFormat.java new file mode 100644 index 00000000000000..baa7b07f2be578 --- /dev/null +++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicOutputFormat.java @@ -0,0 +1,200 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.table; + +import org.apache.doris.flink.cfg.DorisExecutionOptions; +import org.apache.doris.flink.cfg.DorisOptions; +import org.apache.doris.flink.cfg.DorisReadOptions; +import org.apache.doris.flink.exception.DorisException; +import org.apache.doris.flink.exception.StreamLoadException; +import org.apache.doris.flink.rest.RestService; +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.StringJoiner; + + +/** + * DorisDynamicOutputFormat + **/ +public class DorisDynamicOutputFormat extends RichOutputFormat { + + private static final Logger LOG = LoggerFactory.getLogger(DorisDynamicOutputFormat.class); + + private DorisOptions options ; + private DorisReadOptions readOptions; + private DorisExecutionOptions executionOptions; + private DorisStreamLoad dorisStreamLoad; + private final String fieldDelimiter = "\t"; + private final String lineDelimiter = "\n"; + private final List batch = new ArrayList<>(); + private transient volatile boolean closed = false; + + public DorisDynamicOutputFormat(DorisOptions option,DorisReadOptions readOptions,DorisExecutionOptions executionOptions) { + this.options = option; + this.readOptions = readOptions; + this.executionOptions = executionOptions; + } + + @Override + public void configure(Configuration configuration) { + + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + dorisStreamLoad = new DorisStreamLoad( + getBackend(), + options.getTableIdentifier().split("\\.")[0], + options.getTableIdentifier().split("\\.")[1], + options.getUsername(), + options.getPassword()); + LOG.info("Steamload BE:{}",dorisStreamLoad.getLoadUrlStr()); + } + + @Override + public void writeRecord(RowData row) throws IOException { + addBatch(row); + if (executionOptions.getBatchSize() > 0 && batch.size() >= executionOptions.getBatchSize()) { + flush(); + } + } + + private void addBatch(RowData row) { + StringJoiner value = new StringJoiner(this.fieldDelimiter); + GenericRowData rowData = (GenericRowData) row; + for(int i = 0; i < row.getArity(); ++i) { + value.add(rowData.getField(i).toString()); + } + batch.add(value.toString()); + } + + @Override + public void close() throws IOException { + if (!closed) { + closed = true; + if (batch.size() > 0) { + try { + flush(); + } catch (Exception e) { + LOG.warn("Writing records to doris failed.", e); + throw new RuntimeException("Writing records to doris failed.", e); + } + } + } + } + + + public void flush() throws IOException { + for (int i = 0; i <= executionOptions.getMaxRetries(); i++) { + try { + dorisStreamLoad.load(String.join(lineDelimiter,batch)); + batch.clear(); + break; + } catch (StreamLoadException e) { + LOG.error("doris sink error, retry times = {}", i, e); + if (i >= executionOptions.getMaxRetries()) { + throw new IOException(e); + } + try { + dorisStreamLoad.setHostPort(getBackend()); + LOG.warn("streamload error,switch be: {}",dorisStreamLoad.getLoadUrlStr(), e); + Thread.sleep(1000 * i); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + throw new IOException("unable to flush; interrupted while doing another attempt", e); + } + } + } + } + + private String getBackend() throws IOException{ + try { + //get be url from fe + return RestService.randomBackend(options,readOptions, LOG); + } catch (IOException | DorisException e) { + LOG.error("get backends info fail"); + throw new IOException(e); + } + } + + + /** + * A builder used to set parameters to the output format's configuration in a fluent way. + * + * @return builder + */ + public static Builder builder() { + return new Builder(); + } + + /** + * Builder for {@link DorisDynamicOutputFormat}. + */ + public static class Builder { + private DorisOptions.Builder optionsBuilder; + private DorisReadOptions readOptions; + private DorisExecutionOptions executionOptions; + + public Builder() { + this.optionsBuilder = DorisOptions.builder(); + } + + public Builder setFenodes(String fenodes) { + this.optionsBuilder.setFenodes(fenodes); + return this; + } + + public Builder setUsername(String username) { + this.optionsBuilder.setUsername(username); + return this; + } + + public Builder setPassword(String password) { + this.optionsBuilder.setPassword(password); + return this; + } + + public Builder setTableIdentifier(String tableIdentifier) { + this.optionsBuilder.setTableIdentifier(tableIdentifier); + return this; + } + + public Builder setReadOptions(DorisReadOptions readOptions) { + this.readOptions = readOptions; + return this; + } + + public Builder setExecutionOptions(DorisExecutionOptions executionOptions) { + this.executionOptions = executionOptions; + return this; + } + + public DorisDynamicOutputFormat build() { + return new DorisDynamicOutputFormat( + optionsBuilder.build(),readOptions,executionOptions + ); + } + } +} diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableFactory.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableFactory.java new file mode 100644 index 00000000000000..aecda37d5e64a1 --- /dev/null +++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableFactory.java @@ -0,0 +1,250 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.table; + +import org.apache.doris.flink.cfg.DorisExecutionOptions; +import org.apache.doris.flink.cfg.DorisOptions; +import org.apache.doris.flink.cfg.DorisReadOptions; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.utils.TableSchemaUtils; + +import java.util.HashSet; +import java.util.Set; + +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_BATCH_SIZE_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_DESERIALIZE_ARROW_ASYNC_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_DESERIALIZE_QUEUE_SIZE_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_EXEC_MEM_LIMIT_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_REQUEST_CONNECT_TIMEOUT_MS_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_REQUEST_QUERY_TIMEOUT_S_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_REQUEST_READ_TIMEOUT_MS_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_REQUEST_RETRIES_DEFAULT; +import static org.apache.doris.flink.cfg.ConfigurationOptions.DORIS_TABLET_SIZE_DEFAULT; + +/** + * The {@link DorisDynamicTableFactory} translates the catalog table to a table source. + * + *

Because the table source requires a decoding format, we are discovering the format using the + * provided {@link FactoryUtil} for convenience. + */ +public final class DorisDynamicTableFactory implements DynamicTableSourceFactory , DynamicTableSinkFactory { + + public static final ConfigOption FENODES = ConfigOptions.key("fenodes").stringType().noDefaultValue().withDescription("doris fe http address."); + public static final ConfigOption TABLE_IDENTIFIER = ConfigOptions.key("table.identifier").stringType().noDefaultValue().withDescription("the jdbc table name."); + public static final ConfigOption USERNAME = ConfigOptions.key("username").stringType().noDefaultValue().withDescription("the jdbc user name."); + public static final ConfigOption PASSWORD = ConfigOptions.key("password").stringType().noDefaultValue().withDescription("the jdbc password."); + + // doris options + private static final ConfigOption DORIS_READ_FIELD = ConfigOptions + .key("doris.read.field") + .stringType() + .noDefaultValue() + .withDescription("List of column names in the Doris table, separated by commas"); + + private static final ConfigOption DORIS_FILTER_QUERY = ConfigOptions + .key("doris.filter.query") + .stringType() + .noDefaultValue() + .withDescription("Filter expression of the query, which is transparently transmitted to Doris. Doris uses this expression to complete source-side data filtering"); + + private static final ConfigOption DORIS_TABLET_SIZE = ConfigOptions + .key("doris.request.tablet.size") + .intType() + .defaultValue(DORIS_TABLET_SIZE_DEFAULT) + .withDescription(""); + + private static final ConfigOption DORIS_REQUEST_CONNECT_TIMEOUT_MS = ConfigOptions + .key("doris.request.connect.timeout.ms") + .intType() + .defaultValue(DORIS_REQUEST_CONNECT_TIMEOUT_MS_DEFAULT) + .withDescription(""); + + private static final ConfigOption DORIS_REQUEST_READ_TIMEOUT_MS = ConfigOptions + .key("doris.request.read.timeout.ms") + .intType() + .defaultValue(DORIS_REQUEST_READ_TIMEOUT_MS_DEFAULT) + .withDescription(""); + + private static final ConfigOption DORIS_REQUEST_QUERY_TIMEOUT_S = ConfigOptions + .key("doris.request.query.timeout.s") + .intType() + .defaultValue(DORIS_REQUEST_QUERY_TIMEOUT_S_DEFAULT) + .withDescription(""); + + private static final ConfigOption DORIS_REQUEST_RETRIES = ConfigOptions + .key("doris.request.retries") + .intType() + .defaultValue(DORIS_REQUEST_RETRIES_DEFAULT) + .withDescription(""); + + private static final ConfigOption DORIS_DESERIALIZE_ARROW_ASYNC = ConfigOptions + .key("doris.deserialize.arrow.async") + .booleanType() + .defaultValue(DORIS_DESERIALIZE_ARROW_ASYNC_DEFAULT) + .withDescription(""); + + private static final ConfigOption DORIS_DESERIALIZE_QUEUE_SIZE = ConfigOptions + .key("doris.request.retriesdoris.deserialize.queue.size") + .intType() + .defaultValue(DORIS_DESERIALIZE_QUEUE_SIZE_DEFAULT) + .withDescription(""); + + + private static final ConfigOption DORIS_BATCH_SIZE = ConfigOptions + .key("doris.batch.size") + .intType() + .defaultValue(DORIS_BATCH_SIZE_DEFAULT) + .withDescription(""); + + private static final ConfigOption DORIS_EXEC_MEM_LIMIT = ConfigOptions + .key("doris.exec.mem.limit") + .longType() + .defaultValue(DORIS_EXEC_MEM_LIMIT_DEFAULT) + .withDescription(""); + + // flink write config options + private static final ConfigOption SINK_BUFFER_FLUSH_MAX_ROWS = ConfigOptions + .key("sink.batch.size") + .intType() + .defaultValue(100) + .withDescription("the flush max size (includes all append, upsert and delete records), over this number" + + " of records, will flush data. The default value is 100."); + + private static final ConfigOption SINK_MAX_RETRIES = ConfigOptions + .key("sink.max-retries") + .intType() + .defaultValue(3) + .withDescription("the max retry times if writing records to database failed."); + + + @Override + public String factoryIdentifier() { + return "doris"; // used for matching to `connector = '...'` + } + + @Override + public Set> requiredOptions() { + final Set> options = new HashSet<>(); + options.add(FENODES); + options.add(TABLE_IDENTIFIER); + return options; + } + + @Override + public Set> optionalOptions() { + final Set> options = new HashSet<>(); + options.add(FENODES); + options.add(TABLE_IDENTIFIER); + options.add(USERNAME); + options.add(PASSWORD); + + options.add(DORIS_READ_FIELD); + options.add(DORIS_FILTER_QUERY); + options.add(DORIS_TABLET_SIZE); + options.add(DORIS_REQUEST_CONNECT_TIMEOUT_MS); + options.add(DORIS_REQUEST_READ_TIMEOUT_MS); + options.add(DORIS_REQUEST_QUERY_TIMEOUT_S); + options.add(DORIS_REQUEST_RETRIES); + options.add(DORIS_DESERIALIZE_ARROW_ASYNC); + options.add(DORIS_DESERIALIZE_QUEUE_SIZE); + options.add(DORIS_BATCH_SIZE); + options.add(DORIS_EXEC_MEM_LIMIT); + + options.add(SINK_BUFFER_FLUSH_MAX_ROWS); + options.add(SINK_MAX_RETRIES); + return options; + } + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + // either implement your custom validation logic here ... + // or use the provided helper utility + final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + // validate all options + helper.validate(); + // get the validated options + final ReadableConfig options = helper.getOptions(); + // derive the produced data type (excluding computed columns) from the catalog table + final DataType producedDataType = context.getCatalogTable().getSchema().toPhysicalRowDataType(); + TableSchema physicalSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema()); + // create and return dynamic table source + return new DorisDynamicTableSource( + getDorisOptions(helper.getOptions()), + getDorisReadOptions(helper.getOptions()), + physicalSchema); + } + + private DorisOptions getDorisOptions(ReadableConfig readableConfig) { + final String fenodes = readableConfig.get(FENODES); + final DorisOptions.Builder builder = DorisOptions.builder() + .setFenodes(fenodes) + .setTableIdentifier(readableConfig.get(TABLE_IDENTIFIER)); + + readableConfig.getOptional(USERNAME).ifPresent(builder::setUsername); + readableConfig.getOptional(PASSWORD).ifPresent(builder::setPassword); + return builder.build(); + } + + private DorisReadOptions getDorisReadOptions(ReadableConfig readableConfig) { + final DorisReadOptions.Builder builder = DorisReadOptions.builder(); + builder.setDeserializeArrowAsync(readableConfig.get(DORIS_DESERIALIZE_ARROW_ASYNC)) + .setDeserializeQueueSize(readableConfig.get(DORIS_DESERIALIZE_QUEUE_SIZE)) + .setExecMemLimit(readableConfig.get(DORIS_EXEC_MEM_LIMIT)) + .setFilterQuery(readableConfig.get(DORIS_FILTER_QUERY)) + .setReadFields(readableConfig.get(DORIS_READ_FIELD)) + .setRequestQueryTimeoutS(readableConfig.get(DORIS_REQUEST_QUERY_TIMEOUT_S)) + .setRequestBatchSize(readableConfig.get(DORIS_BATCH_SIZE)) + .setRequestConnectTimeoutMs(readableConfig.get(DORIS_REQUEST_CONNECT_TIMEOUT_MS)) + .setRequestReadTimeoutMs(readableConfig.get(DORIS_REQUEST_READ_TIMEOUT_MS)) + .setRequestRetries(readableConfig.get(DORIS_REQUEST_RETRIES)) + .setRequestTabletSize(readableConfig.get(DORIS_TABLET_SIZE)); + return builder.build(); + } + + private DorisExecutionOptions getDorisExecutionOptions(ReadableConfig readableConfig) { + final DorisExecutionOptions.Builder builder = DorisExecutionOptions.builder(); + builder.setBatchSize(readableConfig.get(SINK_BUFFER_FLUSH_MAX_ROWS)); + builder.setMaxRetries(readableConfig.get(SINK_MAX_RETRIES)); + return builder.build(); + } + + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + // either implement your custom validation logic here ... + // or use the provided helper utility + final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + // validate all options + helper.validate(); + // create and return dynamic table source + return new DorisDynamicTableSink( + getDorisOptions(helper.getOptions()), + getDorisReadOptions(helper.getOptions()), + getDorisExecutionOptions(helper.getOptions()) + ); + } +} diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSink.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSink.java new file mode 100644 index 00000000000000..feeab969d4e76c --- /dev/null +++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSink.java @@ -0,0 +1,73 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +package org.apache.doris.flink.table; + +import org.apache.doris.flink.cfg.DorisExecutionOptions; +import org.apache.doris.flink.cfg.DorisOptions; +import org.apache.doris.flink.cfg.DorisReadOptions; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.OutputFormatProvider; +import org.apache.flink.types.RowKind; + +/** + * DorisDynamicTableSink + **/ +public class DorisDynamicTableSink implements DynamicTableSink { + + private final DorisOptions options; + private final DorisReadOptions readOptions; + private final DorisExecutionOptions executionOptions; + + public DorisDynamicTableSink(DorisOptions options, DorisReadOptions readOptions, DorisExecutionOptions executionOptions) { + this.options = options; + this.readOptions = readOptions; + this.executionOptions = executionOptions; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode changelogMode) { + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.DELETE) + .addContainedKind(RowKind.UPDATE_AFTER) + .build(); + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + DorisDynamicOutputFormat.Builder builder = DorisDynamicOutputFormat.builder() + .setFenodes(options.getFenodes()) + .setUsername(options.getUsername()) + .setPassword(options.getPassword()) + .setTableIdentifier(options.getTableIdentifier()) + .setReadOptions(readOptions) + .setExecutionOptions(executionOptions); + + return OutputFormatProvider.of(builder.build()); + } + + @Override + public DynamicTableSink copy() { + return new DorisDynamicTableSink(options,readOptions,executionOptions); + } + + @Override + public String asSummaryString() { + return "Doris Table Sink"; + } +} diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSource.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSource.java new file mode 100644 index 00000000000000..ab523aad08bd14 --- /dev/null +++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisDynamicTableSource.java @@ -0,0 +1,101 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.flink.table; + +import org.apache.doris.flink.cfg.DorisOptions; +import org.apache.doris.flink.cfg.DorisReadOptions; +import org.apache.doris.flink.exception.DorisException; +import org.apache.doris.flink.rest.PartitionDefinition; +import org.apache.doris.flink.rest.RestService; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.InputFormatProvider; +import org.apache.flink.table.connector.source.LookupTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; +import org.apache.flink.table.data.RowData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +/** + * The {@link DorisDynamicTableSource} is used during planning. + * + *

In our example, we don't implement any of the available ability interfaces such as {@link SupportsFilterPushDown} + * or {@link SupportsProjectionPushDown}. Therefore, the main logic can be found in {@link #getScanRuntimeProvider(ScanContext)} + * where we instantiate the required {@link SourceFunction} and its {@link DeserializationSchema} for + * runtime. Both instances are parameterized to return internal data structures (i.e. {@link RowData}). + */ +public final class DorisDynamicTableSource implements ScanTableSource ,LookupTableSource { + + private final DorisOptions options; + private final DorisReadOptions readOptions; + private TableSchema physicalSchema; + private static final Logger LOG = LoggerFactory.getLogger(DorisRowDataInputFormat.class); + + public DorisDynamicTableSource(DorisOptions options, DorisReadOptions readOptions,TableSchema physicalSchema) { + this.options = options; + this.readOptions = readOptions; + this.physicalSchema = physicalSchema; + } + + @Override + public ChangelogMode getChangelogMode() { + // in our example the format decides about the changelog mode + // but it could also be the source itself + return ChangelogMode.insertOnly(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { + List dorisPartitions ; + try { + dorisPartitions = RestService.findPartitions(options,readOptions,LOG); + } catch (DorisException e) { + throw new RuntimeException("can not fetch partitions"); + } + DorisRowDataInputFormat.Builder builder = DorisRowDataInputFormat.builder() + .setFenodes(options.getFenodes()) + .setUsername(options.getUsername()) + .setPassword(options.getPassword()) + .setTableIdentifier(options.getTableIdentifier()) + .setPartitions(dorisPartitions) + .setReadOptions(readOptions); + return InputFormatProvider.of(builder.build()); + } + + @Override + public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext lookupContext) { + return null; + } + + @Override + public DynamicTableSource copy() { + return new DorisDynamicTableSource(options,readOptions,physicalSchema); + } + + @Override + public String asSummaryString() { + return "Doris Table Source"; + } +} diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisRowDataInputFormat.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisRowDataInputFormat.java new file mode 100644 index 00000000000000..75e7fc90d1e020 --- /dev/null +++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisRowDataInputFormat.java @@ -0,0 +1,227 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.table; + +import org.apache.doris.flink.cfg.DorisOptions; +import org.apache.doris.flink.cfg.DorisReadOptions; +import org.apache.doris.flink.datastream.ScalaValueReader; +import org.apache.doris.flink.rest.PartitionDefinition; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.io.DefaultInputSplitAssigner; +import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.RichInputFormat; +import org.apache.flink.api.common.io.statistics.BaseStatistics; +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.core.io.InputSplitAssigner; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.sql.PreparedStatement; +import java.util.ArrayList; +import java.util.List; + +/** + * InputFormat for {@link DorisDynamicTableSource}. + */ +@Internal +public class DorisRowDataInputFormat extends RichInputFormat implements ResultTypeQueryable { + + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(DorisRowDataInputFormat.class); + + private DorisOptions options; + private DorisReadOptions readOptions; + private List dorisPartitions; + private TypeInformation rowDataTypeInfo; + + private ScalaValueReader scalaValueReader; + private transient boolean hasNext; + + public DorisRowDataInputFormat(DorisOptions options,List dorisPartitions,DorisReadOptions readOptions) { + this.options = options; + this.dorisPartitions = dorisPartitions; + this.readOptions = readOptions; + } + + @Override + public void configure(Configuration parameters) { + //do nothing here + } + + @Override + public void openInputFormat() { + //called once per inputFormat (on open) + } + + @Override + public void closeInputFormat() { + //called once per inputFormat (on close) + } + + /** + * Connects to the source database and executes the query in a parallel + * fashion if + * this {@link InputFormat} is built using a parameterized query (i.e. using + * a {@link PreparedStatement}) + * and a proper {@link }, in a non-parallel + * fashion otherwise. + * + * @param inputSplit which is ignored if this InputFormat is executed as a + * non-parallel source, + * a "hook" to the query parameters otherwise (using its + * splitNumber) + * @throws IOException if there's an error during the execution of the query + */ + @Override + public void open(DorisTableInputSplit inputSplit) throws IOException { + scalaValueReader = new ScalaValueReader(inputSplit.partition, options,readOptions); + hasNext = scalaValueReader.hasNext(); + } + + /** + * Closes all resources used. + * + * @throws IOException Indicates that a resource could not be closed. + */ + @Override + public void close() throws IOException { + + } + + @Override + public TypeInformation getProducedType() { + return rowDataTypeInfo; + } + + /** + * Checks whether all data has been read. + * + * @return boolean value indication whether all data has been read. + * @throws IOException + */ + @Override + public boolean reachedEnd() throws IOException { + return !hasNext; + } + + /** + * Stores the next resultSet row in a tuple. + * + * @param reuse row to be reused. + * @return row containing next {@link RowData} + * @throws IOException + */ + @Override + public RowData nextRecord(RowData reuse) throws IOException { + if (!hasNext) { + return null; + } + List next = (List)scalaValueReader.next(); + GenericRowData genericRowData = new GenericRowData(next.size()); + for(int i =0;i dorisSplits = new ArrayList<>(); + int splitNum = 0; + for (PartitionDefinition partition : dorisPartitions) { + dorisSplits.add(new DorisTableInputSplit(splitNum++,partition)); + } + LOG.info("DorisTableInputSplit Num:{}",dorisSplits.size()); + return dorisSplits.toArray(new DorisTableInputSplit[0]); + } + + @Override + public InputSplitAssigner getInputSplitAssigner(DorisTableInputSplit[] inputSplits) { + return new DefaultInputSplitAssigner(inputSplits); + } + + /** + * A builder used to set parameters to the output format's configuration in a fluent way. + * + * @return builder + */ + public static Builder builder() { + return new Builder(); + } + + /** + * Builder for {@link DorisRowDataInputFormat}. + */ + public static class Builder { + private DorisOptions.Builder optionsBuilder; + private List partitions; + private DorisReadOptions readOptions; + + + public Builder() { + this.optionsBuilder = DorisOptions.builder(); + } + + public Builder setFenodes(String fenodes) { + this.optionsBuilder.setFenodes(fenodes); + return this; + } + + public Builder setUsername(String username) { + this.optionsBuilder.setUsername(username); + return this; + } + + public Builder setPassword(String password) { + this.optionsBuilder.setPassword(password); + return this; + } + + public Builder setTableIdentifier(String tableIdentifier) { + this.optionsBuilder.setTableIdentifier(tableIdentifier); + return this; + } + + public Builder setPartitions(List partitions) { + this.partitions = partitions; + return this; + } + + public Builder setReadOptions(DorisReadOptions readOptions) { + this.readOptions = readOptions; + return this; + } + + public DorisRowDataInputFormat build() { + return new DorisRowDataInputFormat( + optionsBuilder.build(),partitions,readOptions + ); + } + } +} diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisStreamLoad.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisStreamLoad.java new file mode 100644 index 00000000000000..fa001a60382771 --- /dev/null +++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisStreamLoad.java @@ -0,0 +1,179 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.table; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.doris.flink.exception.StreamLoadException; +import org.apache.doris.flink.rest.models.RespContent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedOutputStream; +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +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.Calendar; +import java.util.List; +import java.util.UUID; + +/** + * DorisStreamLoad + **/ +public class DorisStreamLoad implements Serializable{ + + private static final Logger LOG = LoggerFactory.getLogger(DorisStreamLoad.class); + + private final static List DORIS_SUCCESS_STATUS = new ArrayList<>(Arrays.asList("Success", "Publish Timeout")); + private static String loadUrlPattern = "http://%s/api/%s/%s/_stream_load?"; + private String user; + private String passwd; + private String loadUrlStr; + private String hostPort; + private String db; + private String tbl; + private String authEncoding; + + public DorisStreamLoad(String hostPort, String db, String tbl, String user, String passwd) { + this.hostPort = hostPort; + this.db = db; + this.tbl = tbl; + this.user = user; + this.passwd = passwd; + this.loadUrlStr = String.format(loadUrlPattern, hostPort, db, tbl); + this.authEncoding = Base64.getEncoder().encodeToString(String.format("%s:%s", user, passwd).getBytes(StandardCharsets.UTF_8)); + } + + public String getLoadUrlStr() { + return loadUrlStr; + } + + public String getHostPort() { + return hostPort; + } + + public void setHostPort(String hostPort) { + this.hostPort = hostPort; + this.loadUrlStr = String.format(loadUrlPattern, hostPort, this.db, this.tbl); + } + + + private HttpURLConnection getConnection(String urlStr, String label) throws IOException { + URL url = new URL(urlStr); + HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + conn.setInstanceFollowRedirects(false); + conn.setRequestMethod("PUT"); + String authEncoding = Base64.getEncoder().encodeToString(String.format("%s:%s", user, passwd).getBytes(StandardCharsets.UTF_8)); + conn.setRequestProperty("Authorization", "Basic " + authEncoding); + conn.addRequestProperty("Expect", "100-continue"); + conn.addRequestProperty("Content-Type", "text/plain; charset=UTF-8"); + conn.addRequestProperty("label", label); + conn.setDoOutput(true); + conn.setDoInput(true); + return conn; + } + + public static class LoadResponse { + public int status; + public String respMsg; + public String respContent; + + public LoadResponse(int status, String respMsg, String respContent) { + this.status = status; + this.respMsg = respMsg; + this.respContent = respContent; + } + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("status: ").append(status); + sb.append(", resp msg: ").append(respMsg); + sb.append(", resp content: ").append(respContent); + return sb.toString(); + } + } + + public void load(String value) throws StreamLoadException { + LoadResponse loadResponse = loadBatch(value); + LOG.info("Streamload Response:{}",loadResponse); + if(loadResponse.status != 200){ + throw new StreamLoadException("stream load error: " + loadResponse.respContent); + }else{ + ObjectMapper obj = new ObjectMapper(); + try { + RespContent respContent = obj.readValue(loadResponse.respContent, RespContent.class); + if(!DORIS_SUCCESS_STATUS.contains(respContent.getStatus())){ + throw new StreamLoadException("stream load error: " + respContent.getMessage()); + } + } catch (IOException e) { + throw new StreamLoadException(e); + } + } + } + + private LoadResponse loadBatch(String value) { + Calendar calendar = Calendar.getInstance(); + String label = String.format("audit_%s%02d%02d_%02d%02d%02d_%s", + calendar.get(Calendar.YEAR), calendar.get(Calendar.MONTH) + 1, calendar.get(Calendar.DAY_OF_MONTH), + calendar.get(Calendar.HOUR_OF_DAY), calendar.get(Calendar.MINUTE), calendar.get(Calendar.SECOND), + UUID.randomUUID().toString().replaceAll("-", "")); + + HttpURLConnection feConn = null; + HttpURLConnection beConn = null; + try { + // build request and send to new be location + beConn = getConnection(loadUrlStr, label); + // send data to be + BufferedOutputStream bos = new BufferedOutputStream(beConn.getOutputStream()); + bos.write(value.getBytes()); + bos.close(); + + // get respond + int status = beConn.getResponseCode(); + String respMsg = beConn.getResponseMessage(); + InputStream stream = (InputStream) beConn.getContent(); + BufferedReader br = new BufferedReader(new InputStreamReader(stream)); + StringBuilder response = new StringBuilder(); + String line; + while ((line = br.readLine()) != null) { + response.append(line); + } +// log.info("AuditLoader plugin load with label: {}, response code: {}, msg: {}, content: {}",label, status, respMsg, response.toString()); + return new LoadResponse(status, respMsg, response.toString()); + + } catch (Exception e) { + e.printStackTrace(); + String err = "failed to load audit via AuditLoader plugin with label: " + label; + LOG.warn(err, e); + return new LoadResponse(-1, e.getMessage(), err); + } finally { + if (feConn != null) { + feConn.disconnect(); + } + if (beConn != null) { + beConn.disconnect(); + } + } + } +} diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisTableInputSplit.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisTableInputSplit.java new file mode 100644 index 00000000000000..f245dacb003e25 --- /dev/null +++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/table/DorisTableInputSplit.java @@ -0,0 +1,44 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +package org.apache.doris.flink.table; + +import org.apache.doris.flink.rest.PartitionDefinition; +import org.apache.flink.core.io.InputSplit; + +/** + * DorisTableInputSplit + **/ +public class DorisTableInputSplit implements InputSplit, java.io.Serializable{ + + /** The number of the split. */ + private final int splitNumber; + + protected final PartitionDefinition partition; + + public DorisTableInputSplit(int splitNumber,PartitionDefinition partition) { + super(); + this.splitNumber = splitNumber; + this.partition = partition; + } + + + @Override + public int getSplitNumber() { + return splitNumber; + } + +} diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/util/ErrorMessages.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/util/ErrorMessages.java new file mode 100644 index 00000000000000..2e02daae694ba5 --- /dev/null +++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/util/ErrorMessages.java @@ -0,0 +1,27 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util; + +public abstract class ErrorMessages { + public static final String PARSE_NUMBER_FAILED_MESSAGE = "Parse '{}' to number failed. Original string is '{}'."; + public static final String PARSE_BOOL_FAILED_MESSAGE = "Parse '{}' to boolean failed. Original string is '{}'."; + public static final String CONNECT_FAILED_MESSAGE = "Connect to doris {} failed."; + public static final String ILLEGAL_ARGUMENT_MESSAGE = "argument '{}' is illegal, value is '{}'."; + public static final String SHOULD_NOT_HAPPEN_MESSAGE = "Should not come here."; + public static final String DORIS_INTERNAL_FAIL_MESSAGE = "Doris server '{}' internal failed, status is '{}', error message is '{}'"; +} diff --git a/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/util/IOUtils.java b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/util/IOUtils.java new file mode 100644 index 00000000000000..203010cbed1c3c --- /dev/null +++ b/extension/flink-doris-connector/src/main/java/org/apache/doris/flink/util/IOUtils.java @@ -0,0 +1,49 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.flink.util; + +import java.io.IOException; +import java.io.StringReader; +import java.io.StringWriter; +import java.util.Properties; + +public class IOUtils { + public static String propsToString(Properties props) throws IllegalArgumentException { + StringWriter sw = new StringWriter(); + if (props != null) { + try { + props.store(sw, ""); + } catch (IOException ex) { + throw new IllegalArgumentException("Cannot parse props to String.", ex); + } + } + return sw.toString(); + } + + public static Properties propsFromString(String source) throws IllegalArgumentException { + Properties copy = new Properties(); + if (source != null) { + try { + copy.load(new StringReader(source)); + } catch (IOException ex) { + throw new IllegalArgumentException("Cannot parse props from String.", ex); + } + } + return copy; + } +} diff --git a/extension/flink-doris-connector/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/extension/flink-doris-connector/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 00000000000000..e625cc782df43e --- /dev/null +++ b/extension/flink-doris-connector/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,18 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +org.apache.doris.flink.table.DorisDynamicTableFactory \ No newline at end of file diff --git a/extension/flink-doris-connector/src/main/resources/log4j.properties b/extension/flink-doris-connector/src/main/resources/log4j.properties new file mode 100644 index 00000000000000..da32ea0f44d375 --- /dev/null +++ b/extension/flink-doris-connector/src/main/resources/log4j.properties @@ -0,0 +1,23 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +log4j.rootLogger=INFO, console + +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n diff --git a/extension/flink-doris-connector/src/main/scala/org/apache/doris/flink/datastream/ScalaValueReader.scala b/extension/flink-doris-connector/src/main/scala/org/apache/doris/flink/datastream/ScalaValueReader.scala new file mode 100644 index 00000000000000..bdf948744a405e --- /dev/null +++ b/extension/flink-doris-connector/src/main/scala/org/apache/doris/flink/datastream/ScalaValueReader.scala @@ -0,0 +1,223 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 java.util.concurrent._ +import java.util.concurrent.atomic.AtomicBoolean + +import org.apache.doris.flink.backend.BackendClient +import org.apache.doris.flink.cfg.ConfigurationOptions._ +import org.apache.doris.flink.cfg.{DorisOptions, DorisReadOptions} +import org.apache.doris.flink.exception.ShouldNeverHappenException +import org.apache.doris.flink.rest.{PartitionDefinition, SchemaUtils} +import org.apache.doris.flink.rest.models.Schema +import org.apache.doris.flink.serialization.{Routing, RowBatch} +import org.apache.doris.flink.util.ErrorMessages +import org.apache.doris.flink.util.ErrorMessages._ +import org.apache.doris.thrift.{TScanCloseParams, TScanNextBatchParams, TScanOpenParams, TScanOpenResult} +import org.apache.log4j.Logger + +import scala.collection.JavaConversions._ +import scala.util.Try +import scala.util.control.Breaks + +/** + * read data from Doris BE to array. + * @param partition Doris RDD partition + * @param options request configuration + */ +class ScalaValueReader(partition: PartitionDefinition, options: DorisOptions, readOptions: DorisReadOptions) { + protected val logger = Logger.getLogger(classOf[ScalaValueReader]) + + protected val client = new BackendClient(new Routing(partition.getBeAddress), readOptions) + protected var offset = 0 + protected var eos: AtomicBoolean = new AtomicBoolean(false) + protected var rowBatch: RowBatch = _ + // flag indicate if support deserialize Arrow to RowBatch asynchronously + protected var deserializeArrowToRowBatchAsync: java.lang.Boolean = Try { + if(readOptions.getDeserializeArrowAsync == null ) DORIS_DESERIALIZE_ARROW_ASYNC_DEFAULT else readOptions.getDeserializeArrowAsync + } getOrElse { + logger.warn(ErrorMessages.PARSE_BOOL_FAILED_MESSAGE, DORIS_DESERIALIZE_ARROW_ASYNC, readOptions.getDeserializeArrowAsync) + DORIS_DESERIALIZE_ARROW_ASYNC_DEFAULT + } + + protected var rowBatchBlockingQueue: BlockingQueue[RowBatch] = { + val blockingQueueSize = Try { + if(readOptions.getDeserializeQueueSize == null) DORIS_DESERIALIZE_QUEUE_SIZE_DEFAULT else readOptions.getDeserializeQueueSize + } getOrElse { + logger.warn(ErrorMessages.PARSE_NUMBER_FAILED_MESSAGE, DORIS_DESERIALIZE_QUEUE_SIZE, readOptions.getDeserializeQueueSize) + DORIS_DESERIALIZE_QUEUE_SIZE_DEFAULT + } + + var queue: BlockingQueue[RowBatch] = null + if (deserializeArrowToRowBatchAsync) { + queue = new ArrayBlockingQueue(blockingQueueSize) + } + queue + } + + private val openParams: TScanOpenParams = { + val params = new TScanOpenParams + params.cluster = DORIS_DEFAULT_CLUSTER + params.database = partition.getDatabase + params.table = partition.getTable + + params.tablet_ids = partition.getTabletIds.toList + params.opaqued_query_plan = partition.getQueryPlan + + // max row number of one read batch + val batchSize = Try { + if(readOptions.getRequestBatchSize == null) DORIS_BATCH_SIZE_DEFAULT else readOptions.getRequestBatchSize; + } getOrElse { + logger.warn(ErrorMessages.PARSE_NUMBER_FAILED_MESSAGE, DORIS_BATCH_SIZE, readOptions.getRequestBatchSize) + DORIS_BATCH_SIZE_DEFAULT + } + + val queryDorisTimeout = Try { + if(readOptions.getRequestQueryTimeoutS == null) DORIS_REQUEST_QUERY_TIMEOUT_S_DEFAULT else readOptions.getRequestQueryTimeoutS + } getOrElse { + logger.warn(ErrorMessages.PARSE_NUMBER_FAILED_MESSAGE, DORIS_REQUEST_QUERY_TIMEOUT_S, readOptions.getRequestQueryTimeoutS) + DORIS_REQUEST_QUERY_TIMEOUT_S_DEFAULT + } + + val execMemLimit = Try { + if(readOptions.getExecMemLimit == null) DORIS_EXEC_MEM_LIMIT_DEFAULT else readOptions.getExecMemLimit + } getOrElse { + logger.warn(ErrorMessages.PARSE_NUMBER_FAILED_MESSAGE, DORIS_EXEC_MEM_LIMIT, readOptions.getExecMemLimit) + DORIS_EXEC_MEM_LIMIT_DEFAULT + } + + params.setBatch_size(batchSize) + params.setQuery_timeout(queryDorisTimeout) + params.setMem_limit(execMemLimit) + params.setUser(options.getUsername) + params.setPasswd(options.getPassword) + + logger.debug(s"Open scan params is, " + + s"cluster: ${params.getCluster}, " + + s"database: ${params.getDatabase}, " + + s"table: ${params.getTable}, " + + s"tabletId: ${params.getTablet_ids}, " + + s"batch size: $batchSize, " + + s"query timeout: $queryDorisTimeout, " + + s"execution memory limit: $execMemLimit, " + + s"user: ${params.getUser}, " + + s"query plan: ${params.opaqued_query_plan}") + + params + } + + protected val openResult: TScanOpenResult = client.openScanner(openParams) + protected val contextId: String = openResult.getContext_id + protected val schema: Schema = + SchemaUtils.convertToSchema(openResult.getSelected_columns) + + protected val asyncThread: Thread = new Thread { + override def run { + val nextBatchParams = new TScanNextBatchParams + nextBatchParams.setContext_id(contextId) + while (!eos.get) { + nextBatchParams.setOffset(offset) + val nextResult = client.getNext(nextBatchParams) + eos.set(nextResult.isEos) + if (!eos.get) { + val rowBatch = new RowBatch(nextResult, schema).readArrow() + offset += rowBatch.getReadRowCount + rowBatch.close + rowBatchBlockingQueue.put(rowBatch) + } + } + } + } + + protected val asyncThreadStarted: Boolean = { + var started = false + if (deserializeArrowToRowBatchAsync) { + asyncThread.start + started = true + } + started + } + + logger.debug(s"Open scan result is, contextId: $contextId, schema: $schema.") + + /** + * read data and cached in rowBatch. + * @return true if hax next value + */ + def hasNext: Boolean = { + var hasNext = false + if (deserializeArrowToRowBatchAsync && asyncThreadStarted) { + // support deserialize Arrow to RowBatch asynchronously + if (rowBatch == null || !rowBatch.hasNext) { + val loop = new Breaks + loop.breakable { + while (!eos.get || !rowBatchBlockingQueue.isEmpty) { + if (!rowBatchBlockingQueue.isEmpty) { + rowBatch = rowBatchBlockingQueue.take + hasNext = true + loop.break + } else { + // wait for rowBatch put in queue or eos change + Thread.sleep(5) + } + } + } + } else { + hasNext = true + } + } else { + // Arrow data was acquired synchronously during the iterative process + if (!eos.get && (rowBatch == null || !rowBatch.hasNext)) { + if (rowBatch != null) { + offset += rowBatch.getReadRowCount + rowBatch.close + } + val nextBatchParams = new TScanNextBatchParams + nextBatchParams.setContext_id(contextId) + nextBatchParams.setOffset(offset) + val nextResult = client.getNext(nextBatchParams) + eos.set(nextResult.isEos) + if (!eos.get) { + rowBatch = new RowBatch(nextResult, schema).readArrow() + } + } + hasNext = !eos.get + } + hasNext + } + + /** + * get next value. + * @return next value + */ + def next: AnyRef = { + if (!hasNext) { + logger.error(SHOULD_NOT_HAPPEN_MESSAGE) + throw new ShouldNeverHappenException + } + rowBatch.next + } + + def close(): Unit = { + val closeParams = new TScanCloseParams + closeParams.context_id = contextId + client.closeScanner(closeParams) + } + +} diff --git a/extension/flink-doris-connector/src/main/thrift/doris/DorisExternalService.thrift b/extension/flink-doris-connector/src/main/thrift/doris/DorisExternalService.thrift new file mode 100644 index 00000000000000..c1698748878fbe --- /dev/null +++ b/extension/flink-doris-connector/src/main/thrift/doris/DorisExternalService.thrift @@ -0,0 +1,122 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +namespace java org.apache.doris.thrift +namespace cpp doris + +include "Types.thrift" +include "Status.thrift" + + +// Parameters to open(). +struct TScanOpenParams { + + 1: required string cluster + + 2: required string database + + 3: required string table + + // tablets to scan + 4: required list tablet_ids + + // base64 encoded binary plan fragment + 5: required string opaqued_query_plan + + // A string specified for the table that is passed to the external data source. + // Always set, may be an empty string. + 6: optional i32 batch_size + + // reserved params for use + 7: optional map properties + + // The query limit, if specified. + 8: optional i64 limit + + // The authenticated user name. Always set. + // maybe usefullless + 9: optional string user + + 10: optional string passwd + // max keep alive time min + 11: optional i16 keep_alive_min + + 12: optional i32 query_timeout + + // memory limit for a single query + 13: optional i64 mem_limit +} + +struct TScanColumnDesc { + // The column name + 1: optional string name + // The column type. Always set. + 2: optional Types.TPrimitiveType type +} + +// Returned by open(). +struct TScanOpenResult { + 1: required Status.TStatus status + // An opaque context_id used in subsequent getNext()/close() calls. Required. + 2: optional string context_id + // selected fields + 3: optional list selected_columns + +} + +// Parameters to getNext() +struct TScanNextBatchParams { + // The opaque handle returned by the previous open() call. Always set. + 1: optional string context_id // doris olap engine context id + 2: optional i64 offset // doris should check the offset to prevent duplicate rpc calls +} + +// Returned by getNext(). +struct TScanBatchResult { + 1: required Status.TStatus status + + // If true, reached the end of the result stream; subsequent calls to + // getNext() won’t return any more results. Required. + 2: optional bool eos + + // A batch of rows of arrow format to return, if any exist. The number of rows in the batch + // should be less than or equal to the batch_size specified in TOpenParams. + 3: optional binary rows +} + +// Parameters to close() +struct TScanCloseParams { + // The opaque handle returned by the previous open() call. Always set. + 1: optional string context_id +} + +// Returned by close(). +struct TScanCloseResult { + 1: required Status.TStatus status +} + +// scan service expose ability of scanning data ability to other compute system +service TDorisExternalService { + // doris will build a scan context for this session, context_id returned if success + TScanOpenResult open_scanner(1: TScanOpenParams params); + + // return the batch_size of data + TScanBatchResult get_next(1: TScanNextBatchParams params); + + // release the context resource associated with the context_id + TScanCloseResult close_scanner(1: TScanCloseParams params); +} diff --git a/extension/flink-doris-connector/src/main/thrift/doris/Status.thrift b/extension/flink-doris-connector/src/main/thrift/doris/Status.thrift new file mode 100644 index 00000000000000..2966a8a5353b9e --- /dev/null +++ b/extension/flink-doris-connector/src/main/thrift/doris/Status.thrift @@ -0,0 +1,66 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +namespace cpp doris +namespace java org.apache.doris.thrift + +enum TStatusCode { + OK, + CANCELLED, + ANALYSIS_ERROR, + NOT_IMPLEMENTED_ERROR, + RUNTIME_ERROR, + MEM_LIMIT_EXCEEDED, + INTERNAL_ERROR, + THRIFT_RPC_ERROR, + TIMEOUT, + KUDU_NOT_ENABLED, // Deprecated + KUDU_NOT_SUPPORTED_ON_OS, // Deprecated + MEM_ALLOC_FAILED, + BUFFER_ALLOCATION_FAILED, + MINIMUM_RESERVATION_UNAVAILABLE, + PUBLISH_TIMEOUT, + LABEL_ALREADY_EXISTS, + ES_INTERNAL_ERROR, + ES_INDEX_NOT_FOUND, + ES_SHARD_NOT_FOUND, + ES_INVALID_CONTEXTID, + ES_INVALID_OFFSET, + ES_REQUEST_ERROR, + + // end of file + END_OF_FILE = 30, + NOT_FOUND = 31, + CORRUPTION = 32, + INVALID_ARGUMENT = 33, + IO_ERROR = 34, + ALREADY_EXIST = 35, + NETWORK_ERROR = 36, + ILLEGAL_STATE = 37, + NOT_AUTHORIZED = 38, + ABORTED = 39, + REMOTE_ERROR = 40, + SERVICE_UNAVAILABLE = 41, + UNINITIALIZED = 42, + CONFIGURATION_ERROR = 43, + INCOMPLETE = 44 +} + +struct TStatus { + 1: required TStatusCode status_code + 2: optional list error_msgs +} diff --git a/extension/flink-doris-connector/src/main/thrift/doris/Types.thrift b/extension/flink-doris-connector/src/main/thrift/doris/Types.thrift new file mode 100644 index 00000000000000..2d902baef1326c --- /dev/null +++ b/extension/flink-doris-connector/src/main/thrift/doris/Types.thrift @@ -0,0 +1,370 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +namespace cpp doris +namespace java org.apache.doris.thrift + + +typedef i64 TTimestamp +typedef i32 TPlanNodeId +typedef i32 TTupleId +typedef i32 TSlotId +typedef i64 TTableId +typedef i64 TTabletId +typedef i64 TVersion +typedef i64 TVersionHash +typedef i32 TSchemaHash +typedef i32 TPort +typedef i64 TCount +typedef i64 TSize +typedef i32 TClusterId +typedef i64 TEpoch + +// add for real time load, partitionid is not defined previously, define it here +typedef i64 TTransactionId +typedef i64 TPartitionId + +enum TStorageType { + ROW, + COLUMN, +} + +enum TStorageMedium { + HDD, + SSD, +} + +enum TVarType { + SESSION, + GLOBAL +} + +enum TPrimitiveType { + INVALID_TYPE, + NULL_TYPE, + BOOLEAN, + TINYINT, + SMALLINT, + INT, + BIGINT, + FLOAT, + DOUBLE, + DATE, + DATETIME, + BINARY, + DECIMAL, + // CHAR(n). Currently only supported in UDAs + CHAR, + LARGEINT, + VARCHAR, + HLL, + DECIMALV2, + TIME +} + +enum TTypeNodeType { + SCALAR, + ARRAY, + MAP, + STRUCT +} + +struct TScalarType { + 1: required TPrimitiveType type + + // Only set if type == CHAR or type == VARCHAR + 2: optional i32 len + + // Only set for DECIMAL + 3: optional i32 precision + 4: optional i32 scale +} + +// Represents a field in a STRUCT type. +// TODO: Model column stats for struct fields. +struct TStructField { + 1: required string name + 2: optional string comment +} + +struct TTypeNode { + 1: required TTypeNodeType type + + // only set for scalar types + 2: optional TScalarType scalar_type + + // only used for structs; has struct_fields.size() corresponding child types + 3: optional list struct_fields +} + +// A flattened representation of a tree of column types obtained by depth-first +// traversal. Complex types such as map, array and struct have child types corresponding +// to the map key/value, array item type, and struct fields, respectively. +// For scalar types the list contains only a single node. +// Note: We cannot rename this to TType because it conflicts with Thrift's internal TType +// and the generated Python thrift files will not work. +// Note: TTypeDesc in impala is TColumnType, but we already use TColumnType, so we name this +// to TTypeDesc. In future, we merge these two to one +struct TTypeDesc { + 1: list types +} + +enum TAggregationType { + SUM, + MAX, + MIN, + REPLACE, + HLL_UNION, + NONE +} + +enum TPushType { + LOAD, + DELETE, + LOAD_DELETE +} + +enum TTaskType { + CREATE, + DROP, + PUSH, + CLONE, + STORAGE_MEDIUM_MIGRATE, + ROLLUP, + SCHEMA_CHANGE, + CANCEL_DELETE, // Deprecated + MAKE_SNAPSHOT, + RELEASE_SNAPSHOT, + CHECK_CONSISTENCY, + UPLOAD, + DOWNLOAD, + CLEAR_REMOTE_FILE, + MOVE + REALTIME_PUSH, + PUBLISH_VERSION, + CLEAR_ALTER_TASK, + CLEAR_TRANSACTION_TASK, + RECOVER_TABLET, + STREAM_LOAD, + UPDATE_TABLET_META_INFO, + ALTER_TASK +} + +enum TStmtType { + QUERY, + DDL, // Data definition, e.g. CREATE TABLE (includes read-only functions e.g. SHOW) + DML, // Data modification e.g. INSERT + EXPLAIN // EXPLAIN +} + +// level of verboseness for "explain" output +// TODO: should this go somewhere else? +enum TExplainLevel { + NORMAL, + VERBOSE +} + +struct TColumnType { + 1: required TPrimitiveType type + // Only set if type == CHAR_ARRAY + 2: optional i32 len + 3: optional i32 index_len + 4: optional i32 precision + 5: optional i32 scale +} + +// A TNetworkAddress is the standard host, port representation of a +// network address. The hostname field must be resolvable to an IPv4 +// address. +struct TNetworkAddress { + 1: required string hostname + 2: required i32 port +} + +// Wire format for UniqueId +struct TUniqueId { + 1: required i64 hi + 2: required i64 lo +} + +enum QueryState { + CREATED, + INITIALIZED, + COMPILED, + RUNNING, + FINISHED, + EXCEPTION +} + +enum TFunctionType { + SCALAR, + AGGREGATE, +} + +enum TFunctionBinaryType { + // Palo builtin. We can either run this interpreted or via codegen + // depending on the query option. + BUILTIN, + + // Hive UDFs, loaded from *.jar + HIVE, + + // Native-interface, precompiled UDFs loaded from *.so + NATIVE, + + // Native-interface, precompiled to IR; loaded from *.ll + IR, +} + +// Represents a fully qualified function name. +struct TFunctionName { + // Name of the function's parent database. Not set if in global + // namespace (e.g. builtins) + 1: optional string db_name + + // Name of the function + 2: required string function_name +} + +struct TScalarFunction { + // Symbol for the function + 1: required string symbol + 2: optional string prepare_fn_symbol + 3: optional string close_fn_symbol +} + +struct TAggregateFunction { + 1: required TTypeDesc intermediate_type + 2: optional string update_fn_symbol + 3: optional string init_fn_symbol + 4: optional string serialize_fn_symbol + 5: optional string merge_fn_symbol + 6: optional string finalize_fn_symbol + 8: optional string get_value_fn_symbol + 9: optional string remove_fn_symbol + 10: optional bool is_analytic_only_fn = false +} + +// Represents a function in the Catalog. +struct TFunction { + // Fully qualified function name. + 1: required TFunctionName name + + // Type of the udf. e.g. hive, native, ir + 2: required TFunctionBinaryType binary_type + + // The types of the arguments to the function + 3: required list arg_types + + // Return type for the function. + 4: required TTypeDesc ret_type + + // If true, this function takes var args. + 5: required bool has_var_args + + // Optional comment to attach to the function + 6: optional string comment + + 7: optional string signature + + // HDFS path for the function binary. This binary must exist at the time the + // function is created. + 8: optional string hdfs_location + + // One of these should be set. + 9: optional TScalarFunction scalar_fn + 10: optional TAggregateFunction aggregate_fn + + 11: optional i64 id + 12: optional string checksum +} + +enum TLoadJobState { + PENDING, + ETL, + LOADING, + FINISHED, + CANCELLED +} + +enum TEtlState { + RUNNING, + FINISHED, + CANCELLED, + UNKNOWN +} + +enum TTableType { + MYSQL_TABLE, + OLAP_TABLE, + SCHEMA_TABLE, + KUDU_TABLE, // Deprecated + BROKER_TABLE, + ES_TABLE +} + +enum TKeysType { + PRIMARY_KEYS, + DUP_KEYS, + UNIQUE_KEYS, + AGG_KEYS +} + +enum TPriority { + NORMAL, + HIGH +} + +struct TBackend { + 1: required string host + 2: required TPort be_port + 3: required TPort http_port +} + +struct TResourceInfo { + 1: required string user + 2: required string group +} + +enum TExportState { + RUNNING, + FINISHED, + CANCELLED, + UNKNOWN +} + +enum TFileType { + FILE_LOCAL, + FILE_BROKER, + FILE_STREAM, // file content is streaming in the buffer +} + +struct TTabletCommitInfo { + 1: required i64 tabletId + 2: required i64 backendId +} + +enum TLoadType { + MANUL_LOAD, + ROUTINE_LOAD, + MINI_LOAD +} + +enum TLoadSourceType { + RAW, + KAFKA, +} diff --git a/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisSinkExample.java b/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisSinkExample.java new file mode 100644 index 00000000000000..dde97def1ffe3f --- /dev/null +++ b/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisSinkExample.java @@ -0,0 +1,56 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +package org.apache.doris.flink; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.table.api.Expressions.$; + +public class DorisSinkExample { + + public static void main(String[] args) { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + + List> data = new ArrayList<>(); + data.add(new Tuple2<>("doris",1)); + DataStreamSource> source = env.fromCollection(data); + tEnv.createTemporaryView("doris_test",source,$("name"),$("age")); + + tEnv.executeSql( + "CREATE TABLE doris_test_sink (" + + "name STRING," + + "age INT" + + ") " + + "WITH (\n" + + " 'connector' = 'doris',\n" + + " 'fenodes' = 'FE_IP:8030',\n" + + " 'table.identifier' = 'db.table',\n" + + " 'username' = 'root',\n" + + " 'password' = ''\n" + + ")"); + + tEnv.executeSql("INSERT INTO doris_test_sink select name,age from doris_test"); + } +} diff --git a/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisSourceDataStream.java b/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisSourceDataStream.java new file mode 100644 index 00000000000000..68152347deef61 --- /dev/null +++ b/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisSourceDataStream.java @@ -0,0 +1,46 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.flink; + +import org.apache.doris.flink.cfg.DorisStreamOptions; +import org.apache.doris.flink.datastream.DorisSourceFunction; +import org.apache.doris.flink.deserialization.SimpleListDeserializationSchema; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import java.util.Properties; + + + +public class DorisSourceDataStream { + + public static void main(String[] args) throws Exception { + Properties properties = new Properties(); + properties.put("fenodes","FE_IP:8030"); + properties.put("username","root"); + properties.put("password",""); + properties.put("table.identifier","db.table"); + properties.put("doris.read.field","id,code,name"); + properties.put("doris.filter.query","name='doris'"); + DorisStreamOptions options = new DorisStreamOptions(properties); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + env.addSource(new DorisSourceFunction(options,new SimpleListDeserializationSchema())).print(); + env.execute("Flink doris test"); + } +} diff --git a/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisSourceExample.java b/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisSourceExample.java new file mode 100644 index 00000000000000..eb1d81937c1668 --- /dev/null +++ b/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisSourceExample.java @@ -0,0 +1,64 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.types.Row; + +public class DorisSourceExample { + + public static void main(String[] args) throws Exception { + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); // source only supports parallelism of 1 + + final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + + // register a table in the catalog + tEnv.executeSql( + "CREATE TABLE doris_source (" + + "bigint_1 BIGINT," + + "char_1 STRING," + + "date_1 STRING," + + "datetime_1 STRING," + + "decimal_1 DECIMAL(5,2)," + + "double_1 DOUBLE," + + "float_1 FLOAT ," + + "int_1 INT ," + + "largeint_1 STRING, " + + "smallint_1 SMALLINT, " + + "tinyint_1 TINYINT, " + + "varchar_1 STRING " + + ") " + + "WITH (\n" + + " 'connector' = 'doris',\n" + + " 'fenodes' = 'FE_IP:8030',\n" + + " 'table.identifier' = 'db.table',\n" + + " 'username' = 'root',\n" + + " 'password' = ''\n" + + ")"); + + // define a dynamic aggregating query + final Table result = tEnv.sqlQuery("SELECT * from doris_source "); + + // print the result to the console + tEnv.toRetractStream(result, Row.class).print(); + env.execute(); + } +} diff --git a/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisSourceSinkExample.java b/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisSourceSinkExample.java new file mode 100644 index 00000000000000..c4ce1a5b5f7817 --- /dev/null +++ b/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/DorisSourceSinkExample.java @@ -0,0 +1,64 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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; + +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; + +public class DorisSourceSinkExample { + + public static void main(String[] args) { + EnvironmentSettings settings = EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build(); + TableEnvironment tEnv = TableEnvironment.create(settings); + tEnv.executeSql( + "CREATE TABLE doris_test (" + + "name STRING," + + "age INT," + + "price DECIMAL(5,2)," + + "sale DOUBLE" + + ") " + + "WITH (\n" + + " 'connector' = 'doris',\n" + + " 'fenodes' = 'FE_IP:8030',\n" + + " 'table.identifier' = 'db.table',\n" + + " 'username' = 'root',\n" + + " 'password' = ''" + + ")"); + tEnv.executeSql( + "CREATE TABLE doris_test_sink (" + + "name STRING," + + "age INT," + + "price DECIMAL(5,2)," + + "sale DOUBLE" + + ") " + + "WITH (\n" + + " 'connector' = 'doris',\n" + + " 'fenodes' = 'FE_IP:8030',\n" + + " 'table.identifier' = 'db.table',\n" + + " 'username' = 'root',\n" + + " 'password' = '',\n" + + " 'sink.batch.size' = '3',\n" + + " 'sink.max-retries' = '2'\n" + + ")"); + + tEnv.executeSql("INSERT INTO doris_test_sink select name,age,price,sale from doris_test"); + } +} diff --git a/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/serialization/TestRowBatch.java b/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/serialization/TestRowBatch.java new file mode 100644 index 00000000000000..a9d24920e3e619 --- /dev/null +++ b/extension/flink-doris-connector/src/test/java/org/apache/doris/flink/serialization/TestRowBatch.java @@ -0,0 +1,439 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.dictionary.DictionaryProvider; +import org.apache.arrow.vector.ipc.ArrowStreamWriter; +import org.apache.arrow.vector.types.FloatingPointPrecision; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.doris.flink.rest.RestService; +import org.apache.doris.flink.rest.models.Schema; +import org.apache.doris.thrift.TScanBatchResult; +import org.apache.doris.thrift.TStatus; +import org.apache.doris.thrift.TStatusCode; +import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableList; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.StringData; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayOutputStream; +import java.math.BigDecimal; +import java.util.Arrays; +import java.util.List; +import java.util.NoSuchElementException; + +import static org.hamcrest.core.StringStartsWith.startsWith; + +public class TestRowBatch { + private static Logger logger = LoggerFactory.getLogger(TestRowBatch.class); + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testRowBatch() throws Exception { + // schema + ImmutableList.Builder childrenBuilder = ImmutableList.builder(); + childrenBuilder.add(new Field("k0", FieldType.nullable(new ArrowType.Bool()), null)); + childrenBuilder.add(new Field("k1", FieldType.nullable(new ArrowType.Int(8, true)), null)); + childrenBuilder.add(new Field("k2", FieldType.nullable(new ArrowType.Int(16, true)), null)); + childrenBuilder.add(new Field("k3", FieldType.nullable(new ArrowType.Int(32, true)), null)); + childrenBuilder.add(new Field("k4", FieldType.nullable(new ArrowType.Int(64, true)), null)); + childrenBuilder.add(new Field("k9", FieldType.nullable(new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)), null)); + childrenBuilder.add(new Field("k8", FieldType.nullable(new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)), null)); + childrenBuilder.add(new Field("k10", FieldType.nullable(new ArrowType.Utf8()), null)); + childrenBuilder.add(new Field("k11", FieldType.nullable(new ArrowType.Utf8()), null)); + childrenBuilder.add(new Field("k5", FieldType.nullable(new ArrowType.Decimal(9,2)), null)); + childrenBuilder.add(new Field("k6", FieldType.nullable(new ArrowType.Utf8()), null)); + + VectorSchemaRoot root = VectorSchemaRoot.create( + new org.apache.arrow.vector.types.pojo.Schema(childrenBuilder.build(), null), + new RootAllocator(Integer.MAX_VALUE)); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + ArrowStreamWriter arrowStreamWriter = new ArrowStreamWriter( + root, + new DictionaryProvider.MapDictionaryProvider(), + outputStream); + + arrowStreamWriter.start(); + root.setRowCount(3); + + FieldVector vector = root.getVector("k0"); + BitVector bitVector = (BitVector)vector; + bitVector.setInitialCapacity(3); + bitVector.allocateNew(3); + bitVector.setSafe(0, 1); + bitVector.setSafe(1, 0); + bitVector.setSafe(2, 1); + vector.setValueCount(3); + + vector = root.getVector("k1"); + TinyIntVector tinyIntVector = (TinyIntVector)vector; + tinyIntVector.setInitialCapacity(3); + tinyIntVector.allocateNew(3); + tinyIntVector.setSafe(0, 1); + tinyIntVector.setSafe(1, 2); + tinyIntVector.setSafe(2, 3); + vector.setValueCount(3); + + vector = root.getVector("k2"); + SmallIntVector smallIntVector = (SmallIntVector)vector; + smallIntVector.setInitialCapacity(3); + smallIntVector.allocateNew(3); + smallIntVector.setSafe(0, 1); + smallIntVector.setSafe(1, 2); + smallIntVector.setSafe(2, 3); + vector.setValueCount(3); + + vector = root.getVector("k3"); + IntVector intVector = (IntVector)vector; + intVector.setInitialCapacity(3); + intVector.allocateNew(3); + intVector.setSafe(0, 1); + intVector.setNull(1); + intVector.setSafe(2, 3); + vector.setValueCount(3); + + vector = root.getVector("k4"); + BigIntVector bigIntVector = (BigIntVector)vector; + bigIntVector.setInitialCapacity(3); + bigIntVector.allocateNew(3); + bigIntVector.setSafe(0, 1); + bigIntVector.setSafe(1, 2); + bigIntVector.setSafe(2, 3); + vector.setValueCount(3); + + vector = root.getVector("k5"); + DecimalVector decimalVector = (DecimalVector)vector; + decimalVector.setInitialCapacity(3); + decimalVector.allocateNew(); + decimalVector.setIndexDefined(0); + decimalVector.setSafe(0, new BigDecimal("12.34")); + decimalVector.setIndexDefined(1); + decimalVector.setSafe(1, new BigDecimal("88.88")); + decimalVector.setIndexDefined(2); + decimalVector.setSafe(2, new BigDecimal("10.22")); + vector.setValueCount(3); + + vector = root.getVector("k6"); + VarCharVector charVector = (VarCharVector)vector; + charVector.setInitialCapacity(3); + charVector.allocateNew(); + charVector.setIndexDefined(0); + charVector.setValueLengthSafe(0, 5); + charVector.setSafe(0, "char1".getBytes()); + charVector.setIndexDefined(1); + charVector.setValueLengthSafe(1, 5); + charVector.setSafe(1, "char2".getBytes()); + charVector.setIndexDefined(2); + charVector.setValueLengthSafe(2, 5); + charVector.setSafe(2, "char3".getBytes()); + vector.setValueCount(3); + + vector = root.getVector("k8"); + Float8Vector float8Vector = (Float8Vector)vector; + float8Vector.setInitialCapacity(3); + float8Vector.allocateNew(3); + float8Vector.setSafe(0, 1.1); + float8Vector.setSafe(1, 2.2); + float8Vector.setSafe(2, 3.3); + vector.setValueCount(3); + + vector = root.getVector("k9"); + Float4Vector float4Vector = (Float4Vector)vector; + float4Vector.setInitialCapacity(3); + float4Vector.allocateNew(3); + float4Vector.setSafe(0, 1.1f); + float4Vector.setSafe(1, 2.2f); + float4Vector.setSafe(2, 3.3f); + vector.setValueCount(3); + + vector = root.getVector("k10"); + VarCharVector datecharVector = (VarCharVector)vector; + datecharVector.setInitialCapacity(3); + datecharVector.allocateNew(); + datecharVector.setIndexDefined(0); + datecharVector.setValueLengthSafe(0, 5); + datecharVector.setSafe(0, "2008-08-08".getBytes()); + datecharVector.setIndexDefined(1); + datecharVector.setValueLengthSafe(1, 5); + datecharVector.setSafe(1, "1900-08-08".getBytes()); + datecharVector.setIndexDefined(2); + datecharVector.setValueLengthSafe(2, 5); + datecharVector.setSafe(2, "2100-08-08".getBytes()); + vector.setValueCount(3); + + vector = root.getVector("k11"); + VarCharVector timecharVector = (VarCharVector)vector; + timecharVector.setInitialCapacity(3); + timecharVector.allocateNew(); + timecharVector.setIndexDefined(0); + timecharVector.setValueLengthSafe(0, 5); + timecharVector.setSafe(0, "2008-08-08 00:00:00".getBytes()); + timecharVector.setIndexDefined(1); + timecharVector.setValueLengthSafe(1, 5); + timecharVector.setSafe(1, "1900-08-08 00:00:00".getBytes()); + timecharVector.setIndexDefined(2); + timecharVector.setValueLengthSafe(2, 5); + timecharVector.setSafe(2, "2100-08-08 00:00:00".getBytes()); + vector.setValueCount(3); + + arrowStreamWriter.writeBatch(); + + arrowStreamWriter.end(); + arrowStreamWriter.close(); + + TStatus status = new TStatus(); + status.setStatus_code(TStatusCode.OK); + TScanBatchResult scanBatchResult = new TScanBatchResult(); + scanBatchResult.setStatus(status); + scanBatchResult.setEos(false); + scanBatchResult.setRows(outputStream.toByteArray()); + + String schemaStr = "{\"properties\":[{\"type\":\"BOOLEAN\",\"name\":\"k0\",\"comment\":\"\"}," + + "{\"type\":\"TINYINT\",\"name\":\"k1\",\"comment\":\"\"},{\"type\":\"SMALLINT\",\"name\":\"k2\"," + + "\"comment\":\"\"},{\"type\":\"INT\",\"name\":\"k3\",\"comment\":\"\"},{\"type\":\"BIGINT\"," + + "\"name\":\"k4\",\"comment\":\"\"},{\"type\":\"FLOAT\",\"name\":\"k9\",\"comment\":\"\"}," + + "{\"type\":\"DOUBLE\",\"name\":\"k8\",\"comment\":\"\"},{\"type\":\"DATE\",\"name\":\"k10\"," + + "\"comment\":\"\"},{\"type\":\"DATETIME\",\"name\":\"k11\",\"comment\":\"\"}," + + "{\"name\":\"k5\",\"scale\":\"9\",\"comment\":\"\"," + + "\"type\":\"DECIMAL\",\"precision\":\"2\"},{\"type\":\"CHAR\",\"name\":\"k6\",\"comment\":\"\"}]," + + "\"status\":200}"; + + Schema schema = RestService.parseSchema(schemaStr, logger); + + RowBatch rowBatch = new RowBatch(scanBatchResult, schema).readArrow(); + + List expectedRow1 = Lists.newArrayList( + Boolean.TRUE, + (byte) 1, + (short) 1, + 1, + 1L, + (float) 1.1, + (double) 1.1, + StringData.fromString("2008-08-08"), + StringData.fromString("2008-08-08 00:00:00"), + DecimalData.fromBigDecimal(new BigDecimal(12.34), 4, 2), + StringData.fromString("char1") + ); + + List expectedRow2 = Arrays.asList( + Boolean.FALSE, + (byte) 2, + (short) 2, + null, + 2L, + (float) 2.2, + (double) 2.2, + StringData.fromString("1900-08-08"), + StringData.fromString("1900-08-08 00:00:00"), + DecimalData.fromBigDecimal(new BigDecimal(88.88), 4, 2), + StringData.fromString("char2") + ); + + List expectedRow3 = Arrays.asList( + Boolean.TRUE, + (byte) 3, + (short) 3, + 3, + 3L, + (float) 3.3, + (double) 3.3, + StringData.fromString("2100-08-08"), + StringData.fromString("2100-08-08 00:00:00"), + DecimalData.fromBigDecimal(new BigDecimal(10.22), 4, 2), + StringData.fromString("char3") + ); + + Assert.assertTrue(rowBatch.hasNext()); + List actualRow1 = rowBatch.next(); + Assert.assertEquals(expectedRow1, actualRow1); + + Assert.assertTrue(rowBatch.hasNext()); + List actualRow2 = rowBatch.next(); + Assert.assertEquals(expectedRow2, actualRow2); + + Assert.assertTrue(rowBatch.hasNext()); + List actualRow3 = rowBatch.next(); + Assert.assertEquals(expectedRow3, actualRow3); + + Assert.assertFalse(rowBatch.hasNext()); + thrown.expect(NoSuchElementException.class); + thrown.expectMessage(startsWith("Get row offset:")); + rowBatch.next(); + } + + @Test + public void testBinary() throws Exception { + byte[] binaryRow0 = {'a', 'b', 'c'}; + byte[] binaryRow1 = {'d', 'e', 'f'}; + byte[] binaryRow2 = {'g', 'h', 'i'}; + + ImmutableList.Builder childrenBuilder = ImmutableList.builder(); + childrenBuilder.add(new Field("k7", FieldType.nullable(new ArrowType.Binary()), null)); + + VectorSchemaRoot root = VectorSchemaRoot.create( + new org.apache.arrow.vector.types.pojo.Schema(childrenBuilder.build(), null), + new RootAllocator(Integer.MAX_VALUE)); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + ArrowStreamWriter arrowStreamWriter = new ArrowStreamWriter( + root, + new DictionaryProvider.MapDictionaryProvider(), + outputStream); + + arrowStreamWriter.start(); + root.setRowCount(3); + + FieldVector vector = root.getVector("k7"); + VarBinaryVector varBinaryVector = (VarBinaryVector) vector; + varBinaryVector.setInitialCapacity(3); + varBinaryVector.allocateNew(); + varBinaryVector.setIndexDefined(0); + varBinaryVector.setValueLengthSafe(0, 3); + varBinaryVector.setSafe(0, binaryRow0); + varBinaryVector.setIndexDefined(1); + varBinaryVector.setValueLengthSafe(1, 3); + varBinaryVector.setSafe(1, binaryRow1); + varBinaryVector.setIndexDefined(2); + varBinaryVector.setValueLengthSafe(2, 3); + varBinaryVector.setSafe(2, binaryRow2); + vector.setValueCount(3); + + arrowStreamWriter.writeBatch(); + + arrowStreamWriter.end(); + arrowStreamWriter.close(); + + TStatus status = new TStatus(); + status.setStatus_code(TStatusCode.OK); + TScanBatchResult scanBatchResult = new TScanBatchResult(); + scanBatchResult.setStatus(status); + scanBatchResult.setEos(false); + scanBatchResult.setRows(outputStream.toByteArray()); + + String schemaStr = "{\"properties\":[{\"type\":\"BINARY\",\"name\":\"k7\",\"comment\":\"\"}], \"status\":200}"; + + Schema schema = RestService.parseSchema(schemaStr, logger); + + RowBatch rowBatch = new RowBatch(scanBatchResult, schema).readArrow(); + + Assert.assertTrue(rowBatch.hasNext()); + List actualRow0 = rowBatch.next(); + Assert.assertArrayEquals(binaryRow0, (byte[])actualRow0.get(0)); + + Assert.assertTrue(rowBatch.hasNext()); + List actualRow1 = rowBatch.next(); + Assert.assertArrayEquals(binaryRow1, (byte[])actualRow1.get(0)); + + Assert.assertTrue(rowBatch.hasNext()); + List actualRow2 = rowBatch.next(); + Assert.assertArrayEquals(binaryRow2, (byte[])actualRow2.get(0)); + + Assert.assertFalse(rowBatch.hasNext()); + thrown.expect(NoSuchElementException.class); + thrown.expectMessage(startsWith("Get row offset:")); + rowBatch.next(); + } + + @Test + public void testDecimalV2() throws Exception { + ImmutableList.Builder childrenBuilder = ImmutableList.builder(); + childrenBuilder.add(new Field("k7", FieldType.nullable(new ArrowType.Decimal(27, 9)), null)); + + VectorSchemaRoot root = VectorSchemaRoot.create( + new org.apache.arrow.vector.types.pojo.Schema(childrenBuilder.build(), null), + new RootAllocator(Integer.MAX_VALUE)); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + ArrowStreamWriter arrowStreamWriter = new ArrowStreamWriter( + root, + new DictionaryProvider.MapDictionaryProvider(), + outputStream); + + arrowStreamWriter.start(); + root.setRowCount(3); + + FieldVector vector = root.getVector("k7"); + DecimalVector decimalVector = (DecimalVector) vector; + decimalVector.setInitialCapacity(3); + decimalVector.allocateNew(3); + decimalVector.setSafe(0, new BigDecimal("12.340000000")); + decimalVector.setSafe(1, new BigDecimal("88.880000000")); + decimalVector.setSafe(2, new BigDecimal("10.000000000")); + vector.setValueCount(3); + + arrowStreamWriter.writeBatch(); + + arrowStreamWriter.end(); + arrowStreamWriter.close(); + + TStatus status = new TStatus(); + status.setStatus_code(TStatusCode.OK); + TScanBatchResult scanBatchResult = new TScanBatchResult(); + scanBatchResult.setStatus(status); + scanBatchResult.setEos(false); + scanBatchResult.setRows(outputStream.toByteArray()); + + String schemaStr = "{\"properties\":[{\"type\":\"DECIMALV2\",\"scale\": 0," + + "\"precision\": 9, \"name\":\"k7\",\"comment\":\"\"}], " + + "\"status\":200}"; + + Schema schema = RestService.parseSchema(schemaStr, logger); + + RowBatch rowBatch = new RowBatch(scanBatchResult, schema).readArrow(); + + Assert.assertTrue(rowBatch.hasNext()); + List actualRow0 = rowBatch.next(); + Assert.assertEquals(DecimalData.fromBigDecimal(new BigDecimal(12.340000000), 11, 9), actualRow0.get(0)); + + Assert.assertTrue(rowBatch.hasNext()); + List actualRow1 = rowBatch.next(); + + Assert.assertEquals(DecimalData.fromBigDecimal(new BigDecimal(88.880000000), 11, 9), actualRow1.get(0)); + + Assert.assertTrue(rowBatch.hasNext()); + List actualRow2 = rowBatch.next(); + Assert.assertEquals(DecimalData.fromBigDecimal(new BigDecimal(10.000000000),11, 9), actualRow2.get(0)); + + Assert.assertFalse(rowBatch.hasNext()); + thrown.expect(NoSuchElementException.class); + thrown.expectMessage(startsWith("Get row offset:")); + rowBatch.next(); + } +}