From 7ac4301fc5e906a1c0a9a07b0192c0d33aa46228 Mon Sep 17 00:00:00 2001 From: "jiafeng.zhang" Date: Thu, 20 Jan 2022 12:58:07 +0800 Subject: [PATCH 01/13] Add support for writing data to Apache Doris Add support for writing data to Apache Doris --- core/pom.xml | 2 +- doriswriter/doc/doriswriter.md | 183 ++++++++++++ doriswriter/doc/mysql2doris.json | 44 +++ doriswriter/pom.xml | 107 +++++++ doriswriter/src/main/assembly/package.xml | 56 ++++ .../plugin/writer/doriswriter/DorisCodec.java | 75 +++++ .../writer/doriswriter/DorisFlushBatch.java | 58 ++++ .../writer/doriswriter/DorisJsonCodec.java | 50 ++++ .../writer/doriswriter/DorisWriter.java | 259 +++++++++++++++++ .../doriswriter/DorisWriterEmitter.java | 274 ++++++++++++++++++ .../datax/plugin/writer/doriswriter/Key.java | 171 +++++++++++ doriswriter/src/main/resources/plugin.json | 6 + .../main/resources/plugin_job_template.json | 15 + pom.xml | 1 + 14 files changed, 1300 insertions(+), 1 deletion(-) create mode 100644 doriswriter/doc/doriswriter.md create mode 100644 doriswriter/doc/mysql2doris.json create mode 100644 doriswriter/pom.xml create mode 100644 doriswriter/src/main/assembly/package.xml create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisFlushBatch.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java create mode 100644 doriswriter/src/main/resources/plugin.json create mode 100644 doriswriter/src/main/resources/plugin_job_template.json diff --git a/core/pom.xml b/core/pom.xml index 174a18d3..2e085e14 100755 --- a/core/pom.xml +++ b/core/pom.xml @@ -41,7 +41,7 @@ org.apache.httpcomponents httpclient - 4.5 + 4.5.13 org.apache.httpcomponents diff --git a/doriswriter/doc/doriswriter.md b/doriswriter/doc/doriswriter.md new file mode 100644 index 00000000..7a072620 --- /dev/null +++ b/doriswriter/doc/doriswriter.md @@ -0,0 +1,183 @@ +# DorisWriter 插件文档 + +## 1 快速介绍 +Apache Doris是一个现代化的MPP分析型数据库产品。仅需亚秒级响应时间即可获得查询结果,有效地支持实时数据分析。Apache Doris的分布式架构非常简洁,易于运维,并且可以支持10PB以上的超大数据集。 + +Apache Doris可以满足多种数据分析需求,例如固定历史报表,实时数据分析,交互式数据分析和探索式数据分析等。令您的数据分析工作更加简单高效! + +DorisWriter是一个支持将大批量数据写入Doris中的一个插件,可以结合Datax其他读取数据的插件结合使用完成数据的整合。 + +Doris是完全支持Mysql协议的,所以如果你需要读取Doris的数据,可以使用mysqlreader插件完成,这里不单独提供Doris数据读取插件。 + +## 2 实现原理 +DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter会将`reader`读取的数据进行缓存在内存中,拼接成Json文本,然后批量导入至Doris。 + +## 3 功能说明 + +### 3.1 配置样例 + +这里是一份从Stream读取数据后导入至Doris的配置文件。 + +``` +{ + "job": { + "setting": { + "speed": { + "channel": 1 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + } + }, + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "value": "皮蛋1", + "type": "string" + }, + { + "value": "皮蛋2", + "type": "string" + }, + { + "value": "111", + "type": "long" + }, + { + "value": "222", + "type": "long" + } + ], + "sliceRecordCount": 100 + } + }, + "writer": { + "name": "doriswriter", + "parameter": { + "feLoadUrl": ["127.0.0.1:8030", "127.0.0.2:8030", "127.0.0.3:8030"], + "beLoadUrl": ["192.168.10.1:8040", "192.168.10.2:8040", "192.168.10.3:8040"], + "jdbcUrl": "jdbc:mysql://127.0.0.1:9030/", + "database": "db1", + "table": "t1", + "column": ["k1", "k2", "v1", "v2"], + "username": "root", + "password": "", + "postSql": [], + "preSql": [], + "loadProps": { + }, + "maxBatchRows" : 500000, + "maxBatchByteSize" : 104857600, + "labelPrefix": "my_prefix", + "lineDelimiter": "\n" + } + } + } + ] + } +} +``` + +### 3.2 参数说明 + +* **jdbcUrl** + + - 描述:Doris 的 JDBC 连接串,用户执行 preSql 或 postSQL。 + - 必选:是 + - 默认值:无 + +* **feLoadUrl** + + - 描述:和 **beLoadUrl** 二选一。作为 Stream Load 的连接目标。格式为 "ip:port"。其中 IP 是 FE 节点 IP,port 是 FE 节点的 http_port。可以填写多个,doriswriter 将以轮询的方式访问。 + - 必选:否 + - 默认值:无 + +* **beLoadUrl** + + - 描述:和 **feLoadUrl** 二选一。作为 Stream Load 的连接目标。格式为 "ip:port"。其中 IP 是 BE 节点 IP,port 是 BE 节点的 webserver_port。可以填写多个,doriswriter 将以轮询的方式访问。 + - 必选:否 + - 默认值:无 + +* **username** + + - 描述:访问Doris数据库的用户名 + - 必选:是 + - 默认值:无 + +* **password** + + - 描述:访问Doris数据库的密码 + - 必选:否 + - 默认值:空 + +* **database** + + - 描述:需要写入的Doris数据库名称。 + - 必选:是 + - 默认值:无 + +* **table** + + - 描述:需要写入的Doris表名称。 + - 必选:是 + - 默认值:无 + +* **column** + + - 描述:目的表**需要写入数据**的字段,这些字段将作为生成的 Json 数据的字段名。字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 + - 必选:是 + - 默认值:否 + +* **preSql** + + - 描述:写入数据到目的表前,会先执行这里的标准语句。 + - 必选:否 + - 默认值:无 + +* **postSql** + + - 描述:写入数据到目的表后,会执行这里的标准语句。 + - 必选:否 + - 默认值:无 + + +* **maxBatchRows** + + - 描述:每批次导入数据的最大行数。和 **maxBatchByteSize** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 + - 必选:否 + - 默认值:500000 + +* **maxBatchByteSize** + + - 描述:每批次导入数据的最大数据量。和 ** maxBatchRows** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 + - 必选:否 + - 默认值:104857600 + +* **labelPrefix** + + - 描述:每批次导入任务的 label 前缀。最终的 label 将有 `labelPrefix + UUID + 序号` 组成 + - 必选:否 + - 默认值:`datax_doris_writer_` + +* **lineDelimiter** + + - 描述:每批次数据包含多行,每行为 Json 格式,每行的的分隔符即为 lineDelimiter。支持多个字节, 例如'\x02\x03'。 + - 必选:否 + - 默认值:`\n` + +* **loadProps** + + - 描述:StreamLoad 的请求参数,详情参照StreamLoad介绍页面。 + - 必选:否 + - 默认值:无 + +* **connectTimeout** + + - 描述:StreamLoad单次请求的超时时间, 单位毫秒(ms)。 + - 必选:否 + - 默认值:-1 diff --git a/doriswriter/doc/mysql2doris.json b/doriswriter/doc/mysql2doris.json new file mode 100644 index 00000000..f6e9d9bd --- /dev/null +++ b/doriswriter/doc/mysql2doris.json @@ -0,0 +1,44 @@ +{ + "job": { + "content": [ + { + "reader": { + "name": "mysqlreader", + "parameter": { + "column": ["k1", "k2", "k3"], + "connection": [ + { + "jdbcUrl": ["jdbc:mysql://192.168.10.10:3306/db1"], + "table": ["t1"] + } + ], + "username": "root", + "password": "123456", + "where": "" + } + }, + "writer": { + "name": "doriswriter", + "parameter": { + "feLoadUrl": ["192.168.1.1:8030"], + "jdbcUrl": "jdbc:mysql://192.168.1.1:9030/", + "loadProps": { + }, + "database": "db1", + "table": "t3", + "column": ["k1", "k2", "k3"], + "username": "root", + "password": "12345", + "postSql": [], + "preSql": [] + } + } + } + ], + "setting": { + "speed": { + "channel": "1" + } + } + } +} diff --git a/doriswriter/pom.xml b/doriswriter/pom.xml new file mode 100644 index 00000000..1051436a --- /dev/null +++ b/doriswriter/pom.xml @@ -0,0 +1,107 @@ + + + + + datax-all + com.alibaba.datax + 0.0.1-SNAPSHOT + ../pom.xml + + + 4.0.0 + doriswriter + doriswriter + jar + + + + com.alibaba.datax + datax-common + ${datax-project-version} + + + slf4j-log4j12 + org.slf4j + + + + + org.slf4j + slf4j-api + + + ch.qos.logback + logback-classic + + + + com.alibaba.datax + plugin-rdbms-util + ${datax-project-version} + + + + mysql + mysql-connector-java + ${mysql.driver.version} + + + + org.apache.httpcomponents + httpclient + 4.5.13 + + + + + + + maven-compiler-plugin + + ${jdk-version} + ${jdk-version} + ${project-sourceEncoding} + + + + + + maven-assembly-plugin + + + src/main/assembly/package.xml + + datax + + + + dwzip + package + + single + + + + + + + diff --git a/doriswriter/src/main/assembly/package.xml b/doriswriter/src/main/assembly/package.xml new file mode 100644 index 00000000..9fca5e93 --- /dev/null +++ b/doriswriter/src/main/assembly/package.xml @@ -0,0 +1,56 @@ + + + + + + + + dir + + false + + + src/main/resources + + plugin.json + plugin_job_template.json + + plugin/writer/doriswriter + + + target/ + + doriswriter-0.0.1-SNAPSHOT.jar + + plugin/writer/doriswriter + + + + + + false + plugin/writer/doriswriter/libs + runtime + + + diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java new file mode 100644 index 00000000..9a364f40 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java @@ -0,0 +1,75 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.element.Column; +import com.alibaba.datax.common.element.DateColumn; +import com.alibaba.datax.common.element.Record; +import org.apache.commons.lang3.time.DateFormatUtils; + +import java.util.List; +import java.util.TimeZone; + +public abstract class DorisCodec { + protected static String timeZone = "GMT+8"; + protected static TimeZone timeZoner = TimeZone.getTimeZone(timeZone); + protected final List fieldNames; + + public DorisCodec(final List fieldNames) { + this.fieldNames = fieldNames; + } + + public abstract String serialize(Record row); + + /** + * convert datax internal data to string + * + * @param col + * @return + */ + protected Object convertColumn(final Column col) { + if (null == col.getRawData()) { + return null; + } + Column.Type type = col.getType(); + switch (type) { + case BOOL: + case INT: + case LONG: + return col.asLong(); + case DOUBLE: + return col.asDouble(); + case STRING: + return col.asString(); + case DATE: { + final DateColumn.DateType dateType = ((DateColumn) col).getSubType(); + switch (dateType) { + case DATE: + return DateFormatUtils.format(col.asDate(), "yyyy-MM-dd", timeZoner); + case DATETIME: + return DateFormatUtils.format(col.asDate(), "yyyy-MM-dd HH:mm:ss", timeZoner); + default: + return col.asString(); + } + } + default: + // BAD, NULL, BYTES + return null; + } + } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisFlushBatch.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisFlushBatch.java new file mode 100644 index 00000000..493ddec9 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisFlushBatch.java @@ -0,0 +1,58 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package com.alibaba.datax.plugin.writer.doriswriter; + +// A wrapper class to hold a batch of loaded rows +public class DorisFlushBatch { + private String lineDelimiter; + private String label; + private long rows = 0; + private StringBuilder data = new StringBuilder(); + + public DorisFlushBatch(String lineDelimiter) { + this.lineDelimiter = lineDelimiter; + } + + public void setLabel(String label) { + this.label = label; + } + + public String getLabel() { + return label; + } + + public long getRows() { + return rows; + } + + public void putData(String row) { + if (data.length() > 0) { + data.append(lineDelimiter); + } + data.append(row); + rows++; + } + + public StringBuilder getData() { + return data; + } + + public long getSize() { + return data.length(); + } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java new file mode 100644 index 00000000..8d4568c2 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java @@ -0,0 +1,50 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.fastjson.JSON; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +// Convert DataX data to json +public class DorisJsonCodec extends DorisCodec { + private Map rowMap; + + public DorisJsonCodec(final List fieldNames) { + super(fieldNames); + this.rowMap = new HashMap<>(this.fieldNames.size()); + } + + @Override + public String serialize(final Record row) { + if (null == this.fieldNames) { + return ""; + } + + rowMap.clear(); + int idx = 0; + for (final String fieldName : this.fieldNames) { + rowMap.put(fieldName, this.convertColumn(row.getColumn(idx))); + ++idx; + } + return JSON.toJSONString(rowMap); + } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java new file mode 100644 index 00000000..71e2d1ad --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java @@ -0,0 +1,259 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.common.plugin.RecordReceiver; +import com.alibaba.datax.common.spi.Writer; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.rdbms.util.DBUtil; +import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; +import com.alibaba.datax.plugin.rdbms.util.DataBaseType; +import com.alibaba.datax.plugin.rdbms.util.RdbmsException; +import com.alibaba.datax.plugin.rdbms.writer.Constant; +import com.alibaba.druid.sql.parser.ParserException; +import com.google.common.base.Strings; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.UUID; + +public class DorisWriter extends Writer { + public DorisWriter() { + } + + public static class Task extends com.alibaba.datax.common.spi.Writer.Task { + private DorisWriterEmitter dorisWriterEmitter; + private Key keys; + private DorisCodec rowCodec; + private int batchNum = 0; + + public Task() { + } + + @Override + public void init() { + this.keys = new Key(super.getPluginJobConf()); + this.rowCodec = new DorisJsonCodec(this.keys.getColumns()); + this.dorisWriterEmitter = new DorisWriterEmitter(keys); + } + + @Override + public void prepare() { + } + + @Override + public void startWrite(RecordReceiver recordReceiver) { + String lineDelimiter = this.keys.getLineDelimiter(); + DorisFlushBatch flushBatch = new DorisFlushBatch(lineDelimiter); + long batchCount = 0; + long batchByteSize = 0L; + Record record; + // loop to get record from datax + while ((record = recordReceiver.getFromReader()) != null) { + // check column size + if (record.getColumnNumber() != this.keys.getColumns().size()) { + throw DataXException.asDataXException(DBUtilErrorCode.CONF_ERROR, + String.format("config writer column info error. because the column number of reader is :%s" + + "and the column number of writer is:%s. please check you datax job config json.", + record.getColumnNumber(), this.keys.getColumns().size())); + } + // codec record + final String recordStr = this.rowCodec.serialize(record); + + // put into buffer + flushBatch.putData(recordStr); + batchCount += 1; + batchByteSize += recordStr.length(); + // trigger buffer + if (batchCount >= this.keys.getBatchRows() || batchByteSize >= this.keys.getBatchByteSize()) { + // generate doris stream load label + flush(flushBatch); + // clear buffer + batchCount = 0; + batchByteSize = 0L; + flushBatch = new DorisFlushBatch(lineDelimiter); + } + } // end of while + + if (flushBatch.getSize() > 0) { + flush(flushBatch); + } + } + + private void flush(DorisFlushBatch flushBatch) { + final String label = getStreamLoadLabel(); + flushBatch.setLabel(label); + dorisWriterEmitter.doStreamLoad(flushBatch); + } + + private String getStreamLoadLabel() { + String labelPrefix = this.keys.getLabelPrefix(); + return labelPrefix + UUID.randomUUID().toString() + "_" + (batchNum++); + } + + @Override + public void post() { + + } + + @Override + public void destroy() { + } + + @Override + public boolean supportFailOver() { + return false; + } + } + + public static class Job extends com.alibaba.datax.common.spi.Writer.Job { + private static final Logger LOG = LoggerFactory.getLogger(DorisWriter.Job.class); + private Configuration originalConfig = null; + private Key keys; + + public Job() { + } + + @Override + public void init() { + this.originalConfig = super.getPluginJobConf(); + this.keys = new Key(super.getPluginJobConf()); + this.keys.doPretreatment(); + } + + @Override + public void preCheck() { + this.init(); + this.preCheckPrePareSQL(this.keys); + this.preCheckPostSQL(this.keys); + } + + @Override + public void prepare() { + String username = this.keys.getUsername(); + String password = this.keys.getPassword(); + String jdbcUrl = this.keys.getJdbcUrl(); + List renderedPreSqls = this.renderPreOrPostSqls(this.keys.getPreSqlList(), this.keys.getTable()); + if (!renderedPreSqls.isEmpty()) { + Connection conn = DBUtil.getConnection(DataBaseType.MySql, jdbcUrl, username, password); + LOG.info("prepare execute preSqls:[{}]. doris jdbc url:{}.", String.join(";", renderedPreSqls), jdbcUrl); + this.executeSqls(conn, renderedPreSqls); + DBUtil.closeDBResources(null, null, conn); + } + } + + @Override + public List split(int mandatoryNumber) { + List configurations = new ArrayList<>(mandatoryNumber); + + for (int i = 0; i < mandatoryNumber; ++i) { + configurations.add(this.originalConfig); + } + + return configurations; + } + + @Override + public void post() { + String username = this.keys.getUsername(); + String password = this.keys.getPassword(); + String jdbcUrl = this.keys.getJdbcUrl(); + List renderedPostSqls = this.renderPreOrPostSqls(this.keys.getPostSqlList(), this.keys.getTable()); + if (!renderedPostSqls.isEmpty()) { + Connection conn = DBUtil.getConnection(DataBaseType.MySql, jdbcUrl, username, password); + LOG.info("prepare execute postSqls:[{}]. doris jdbc url为:{}.", String.join(";", renderedPostSqls), jdbcUrl); + this.executeSqls(conn, renderedPostSqls); + DBUtil.closeDBResources(null, null, conn); + } + + } + + @Override + public void destroy() { + } + + private List renderPreOrPostSqls(final List preOrPostSqls, final String tableName) { + if (null == preOrPostSqls) { + return Collections.emptyList(); + } + final List renderedSqls = new ArrayList<>(); + for (final String sql : preOrPostSqls) { + if (!Strings.isNullOrEmpty(sql)) { + renderedSqls.add(sql.replace(Constant.TABLE_NAME_PLACEHOLDER, tableName)); + } + } + return renderedSqls; + } + + private void executeSqls(final Connection conn, final List sqls) { + Statement stmt = null; + String currentSql = null; + try { + stmt = conn.createStatement(); + for (String s : sqls) { + final String sql = currentSql = s; + DBUtil.executeSqlWithoutResultSet(stmt, sql); + } + } catch (Exception e) { + throw RdbmsException.asQueryException(DataBaseType.MySql, e, currentSql, null, null); + } finally { + DBUtil.closeDBResources(null, stmt, null); + } + } + + private void preCheckPrePareSQL(final Key keys) { + final String table = keys.getTable(); + final List preSqls = keys.getPreSqlList(); + final List renderedPreSqls = renderPreOrPostSqls(preSqls, table); + if (!renderedPreSqls.isEmpty()) { + LOG.info("prepare check preSqls:[{}].", String.join(";", renderedPreSqls)); + for (final String sql : renderedPreSqls) { + try { + DBUtil.sqlValid(sql, DataBaseType.MySql); + } catch (ParserException e) { + throw RdbmsException.asPreSQLParserException(DataBaseType.MySql, e, sql); + } + } + } + } + + private void preCheckPostSQL(final Key keys) { + final String table = keys.getTable(); + final List postSqls = keys.getPostSqlList(); + final List renderedPostSqls = renderPreOrPostSqls(postSqls, table); + if (!renderedPostSqls.isEmpty()) { + LOG.info("prepare check postSqls:[{}].", String.join(";", renderedPostSqls)); + for (final String sql : renderedPostSqls) { + try { + DBUtil.sqlValid(sql, DataBaseType.MySql); + } catch (ParserException e) { + throw RdbmsException.asPostSQLParserException(DataBaseType.MySql, e, sql); + } + } + } + } + + } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java new file mode 100644 index 00000000..fb4e46b0 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java @@ -0,0 +1,274 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; +import com.alibaba.fastjson.JSON; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.http.HttpEntity; +import org.apache.http.HttpHeaders; +import org.apache.http.HttpRequest; +import org.apache.http.HttpResponse; +import org.apache.http.HttpStatus; +import org.apache.http.ProtocolException; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpHead; +import org.apache.http.client.methods.HttpPut; +import org.apache.http.client.methods.HttpUriRequest; +import org.apache.http.client.methods.RequestBuilder; +import org.apache.http.entity.ByteArrayEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.DefaultRedirectStrategy; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.protocol.HttpContext; +import org.apache.http.util.EntityUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.URI; +import java.net.URL; +import java.util.Base64; +import java.util.List; +import java.util.Map; + +// Used to load batch of rows to Doris using stream load +public class DorisWriterEmitter { + private static final Logger LOG = LoggerFactory.getLogger(DorisWriterEmitter.class); + private final Key keys; + private int hostPos = 0; + private List targetHosts = Lists.newArrayList(); + + private RequestConfig requestConfig; + + public DorisWriterEmitter(final Key keys) { + this.keys = keys; + initHostList(); + initRequestConfig(); + } + + private void initRequestConfig() { + requestConfig = RequestConfig.custom().setConnectTimeout(this.keys.getConnectTimeout()).build(); + } + + // get target host from config + private void initHostList() { + List hosts = this.keys.getBeLoadUrlList(); + if (hosts == null || hosts.isEmpty()) { + hosts = this.keys.getFeLoadUrlList(); + } + if (hosts == null || hosts.isEmpty()) { + DataXException.asDataXException(DBUtilErrorCode.CONF_ERROR, + "Either beLoadUrl or feLoadUrl must be set"); + } + for (String beHost : hosts) { + targetHosts.add("http://" + beHost); + } + } + + /** + * execute doris stream load + */ + public void doStreamLoad(final DorisFlushBatch flushData) { + long start = System.currentTimeMillis(); + final String host = this.getAvailableHost(); + if (null == host) { + throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, "None of the load url can be connected."); + } + final String loadUrl = host + "/api/" + this.keys.getDatabase() + "/" + this.keys.getTable() + "/_stream_load"; + // do http put request and get response + final Map loadResult; + try { + loadResult = this.doHttpPut(loadUrl, flushData); + } catch (IOException e) { + throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, e); + } + + long cost = System.currentTimeMillis() - start; + LOG.info("StreamLoad response: " + JSON.toJSONString(loadResult) + ", cost(ms): " + cost); + final String keyStatus = "Status"; + if (null == loadResult || !loadResult.containsKey(keyStatus)) { + throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, "Unable to flush data to doris: unknown result status."); + } + if (loadResult.get(keyStatus).equals("Fail")) { + throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, "Failed to flush data to doris.\n" + JSON.toJSONString(loadResult)); + } + } + + /** + * loop to get target host + * + * @return + */ + private String getAvailableHost() { + if (this.hostPos >= targetHosts.size()) { + this.hostPos = 0; + } + + while (this.hostPos < targetHosts.size()) { + final String host = targetHosts.get(hostPos); + ++this.hostPos; + if (this.tryHttpConnection(host)) { + return host; + } + } + + return null; + } + + private boolean tryHttpConnection(final String host) { + try { + final URL url = new URL(host); + final HttpURLConnection co = (HttpURLConnection) url.openConnection(); + co.setConnectTimeout(1000); + co.connect(); + co.disconnect(); + return true; + } catch (Exception e) { + LOG.warn("Failed to connect to address:{} , Exception ={}", host, e); + return false; + } + } + + private Map doHttpPut(final String loadUrl, final DorisFlushBatch flushBatch) throws IOException { + LOG.info(String.format("Executing stream load to: '%s', size: %s, rows: %d", + loadUrl, flushBatch.getSize(), flushBatch.getRows())); + + final HttpClientBuilder httpClientBuilder = HttpClients.custom().setRedirectStrategy(new DefaultRedirectStrategy() { + @Override + protected boolean isRedirectable(final String method) { + return true; + } + + @Override + public HttpUriRequest getRedirect(HttpRequest request, HttpResponse response, HttpContext context) throws ProtocolException { + URI uri = this.getLocationURI(request, response, context); + String method = request.getRequestLine().getMethod(); + if (method.equalsIgnoreCase("HEAD")) { + return new HttpHead(uri); + } else if (method.equalsIgnoreCase("GET")) { + return new HttpGet(uri); + } else { + int status = response.getStatusLine().getStatusCode(); + return (HttpUriRequest) (status == 307 ? RequestBuilder.copy(request).setUri(uri).build() : new HttpGet(uri)); + } + } + }); + + try (final CloseableHttpClient httpclient = httpClientBuilder.build()) { + final HttpPut httpPut = new HttpPut(loadUrl); + final List cols = this.keys.getColumns(); + if (null != cols && !cols.isEmpty()) { + httpPut.setHeader("columns", String.join(",", cols)); + } + + // put loadProps to http header + final Map loadProps = this.keys.getLoadProps(); + if (null != loadProps) { + for (final Map.Entry entry : loadProps.entrySet()) { + httpPut.setHeader(entry.getKey(), String.valueOf(entry.getValue())); + } + } + + // set other required headers + httpPut.setHeader(HttpHeaders.EXPECT, "100-continue"); + httpPut.setHeader(HttpHeaders.AUTHORIZATION, this.getBasicAuthHeader(this.keys.getUsername(), this.keys.getPassword())); + httpPut.setHeader("label", flushBatch.getLabel()); + httpPut.setHeader("format", "json"); + httpPut.setHeader("line_delimiter", this.keys.getLineDelimiterDesc()); + httpPut.setHeader("read_json_by_line", "true"); + httpPut.setHeader("fuzzy_parse", "true"); + + // Use ByteArrayEntity instead of StringEntity to handle Chinese correctly + httpPut.setEntity(new ByteArrayEntity(flushBatch.getData().toString().getBytes())); + + httpPut.setConfig(requestConfig); + + try (final CloseableHttpResponse resp = httpclient.execute(httpPut)) { + final int code = resp.getStatusLine().getStatusCode(); + if (HttpStatus.SC_OK != code) { + LOG.warn("Request failed with code:{}", code); + return null; + } + final HttpEntity respEntity = resp.getEntity(); + if (null == respEntity) { + LOG.warn("Request failed with empty response."); + return null; + } + return (Map) JSON.parse(EntityUtils.toString(respEntity)); + } + } + } + + private String getBasicAuthHeader(final String username, final String password) { + final String auth = username + ":" + password; + final byte[] encodedAuth = Base64.getEncoder().encode(auth.getBytes()); + return "Basic " + new String(encodedAuth); + } + + // for test + public static void main(String[] args) throws IOException { + String json = "{\n" + + " \"feLoadUrl\": [\"127.0.0.1:8030\"],\n" + + " \"column\": [\"k1\", \"k2\", \"k3\"],\n" + + " \"database\": \"db1\",\n" + + " \"jdbcUrl\": \"jdbc:mysql://127.0.0.1:9030/\",\n" + + " \"loadProps\": {\n" + + " },\n" + + " \"password\": \"12345\",\n" + + " \"postSql\": [],\n" + + " \"preSql\": [],\n" + + " \"table\": \"t1\",\n" + + " \"username\": \"root\"\n" + + " }"; + Configuration configuration = Configuration.from(json); + Key key = new Key(configuration); + + DorisWriterEmitter emitter = new DorisWriterEmitter(key); + DorisFlushBatch flushBatch = new DorisFlushBatch("\n"); + flushBatch.setLabel("test4"); + Map row1 = Maps.newHashMap(); + row1.put("k1", "2021-02-02"); + row1.put("k2", "2021-02-02 00:00:00"); + row1.put("k3", "3"); + String rowStr1 = JSON.toJSONString(row1); + System.out.println("rows1: " + rowStr1); + flushBatch.putData(rowStr1); + + Map row2 = Maps.newHashMap(); + row2.put("k1", "2021-02-03"); + row2.put("k2", "2021-02-03 00:00:00"); + row2.put("k3", "4"); + String rowStr2 = JSON.toJSONString(row2); + System.out.println("rows2: " + rowStr2); + flushBatch.putData(rowStr2); + + for (int i = 0; i < 500000; ++i) { + flushBatch.putData(rowStr2); + } + + emitter.doStreamLoad(flushBatch); + } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java new file mode 100644 index 00000000..cfbef96c --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java @@ -0,0 +1,171 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; +import com.google.common.base.Strings; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +public class Key implements Serializable { + public static final String FE_LOAD_URL = "feLoadUrl"; + public static final String BE_LOAD_URL = "beLoadUrl"; + public static final String JDBC_URL = "jdbcUrl"; + + public static final String DATABASE = "database"; + public static final String TABLE = "table"; + public static final String COLUMN = "column"; + + public static final String USERNAME = "username"; + public static final String PASSWORD = "password"; + + public static final String PRE_SQL = "preSql"; + public static final String POST_SQL = "postSql"; + + public static final String LOAD_PROPS = "loadProps"; + public static final String MAX_BATCH_ROWS = "maxBatchRows"; + public static final String MAX_BATCH_BYTE_SIZE = "maxBatchByteSize"; + public static final String LABEL_PREFIX = "labelPrefix"; + public static final String LINE_DELIMITER = "lineDelimiter"; + public static final String CONNECT_TIMEOUT = "connectTimeout"; + private final Configuration options; + private final String lineDelimiterDesc; + + private static final long DEFAULT_MAX_BATCH_ROWS = 50_0000; + private static final long DEFAULT_MAX_BATCH_BYTE_SIZE = 100 * 1024 * 1024; // 100MB + private static final String DEFAULT_LABEL_PREFIX = "datax_doris_writer_"; + private static final String DEFAULT_LINE_DELIMITER = "\n"; + private static final int DEFAULT_CONNECT_TIMEOUT = -1; + + public Key(final Configuration options) { + this.options = options; + this.lineDelimiterDesc = parseHexReadable(this.getLineDelimiter()); + } + + public void doPretreatment() { + this.validateRequired(); + this.validateStreamLoadUrl(); + } + + public String getJdbcUrl() { + return this.options.getString(JDBC_URL); + } + + public String getDatabase() { + return this.options.getString(DATABASE); + } + + public String getTable() { + return this.options.getString(TABLE); + } + + public String getUsername() { + return this.options.getString(USERNAME); + } + + public String getPassword() { + return Strings.nullToEmpty(this.options.getString(PASSWORD)); + } + + public List getBeLoadUrlList() { + return this.options.getList(BE_LOAD_URL, String.class); + } + + public List getFeLoadUrlList() { + return this.options.getList(FE_LOAD_URL, String.class); + } + + public List getColumns() { + return this.options.getList(COLUMN, String.class); + } + + public List getPreSqlList() { + return this.options.getList(PRE_SQL, String.class); + } + + public List getPostSqlList() { + return this.options.getList(POST_SQL, String.class); + } + + public Map getLoadProps() { + return this.options.getMap(LOAD_PROPS); + } + + public long getBatchRows() { + return this.options.getLong(MAX_BATCH_ROWS, DEFAULT_MAX_BATCH_ROWS); + } + + public long getBatchByteSize() { + return this.options.getLong(MAX_BATCH_BYTE_SIZE, DEFAULT_MAX_BATCH_BYTE_SIZE); + } + + public String getLabelPrefix() { + return this.options.getString(LABEL_PREFIX, DEFAULT_LABEL_PREFIX); + } + + public String getLineDelimiter() { + return this.options.getString(LINE_DELIMITER, DEFAULT_LINE_DELIMITER); + } + + public int getConnectTimeout() { + return this.options.getInt(CONNECT_TIMEOUT, DEFAULT_CONNECT_TIMEOUT); + } + + public String getLineDelimiterDesc() { + return lineDelimiterDesc; + } + + private void validateStreamLoadUrl() { + List urlList = this.getBeLoadUrlList(); + if (urlList == null) { + urlList = this.getFeLoadUrlList(); + } + if (urlList == null || urlList.isEmpty()) { + throw DataXException.asDataXException(DBUtilErrorCode.CONF_ERROR, "Either beLoadUrl or feLoadUrl must be set"); + } + + for (final String host : urlList) { + if (host.split(":").length < 2) { + throw DataXException.asDataXException(DBUtilErrorCode.CONF_ERROR, + "Invalid load url format. IF use FE hosts, should be like: fe_host:fe_http_port." + + " If use BE hosts, should be like: be_host:be_webserver_port"); + } + } + } + + private String parseHexReadable(String s) { + byte[] separatorBytes = s.getBytes(); + StringBuilder desc = new StringBuilder(); + + for (byte separatorByte : separatorBytes) { + desc.append(String.format("\\x%02x", separatorByte)); + } + return desc.toString(); + } + + private void validateRequired() { + final String[] requiredOptionKeys = new String[]{JDBC_URL, USERNAME, DATABASE, TABLE, COLUMN}; + for (final String optionKey : requiredOptionKeys) { + this.options.getNecessaryValue(optionKey, DBUtilErrorCode.REQUIRED_VALUE); + } + } +} diff --git a/doriswriter/src/main/resources/plugin.json b/doriswriter/src/main/resources/plugin.json new file mode 100644 index 00000000..9d2ad497 --- /dev/null +++ b/doriswriter/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "doriswriter", + "class": "com.alibaba.datax.plugin.writer.doriswriter.DorisWriter", + "description": "", + "developer": "" +} diff --git a/doriswriter/src/main/resources/plugin_job_template.json b/doriswriter/src/main/resources/plugin_job_template.json new file mode 100644 index 00000000..152f1eee --- /dev/null +++ b/doriswriter/src/main/resources/plugin_job_template.json @@ -0,0 +1,15 @@ +{ + "name": "doriswriter", + "parameter": { + "username": "", + "password": "", + "database": "", + "table": "", + "column": [], + "preSql": [], + "postSql": [], + "jdbcUrl": "", + "beLoadUrl": [], + "loadProps": {} + } +} \ No newline at end of file diff --git a/pom.xml b/pom.xml index 3f64b929..f57306a4 100644 --- a/pom.xml +++ b/pom.xml @@ -108,6 +108,7 @@ hbase20xsqlreader hbase20xsqlwriter kuduwriter + doriswriter From 9e3e08987dceeae8d5b825404f2eb15d21bf3527 Mon Sep 17 00:00:00 2001 From: "jiafeng.zhang" Date: Tue, 25 Jan 2022 23:37:20 +0800 Subject: [PATCH 02/13] test case and Performance test report test case and Performance test report --- doriswriter/doc/doriswriter.md | 92 ++++++++++++++++--- .../doriswriter/TestDorisWriterLoad.java | 88 ++++++++++++++++++ 2 files changed, 166 insertions(+), 14 deletions(-) create mode 100644 doriswriter/src/test/java/com/alibaba/datax/plugin/writer/doriswriter/TestDorisWriterLoad.java diff --git a/doriswriter/doc/doriswriter.md b/doriswriter/doc/doriswriter.md index 7a072620..fedc2171 100644 --- a/doriswriter/doc/doriswriter.md +++ b/doriswriter/doc/doriswriter.md @@ -9,12 +9,60 @@ DorisWriter是一个支持将大批量数据写入Doris中的一个插件,可 Doris是完全支持Mysql协议的,所以如果你需要读取Doris的数据,可以使用mysqlreader插件完成,这里不单独提供Doris数据读取插件。 -## 2 实现原理 +## 2.支持版本 + +DorisWriter目前支持的Doris版本如下: + +| Doris版本号 | 说明 | +| -------------------- | ----------------------------------------------- | +| Apahce Doris 0.13.0 | 包括百度发行版palo 0.13.15 | +| Apache Doris 0.14.0 | 包括百度发行版palo 0.14.7、0.14.12.1、0.14.13.1 | +| Apache Doris 0.15.0 | 包括百度发行版palo 0.15.1 RC09 | +| Apahce Doris后续版本 | | + +大家在使用过程中如果遇到什么问题,可以通过邮件或者在Doris的[Issues · apache/incubator-doris](https://github.com/apache/incubator-doris/issues)上提问,我们会及时解决,或者可以给Doris的开发组邮箱发送邮件:dev@doris.apache.org,我们也会及时查看回复及修复。 + +## 3 实现原理 DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter会将`reader`读取的数据进行缓存在内存中,拼接成Json文本,然后批量导入至Doris。 -## 3 功能说明 +Stream load 是一个同步的导入方式,用户通过发送 HTTP 协议发送请求将本地文件或数据流导入到 Doris 中。Stream load 同步执行导入并返回导入结果。用户可直接通过请求的返回体判断本次导入是否成功。 -### 3.1 配置样例 +Stream load 主要适用于导入本地文件,或通过程序导入数据流中的数据 + +Stream Load的数据导入流程如下: + +```text + ^ + + | | + | | 1A. User submit load to FE + | | + | +--v-----------+ + | | FE | +5. Return result to user | +--+-----------+ + | | + | | 2. Redirect to BE + | | + | +--v-----------+ + +---+Coordinator BE| 1B. User submit load to BE + +-+-----+----+-+ + | | | + +-----+ | +-----+ + | | | 3. Distrbute data + | | | + +-v-+ +-v-+ +-v-+ + |BE | |BE | |BE | + +---+ +---+ +---+ +``` + +Stream load 中,Doris 会选定一个节点作为 Coordinator 节点。该节点负责接数据并分发数据到其他数据节点。 + +用户通过 HTTP 协议提交导入命令。如果提交到 FE,则 FE 会通过 HTTP redirect 指令将请求转发给某一个 BE。用户也可以直接提交导入命令给某一指定 BE。 + +导入的最终结果由 Coordinator BE 返回给用户。 + +## 4 功能说明 + +### 4.1 配置样例 这里是一份从Stream读取数据后导入至Doris的配置文件。 @@ -83,7 +131,7 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter } ``` -### 3.2 参数说明 +### 4.2 参数说明 * **jdbcUrl** @@ -147,37 +195,53 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter * **maxBatchRows** - - 描述:每批次导入数据的最大行数。和 **maxBatchByteSize** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 - 必选:否 - 默认值:500000 - * **maxBatchByteSize** - - 描述:每批次导入数据的最大数据量。和 ** maxBatchRows** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 - 必选:否 - 默认值:104857600 - * **labelPrefix** - - 描述:每批次导入任务的 label 前缀。最终的 label 将有 `labelPrefix + UUID + 序号` 组成 - 必选:否 - 默认值:`datax_doris_writer_` - * **lineDelimiter** - - 描述:每批次数据包含多行,每行为 Json 格式,每行的的分隔符即为 lineDelimiter。支持多个字节, 例如'\x02\x03'。 - 必选:否 - 默认值:`\n` - * **loadProps** - - 描述:StreamLoad 的请求参数,详情参照StreamLoad介绍页面。 - 必选:否 - 默认值:无 - * **connectTimeout** - 描述:StreamLoad单次请求的超时时间, 单位毫秒(ms)。 - 必选:否 - 默认值:-1 + +### 4.3 doriswriter插件的约束与限制 + +DorisWriter是借助于Apache Doris提供的Stream Load方式来实现数据导入,为了避免频繁的数据插入引发Doris写失败,建议批到的方式,具体参照[常见报错 | Apache Doris](https://doris.apache.org/zh-CN/faq/error.html#e3-tablet-writer-write-failed-tablet-id-27306172-txn-id-28573520-err-235-or-215-or-238),建议将参数列表中的下列参数设大,下面给出建议值: + +1. maxBatchRows:10000,表示每10000条提交导入一次,如果你的数据量没那么可以适当调小 +2. maxBatchByteSize:这个参数表示你每个批到导入数据量大的大小,具体值=maxBatchRows * 单条记录的大小,如果一个批次导入的数据量大小超过这个值将被阻塞导入,导入数据格式是JSON格式所以这个值可以适当放大,通过上面的导入记录数来控制每个批次导入的数据量就可以了 +3. column:这个要和你在Doris里建表的字段顺序一致。 + +## 4.性能测试 + +下面是通过读取Mysql数据表的数据,插入到Doris进行的性能测试结果,仅供参考 + +``` +2022-01-25 23:32:53.638 [job-0] INFO JobContainer - PerfTrace not enable! +2022-01-25 23:32:53.638 [job-0] INFO StandAloneJobContainerCommunicator - Total 2000000 records, 80888896 bytes | Speed 3.86MB/s, 100000 records/s | Error 0 records, 0 bytes | All Task WaitWriterTime 14.270s | All Task WaitReaderTime 0.147s | Percentage 100.00% +2022-01-25 23:32:53.639 [job-0] INFO JobContainer - +任务启动时刻 : 2022-01-25 23:32:33 +任务结束时刻 : 2022-01-25 23:32:53 +任务总计耗时 : 20s +任务平均流量 : 3.86MB/s +记录写入速度 : 100000rec/s +读出记录总数 : 2000000 +读写失败总数 : 0 +``` + diff --git a/doriswriter/src/test/java/com/alibaba/datax/plugin/writer/doriswriter/TestDorisWriterLoad.java b/doriswriter/src/test/java/com/alibaba/datax/plugin/writer/doriswriter/TestDorisWriterLoad.java new file mode 100644 index 00000000..35b6e3a4 --- /dev/null +++ b/doriswriter/src/test/java/com/alibaba/datax/plugin/writer/doriswriter/TestDorisWriterLoad.java @@ -0,0 +1,88 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.fastjson.JSON; +import com.google.common.collect.Maps; +import com.alibaba.datax.common.util.Configuration; + +import java.io.IOException; +import java.util.Map; + +public class TestDorisWriterLoad { + + + // for test + public static void main(String[] args) throws IOException { + /** + * 下面示例使用的建表语句,要首先有一套Ddoris的环境,创建数据库demo,然后使用下面的建表语句创建表 + * 修改feLoadUrl中的IP地址,username,password,然后运行 + * CREATE TABLE `doris_test` ( + * `k1` varchar(30) NULL , + * `k2` varchar(255) NULL, + * `k3` varchar(200) + * ) ENGINE=OLAP + * Duplicate KEY(k1) + * COMMENT "OLAP" + * DISTRIBUTED BY HASH(k1) BUCKETS 1 + * PROPERTIES ( + * "replication_allocation" = "tag.location.default: 1", + * "in_memory" = "false", + * "storage_format" = "V2" + * ) + */ + String json = "{\n" + + " \"feLoadUrl\": [\"127.0.0.1:8030\"],\n" + + " \"column\": [\"k1\", \"k2\", \"k3\"],\n" + + " \"database\": \"demo\",\n" + + " \"jdbcUrl\": \"\",\n" + + " \"loadProps\": {},\n" + + " \"password\": \"12345\",\n" + + " \"postSql\": [],\n" + + " \"preSql\": [],\n" + + " \"table\": \"doris_test\",\n" + + " \"username\": \"root\"\n" + + "}"; + Configuration configuration = Configuration.from(json); + Key key = new Key(configuration); + + DorisWriterEmitter emitter = new DorisWriterEmitter(key); + DorisFlushBatch flushBatch = new DorisFlushBatch("\n"); + flushBatch.setLabel("test4"); + Map row1 = Maps.newHashMap(); + row1.put("k1", "2021-02-02"); + row1.put("k2", "2021-02-02 00:00:00"); + row1.put("k3", "3"); + String rowStr1 = JSON.toJSONString(row1); + System.out.println("rows1: " + rowStr1); + flushBatch.putData(rowStr1); + + Map row2 = Maps.newHashMap(); + row2.put("k1", "2021-02-03"); + row2.put("k2", "2021-02-03 00:00:00"); + row2.put("k3", "4"); + String rowStr2 = JSON.toJSONString(row2); + System.out.println("rows2: " + rowStr2); + flushBatch.putData(rowStr2); + + for (int i = 0; i < 50000; ++i) { + flushBatch.putData(rowStr2); + } + emitter.doStreamLoad(flushBatch); + } +} From 79db76dd8f597d0eaa00b039cc9b4a39b116aede Mon Sep 17 00:00:00 2001 From: "jiafeng.zhang" Date: Tue, 25 Jan 2022 23:57:15 +0800 Subject: [PATCH 03/13] 1 1 --- doriswriter/doc/doriswriter.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/doriswriter/doc/doriswriter.md b/doriswriter/doc/doriswriter.md index fedc2171..a9e7101b 100644 --- a/doriswriter/doc/doriswriter.md +++ b/doriswriter/doc/doriswriter.md @@ -232,6 +232,8 @@ DorisWriter是借助于Apache Doris提供的Stream Load方式来实现数据导 下面是通过读取Mysql数据表的数据,插入到Doris进行的性能测试结果,仅供参考 +测试是单机测试,Mysql 8.0.26,Doris 0.15 (单机),mysql和Doris部署在同一台服务器上,服务器配置:4核 16 GiB + ``` 2022-01-25 23:32:53.638 [job-0] INFO JobContainer - PerfTrace not enable! 2022-01-25 23:32:53.638 [job-0] INFO StandAloneJobContainerCommunicator - Total 2000000 records, 80888896 bytes | Speed 3.86MB/s, 100000 records/s | Error 0 records, 0 bytes | All Task WaitWriterTime 14.270s | All Task WaitReaderTime 0.147s | Percentage 100.00% From 6c41fe832b79675171a7fefe4777c3a4887e1b39 Mon Sep 17 00:00:00 2001 From: "jiafeng.zhang" Date: Thu, 29 Sep 2022 09:25:25 +0800 Subject: [PATCH 04/13] support csv format import support csv format import --- doriswriter/src/main/assembly/package.xml | 12 +- .../plugin/writer/doriswriter/DorisCodec.java | 11 +- .../writer/doriswriter/DorisCsvCodec.java | 49 +++++++ .../writer/doriswriter/DorisFlushBatch.java | 36 ++++-- .../writer/doriswriter/DorisJsonCodec.java | 4 +- .../writer/doriswriter/DorisWriter.java | 20 +-- .../doriswriter/DorisWriterEmitter.java | 120 ++++++++++-------- .../writer/doriswriter/EscapeHandler.java | 42 ++++++ .../datax/plugin/writer/doriswriter/Key.java | 56 +++++--- .../main/resources/plugin_job_template.json | 1 + 10 files changed, 245 insertions(+), 106 deletions(-) create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/EscapeHandler.java diff --git a/doriswriter/src/main/assembly/package.xml b/doriswriter/src/main/assembly/package.xml index 9fca5e93..71596332 100644 --- a/doriswriter/src/main/assembly/package.xml +++ b/doriswriter/src/main/assembly/package.xml @@ -1,5 +1,4 @@ - - - - + + dir @@ -45,7 +42,6 @@ under the License. plugin/writer/doriswriter - false diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java index 9a364f40..51bc6881 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java @@ -22,16 +22,17 @@ import com.alibaba.datax.common.element.DateColumn; import com.alibaba.datax.common.element.Record; import org.apache.commons.lang3.time.DateFormatUtils; +import java.time.ZoneId; import java.util.List; import java.util.TimeZone; public abstract class DorisCodec { - protected static String timeZone = "GMT+8"; - protected static TimeZone timeZoner = TimeZone.getTimeZone(timeZone); + protected final TimeZone timeZone; protected final List fieldNames; - public DorisCodec(final List fieldNames) { + public DorisCodec(final List fieldNames, final String timeZone) { this.fieldNames = fieldNames; + this.timeZone = TimeZone.getTimeZone(ZoneId.of(timeZone)); } public abstract String serialize(Record row); @@ -60,9 +61,9 @@ public abstract class DorisCodec { final DateColumn.DateType dateType = ((DateColumn) col).getSubType(); switch (dateType) { case DATE: - return DateFormatUtils.format(col.asDate(), "yyyy-MM-dd", timeZoner); + return DateFormatUtils.format(col.asDate(), "yyyy-MM-dd", timeZone); case DATETIME: - return DateFormatUtils.format(col.asDate(), "yyyy-MM-dd HH:mm:ss", timeZoner); + return DateFormatUtils.format(col.asDate(), "yyyy-MM-dd HH:mm:ss", timeZone); default: return col.asString(); } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java new file mode 100644 index 00000000..d433af38 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.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 com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.element.Record; + +import java.util.ArrayList; +import java.util.List; + +public class DorisCsvCodec extends DorisCodec { + + private final String columnSeparator; + + public DorisCsvCodec(final List fieldNames, String columnSeparator, String timeZone) { + super(fieldNames, timeZone); + this.columnSeparator = EscapeHandler.escapeString(columnSeparator); + } + + @Override + public String serialize(final Record row) { + if (null == this.fieldNames) { + return ""; + } + List list = new ArrayList<>(); + + for (int i = 0; i < this.fieldNames.size(); i++) { + Object value = this.convertColumn(row.getColumn(i)); + list.add(value != null ? value.toString() : "\\N"); + } + + return String.join(columnSeparator, list); + } + +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisFlushBatch.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisFlushBatch.java index 493ddec9..9980c937 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisFlushBatch.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisFlushBatch.java @@ -17,15 +17,20 @@ package com.alibaba.datax.plugin.writer.doriswriter; +import java.util.ArrayList; +import java.util.List; + // A wrapper class to hold a batch of loaded rows public class DorisFlushBatch { - private String lineDelimiter; + private final String format; + private final String lineDelimiter; private String label; - private long rows = 0; - private StringBuilder data = new StringBuilder(); + private long byteSize = 0; + private List data = new ArrayList<>(); - public DorisFlushBatch(String lineDelimiter) { - this.lineDelimiter = lineDelimiter; + public DorisFlushBatch(String lineDelimiter, String format) { + this.lineDelimiter = EscapeHandler.escapeString(lineDelimiter); + this.format = format; } public void setLabel(String label) { @@ -37,22 +42,25 @@ public class DorisFlushBatch { } public long getRows() { - return rows; + return data.size(); } public void putData(String row) { - if (data.length() > 0) { - data.append(lineDelimiter); - } - data.append(row); - rows++; + data.add(row); + byteSize += row.getBytes().length; } - public StringBuilder getData() { - return data; + public String getData() { + String result; + if (Key.DEFAULT_FORMAT_CSV.equalsIgnoreCase(format)) { + result = String.join(this.lineDelimiter, data); + } else { + result = data.toString(); + } + return result; } public long getSize() { - return data.length(); + return byteSize; } } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java index 8d4568c2..664b9d36 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java @@ -28,8 +28,8 @@ import java.util.Map; public class DorisJsonCodec extends DorisCodec { private Map rowMap; - public DorisJsonCodec(final List fieldNames) { - super(fieldNames); + public DorisJsonCodec(final List fieldNames, final String timeZone) { + super(fieldNames, timeZone); this.rowMap = new HashMap<>(this.fieldNames.size()); } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java index 71e2d1ad..a2bdfd7e 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java @@ -48,6 +48,7 @@ public class DorisWriter extends Writer { private Key keys; private DorisCodec rowCodec; private int batchNum = 0; + private String labelPrefix; public Task() { } @@ -55,7 +56,12 @@ public class DorisWriter extends Writer { @Override public void init() { this.keys = new Key(super.getPluginJobConf()); - this.rowCodec = new DorisJsonCodec(this.keys.getColumns()); + if (Key.DEFAULT_FORMAT_CSV.equalsIgnoreCase(this.keys.getFormat())) { + this.rowCodec = new DorisCsvCodec(this.keys.getColumns(), this.keys.getColumnSeparator(), this.keys.getTimeZone()); + } else { + this.rowCodec = new DorisJsonCodec(this.keys.getColumns(), this.keys.getTimeZone()); + } + this.labelPrefix = this.keys.getLabelPrefix() + UUID.randomUUID(); this.dorisWriterEmitter = new DorisWriterEmitter(keys); } @@ -66,7 +72,7 @@ public class DorisWriter extends Writer { @Override public void startWrite(RecordReceiver recordReceiver) { String lineDelimiter = this.keys.getLineDelimiter(); - DorisFlushBatch flushBatch = new DorisFlushBatch(lineDelimiter); + DorisFlushBatch flushBatch = new DorisFlushBatch(lineDelimiter, this.keys.getFormat()); long batchCount = 0; long batchByteSize = 0L; Record record; @@ -93,7 +99,7 @@ public class DorisWriter extends Writer { // clear buffer batchCount = 0; batchByteSize = 0L; - flushBatch = new DorisFlushBatch(lineDelimiter); + flushBatch = new DorisFlushBatch(lineDelimiter, this.keys.getFormat()); } } // end of while @@ -103,14 +109,12 @@ public class DorisWriter extends Writer { } private void flush(DorisFlushBatch flushBatch) { - final String label = getStreamLoadLabel(); - flushBatch.setLabel(label); - dorisWriterEmitter.doStreamLoad(flushBatch); + flushBatch.setLabel(getStreamLoadLabel()); + dorisWriterEmitter.emit(flushBatch); } private String getStreamLoadLabel() { - String labelPrefix = this.keys.getLabelPrefix(); - return labelPrefix + UUID.randomUUID().toString() + "_" + (batchNum++); + return labelPrefix + "_" + (batchNum++); } @Override diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java index fb4e46b0..ccd9fc4d 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java @@ -23,6 +23,7 @@ import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; import com.alibaba.fastjson.JSON; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import org.apache.commons.lang3.StringUtils; import org.apache.http.HttpEntity; import org.apache.http.HttpHeaders; import org.apache.http.HttpRequest; @@ -47,12 +48,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.net.HttpURLConnection; import java.net.URI; -import java.net.URL; import java.util.Base64; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; // Used to load batch of rows to Doris using stream load public class DorisWriterEmitter { @@ -88,13 +88,28 @@ public class DorisWriterEmitter { } } + public void emit(final DorisFlushBatch flushData) { + String host = this.getAvailableHost(); + for (int i = 0; i <= this.keys.getMaxRetries(); i++) { + try { + doStreamLoad(flushData, host); + return; + } catch (DataXException ex) { + if (i >= this.keys.getMaxRetries()) { + throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, ex); + } + LOG.error("StreamLoad error, switch host {} and retry: ", host, ex); + host = this.getAvailableHost(); + } + } + } + /** * execute doris stream load */ - public void doStreamLoad(final DorisFlushBatch flushData) { + private void doStreamLoad(final DorisFlushBatch flushData, String host) { long start = System.currentTimeMillis(); - final String host = this.getAvailableHost(); - if (null == host) { + if (StringUtils.isEmpty(host)) { throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, "None of the load url can be connected."); } final String loadUrl = host + "/api/" + this.keys.getDatabase() + "/" + this.keys.getTable() + "/_stream_load"; @@ -130,28 +145,12 @@ public class DorisWriterEmitter { while (this.hostPos < targetHosts.size()) { final String host = targetHosts.get(hostPos); ++this.hostPos; - if (this.tryHttpConnection(host)) { - return host; - } + return host; } return null; } - private boolean tryHttpConnection(final String host) { - try { - final URL url = new URL(host); - final HttpURLConnection co = (HttpURLConnection) url.openConnection(); - co.setConnectTimeout(1000); - co.connect(); - co.disconnect(); - return true; - } catch (Exception e) { - LOG.warn("Failed to connect to address:{} , Exception ={}", host, e); - return false; - } - } - private Map doHttpPut(final String loadUrl, final DorisFlushBatch flushBatch) throws IOException { LOG.info(String.format("Executing stream load to: '%s', size: %s, rows: %d", loadUrl, flushBatch.getSize(), flushBatch.getRows())); @@ -181,10 +180,12 @@ public class DorisWriterEmitter { final HttpPut httpPut = new HttpPut(loadUrl); final List cols = this.keys.getColumns(); if (null != cols && !cols.isEmpty()) { - httpPut.setHeader("columns", String.join(",", cols)); + httpPut.setHeader("columns", String.join(",", cols.stream().map(item -> String.format("`%s`", item.trim().replace("`", ""))).collect(Collectors.toList()))); } - // put loadProps to http header + //set default header + setDefaultHeader(httpPut); + // put custom loadProps to http header final Map loadProps = this.keys.getLoadProps(); if (null != loadProps) { for (final Map.Entry entry : loadProps.entrySet()) { @@ -196,14 +197,9 @@ public class DorisWriterEmitter { httpPut.setHeader(HttpHeaders.EXPECT, "100-continue"); httpPut.setHeader(HttpHeaders.AUTHORIZATION, this.getBasicAuthHeader(this.keys.getUsername(), this.keys.getPassword())); httpPut.setHeader("label", flushBatch.getLabel()); - httpPut.setHeader("format", "json"); - httpPut.setHeader("line_delimiter", this.keys.getLineDelimiterDesc()); - httpPut.setHeader("read_json_by_line", "true"); - httpPut.setHeader("fuzzy_parse", "true"); // Use ByteArrayEntity instead of StringEntity to handle Chinese correctly - httpPut.setEntity(new ByteArrayEntity(flushBatch.getData().toString().getBytes())); - + httpPut.setEntity(new ByteArrayEntity(flushBatch.getData().getBytes())); httpPut.setConfig(requestConfig); try (final CloseableHttpResponse resp = httpclient.execute(httpPut)) { @@ -222,6 +218,21 @@ public class DorisWriterEmitter { } } + /** + * Set default request headers in json and csv formats. + * csv default delimiters are \x01 and \x02 + */ + private void setDefaultHeader(HttpPut httpPut) { + if (Key.DEFAULT_FORMAT_CSV.equalsIgnoreCase(this.keys.getFormat())) { + httpPut.setHeader("line_delimiter", this.keys.getLineDelimiter()); + httpPut.setHeader("column_separator", this.keys.getColumnSeparator()); + } else { + httpPut.setHeader("format", "json"); + httpPut.setHeader("strip_outer_array", "true"); + httpPut.setHeader("fuzzy_parse", "true"); + } + } + private String getBasicAuthHeader(final String username, final String password) { final String auth = username + ":" + password; final byte[] encodedAuth = Base64.getEncoder().encode(auth.getBytes()); @@ -231,44 +242,51 @@ public class DorisWriterEmitter { // for test public static void main(String[] args) throws IOException { String json = "{\n" + - " \"feLoadUrl\": [\"127.0.0.1:8030\"],\n" + - " \"column\": [\"k1\", \"k2\", \"k3\"],\n" + - " \"database\": \"db1\",\n" + + " \"beLoadUrl\": [\"127.0.0.1:8040\"],\n" + + " \"column\": [\"name\", \"age\", \"cdate\", \"cdatetime\"],\n" + + " \"database\": \"test\",\n" + " \"jdbcUrl\": \"jdbc:mysql://127.0.0.1:9030/\",\n" + " \"loadProps\": {\n" + +// " \"line_delimiter\": \"\\\\x03\",\n" + +// " \"column_separator\": \"\\\\x04\",\n" + " },\n" + - " \"password\": \"12345\",\n" + + " \"format\": \"csv\",\n" + + " \"password\": \"\",\n" + " \"postSql\": [],\n" + " \"preSql\": [],\n" + - " \"table\": \"t1\",\n" + + " \"table\": \"test_datax\",\n" + + " \"maxRetries\": \"0\",\n" + " \"username\": \"root\"\n" + " }"; Configuration configuration = Configuration.from(json); Key key = new Key(configuration); DorisWriterEmitter emitter = new DorisWriterEmitter(key); - DorisFlushBatch flushBatch = new DorisFlushBatch("\n"); - flushBatch.setLabel("test4"); + DorisFlushBatch flushBatch = new DorisFlushBatch(key.getLineDelimiter(), key.getFormat()); + Map row1 = Maps.newHashMap(); - row1.put("k1", "2021-02-02"); - row1.put("k2", "2021-02-02 00:00:00"); - row1.put("k3", "3"); - String rowStr1 = JSON.toJSONString(row1); - System.out.println("rows1: " + rowStr1); - flushBatch.putData(rowStr1); - + row1.put("cdate", "2021-02-02"); + row1.put("cdatetime", "2021-02-02 00:00:00"); + row1.put("name", "zhangsan"); + row1.put("age", "18"); Map row2 = Maps.newHashMap(); - row2.put("k1", "2021-02-03"); - row2.put("k2", "2021-02-03 00:00:00"); - row2.put("k3", "4"); + row2.put("cdate", "2022-02-02"); + row2.put("cdatetime", "2022-02-02 10:00:00"); + row2.put("name", "lisi"); + row2.put("age", "180"); + String rowStr1 = JSON.toJSONString(row1); String rowStr2 = JSON.toJSONString(row2); + if ("csv".equals(key.getFormat())) { + rowStr1 = String.join(EscapeHandler.escapeString(key.getColumnSeparator()), "2021-02-02", "2021-02-02 00:00:00", "zhangsan", "18"); + rowStr2 = String.join(EscapeHandler.escapeString(key.getColumnSeparator()), "2022-02-02", "2022-02-02 10:00:00", "lisi", "180"); + } + System.out.println("rows1: " + rowStr1); System.out.println("rows2: " + rowStr2); - flushBatch.putData(rowStr2); - for (int i = 0; i < 500000; ++i) { + for (int i = 0; i < 1; ++i) { + flushBatch.putData(rowStr1); flushBatch.putData(rowStr2); } - - emitter.doStreamLoad(flushBatch); + emitter.emit(flushBatch); } } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/EscapeHandler.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/EscapeHandler.java new file mode 100644 index 00000000..91b0fbdb --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/EscapeHandler.java @@ -0,0 +1,42 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package com.alibaba.datax.plugin.writer.doriswriter; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Handler for escape in properties. + */ +public class EscapeHandler { + public static final String ESCAPE_DELIMITERS_FLAGS = "\\x"; + public static final Pattern ESCAPE_PATTERN = Pattern.compile("\\\\x([0-9|a-f|A-F]{2})"); + + public static String escapeString(String source) { + if (source.startsWith(ESCAPE_DELIMITERS_FLAGS)) { + Matcher m = ESCAPE_PATTERN.matcher(source); + StringBuffer buf = new StringBuffer(); + while (m.find()) { + m.appendReplacement(buf, String.format("%s", (char) Integer.parseInt(m.group(1), 16))); + } + m.appendTail(buf); + return buf.toString(); + } + return source; + } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java index cfbef96c..72812a9c 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java @@ -23,6 +23,8 @@ import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; import com.google.common.base.Strings; import java.io.Serializable; +import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -34,6 +36,7 @@ public class Key implements Serializable { public static final String DATABASE = "database"; public static final String TABLE = "table"; public static final String COLUMN = "column"; + public static final String TIME_ZONE = "timeZone"; public static final String USERNAME = "username"; public static final String PASSWORD = "password"; @@ -42,28 +45,36 @@ public class Key implements Serializable { public static final String POST_SQL = "postSql"; public static final String LOAD_PROPS = "loadProps"; + public static final String LOAD_PROPS_LINE_DELIMITER = "line_delimiter"; + public static final String LOAD_PROPS_COLUMN_SEPARATOR = "column_separator"; + public static final String MAX_BATCH_ROWS = "maxBatchRows"; public static final String MAX_BATCH_BYTE_SIZE = "maxBatchByteSize"; + public static final String MAX_RETRIES = "maxRetries"; public static final String LABEL_PREFIX = "labelPrefix"; - public static final String LINE_DELIMITER = "lineDelimiter"; + public static final String FORMAT = "format"; public static final String CONNECT_TIMEOUT = "connectTimeout"; private final Configuration options; - private final String lineDelimiterDesc; private static final long DEFAULT_MAX_BATCH_ROWS = 50_0000; - private static final long DEFAULT_MAX_BATCH_BYTE_SIZE = 100 * 1024 * 1024; // 100MB + private static final long DEFAULT_MAX_BATCH_BYTE_SIZE = 90 * 1024 * 1024; // 90MB + private static final int DEFAULT_MAX_RETRIES = 0; + private static final String DEFAULT_LABEL_PREFIX = "datax_doris_writer_"; - private static final String DEFAULT_LINE_DELIMITER = "\n"; + private static final String DEFAULT_COLUMN_SEPARATOR = "\\x01"; + private static final String DEFAULT_LINE_DELIMITER = "\\x02"; + public static final String DEFAULT_FORMAT_CSV = "csv"; + private static final String DEFAULT_TIME_ZONE = "+08:00"; private static final int DEFAULT_CONNECT_TIMEOUT = -1; public Key(final Configuration options) { this.options = options; - this.lineDelimiterDesc = parseHexReadable(this.getLineDelimiter()); } public void doPretreatment() { this.validateRequired(); this.validateStreamLoadUrl(); + this.validateFormat(); } public String getJdbcUrl() { @@ -98,6 +109,10 @@ public class Key implements Serializable { return this.options.getList(COLUMN, String.class); } + public String getTimeZone() { + return this.options.getString(TIME_ZONE, DEFAULT_TIME_ZONE); + } + public List getPreSqlList() { return this.options.getList(PRE_SQL, String.class); } @@ -107,7 +122,7 @@ public class Key implements Serializable { } public Map getLoadProps() { - return this.options.getMap(LOAD_PROPS); + return this.options.getMap(LOAD_PROPS, new HashMap<>()); } public long getBatchRows() { @@ -118,22 +133,30 @@ public class Key implements Serializable { return this.options.getLong(MAX_BATCH_BYTE_SIZE, DEFAULT_MAX_BATCH_BYTE_SIZE); } + public int getMaxRetries() { + return this.options.getInt(MAX_RETRIES, DEFAULT_MAX_RETRIES); + } + public String getLabelPrefix() { return this.options.getString(LABEL_PREFIX, DEFAULT_LABEL_PREFIX); } public String getLineDelimiter() { - return this.options.getString(LINE_DELIMITER, DEFAULT_LINE_DELIMITER); + return getLoadProps().getOrDefault(LOAD_PROPS_LINE_DELIMITER, DEFAULT_LINE_DELIMITER).toString(); + } + + public String getFormat() { + return this.options.getString(FORMAT, DEFAULT_FORMAT_CSV); + } + + public String getColumnSeparator() { + return getLoadProps().getOrDefault(LOAD_PROPS_COLUMN_SEPARATOR, DEFAULT_COLUMN_SEPARATOR).toString(); } public int getConnectTimeout() { return this.options.getInt(CONNECT_TIMEOUT, DEFAULT_CONNECT_TIMEOUT); } - public String getLineDelimiterDesc() { - return lineDelimiterDesc; - } - private void validateStreamLoadUrl() { List urlList = this.getBeLoadUrlList(); if (urlList == null) { @@ -152,14 +175,11 @@ public class Key implements Serializable { } } - private String parseHexReadable(String s) { - byte[] separatorBytes = s.getBytes(); - StringBuilder desc = new StringBuilder(); - - for (byte separatorByte : separatorBytes) { - desc.append(String.format("\\x%02x", separatorByte)); + private void validateFormat() { + String format = this.getFormat(); + if (!Arrays.asList("csv", "json").contains(format.toLowerCase())) { + throw DataXException.asDataXException(DBUtilErrorCode.CONF_ERROR, "format only supports csv or json"); } - return desc.toString(); } private void validateRequired() { diff --git a/doriswriter/src/main/resources/plugin_job_template.json b/doriswriter/src/main/resources/plugin_job_template.json index 152f1eee..9cd9bb18 100644 --- a/doriswriter/src/main/resources/plugin_job_template.json +++ b/doriswriter/src/main/resources/plugin_job_template.json @@ -6,6 +6,7 @@ "database": "", "table": "", "column": [], + "timeZone": "", "preSql": [], "postSql": [], "jdbcUrl": "", From 78ac25cb080520b562f360593f8bd4483bca8ac4 Mon Sep 17 00:00:00 2001 From: "jiafeng.zhang" Date: Thu, 29 Sep 2022 09:58:38 +0800 Subject: [PATCH 05/13] fix --- .../alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java index 664b9d36..2a68d050 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java @@ -45,6 +45,6 @@ public class DorisJsonCodec extends DorisCodec { rowMap.put(fieldName, this.convertColumn(row.getColumn(idx))); ++idx; } - return JSON.toJSONString(rowMap); + return JSON.toJSONString(rowMap, SerializerFeature.WriteMapNullValue); } } From 15cca65722a451cec75d92bc914e99c83b28acf9 Mon Sep 17 00:00:00 2001 From: "jiafeng.zhang" Date: Fri, 30 Sep 2022 13:11:57 +0800 Subject: [PATCH 06/13] fix --- doriswriter/doc/doriswriter.md | 130 +++++------------- doriswriter/doc/mysql2doris.json | 4 +- doriswriter/pom.xml | 10 +- .../plugin/writer/doriswriter/DorisCodec.java | 9 +- .../writer/doriswriter/DorisCsvCodec.java | 4 +- .../writer/doriswriter/DorisJsonCodec.java | 5 +- .../writer/doriswriter/DorisWriter.java | 4 +- .../doriswriter/DorisWriterEmitter.java | 53 +------ .../datax/plugin/writer/doriswriter/Key.java | 6 +- package.xml | 7 + 10 files changed, 57 insertions(+), 175 deletions(-) diff --git a/doriswriter/doc/doriswriter.md b/doriswriter/doc/doriswriter.md index a9e7101b..3bcbf9ff 100644 --- a/doriswriter/doc/doriswriter.md +++ b/doriswriter/doc/doriswriter.md @@ -1,68 +1,14 @@ # DorisWriter 插件文档 ## 1 快速介绍 -Apache Doris是一个现代化的MPP分析型数据库产品。仅需亚秒级响应时间即可获得查询结果,有效地支持实时数据分析。Apache Doris的分布式架构非常简洁,易于运维,并且可以支持10PB以上的超大数据集。 +DorisWriter支持将大批量数据写入Doris中。 -Apache Doris可以满足多种数据分析需求,例如固定历史报表,实时数据分析,交互式数据分析和探索式数据分析等。令您的数据分析工作更加简单高效! - -DorisWriter是一个支持将大批量数据写入Doris中的一个插件,可以结合Datax其他读取数据的插件结合使用完成数据的整合。 - -Doris是完全支持Mysql协议的,所以如果你需要读取Doris的数据,可以使用mysqlreader插件完成,这里不单独提供Doris数据读取插件。 - -## 2.支持版本 - -DorisWriter目前支持的Doris版本如下: - -| Doris版本号 | 说明 | -| -------------------- | ----------------------------------------------- | -| Apahce Doris 0.13.0 | 包括百度发行版palo 0.13.15 | -| Apache Doris 0.14.0 | 包括百度发行版palo 0.14.7、0.14.12.1、0.14.13.1 | -| Apache Doris 0.15.0 | 包括百度发行版palo 0.15.1 RC09 | -| Apahce Doris后续版本 | | - -大家在使用过程中如果遇到什么问题,可以通过邮件或者在Doris的[Issues · apache/incubator-doris](https://github.com/apache/incubator-doris/issues)上提问,我们会及时解决,或者可以给Doris的开发组邮箱发送邮件:dev@doris.apache.org,我们也会及时查看回复及修复。 - -## 3 实现原理 +## 2 实现原理 DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter会将`reader`读取的数据进行缓存在内存中,拼接成Json文本,然后批量导入至Doris。 -Stream load 是一个同步的导入方式,用户通过发送 HTTP 协议发送请求将本地文件或数据流导入到 Doris 中。Stream load 同步执行导入并返回导入结果。用户可直接通过请求的返回体判断本次导入是否成功。 +## 3 功能说明 -Stream load 主要适用于导入本地文件,或通过程序导入数据流中的数据 - -Stream Load的数据导入流程如下: - -```text - ^ + - | | - | | 1A. User submit load to FE - | | - | +--v-----------+ - | | FE | -5. Return result to user | +--+-----------+ - | | - | | 2. Redirect to BE - | | - | +--v-----------+ - +---+Coordinator BE| 1B. User submit load to BE - +-+-----+----+-+ - | | | - +-----+ | +-----+ - | | | 3. Distrbute data - | | | - +-v-+ +-v-+ +-v-+ - |BE | |BE | |BE | - +---+ +---+ +---+ -``` - -Stream load 中,Doris 会选定一个节点作为 Coordinator 节点。该节点负责接数据并分发数据到其他数据节点。 - -用户通过 HTTP 协议提交导入命令。如果提交到 FE,则 FE 会通过 HTTP redirect 指令将请求转发给某一个 BE。用户也可以直接提交导入命令给某一指定 BE。 - -导入的最终结果由 Coordinator BE 返回给用户。 - -## 4 功能说明 - -### 4.1 配置样例 +### 3.1 配置样例 这里是一份从Stream读取数据后导入至Doris的配置文件。 @@ -122,7 +68,7 @@ Stream load 中,Doris 会选定一个节点作为 Coordinator 节点。该节 "maxBatchRows" : 500000, "maxBatchByteSize" : 104857600, "labelPrefix": "my_prefix", - "lineDelimiter": "\n" + "format":"csv" } } } @@ -131,7 +77,7 @@ Stream load 中,Doris 会选定一个节点作为 Coordinator 节点。该节 } ``` -### 4.2 参数说明 +### 3.2 参数说明 * **jdbcUrl** @@ -142,7 +88,7 @@ Stream load 中,Doris 会选定一个节点作为 Coordinator 节点。该节 * **feLoadUrl** - 描述:和 **beLoadUrl** 二选一。作为 Stream Load 的连接目标。格式为 "ip:port"。其中 IP 是 FE 节点 IP,port 是 FE 节点的 http_port。可以填写多个,doriswriter 将以轮询的方式访问。 - - 必选:否 + - 必选:是 - 默认值:无 * **beLoadUrl** @@ -158,7 +104,7 @@ Stream load 中,Doris 会选定一个节点作为 Coordinator 节点。该节 - 默认值:无 * **password** - + - 描述:访问Doris数据库的密码 - 必选:否 - 默认值:空 @@ -170,16 +116,16 @@ Stream load 中,Doris 会选定一个节点作为 Coordinator 节点。该节 - 默认值:无 * **table** - + - 描述:需要写入的Doris表名称。 - 必选:是 - 默认值:无 * **column** - - 描述:目的表**需要写入数据**的字段,这些字段将作为生成的 Json 数据的字段名。字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 - - 必选:是 - - 默认值:否 + - 描述:目的表**需要写入数据**的字段,这些字段将作为生成的 Json 数据的字段名。字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 + - 必选:是 + - 默认值:否 * **preSql** @@ -195,55 +141,43 @@ Stream load 中,Doris 会选定一个节点作为 Coordinator 节点。该节 * **maxBatchRows** + - 描述:每批次导入数据的最大行数。和 **maxBatchByteSize** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 - 必选:否 - 默认值:500000 -* **maxBatchByteSize** + +* **batchByteSize** + - 描述:每批次导入数据的最大数据量。和 ** maxBatchRows** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 - 必选:否 - 默认值:104857600 + +* **maxRetries** + + - 描述:每批次导入数据失败后的重试次数。 + - 必选:否 + - 默认值:0 + * **labelPrefix** + - 描述:每批次导入任务的 label 前缀。最终的 label 将有 `labelPrefix + UUID + 序号` 组成 - 必选:否 - 默认值:`datax_doris_writer_` -* **lineDelimiter** - - 描述:每批次数据包含多行,每行为 Json 格式,每行的的分隔符即为 lineDelimiter。支持多个字节, 例如'\x02\x03'。 + +* **format** + + - 描述:StreamLoad数据的组装格式,支持csv和json格式。csv默认的行分隔符是\x01,列分隔符是\x02。 - 必选:否 - - 默认值:`\n` + - 默认值:csv + * **loadProps** + - 描述:StreamLoad 的请求参数,详情参照StreamLoad介绍页面。 - 必选:否 - 默认值:无 + * **connectTimeout** - 描述:StreamLoad单次请求的超时时间, 单位毫秒(ms)。 - 必选:否 - 默认值:-1 - -### 4.3 doriswriter插件的约束与限制 - -DorisWriter是借助于Apache Doris提供的Stream Load方式来实现数据导入,为了避免频繁的数据插入引发Doris写失败,建议批到的方式,具体参照[常见报错 | Apache Doris](https://doris.apache.org/zh-CN/faq/error.html#e3-tablet-writer-write-failed-tablet-id-27306172-txn-id-28573520-err-235-or-215-or-238),建议将参数列表中的下列参数设大,下面给出建议值: - -1. maxBatchRows:10000,表示每10000条提交导入一次,如果你的数据量没那么可以适当调小 -2. maxBatchByteSize:这个参数表示你每个批到导入数据量大的大小,具体值=maxBatchRows * 单条记录的大小,如果一个批次导入的数据量大小超过这个值将被阻塞导入,导入数据格式是JSON格式所以这个值可以适当放大,通过上面的导入记录数来控制每个批次导入的数据量就可以了 -3. column:这个要和你在Doris里建表的字段顺序一致。 - -## 4.性能测试 - -下面是通过读取Mysql数据表的数据,插入到Doris进行的性能测试结果,仅供参考 - -测试是单机测试,Mysql 8.0.26,Doris 0.15 (单机),mysql和Doris部署在同一台服务器上,服务器配置:4核 16 GiB - -``` -2022-01-25 23:32:53.638 [job-0] INFO JobContainer - PerfTrace not enable! -2022-01-25 23:32:53.638 [job-0] INFO StandAloneJobContainerCommunicator - Total 2000000 records, 80888896 bytes | Speed 3.86MB/s, 100000 records/s | Error 0 records, 0 bytes | All Task WaitWriterTime 14.270s | All Task WaitReaderTime 0.147s | Percentage 100.00% -2022-01-25 23:32:53.639 [job-0] INFO JobContainer - -任务启动时刻 : 2022-01-25 23:32:33 -任务结束时刻 : 2022-01-25 23:32:53 -任务总计耗时 : 20s -任务平均流量 : 3.86MB/s -记录写入速度 : 100000rec/s -读出记录总数 : 2000000 -读写失败总数 : 0 -``` - diff --git a/doriswriter/doc/mysql2doris.json b/doriswriter/doc/mysql2doris.json index f6e9d9bd..b8be23c9 100644 --- a/doriswriter/doc/mysql2doris.json +++ b/doriswriter/doc/mysql2doris.json @@ -13,7 +13,7 @@ } ], "username": "root", - "password": "123456", + "password": "", "where": "" } }, @@ -28,7 +28,7 @@ "table": "t3", "column": ["k1", "k2", "k3"], "username": "root", - "password": "12345", + "password": "", "postSql": [], "preSql": [] } diff --git a/doriswriter/pom.xml b/doriswriter/pom.xml index 1051436a..aa1e6ff0 100644 --- a/doriswriter/pom.xml +++ b/doriswriter/pom.xml @@ -17,21 +17,17 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - datax-all com.alibaba.datax 0.0.1-SNAPSHOT - ../pom.xml - 4.0.0 doriswriter doriswriter jar - com.alibaba.datax @@ -52,19 +48,16 @@ under the License. ch.qos.logback logback-classic - com.alibaba.datax plugin-rdbms-util ${datax-project-version} - mysql mysql-connector-java ${mysql.driver.version} - org.apache.httpcomponents httpclient @@ -82,7 +75,6 @@ under the License. ${project-sourceEncoding} - maven-assembly-plugin diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java index 51bc6881..ef5889de 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java @@ -24,15 +24,12 @@ import org.apache.commons.lang3.time.DateFormatUtils; import java.time.ZoneId; import java.util.List; -import java.util.TimeZone; public abstract class DorisCodec { - protected final TimeZone timeZone; protected final List fieldNames; - public DorisCodec(final List fieldNames, final String timeZone) { + public DorisCodec(final List fieldNames) { this.fieldNames = fieldNames; - this.timeZone = TimeZone.getTimeZone(ZoneId.of(timeZone)); } public abstract String serialize(Record row); @@ -61,9 +58,9 @@ public abstract class DorisCodec { final DateColumn.DateType dateType = ((DateColumn) col).getSubType(); switch (dateType) { case DATE: - return DateFormatUtils.format(col.asDate(), "yyyy-MM-dd", timeZone); + return DateFormatUtils.format(col.asDate(), "yyyy-MM-dd"); case DATETIME: - return DateFormatUtils.format(col.asDate(), "yyyy-MM-dd HH:mm:ss", timeZone); + return DateFormatUtils.format(col.asDate(), "yyyy-MM-dd HH:mm:ss"); default: return col.asString(); } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java index d433af38..8fa8b2c4 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java @@ -26,8 +26,8 @@ public class DorisCsvCodec extends DorisCodec { private final String columnSeparator; - public DorisCsvCodec(final List fieldNames, String columnSeparator, String timeZone) { - super(fieldNames, timeZone); + public DorisCsvCodec(final List fieldNames, String columnSeparator) { + super(fieldNames); this.columnSeparator = EscapeHandler.escapeString(columnSeparator); } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java index 2a68d050..5486dbde 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java @@ -19,6 +19,7 @@ package com.alibaba.datax.plugin.writer.doriswriter; import com.alibaba.datax.common.element.Record; import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.serializer.SerializerFeature; import java.util.HashMap; import java.util.List; @@ -28,8 +29,8 @@ import java.util.Map; public class DorisJsonCodec extends DorisCodec { private Map rowMap; - public DorisJsonCodec(final List fieldNames, final String timeZone) { - super(fieldNames, timeZone); + public DorisJsonCodec(final List fieldNames) { + super(fieldNames); this.rowMap = new HashMap<>(this.fieldNames.size()); } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java index a2bdfd7e..5f60f9ab 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java @@ -57,9 +57,9 @@ public class DorisWriter extends Writer { public void init() { this.keys = new Key(super.getPluginJobConf()); if (Key.DEFAULT_FORMAT_CSV.equalsIgnoreCase(this.keys.getFormat())) { - this.rowCodec = new DorisCsvCodec(this.keys.getColumns(), this.keys.getColumnSeparator(), this.keys.getTimeZone()); + this.rowCodec = new DorisCsvCodec(this.keys.getColumns(), this.keys.getColumnSeparator()); } else { - this.rowCodec = new DorisJsonCodec(this.keys.getColumns(), this.keys.getTimeZone()); + this.rowCodec = new DorisJsonCodec(this.keys.getColumns()); } this.labelPrefix = this.keys.getLabelPrefix() + UUID.randomUUID(); this.dorisWriterEmitter = new DorisWriterEmitter(keys); diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java index ccd9fc4d..ba5d6742 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java @@ -53,6 +53,7 @@ import java.util.Base64; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.net.URLEncoder; // Used to load batch of rows to Doris using stream load public class DorisWriterEmitter { @@ -112,7 +113,7 @@ public class DorisWriterEmitter { if (StringUtils.isEmpty(host)) { throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, "None of the load url can be connected."); } - final String loadUrl = host + "/api/" + this.keys.getDatabase() + "/" + this.keys.getTable() + "/_stream_load"; + String loadUrl = host + "/api/" + this.keys.getDatabase() + "/" + this.keys.getTable() + "/_stream_load"; // do http put request and get response final Map loadResult; try { @@ -239,54 +240,4 @@ public class DorisWriterEmitter { return "Basic " + new String(encodedAuth); } - // for test - public static void main(String[] args) throws IOException { - String json = "{\n" + - " \"beLoadUrl\": [\"127.0.0.1:8040\"],\n" + - " \"column\": [\"name\", \"age\", \"cdate\", \"cdatetime\"],\n" + - " \"database\": \"test\",\n" + - " \"jdbcUrl\": \"jdbc:mysql://127.0.0.1:9030/\",\n" + - " \"loadProps\": {\n" + -// " \"line_delimiter\": \"\\\\x03\",\n" + -// " \"column_separator\": \"\\\\x04\",\n" + - " },\n" + - " \"format\": \"csv\",\n" + - " \"password\": \"\",\n" + - " \"postSql\": [],\n" + - " \"preSql\": [],\n" + - " \"table\": \"test_datax\",\n" + - " \"maxRetries\": \"0\",\n" + - " \"username\": \"root\"\n" + - " }"; - Configuration configuration = Configuration.from(json); - Key key = new Key(configuration); - - DorisWriterEmitter emitter = new DorisWriterEmitter(key); - DorisFlushBatch flushBatch = new DorisFlushBatch(key.getLineDelimiter(), key.getFormat()); - - Map row1 = Maps.newHashMap(); - row1.put("cdate", "2021-02-02"); - row1.put("cdatetime", "2021-02-02 00:00:00"); - row1.put("name", "zhangsan"); - row1.put("age", "18"); - Map row2 = Maps.newHashMap(); - row2.put("cdate", "2022-02-02"); - row2.put("cdatetime", "2022-02-02 10:00:00"); - row2.put("name", "lisi"); - row2.put("age", "180"); - String rowStr1 = JSON.toJSONString(row1); - String rowStr2 = JSON.toJSONString(row2); - if ("csv".equals(key.getFormat())) { - rowStr1 = String.join(EscapeHandler.escapeString(key.getColumnSeparator()), "2021-02-02", "2021-02-02 00:00:00", "zhangsan", "18"); - rowStr2 = String.join(EscapeHandler.escapeString(key.getColumnSeparator()), "2022-02-02", "2022-02-02 10:00:00", "lisi", "180"); - } - System.out.println("rows1: " + rowStr1); - System.out.println("rows2: " + rowStr2); - - for (int i = 0; i < 1; ++i) { - flushBatch.putData(rowStr1); - flushBatch.putData(rowStr2); - } - emitter.emit(flushBatch); - } } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java index 72812a9c..46f31499 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java @@ -49,7 +49,7 @@ public class Key implements Serializable { public static final String LOAD_PROPS_COLUMN_SEPARATOR = "column_separator"; public static final String MAX_BATCH_ROWS = "maxBatchRows"; - public static final String MAX_BATCH_BYTE_SIZE = "maxBatchByteSize"; + public static final String BATCH_BYTE_SIZE = "batchByteSize"; public static final String MAX_RETRIES = "maxRetries"; public static final String LABEL_PREFIX = "labelPrefix"; public static final String FORMAT = "format"; @@ -57,7 +57,7 @@ public class Key implements Serializable { private final Configuration options; private static final long DEFAULT_MAX_BATCH_ROWS = 50_0000; - private static final long DEFAULT_MAX_BATCH_BYTE_SIZE = 90 * 1024 * 1024; // 90MB + private static final long DEFAULT_BATCH_BYTE_SIZE = 90 * 1024 * 1024; private static final int DEFAULT_MAX_RETRIES = 0; private static final String DEFAULT_LABEL_PREFIX = "datax_doris_writer_"; @@ -130,7 +130,7 @@ public class Key implements Serializable { } public long getBatchByteSize() { - return this.options.getLong(MAX_BATCH_BYTE_SIZE, DEFAULT_MAX_BATCH_BYTE_SIZE); + return this.options.getLong(BATCH_BYTE_SIZE, DEFAULT_BATCH_BYTE_SIZE); } public int getMaxRetries() { diff --git a/package.xml b/package.xml index 5767e3c6..9776ddd6 100755 --- a/package.xml +++ b/package.xml @@ -252,6 +252,13 @@ datax + + doriswriter/target/datax/ + + **/*.* + + datax + txtfilewriter/target/datax/ From 91306ad58618a8a9f0ef19475e9ee6b7b9362864 Mon Sep 17 00:00:00 2001 From: "jiafeng.zhang" Date: Sun, 9 Oct 2022 15:40:33 +0800 Subject: [PATCH 07/13] batchSizeRows batchSizeRows --- doriswriter/doc/doriswriter.md | 6 +++--- .../com/alibaba/datax/plugin/writer/doriswriter/Key.java | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/doriswriter/doc/doriswriter.md b/doriswriter/doc/doriswriter.md index 3bcbf9ff..53f1de34 100644 --- a/doriswriter/doc/doriswriter.md +++ b/doriswriter/doc/doriswriter.md @@ -140,15 +140,15 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter - 默认值:无 -* **maxBatchRows** +* **batchSizeRows** - - 描述:每批次导入数据的最大行数。和 **maxBatchByteSize** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 + - 描述:每批次导入数据的最大行数。和 **batchByteSize** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 - 必选:否 - 默认值:500000 * **batchByteSize** - - 描述:每批次导入数据的最大数据量。和 ** maxBatchRows** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 + - 描述:每批次导入数据的最大数据量。和 **batchSizeRows** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 - 必选:否 - 默认值:104857600 diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java index 46f31499..4335936f 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java @@ -48,7 +48,7 @@ public class Key implements Serializable { public static final String LOAD_PROPS_LINE_DELIMITER = "line_delimiter"; public static final String LOAD_PROPS_COLUMN_SEPARATOR = "column_separator"; - public static final String MAX_BATCH_ROWS = "maxBatchRows"; + public static final String MAX_BATCH_ROWS = "batchSizeRows"; public static final String BATCH_BYTE_SIZE = "batchByteSize"; public static final String MAX_RETRIES = "maxRetries"; public static final String LABEL_PREFIX = "labelPrefix"; @@ -56,7 +56,7 @@ public class Key implements Serializable { public static final String CONNECT_TIMEOUT = "connectTimeout"; private final Configuration options; - private static final long DEFAULT_MAX_BATCH_ROWS = 50_0000; + private static final long DEFAULT_MAX_BATCH_ROWS = 500000; private static final long DEFAULT_BATCH_BYTE_SIZE = 90 * 1024 * 1024; private static final int DEFAULT_MAX_RETRIES = 0; From 2ba4727e14bebfee153e0ce52528ba207f33ccde Mon Sep 17 00:00:00 2001 From: "jiafeng.zhang" Date: Sun, 9 Oct 2022 18:20:23 +0800 Subject: [PATCH 08/13] fix --- doriswriter/doc/mysql2doris.json | 14 ++++++++------ .../plugin/writer/doriswriter/DorisWriter.java | 4 ++-- .../datax/plugin/writer/doriswriter/Key.java | 17 +++++++---------- doriswriter/src/main/resources/plugin.json | 4 ++-- .../src/main/resources/plugin_job_template.json | 13 ++++++++----- .../writer/doriswriter/TestDorisWriterLoad.java | 4 ++-- 6 files changed, 29 insertions(+), 27 deletions(-) diff --git a/doriswriter/doc/mysql2doris.json b/doriswriter/doc/mysql2doris.json index b8be23c9..6992a2be 100644 --- a/doriswriter/doc/mysql2doris.json +++ b/doriswriter/doc/mysql2doris.json @@ -20,17 +20,19 @@ "writer": { "name": "doriswriter", "parameter": { - "feLoadUrl": ["192.168.1.1:8030"], - "jdbcUrl": "jdbc:mysql://192.168.1.1:9030/", - "loadProps": { - }, + "loadUrl": ["192.168.1.1:8030"], + "loadProps": {}, "database": "db1", - "table": "t3", "column": ["k1", "k2", "k3"], "username": "root", "password": "", "postSql": [], - "preSql": [] + "preSql": [], + "connection": [ + "jdbcUrl":"jdbc:mysql://192.168.1.1:9030/", + "table":["xxx"], + "selectedDatabase":"xxxx" + ] } } } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java index 5f60f9ab..5e0bd205 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java @@ -95,11 +95,11 @@ public class DorisWriter extends Writer { // trigger buffer if (batchCount >= this.keys.getBatchRows() || batchByteSize >= this.keys.getBatchByteSize()) { // generate doris stream load label - flush(flushBatch); + flush (flushBatch); // clear buffer batchCount = 0; batchByteSize = 0L; - flushBatch = new DorisFlushBatch(lineDelimiter, this.keys.getFormat()); + flushBatch = new DorisFlushBatch (lineDelimiter, this.keys.getFormat()); } } // end of while diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java index 4335936f..a6ad82c4 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java @@ -31,12 +31,11 @@ import java.util.Map; public class Key implements Serializable { public static final String FE_LOAD_URL = "feLoadUrl"; public static final String BE_LOAD_URL = "beLoadUrl"; - public static final String JDBC_URL = "jdbcUrl"; + public static final String JDBC_URL = "connection[0].jdbcUrl"; - public static final String DATABASE = "database"; - public static final String TABLE = "table"; + public static final String DATABASE = "connection[0].selectedDatabase"; + public static final String TABLE = "connection[0].table[0]"; public static final String COLUMN = "column"; - public static final String TIME_ZONE = "timeZone"; public static final String USERNAME = "username"; public static final String PASSWORD = "password"; @@ -48,8 +47,8 @@ public class Key implements Serializable { public static final String LOAD_PROPS_LINE_DELIMITER = "line_delimiter"; public static final String LOAD_PROPS_COLUMN_SEPARATOR = "column_separator"; - public static final String MAX_BATCH_ROWS = "batchSizeRows"; - public static final String BATCH_BYTE_SIZE = "batchByteSize"; + public static final String MAX_BATCH_ROWS = "maxBatchRows"; + public static final String BATCH_BYTE_SIZE = "maxBatchSize"; public static final String MAX_RETRIES = "maxRetries"; public static final String LABEL_PREFIX = "labelPrefix"; public static final String FORMAT = "format"; @@ -57,6 +56,7 @@ public class Key implements Serializable { private final Configuration options; private static final long DEFAULT_MAX_BATCH_ROWS = 500000; + private static final long DEFAULT_BATCH_BYTE_SIZE = 90 * 1024 * 1024; private static final int DEFAULT_MAX_RETRIES = 0; @@ -109,10 +109,6 @@ public class Key implements Serializable { return this.options.getList(COLUMN, String.class); } - public String getTimeZone() { - return this.options.getString(TIME_ZONE, DEFAULT_TIME_ZONE); - } - public List getPreSqlList() { return this.options.getList(PRE_SQL, String.class); } @@ -157,6 +153,7 @@ public class Key implements Serializable { return this.options.getInt(CONNECT_TIMEOUT, DEFAULT_CONNECT_TIMEOUT); } + private void validateStreamLoadUrl() { List urlList = this.getBeLoadUrlList(); if (urlList == null) { diff --git a/doriswriter/src/main/resources/plugin.json b/doriswriter/src/main/resources/plugin.json index 9d2ad497..69dc31a2 100644 --- a/doriswriter/src/main/resources/plugin.json +++ b/doriswriter/src/main/resources/plugin.json @@ -1,6 +1,6 @@ { "name": "doriswriter", "class": "com.alibaba.datax.plugin.writer.doriswriter.DorisWriter", - "description": "", - "developer": "" + "description": "apache doris writer plugin", + "developer": "apche doris" } diff --git a/doriswriter/src/main/resources/plugin_job_template.json b/doriswriter/src/main/resources/plugin_job_template.json index 9cd9bb18..897fe49b 100644 --- a/doriswriter/src/main/resources/plugin_job_template.json +++ b/doriswriter/src/main/resources/plugin_job_template.json @@ -3,14 +3,17 @@ "parameter": { "username": "", "password": "", - "database": "", - "table": "", "column": [], - "timeZone": "", "preSql": [], "postSql": [], - "jdbcUrl": "", "beLoadUrl": [], - "loadProps": {} + "loadProps": {}, + "connection": [ + { + "jdbcUrl": "", + "selectedDatabase": "", + "table": [] + } + ] } } \ No newline at end of file diff --git a/doriswriter/src/test/java/com/alibaba/datax/plugin/writer/doriswriter/TestDorisWriterLoad.java b/doriswriter/src/test/java/com/alibaba/datax/plugin/writer/doriswriter/TestDorisWriterLoad.java index 35b6e3a4..bbb60a0e 100644 --- a/doriswriter/src/test/java/com/alibaba/datax/plugin/writer/doriswriter/TestDorisWriterLoad.java +++ b/doriswriter/src/test/java/com/alibaba/datax/plugin/writer/doriswriter/TestDorisWriterLoad.java @@ -62,7 +62,7 @@ public class TestDorisWriterLoad { Key key = new Key(configuration); DorisWriterEmitter emitter = new DorisWriterEmitter(key); - DorisFlushBatch flushBatch = new DorisFlushBatch("\n"); + DorisFlushBatch flushBatch = new DorisFlushBatch("\n","csv"); flushBatch.setLabel("test4"); Map row1 = Maps.newHashMap(); row1.put("k1", "2021-02-02"); @@ -83,6 +83,6 @@ public class TestDorisWriterLoad { for (int i = 0; i < 50000; ++i) { flushBatch.putData(rowStr2); } - emitter.doStreamLoad(flushBatch); + emitter.emit (flushBatch); } } From a4c8d2176e027022bb94dc93edb240f3214387c0 Mon Sep 17 00:00:00 2001 From: "jiafeng.zhang" Date: Sun, 9 Oct 2022 18:39:47 +0800 Subject: [PATCH 09/13] fix feLoadUrl to loadUrl --- doriswriter/doc/doriswriter.md | 2 +- .../java/com/alibaba/datax/plugin/writer/doriswriter/Key.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/doriswriter/doc/doriswriter.md b/doriswriter/doc/doriswriter.md index 53f1de34..563de6d8 100644 --- a/doriswriter/doc/doriswriter.md +++ b/doriswriter/doc/doriswriter.md @@ -85,7 +85,7 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter - 必选:是 - 默认值:无 -* **feLoadUrl** +* **loadUrl** - 描述:和 **beLoadUrl** 二选一。作为 Stream Load 的连接目标。格式为 "ip:port"。其中 IP 是 FE 节点 IP,port 是 FE 节点的 http_port。可以填写多个,doriswriter 将以轮询的方式访问。 - 必选:是 diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java index a6ad82c4..a70f6650 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java @@ -29,7 +29,7 @@ import java.util.List; import java.util.Map; public class Key implements Serializable { - public static final String FE_LOAD_URL = "feLoadUrl"; + public static final String FE_LOAD_URL = "loadUrl"; public static final String BE_LOAD_URL = "beLoadUrl"; public static final String JDBC_URL = "connection[0].jdbcUrl"; From 0efbc5df08ee486c878bde812353ff3e12e245b0 Mon Sep 17 00:00:00 2001 From: "jiafeng.zhang" Date: Sun, 9 Oct 2022 19:41:07 +0800 Subject: [PATCH 10/13] fix --- doriswriter/doc/doriswriter.md | 74 +++++++------------ .../doriswriter/DorisBaseSerializer.java | 2 + .../doriswriter/DorisCsvSerializer.java | 2 + .../doriswriter/DorisDelimiterParser.java | 2 + .../doriswriter/DorisJsonSerializer.java | 2 + .../writer/doriswriter/DorisSerializer.java | 2 + .../doriswriter/DorisSerializerFactory.java | 2 + .../doriswriter/DorisStreamLoadExcetion.java | 2 + .../main/resources/plugin_job_template.json | 1 + 9 files changed, 41 insertions(+), 48 deletions(-) create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseSerializer.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvSerializer.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisDelimiterParser.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonSerializer.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializer.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializerFactory.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadExcetion.java diff --git a/doriswriter/doc/doriswriter.md b/doriswriter/doc/doriswriter.md index 563de6d8..344cd943 100644 --- a/doriswriter/doc/doriswriter.md +++ b/doriswriter/doc/doriswriter.md @@ -15,64 +15,48 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter ``` { "job": { - "setting": { - "speed": { - "channel": 1 - }, - "errorLimit": { - "record": 0, - "percentage": 0 - } - }, "content": [ { "reader": { - "name": "streamreader", + "name": "mysqlreader", "parameter": { - "column": [ + "column": ["k1", "k2", "k3"], + "connection": [ { - "value": "皮蛋1", - "type": "string" - }, - { - "value": "皮蛋2", - "type": "string" - }, - { - "value": "111", - "type": "long" - }, - { - "value": "222", - "type": "long" + "jdbcUrl": ["jdbc:mysql://127.0.0.1:3306/db1"], + "table": ["t1"] } ], - "sliceRecordCount": 100 + "username": "root", + "password": "", + "where": "" } }, "writer": { "name": "doriswriter", "parameter": { - "feLoadUrl": ["127.0.0.1:8030", "127.0.0.2:8030", "127.0.0.3:8030"], - "beLoadUrl": ["192.168.10.1:8040", "192.168.10.2:8040", "192.168.10.3:8040"], - "jdbcUrl": "jdbc:mysql://127.0.0.1:9030/", + "loadUrl": ["127.0.0.1:8030"], + "loadProps": {}, "database": "db1", - "table": "t1", - "column": ["k1", "k2", "v1", "v2"], + "column": ["k1", "k2", "k3"], "username": "root", "password": "", "postSql": [], "preSql": [], - "loadProps": { - }, - "maxBatchRows" : 500000, - "maxBatchByteSize" : 104857600, - "labelPrefix": "my_prefix", - "format":"csv" + "connection": [ + "jdbcUrl":"jdbc:mysql://127.0.0.1:9030/demo", + "table":["xxx"], + "selectedDatabase":"xxxx" + ] } } } - ] + ], + "setting": { + "speed": { + "channel": "1" + } + } } } ``` @@ -91,12 +75,6 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter - 必选:是 - 默认值:无 -* **beLoadUrl** - - - 描述:和 **feLoadUrl** 二选一。作为 Stream Load 的连接目标。格式为 "ip:port"。其中 IP 是 BE 节点 IP,port 是 BE 节点的 webserver_port。可以填写多个,doriswriter 将以轮询的方式访问。 - - 必选:否 - - 默认值:无 - * **username** - 描述:访问Doris数据库的用户名 @@ -140,15 +118,15 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter - 默认值:无 -* **batchSizeRows** +* **maxBatchRows** - - 描述:每批次导入数据的最大行数。和 **batchByteSize** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 + - 描述:每批次导入数据的最大行数。和 **maxBatchSize** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 - 必选:否 - 默认值:500000 -* **batchByteSize** +* **maxBatchSize** - - 描述:每批次导入数据的最大数据量。和 **batchSizeRows** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 + - 描述:每批次导入数据的最大数据量。和 **maxBatchRows** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 - 必选:否 - 默认值:104857600 diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseSerializer.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseSerializer.java new file mode 100644 index 00000000..6179ec71 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseSerializer.java @@ -0,0 +1,2 @@ +package com.alibaba.datax.plugin.writer.doriswriter;public class DorisBaseSerializer { +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvSerializer.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvSerializer.java new file mode 100644 index 00000000..298d1142 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvSerializer.java @@ -0,0 +1,2 @@ +package com.alibaba.datax.plugin.writer.doriswriter;public class DorisCsvSerializer { +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisDelimiterParser.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisDelimiterParser.java new file mode 100644 index 00000000..e3f4b46c --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisDelimiterParser.java @@ -0,0 +1,2 @@ +package com.alibaba.datax.plugin.writer.doriswriter;public class DorisDelimiterParser { +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonSerializer.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonSerializer.java new file mode 100644 index 00000000..e747c996 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonSerializer.java @@ -0,0 +1,2 @@ +package com.alibaba.datax.plugin.writer.doriswriter;public class DorisJsonSerializer { +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializer.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializer.java new file mode 100644 index 00000000..a974c5e5 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializer.java @@ -0,0 +1,2 @@ +package com.alibaba.datax.plugin.writer.doriswriter;public class DorisSerializer { +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializerFactory.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializerFactory.java new file mode 100644 index 00000000..2bb1a1c1 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializerFactory.java @@ -0,0 +1,2 @@ +package com.alibaba.datax.plugin.writer.doriswriter;public class DorisSerializerFactory { +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadExcetion.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadExcetion.java new file mode 100644 index 00000000..c8e27e00 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadExcetion.java @@ -0,0 +1,2 @@ +package com.alibaba.datax.plugin.writer.doriswriter;public class DorisStreamLoadExcetion { +} diff --git a/doriswriter/src/main/resources/plugin_job_template.json b/doriswriter/src/main/resources/plugin_job_template.json index 897fe49b..0187e539 100644 --- a/doriswriter/src/main/resources/plugin_job_template.json +++ b/doriswriter/src/main/resources/plugin_job_template.json @@ -7,6 +7,7 @@ "preSql": [], "postSql": [], "beLoadUrl": [], + "loadUrl": [], "loadProps": {}, "connection": [ { From 8b46e82a60d0b5991dff4962a36187e5a3e7d789 Mon Sep 17 00:00:00 2001 From: "jiafeng.zhang" Date: Sun, 9 Oct 2022 20:29:15 +0800 Subject: [PATCH 11/13] Refactoring doris writer code Refactoring doris writer code --- doriswriter/doc/doriswriter.md | 53 +++- .../doriswriter/DorisBaseSerializer.java | 23 +- .../plugin/writer/doriswriter/DorisCodec.java | 73 ------ .../writer/doriswriter/DorisCsvCodec.java | 49 ---- .../doriswriter/DorisCsvSerializer.java | 26 +- .../doriswriter/DorisDelimiterParser.java | 54 +++- .../writer/doriswriter/DorisFlushBatch.java | 66 ----- .../writer/doriswriter/DorisJsonCodec.java | 51 ---- .../doriswriter/DorisJsonSerializer.java | 33 ++- .../writer/doriswriter/DorisSerializer.java | 10 +- .../doriswriter/DorisSerializerFactory.java | 19 +- .../doriswriter/DorisStreamLoadExcetion.java | 29 ++- .../doriswriter/DorisStreamLoadVisitor.java | 233 +++++++++++++++++ .../plugin/writer/doriswriter/DorisUtil.java | 105 ++++++++ .../writer/doriswriter/DorisWriter.java | 242 ++++++----------- .../doriswriter/DorisWriterEmitter.java | 243 ------------------ .../doriswriter/DorisWriterManager.java | 192 ++++++++++++++ .../doriswriter/DorisWriterOptions.java | 174 +++++++++++++ .../writer/doriswriter/DorisWriterTuple.java | 20 ++ .../writer/doriswriter/EscapeHandler.java | 42 --- .../datax/plugin/writer/doriswriter/Key.java | 188 -------------- 21 files changed, 1024 insertions(+), 901 deletions(-) delete mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java delete mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java delete mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisFlushBatch.java delete mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadVisitor.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisUtil.java delete mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterManager.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterOptions.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterTuple.java delete mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/EscapeHandler.java delete mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java diff --git a/doriswriter/doc/doriswriter.md b/doriswriter/doc/doriswriter.md index 344cd943..9f8f510c 100644 --- a/doriswriter/doc/doriswriter.md +++ b/doriswriter/doc/doriswriter.md @@ -20,34 +20,40 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter "reader": { "name": "mysqlreader", "parameter": { - "column": ["k1", "k2", "k3"], + "column": ["emp_no", "birth_date", "first_name","last_name","gender","hire_date"], "connection": [ { - "jdbcUrl": ["jdbc:mysql://127.0.0.1:3306/db1"], - "table": ["t1"] + "jdbcUrl": ["jdbc:mysql://localhost:3306/demo"], + "table": ["employees_1"] } ], "username": "root", - "password": "", + "password": "xxxxx", "where": "" } }, "writer": { "name": "doriswriter", "parameter": { - "loadUrl": ["127.0.0.1:8030"], - "loadProps": {}, - "database": "db1", - "column": ["k1", "k2", "k3"], + "loadUrl": ["172.16.0.13:8030"], + "loadProps": { + }, + "column": ["emp_no", "birth_date", "first_name","last_name","gender","hire_date"], "username": "root", - "password": "", - "postSql": [], + "password": "xxxxxx", + "postSql": ["select count(1) from all_employees_info"], "preSql": [], "connection": [ - "jdbcUrl":"jdbc:mysql://127.0.0.1:9030/demo", - "table":["xxx"], - "selectedDatabase":"xxxx" - ] + { + "jdbcUrl": "jdbc:mysql://172.16.0.13:9030/demo", + "database": "demo", + "table": ["all_employees_info"] + } + ], + "loadProps": { + "format": "json", + "strip_outer_array": true + } } } } @@ -159,3 +165,22 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter - 描述:StreamLoad单次请求的超时时间, 单位毫秒(ms)。 - 必选:否 - 默认值:-1 + +### 类型转换 + +默认传入的数据均会被转为字符串,并以`\t`作为列分隔符,`\n`作为行分隔符,组成`csv`文件进行StreamLoad导入操作。 +如需更改列分隔符, 则正确配置 `loadProps` 即可: +```json +"loadProps": { + "column_separator": "\\x01", + "row_delimiter": "\\x02" +} +``` + +如需更改导入格式为`json`, 则正确配置 `loadProps` 即可: +```json +"loadProps": { + "format": "json", + "strip_outer_array": true +} +``` \ No newline at end of file diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseSerializer.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseSerializer.java index 6179ec71..c9aacc6e 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseSerializer.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseSerializer.java @@ -1,2 +1,23 @@ -package com.alibaba.datax.plugin.writer.doriswriter;public class DorisBaseSerializer { +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.element.Column; + +public class DorisBaseSerializer { + protected String fieldConvertion( Column col) { + if (null == col.getRawData() || Column.Type.NULL == col.getType()) { + return null; + } + if ( Column.Type.BOOL == col.getType()) { + return String.valueOf(col.asLong()); + } + if ( Column.Type.BYTES == col.getType()) { + byte[] bts = (byte[])col.getRawData(); + long value = 0; + for (int i = 0; i < bts.length; i++) { + value += (bts[bts.length - i - 1] & 0xffL) << (8 * i); + } + return String.valueOf(value); + } + return col.asString(); + } } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java deleted file mode 100644 index ef5889de..00000000 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java +++ /dev/null @@ -1,73 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package com.alibaba.datax.plugin.writer.doriswriter; - -import com.alibaba.datax.common.element.Column; -import com.alibaba.datax.common.element.DateColumn; -import com.alibaba.datax.common.element.Record; -import org.apache.commons.lang3.time.DateFormatUtils; - -import java.time.ZoneId; -import java.util.List; - -public abstract class DorisCodec { - protected final List fieldNames; - - public DorisCodec(final List fieldNames) { - this.fieldNames = fieldNames; - } - - public abstract String serialize(Record row); - - /** - * convert datax internal data to string - * - * @param col - * @return - */ - protected Object convertColumn(final Column col) { - if (null == col.getRawData()) { - return null; - } - Column.Type type = col.getType(); - switch (type) { - case BOOL: - case INT: - case LONG: - return col.asLong(); - case DOUBLE: - return col.asDouble(); - case STRING: - return col.asString(); - case DATE: { - final DateColumn.DateType dateType = ((DateColumn) col).getSubType(); - switch (dateType) { - case DATE: - return DateFormatUtils.format(col.asDate(), "yyyy-MM-dd"); - case DATETIME: - return DateFormatUtils.format(col.asDate(), "yyyy-MM-dd HH:mm:ss"); - default: - return col.asString(); - } - } - default: - // BAD, NULL, BYTES - return null; - } - } -} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java deleted file mode 100644 index 8fa8b2c4..00000000 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java +++ /dev/null @@ -1,49 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package com.alibaba.datax.plugin.writer.doriswriter; - -import com.alibaba.datax.common.element.Record; - -import java.util.ArrayList; -import java.util.List; - -public class DorisCsvCodec extends DorisCodec { - - private final String columnSeparator; - - public DorisCsvCodec(final List fieldNames, String columnSeparator) { - super(fieldNames); - this.columnSeparator = EscapeHandler.escapeString(columnSeparator); - } - - @Override - public String serialize(final Record row) { - if (null == this.fieldNames) { - return ""; - } - List list = new ArrayList<>(); - - for (int i = 0; i < this.fieldNames.size(); i++) { - Object value = this.convertColumn(row.getColumn(i)); - list.add(value != null ? value.toString() : "\\N"); - } - - return String.join(columnSeparator, list); - } - -} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvSerializer.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvSerializer.java index 298d1142..93c477d8 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvSerializer.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvSerializer.java @@ -1,2 +1,26 @@ -package com.alibaba.datax.plugin.writer.doriswriter;public class DorisCsvSerializer { +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.element.Record; + +public class DorisCsvSerializer extends DorisBaseSerializer implements DorisSerializer{ + private static final long serialVersionUID = 1L; + + private final String columnSeparator; + + public DorisCsvSerializer(String sp) { + this.columnSeparator = DorisDelimiterParser.parse(sp, "\t"); + } + + @Override + public String serialize( Record row) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < row.getColumnNumber(); i++) { + String value = fieldConvertion(row.getColumn(i)); + sb.append(null == value ? "\\N" : value); + if (i < row.getColumnNumber() - 1) { + sb.append(columnSeparator); + } + } + return sb.toString(); + } } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisDelimiterParser.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisDelimiterParser.java index e3f4b46c..5c92af9f 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisDelimiterParser.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisDelimiterParser.java @@ -1,2 +1,54 @@ -package com.alibaba.datax.plugin.writer.doriswriter;public class DorisDelimiterParser { +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.google.common.base.Strings; + +import java.io.StringWriter; + +public class DorisDelimiterParser { + + private static final String HEX_STRING = "0123456789ABCDEF"; + + public static String parse(String sp, String dSp) throws RuntimeException { + if ( Strings.isNullOrEmpty(sp)) { + return dSp; + } + if (!sp.toUpperCase().startsWith("\\X")) { + return sp; + } + String hexStr = sp.substring(2); + // check hex str + if (hexStr.isEmpty()) { + throw new RuntimeException("Failed to parse delimiter: `Hex str is empty`"); + } + if (hexStr.length() % 2 != 0) { + throw new RuntimeException("Failed to parse delimiter: `Hex str length error`"); + } + for (char hexChar : hexStr.toUpperCase().toCharArray()) { + if (HEX_STRING.indexOf(hexChar) == -1) { + throw new RuntimeException("Failed to parse delimiter: `Hex str format error`"); + } + } + // transform to separator + StringWriter writer = new StringWriter(); + for (byte b : hexStrToBytes(hexStr)) { + writer.append((char) b); + } + return writer.toString(); + } + + private static byte[] hexStrToBytes(String hexStr) { + String upperHexStr = hexStr.toUpperCase(); + int length = upperHexStr.length() / 2; + char[] hexChars = upperHexStr.toCharArray(); + byte[] bytes = new byte[length]; + for (int i = 0; i < length; i++) { + int pos = i * 2; + bytes[i] = (byte) (charToByte(hexChars[pos]) << 4 | charToByte(hexChars[pos + 1])); + } + return bytes; + } + + private static byte charToByte(char c) { + return (byte) HEX_STRING.indexOf(c); + } } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisFlushBatch.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisFlushBatch.java deleted file mode 100644 index 9980c937..00000000 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisFlushBatch.java +++ /dev/null @@ -1,66 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package com.alibaba.datax.plugin.writer.doriswriter; - -import java.util.ArrayList; -import java.util.List; - -// A wrapper class to hold a batch of loaded rows -public class DorisFlushBatch { - private final String format; - private final String lineDelimiter; - private String label; - private long byteSize = 0; - private List data = new ArrayList<>(); - - public DorisFlushBatch(String lineDelimiter, String format) { - this.lineDelimiter = EscapeHandler.escapeString(lineDelimiter); - this.format = format; - } - - public void setLabel(String label) { - this.label = label; - } - - public String getLabel() { - return label; - } - - public long getRows() { - return data.size(); - } - - public void putData(String row) { - data.add(row); - byteSize += row.getBytes().length; - } - - public String getData() { - String result; - if (Key.DEFAULT_FORMAT_CSV.equalsIgnoreCase(format)) { - result = String.join(this.lineDelimiter, data); - } else { - result = data.toString(); - } - return result; - } - - public long getSize() { - return byteSize; - } -} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java deleted file mode 100644 index 5486dbde..00000000 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java +++ /dev/null @@ -1,51 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package com.alibaba.datax.plugin.writer.doriswriter; - -import com.alibaba.datax.common.element.Record; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.serializer.SerializerFeature; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -// Convert DataX data to json -public class DorisJsonCodec extends DorisCodec { - private Map rowMap; - - public DorisJsonCodec(final List fieldNames) { - super(fieldNames); - this.rowMap = new HashMap<>(this.fieldNames.size()); - } - - @Override - public String serialize(final Record row) { - if (null == this.fieldNames) { - return ""; - } - - rowMap.clear(); - int idx = 0; - for (final String fieldName : this.fieldNames) { - rowMap.put(fieldName, this.convertColumn(row.getColumn(idx))); - ++idx; - } - return JSON.toJSONString(rowMap, SerializerFeature.WriteMapNullValue); - } -} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonSerializer.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonSerializer.java index e747c996..315e472c 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonSerializer.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonSerializer.java @@ -1,2 +1,33 @@ -package com.alibaba.datax.plugin.writer.doriswriter;public class DorisJsonSerializer { +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.fastjson.JSON; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class DorisJsonSerializer extends DorisBaseSerializer implements DorisSerializer{ + + private static final long serialVersionUID = 1L; + + private final List fieldNames; + + public DorisJsonSerializer( List fieldNames) { + this.fieldNames = fieldNames; + } + + @Override + public String serialize( Record row) { + if (null == fieldNames) { + return ""; + } + Map rowMap = new HashMap<> (fieldNames.size()); + int idx = 0; + for (String fieldName : fieldNames) { + rowMap.put(fieldName, fieldConvertion(row.getColumn(idx))); + idx++; + } + return JSON.toJSONString(rowMap); + } } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializer.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializer.java index a974c5e5..58759ecf 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializer.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializer.java @@ -1,2 +1,10 @@ -package com.alibaba.datax.plugin.writer.doriswriter;public class DorisSerializer { +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.element.Record; + +import java.io.Serializable; + +public interface DorisSerializer extends Serializable { + + String serialize( Record row); } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializerFactory.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializerFactory.java index 2bb1a1c1..8484e5b7 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializerFactory.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializerFactory.java @@ -1,2 +1,19 @@ -package com.alibaba.datax.plugin.writer.doriswriter;public class DorisSerializerFactory { +package com.alibaba.datax.plugin.writer.doriswriter; + +import java.util.Map; + +public class DorisSerializerFactory { + public DorisSerializerFactory(){ + + } + public static DorisSerializer createSerializer(DorisWriterOptions writerOptions) { + if (DorisWriterOptions.StreamLoadFormat.CSV.equals(writerOptions.getStreamLoadFormat())) { + Map props = writerOptions.getLoadProps(); + return new DorisCsvSerializer(null == props || !props.containsKey("column_separator") ? null : String.valueOf(props.get("column_separator"))); + } + if (DorisWriterOptions.StreamLoadFormat.JSON.equals(writerOptions.getStreamLoadFormat())) { + return new DorisJsonSerializer(writerOptions.getColumns()); + } + throw new RuntimeException("Failed to create row serializer, unsupported `format` from stream load properties."); + } } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadExcetion.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadExcetion.java index c8e27e00..dc19b209 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadExcetion.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadExcetion.java @@ -1,2 +1,29 @@ -package com.alibaba.datax.plugin.writer.doriswriter;public class DorisStreamLoadExcetion { +package com.alibaba.datax.plugin.writer.doriswriter; + +import java.io.IOException; +import java.util.Map; + +public class DorisStreamLoadExcetion extends IOException { + + private final Map response; + private boolean reCreateLabel; + + public DorisStreamLoadExcetion(String message, Map response) { + super(message); + this.response = response; + } + + public DorisStreamLoadExcetion(String message, Map response, boolean reCreateLabel) { + super(message); + this.response = response; + this.reCreateLabel = reCreateLabel; + } + + public Map getFailedResponse() { + return response; + } + + public boolean needReCreateLabel() { + return reCreateLabel; + } } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadVisitor.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadVisitor.java new file mode 100644 index 00000000..a18940cb --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadVisitor.java @@ -0,0 +1,233 @@ +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.fastjson.JSON; +import org.apache.commons.codec.binary.Base64; +import org.apache.http.HttpEntity; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpPut; +import org.apache.http.entity.ByteArrayEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.DefaultRedirectStrategy; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.URL; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class DorisStreamLoadVisitor { + private static final Logger LOG = LoggerFactory.getLogger(DorisStreamLoadVisitor.class); + + private DorisWriterOptions options; + + private long pos; + private static final String RESULT_FAILED = "Fail"; + private static final String RESULT_LABEL_EXISTED = "Label Already Exists"; + private static final String LAEBL_STATE_VISIBLE = "VISIBLE"; + private static final String LAEBL_STATE_COMMITTED = "COMMITTED"; + private static final String RESULT_LABEL_PREPARE = "PREPARE"; + private static final String RESULT_LABEL_ABORTED = "ABORTED"; + private static final String RESULT_LABEL_UNKNOWN = "UNKNOWN"; + + + public DorisStreamLoadVisitor(DorisWriterOptions options){ + this.options = options; + } + + public void streamLoad(DorisWriterTuple data) throws Exception { + String host = getAvailableHost(); + if(host == null){ + throw new IOException ("None of the host in `load_url` could be connected."); + } + String loadUrl = new StringBuilder(host) + .append("/api/") + .append(options.getDatabase()) + .append("/") + .append(options.getTable()) + .append("/_stream_load") + .toString(); + LOG.debug(String.format("Start to join batch data: rows[%d] bytes[%d] label[%s].", data.getRows().size(), data.getBytes(), data.getLabel())); + Map loadResult = doHttpPut(loadUrl, data.getLabel(), joinRows(data.getRows(), data.getBytes().intValue())); + final String keyStatus = "Status"; + if (null == loadResult || !loadResult.containsKey(keyStatus)) { + throw new IOException("Unable to flush data to Doris: unknown result status."); + } + LOG.debug(new StringBuilder("StreamLoad response:\n").append(JSON.toJSONString(loadResult)).toString()); + if (RESULT_FAILED.equals(loadResult.get(keyStatus))) { + throw new IOException( + new StringBuilder("Failed to flush data to Doris.\n").append(JSON.toJSONString(loadResult)).toString() + ); + } else if (RESULT_LABEL_EXISTED.equals(loadResult.get(keyStatus))) { + LOG.debug(new StringBuilder("StreamLoad response:\n").append(JSON.toJSONString(loadResult)).toString()); + // has to block-checking the state to get the final result + checkLabelState(host, data.getLabel()); + } + } + + private void checkLabelState(String host, String label) throws IOException { + int idx = 0; + while(true) { + try { + TimeUnit.SECONDS.sleep(Math.min(++idx, 5)); + } catch (InterruptedException ex) { + break; + } + try (CloseableHttpClient httpclient = HttpClients.createDefault()) { + HttpGet httpGet = new HttpGet(new StringBuilder(host).append("/api/").append(options.getDatabase()).append("/get_load_state?label=").append(label).toString()); + httpGet.setHeader("Authorization", getBasicAuthHeader(options.getUsername(), options.getPassword())); + httpGet.setHeader("Connection", "close"); + + try (CloseableHttpResponse resp = httpclient.execute(httpGet)) { + HttpEntity respEntity = getHttpEntity(resp); + if (respEntity == null) { + throw new IOException(String.format("Failed to flush data to Doris, Error " + + "could not get the final state of label[%s].\n", label), null); + } + Map result = (Map)JSON.parse(EntityUtils.toString(respEntity)); + String labelState = (String)result.get("state"); + if (null == labelState) { + throw new IOException(String.format("Failed to flush data to Doris, Error " + + "could not get the final state of label[%s]. response[%s]\n", label, EntityUtils.toString(respEntity)), null); + } + LOG.info(String.format("Checking label[%s] state[%s]\n", label, labelState)); + switch(labelState) { + case LAEBL_STATE_VISIBLE: + case LAEBL_STATE_COMMITTED: + return; + case RESULT_LABEL_PREPARE: + continue; + case RESULT_LABEL_ABORTED: + throw new DorisStreamLoadExcetion(String.format("Failed to flush data to Doris, Error " + + "label[%s] state[%s]\n", label, labelState), null, true); + case RESULT_LABEL_UNKNOWN: + default: + throw new IOException(String.format("Failed to flush data to Doris, Error " + + "label[%s] state[%s]\n", label, labelState), null); + } + } + } + } + } + + private byte[] joinRows(List rows, int totalBytes) { + if (DorisWriterOptions.StreamLoadFormat.CSV.equals(options.getStreamLoadFormat())) { + Map props = (options.getLoadProps() == null ? new HashMap<> () : options.getLoadProps()); + byte[] lineDelimiter = DorisDelimiterParser.parse((String)props.get("row_delimiter"), "\n").getBytes(StandardCharsets.UTF_8); + ByteBuffer bos = ByteBuffer.allocate(totalBytes + rows.size() * lineDelimiter.length); + for (byte[] row : rows) { + bos.put(row); + bos.put(lineDelimiter); + } + return bos.array(); + } + + if (DorisWriterOptions.StreamLoadFormat.JSON.equals(options.getStreamLoadFormat())) { + ByteBuffer bos = ByteBuffer.allocate(totalBytes + (rows.isEmpty() ? 2 : rows.size() + 1)); + bos.put("[".getBytes(StandardCharsets.UTF_8)); + byte[] jsonDelimiter = ",".getBytes(StandardCharsets.UTF_8); + boolean isFirstElement = true; + for (byte[] row : rows) { + if (!isFirstElement) { + bos.put(jsonDelimiter); + } + bos.put(row); + isFirstElement = false; + } + bos.put("]".getBytes(StandardCharsets.UTF_8)); + return bos.array(); + } + throw new RuntimeException("Failed to join rows data, unsupported `format` from stream load properties:"); + } + private Map doHttpPut(String loadUrl, String label, byte[] data) throws IOException { + LOG.info(String.format("Executing stream load to: '%s', size: '%s'", loadUrl, data.length)); + final HttpClientBuilder httpClientBuilder = HttpClients.custom() + .setRedirectStrategy(new DefaultRedirectStrategy () { + @Override + protected boolean isRedirectable(String method) { + return true; + } + }); + try ( CloseableHttpClient httpclient = httpClientBuilder.build()) { + HttpPut httpPut = new HttpPut(loadUrl); + List cols = options.getColumns(); + if (null != cols && !cols.isEmpty() && DorisWriterOptions.StreamLoadFormat.CSV.equals(options.getStreamLoadFormat())) { + httpPut.setHeader("columns", String.join(",", cols.stream().map(f -> String.format("`%s`", f)).collect(Collectors.toList()))); + } + if (null != options.getLoadProps()) { + for (Map.Entry entry : options.getLoadProps().entrySet()) { + httpPut.setHeader(entry.getKey(), String.valueOf(entry.getValue())); + } + } + httpPut.setHeader("Expect", "100-continue"); + httpPut.setHeader("label", label); + httpPut.setHeader("Content-Type", "application/x-www-form-urlencoded"); + httpPut.setHeader("Authorization", getBasicAuthHeader(options.getUsername(), options.getPassword())); + httpPut.setEntity(new ByteArrayEntity (data)); + httpPut.setConfig(RequestConfig.custom().setRedirectsEnabled(true).build()); + try ( CloseableHttpResponse resp = httpclient.execute(httpPut)) { + HttpEntity respEntity = getHttpEntity(resp); + if (respEntity == null) + return null; + return (Map)JSON.parse(EntityUtils.toString(respEntity)); + } + } + } + + private String getBasicAuthHeader(String username, String password) { + String auth = username + ":" + password; + byte[] encodedAuth = Base64.encodeBase64(auth.getBytes(StandardCharsets.UTF_8)); + return new StringBuilder("Basic ").append(new String(encodedAuth)).toString(); + } + + private HttpEntity getHttpEntity(CloseableHttpResponse resp) { + int code = resp.getStatusLine().getStatusCode(); + if (200 != code) { + LOG.warn("Request failed with code:{}", code); + return null; + } + HttpEntity respEntity = resp.getEntity(); + if (null == respEntity) { + LOG.warn("Request failed with empty response."); + return null; + } + return respEntity; + } + + private String getAvailableHost() { + List hostList = options.getLoadUrlList(); + long tmp = pos + hostList.size(); + for (; pos < tmp; pos++) { + String host = new StringBuilder("http://").append(hostList.get((int) (pos % hostList.size()))).toString(); + if (tryHttpConnection(host)) { + return host; + } + } + return null; + } + + private boolean tryHttpConnection(String host) { + try { + URL url = new URL(host); + HttpURLConnection co = (HttpURLConnection) url.openConnection(); + co.setConnectTimeout(5000); + co.connect(); + co.disconnect(); + return true; + } catch (Exception e1) { + LOG.warn("Failed to connect to address:{}", host, e1); + return false; + } + } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisUtil.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisUtil.java new file mode 100644 index 00000000..7c7d9a92 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisUtil.java @@ -0,0 +1,105 @@ +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.plugin.rdbms.util.DBUtil; +import com.alibaba.datax.plugin.rdbms.util.DataBaseType; +import com.alibaba.datax.plugin.rdbms.util.RdbmsException; +import com.alibaba.datax.plugin.rdbms.writer.Constant; +import com.alibaba.druid.sql.parser.ParserException; +import com.google.common.base.Strings; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * jdbc util + */ +public class DorisUtil { + private static final Logger LOG = LoggerFactory.getLogger(DorisUtil.class); + + private DorisUtil() {} + + public static List getDorisTableColumns( Connection conn, String databaseName, String tableName) { + String currentSql = String.format("SELECT COLUMN_NAME FROM `information_schema`.`COLUMNS` WHERE `TABLE_SCHEMA` = '%s' AND `TABLE_NAME` = '%s' ORDER BY `ORDINAL_POSITION` ASC;", databaseName, tableName); + List columns = new ArrayList<> (); + ResultSet rs = null; + try { + rs = DBUtil.query(conn, currentSql); + while (DBUtil.asyncResultSetNext(rs)) { + String colName = rs.getString("COLUMN_NAME"); + columns.add(colName); + } + return columns; + } catch (Exception e) { + throw RdbmsException.asQueryException(DataBaseType.MySql, e, currentSql, null, null); + } finally { + DBUtil.closeDBResources(rs, null, null); + } + } + + public static List renderPreOrPostSqls(List preOrPostSqls, String tableName) { + if (null == preOrPostSqls) { + return Collections.emptyList(); + } + List renderedSqls = new ArrayList<>(); + for (String sql : preOrPostSqls) { + if (! Strings.isNullOrEmpty(sql)) { + renderedSqls.add(sql.replace(Constant.TABLE_NAME_PLACEHOLDER, tableName)); + } + } + return renderedSqls; + } + + public static void executeSqls(Connection conn, List sqls) { + Statement stmt = null; + String currentSql = null; + try { + stmt = conn.createStatement(); + for (String sql : sqls) { + currentSql = sql; + DBUtil.executeSqlWithoutResultSet(stmt, sql); + } + } catch (Exception e) { + throw RdbmsException.asQueryException(DataBaseType.MySql, e, currentSql, null, null); + } finally { + DBUtil.closeDBResources(null, stmt, null); + } + } + + public static void preCheckPrePareSQL(DorisWriterOptions options) { + String table = options.getTable(); + List preSqls = options.getPreSqlList(); + List renderedPreSqls = DorisUtil.renderPreOrPostSqls(preSqls, table); + if (null != renderedPreSqls && !renderedPreSqls.isEmpty()) { + LOG.info("Begin to preCheck preSqls:[{}].", String.join(";", renderedPreSqls)); + for (String sql : renderedPreSqls) { + try { + DBUtil.sqlValid(sql, DataBaseType.MySql); + } catch ( ParserException e) { + throw RdbmsException.asPreSQLParserException(DataBaseType.MySql,e,sql); + } + } + } + } + + public static void preCheckPostSQL(DorisWriterOptions options) { + String table = options.getTable(); + List postSqls = options.getPostSqlList(); + List renderedPostSqls = DorisUtil.renderPreOrPostSqls(postSqls, table); + if (null != renderedPostSqls && !renderedPostSqls.isEmpty()) { + LOG.info("Begin to preCheck postSqls:[{}].", String.join(";", renderedPostSqls)); + for(String sql : renderedPostSqls) { + try { + DBUtil.sqlValid(sql, DataBaseType.MySql); + } catch (ParserException e){ + throw RdbmsException.asPostSQLParserException(DataBaseType.MySql,e,sql); + } + } + } + } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java index 5e0bd205..f16cd354 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java @@ -39,131 +39,41 @@ import java.util.Collections; import java.util.List; import java.util.UUID; +/** + * doris data writer + */ public class DorisWriter extends Writer { - public DorisWriter() { - } - public static class Task extends com.alibaba.datax.common.spi.Writer.Task { - private DorisWriterEmitter dorisWriterEmitter; - private Key keys; - private DorisCodec rowCodec; - private int batchNum = 0; - private String labelPrefix; + public static class Job extends Writer.Job { - public Task() { - } - - @Override - public void init() { - this.keys = new Key(super.getPluginJobConf()); - if (Key.DEFAULT_FORMAT_CSV.equalsIgnoreCase(this.keys.getFormat())) { - this.rowCodec = new DorisCsvCodec(this.keys.getColumns(), this.keys.getColumnSeparator()); - } else { - this.rowCodec = new DorisJsonCodec(this.keys.getColumns()); - } - this.labelPrefix = this.keys.getLabelPrefix() + UUID.randomUUID(); - this.dorisWriterEmitter = new DorisWriterEmitter(keys); - } - - @Override - public void prepare() { - } - - @Override - public void startWrite(RecordReceiver recordReceiver) { - String lineDelimiter = this.keys.getLineDelimiter(); - DorisFlushBatch flushBatch = new DorisFlushBatch(lineDelimiter, this.keys.getFormat()); - long batchCount = 0; - long batchByteSize = 0L; - Record record; - // loop to get record from datax - while ((record = recordReceiver.getFromReader()) != null) { - // check column size - if (record.getColumnNumber() != this.keys.getColumns().size()) { - throw DataXException.asDataXException(DBUtilErrorCode.CONF_ERROR, - String.format("config writer column info error. because the column number of reader is :%s" + - "and the column number of writer is:%s. please check you datax job config json.", - record.getColumnNumber(), this.keys.getColumns().size())); - } - // codec record - final String recordStr = this.rowCodec.serialize(record); - - // put into buffer - flushBatch.putData(recordStr); - batchCount += 1; - batchByteSize += recordStr.length(); - // trigger buffer - if (batchCount >= this.keys.getBatchRows() || batchByteSize >= this.keys.getBatchByteSize()) { - // generate doris stream load label - flush (flushBatch); - // clear buffer - batchCount = 0; - batchByteSize = 0L; - flushBatch = new DorisFlushBatch (lineDelimiter, this.keys.getFormat()); - } - } // end of while - - if (flushBatch.getSize() > 0) { - flush(flushBatch); - } - } - - private void flush(DorisFlushBatch flushBatch) { - flushBatch.setLabel(getStreamLoadLabel()); - dorisWriterEmitter.emit(flushBatch); - } - - private String getStreamLoadLabel() { - return labelPrefix + "_" + (batchNum++); - } - - @Override - public void post() { - - } - - @Override - public void destroy() { - } - - @Override - public boolean supportFailOver() { - return false; - } - } - - public static class Job extends com.alibaba.datax.common.spi.Writer.Job { - private static final Logger LOG = LoggerFactory.getLogger(DorisWriter.Job.class); + private static final Logger LOG = LoggerFactory.getLogger(Job.class); private Configuration originalConfig = null; - private Key keys; - - public Job() { - } + private DorisWriterOptions options; @Override public void init() { this.originalConfig = super.getPluginJobConf(); - this.keys = new Key(super.getPluginJobConf()); - this.keys.doPretreatment(); + options = new DorisWriterOptions(super.getPluginJobConf()); + options.doPretreatment(); } @Override - public void preCheck() { + public void preCheck(){ this.init(); - this.preCheckPrePareSQL(this.keys); - this.preCheckPostSQL(this.keys); + DorisUtil.preCheckPrePareSQL(options); + DorisUtil.preCheckPostSQL(options); } @Override public void prepare() { - String username = this.keys.getUsername(); - String password = this.keys.getPassword(); - String jdbcUrl = this.keys.getJdbcUrl(); - List renderedPreSqls = this.renderPreOrPostSqls(this.keys.getPreSqlList(), this.keys.getTable()); - if (!renderedPreSqls.isEmpty()) { + String username = options.getUsername(); + String password = options.getPassword(); + String jdbcUrl = options.getJdbcUrl(); + List renderedPreSqls = DorisUtil.renderPreOrPostSqls(options.getPreSqlList(), options.getTable()); + if (null != renderedPreSqls && !renderedPreSqls.isEmpty()) { Connection conn = DBUtil.getConnection(DataBaseType.MySql, jdbcUrl, username, password); - LOG.info("prepare execute preSqls:[{}]. doris jdbc url:{}.", String.join(";", renderedPreSqls), jdbcUrl); - this.executeSqls(conn, renderedPreSqls); + LOG.info("Begin to execute preSqls:[{}]. context info:{}.", String.join(";", renderedPreSqls), jdbcUrl); + DorisUtil.executeSqls(conn, renderedPreSqls); DBUtil.closeDBResources(null, null, conn); } } @@ -171,93 +81,89 @@ public class DorisWriter extends Writer { @Override public List split(int mandatoryNumber) { List configurations = new ArrayList<>(mandatoryNumber); - - for (int i = 0; i < mandatoryNumber; ++i) { - configurations.add(this.originalConfig); + for (int i = 0; i < mandatoryNumber; i++) { + configurations.add(originalConfig); } - return configurations; } @Override public void post() { - String username = this.keys.getUsername(); - String password = this.keys.getPassword(); - String jdbcUrl = this.keys.getJdbcUrl(); - List renderedPostSqls = this.renderPreOrPostSqls(this.keys.getPostSqlList(), this.keys.getTable()); - if (!renderedPostSqls.isEmpty()) { + String username = options.getUsername(); + String password = options.getPassword(); + String jdbcUrl = options.getJdbcUrl(); + LOG.info("userName :{},password:{},jdbcUrl:{}.", username,password,jdbcUrl); + List renderedPostSqls = DorisUtil.renderPreOrPostSqls(options.getPostSqlList(), options.getTable()); + if (null != renderedPostSqls && !renderedPostSqls.isEmpty()) { Connection conn = DBUtil.getConnection(DataBaseType.MySql, jdbcUrl, username, password); - LOG.info("prepare execute postSqls:[{}]. doris jdbc url为:{}.", String.join(";", renderedPostSqls), jdbcUrl); - this.executeSqls(conn, renderedPostSqls); + LOG.info("Begin to execute preSqls:[{}]. context info:{}.", String.join(";", renderedPostSqls), jdbcUrl); + DorisUtil.executeSqls(conn, renderedPostSqls); DBUtil.closeDBResources(null, null, conn); } - } @Override public void destroy() { } - private List renderPreOrPostSqls(final List preOrPostSqls, final String tableName) { - if (null == preOrPostSqls) { - return Collections.emptyList(); + } + + public static class Task extends Writer.Task { + private DorisWriterManager writerManager; + private DorisWriterOptions options; + private DorisSerializer rowSerializer; + + @Override + public void init() { + options = new DorisWriterOptions(super.getPluginJobConf()); + if (options.isWildcardColumn()) { + Connection conn = DBUtil.getConnection(DataBaseType.MySql, options.getJdbcUrl(), options.getUsername(), options.getPassword()); + List columns = DorisUtil.getDorisTableColumns(conn, options.getDatabase(), options.getTable()); + options.setInfoCchemaColumns(columns); } - final List renderedSqls = new ArrayList<>(); - for (final String sql : preOrPostSqls) { - if (!Strings.isNullOrEmpty(sql)) { - renderedSqls.add(sql.replace(Constant.TABLE_NAME_PLACEHOLDER, tableName)); - } - } - return renderedSqls; + writerManager = new DorisWriterManager(options); + rowSerializer = DorisSerializerFactory.createSerializer(options); } - private void executeSqls(final Connection conn, final List sqls) { - Statement stmt = null; - String currentSql = null; + @Override + public void prepare() { + } + + public void startWrite(RecordReceiver recordReceiver) { try { - stmt = conn.createStatement(); - for (String s : sqls) { - final String sql = currentSql = s; - DBUtil.executeSqlWithoutResultSet(stmt, sql); + Record record; + while ((record = recordReceiver.getFromReader()) != null) { + if (record.getColumnNumber() != options.getColumns().size()) { + throw DataXException + .asDataXException( + DBUtilErrorCode.CONF_ERROR, + String.format( + "列配置信息有错误. 因为您配置的任务中,源头读取字段数:%s 与 目的表要写入的字段数:%s 不相等. 请检查您的配置并作出修改.", + record.getColumnNumber(), + options.getColumns().size())); + } + writerManager.writeRecord(rowSerializer.serialize(record)); } } catch (Exception e) { - throw RdbmsException.asQueryException(DataBaseType.MySql, e, currentSql, null, null); - } finally { - DBUtil.closeDBResources(null, stmt, null); + throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, e); } } - private void preCheckPrePareSQL(final Key keys) { - final String table = keys.getTable(); - final List preSqls = keys.getPreSqlList(); - final List renderedPreSqls = renderPreOrPostSqls(preSqls, table); - if (!renderedPreSqls.isEmpty()) { - LOG.info("prepare check preSqls:[{}].", String.join(";", renderedPreSqls)); - for (final String sql : renderedPreSqls) { - try { - DBUtil.sqlValid(sql, DataBaseType.MySql); - } catch (ParserException e) { - throw RdbmsException.asPreSQLParserException(DataBaseType.MySql, e, sql); - } - } + @Override + public void post() { + try { + writerManager.close(); + } catch (Exception e) { + throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, e); } } - private void preCheckPostSQL(final Key keys) { - final String table = keys.getTable(); - final List postSqls = keys.getPostSqlList(); - final List renderedPostSqls = renderPreOrPostSqls(postSqls, table); - if (!renderedPostSqls.isEmpty()) { - LOG.info("prepare check postSqls:[{}].", String.join(";", renderedPostSqls)); - for (final String sql : renderedPostSqls) { - try { - DBUtil.sqlValid(sql, DataBaseType.MySql); - } catch (ParserException e) { - throw RdbmsException.asPostSQLParserException(DataBaseType.MySql, e, sql); - } - } - } - } + @Override + public void destroy() {} + @Override + public boolean supportFailOver(){ + return false; + } } } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java deleted file mode 100644 index ba5d6742..00000000 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterEmitter.java +++ /dev/null @@ -1,243 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package com.alibaba.datax.plugin.writer.doriswriter; - -import com.alibaba.datax.common.exception.DataXException; -import com.alibaba.datax.common.util.Configuration; -import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; -import com.alibaba.fastjson.JSON; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import org.apache.commons.lang3.StringUtils; -import org.apache.http.HttpEntity; -import org.apache.http.HttpHeaders; -import org.apache.http.HttpRequest; -import org.apache.http.HttpResponse; -import org.apache.http.HttpStatus; -import org.apache.http.ProtocolException; -import org.apache.http.client.config.RequestConfig; -import org.apache.http.client.methods.CloseableHttpResponse; -import org.apache.http.client.methods.HttpGet; -import org.apache.http.client.methods.HttpHead; -import org.apache.http.client.methods.HttpPut; -import org.apache.http.client.methods.HttpUriRequest; -import org.apache.http.client.methods.RequestBuilder; -import org.apache.http.entity.ByteArrayEntity; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.impl.client.DefaultRedirectStrategy; -import org.apache.http.impl.client.HttpClientBuilder; -import org.apache.http.impl.client.HttpClients; -import org.apache.http.protocol.HttpContext; -import org.apache.http.util.EntityUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.net.URI; -import java.util.Base64; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.net.URLEncoder; - -// Used to load batch of rows to Doris using stream load -public class DorisWriterEmitter { - private static final Logger LOG = LoggerFactory.getLogger(DorisWriterEmitter.class); - private final Key keys; - private int hostPos = 0; - private List targetHosts = Lists.newArrayList(); - - private RequestConfig requestConfig; - - public DorisWriterEmitter(final Key keys) { - this.keys = keys; - initHostList(); - initRequestConfig(); - } - - private void initRequestConfig() { - requestConfig = RequestConfig.custom().setConnectTimeout(this.keys.getConnectTimeout()).build(); - } - - // get target host from config - private void initHostList() { - List hosts = this.keys.getBeLoadUrlList(); - if (hosts == null || hosts.isEmpty()) { - hosts = this.keys.getFeLoadUrlList(); - } - if (hosts == null || hosts.isEmpty()) { - DataXException.asDataXException(DBUtilErrorCode.CONF_ERROR, - "Either beLoadUrl or feLoadUrl must be set"); - } - for (String beHost : hosts) { - targetHosts.add("http://" + beHost); - } - } - - public void emit(final DorisFlushBatch flushData) { - String host = this.getAvailableHost(); - for (int i = 0; i <= this.keys.getMaxRetries(); i++) { - try { - doStreamLoad(flushData, host); - return; - } catch (DataXException ex) { - if (i >= this.keys.getMaxRetries()) { - throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, ex); - } - LOG.error("StreamLoad error, switch host {} and retry: ", host, ex); - host = this.getAvailableHost(); - } - } - } - - /** - * execute doris stream load - */ - private void doStreamLoad(final DorisFlushBatch flushData, String host) { - long start = System.currentTimeMillis(); - if (StringUtils.isEmpty(host)) { - throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, "None of the load url can be connected."); - } - String loadUrl = host + "/api/" + this.keys.getDatabase() + "/" + this.keys.getTable() + "/_stream_load"; - // do http put request and get response - final Map loadResult; - try { - loadResult = this.doHttpPut(loadUrl, flushData); - } catch (IOException e) { - throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, e); - } - - long cost = System.currentTimeMillis() - start; - LOG.info("StreamLoad response: " + JSON.toJSONString(loadResult) + ", cost(ms): " + cost); - final String keyStatus = "Status"; - if (null == loadResult || !loadResult.containsKey(keyStatus)) { - throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, "Unable to flush data to doris: unknown result status."); - } - if (loadResult.get(keyStatus).equals("Fail")) { - throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, "Failed to flush data to doris.\n" + JSON.toJSONString(loadResult)); - } - } - - /** - * loop to get target host - * - * @return - */ - private String getAvailableHost() { - if (this.hostPos >= targetHosts.size()) { - this.hostPos = 0; - } - - while (this.hostPos < targetHosts.size()) { - final String host = targetHosts.get(hostPos); - ++this.hostPos; - return host; - } - - return null; - } - - private Map doHttpPut(final String loadUrl, final DorisFlushBatch flushBatch) throws IOException { - LOG.info(String.format("Executing stream load to: '%s', size: %s, rows: %d", - loadUrl, flushBatch.getSize(), flushBatch.getRows())); - - final HttpClientBuilder httpClientBuilder = HttpClients.custom().setRedirectStrategy(new DefaultRedirectStrategy() { - @Override - protected boolean isRedirectable(final String method) { - return true; - } - - @Override - public HttpUriRequest getRedirect(HttpRequest request, HttpResponse response, HttpContext context) throws ProtocolException { - URI uri = this.getLocationURI(request, response, context); - String method = request.getRequestLine().getMethod(); - if (method.equalsIgnoreCase("HEAD")) { - return new HttpHead(uri); - } else if (method.equalsIgnoreCase("GET")) { - return new HttpGet(uri); - } else { - int status = response.getStatusLine().getStatusCode(); - return (HttpUriRequest) (status == 307 ? RequestBuilder.copy(request).setUri(uri).build() : new HttpGet(uri)); - } - } - }); - - try (final CloseableHttpClient httpclient = httpClientBuilder.build()) { - final HttpPut httpPut = new HttpPut(loadUrl); - final List cols = this.keys.getColumns(); - if (null != cols && !cols.isEmpty()) { - httpPut.setHeader("columns", String.join(",", cols.stream().map(item -> String.format("`%s`", item.trim().replace("`", ""))).collect(Collectors.toList()))); - } - - //set default header - setDefaultHeader(httpPut); - // put custom loadProps to http header - final Map loadProps = this.keys.getLoadProps(); - if (null != loadProps) { - for (final Map.Entry entry : loadProps.entrySet()) { - httpPut.setHeader(entry.getKey(), String.valueOf(entry.getValue())); - } - } - - // set other required headers - httpPut.setHeader(HttpHeaders.EXPECT, "100-continue"); - httpPut.setHeader(HttpHeaders.AUTHORIZATION, this.getBasicAuthHeader(this.keys.getUsername(), this.keys.getPassword())); - httpPut.setHeader("label", flushBatch.getLabel()); - - // Use ByteArrayEntity instead of StringEntity to handle Chinese correctly - httpPut.setEntity(new ByteArrayEntity(flushBatch.getData().getBytes())); - httpPut.setConfig(requestConfig); - - try (final CloseableHttpResponse resp = httpclient.execute(httpPut)) { - final int code = resp.getStatusLine().getStatusCode(); - if (HttpStatus.SC_OK != code) { - LOG.warn("Request failed with code:{}", code); - return null; - } - final HttpEntity respEntity = resp.getEntity(); - if (null == respEntity) { - LOG.warn("Request failed with empty response."); - return null; - } - return (Map) JSON.parse(EntityUtils.toString(respEntity)); - } - } - } - - /** - * Set default request headers in json and csv formats. - * csv default delimiters are \x01 and \x02 - */ - private void setDefaultHeader(HttpPut httpPut) { - if (Key.DEFAULT_FORMAT_CSV.equalsIgnoreCase(this.keys.getFormat())) { - httpPut.setHeader("line_delimiter", this.keys.getLineDelimiter()); - httpPut.setHeader("column_separator", this.keys.getColumnSeparator()); - } else { - httpPut.setHeader("format", "json"); - httpPut.setHeader("strip_outer_array", "true"); - httpPut.setHeader("fuzzy_parse", "true"); - } - } - - private String getBasicAuthHeader(final String username, final String password) { - final String auth = username + ":" + password; - final byte[] encodedAuth = Base64.getEncoder().encode(auth.getBytes()); - return "Basic " + new String(encodedAuth); - } - -} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterManager.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterManager.java new file mode 100644 index 00000000..df034076 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterManager.java @@ -0,0 +1,192 @@ +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.google.common.base.Strings; +import org.apache.commons.lang3.concurrent.BasicThreadFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +public class DorisWriterManager { + + private static final Logger LOG = LoggerFactory.getLogger(DorisWriterManager.class); + + private final DorisStreamLoadVisitor visitor; + private final DorisWriterOptions options; + private final List buffer = new ArrayList<> (); + private int batchCount = 0; + private long batchSize = 0; + private volatile boolean closed = false; + private volatile Exception flushException; + private final LinkedBlockingDeque flushQueue; + private ScheduledExecutorService scheduler; + private ScheduledFuture scheduledFuture; + + public DorisWriterManager(DorisWriterOptions options) { + this.options = options; + this.visitor = new DorisStreamLoadVisitor(options); + flushQueue = new LinkedBlockingDeque<>(options.getFlushQueueLength()); + this.startScheduler(); + this.startAsyncFlushing(); + } + + public void startScheduler() { + stopScheduler(); + this.scheduler = Executors.newScheduledThreadPool(1, new BasicThreadFactory.Builder().namingPattern("Doris-interval-flush").daemon(true).build()); + this.scheduledFuture = this.scheduler.schedule(() -> { + synchronized (DorisWriterManager.this) { + if (!closed) { + try { + String label = createBatchLabel(); + LOG.info(String.format("Doris interval Sinking triggered: label[%s].", label)); + if (batchCount == 0) { + startScheduler(); + } + flush(label, false); + } catch (Exception e) { + flushException = e; + } + } + } + }, options.getFlushInterval(), TimeUnit.MILLISECONDS); + } + + public void stopScheduler() { + if (this.scheduledFuture != null) { + scheduledFuture.cancel(false); + this.scheduler.shutdown(); + } + } + + public final synchronized void writeRecord(String record) throws IOException { + checkFlushException(); + try { + byte[] bts = record.getBytes(StandardCharsets.UTF_8); + buffer.add(bts); + batchCount++; + batchSize += bts.length; + if (batchCount >= options.getBatchRows() || batchSize >= options.getBatchSize()) { + String label = createBatchLabel(); + LOG.debug(String.format("Doris buffer Sinking triggered: rows[%d] label[%s].", batchCount, label)); + flush(label, false); + } + } catch (Exception e) { + throw new IOException("Writing records to Doris failed.", e); + } + } + + public synchronized void flush(String label, boolean waitUtilDone) throws Exception { + checkFlushException(); + if (batchCount == 0) { + if (waitUtilDone) { + waitAsyncFlushingDone(); + } + return; + } + flushQueue.put(new DorisWriterTuple(label, batchSize, new ArrayList<>(buffer))); + if (waitUtilDone) { + // wait the last flush + waitAsyncFlushingDone(); + } + buffer.clear(); + batchCount = 0; + batchSize = 0; + } + + public synchronized void close() { + if (!closed) { + closed = true; + try { + String label = createBatchLabel(); + if (batchCount > 0) LOG.debug(String.format("Doris Sink is about to close: label[%s].", label)); + flush(label, true); + } catch (Exception e) { + throw new RuntimeException("Writing records to Doris failed.", e); + } + } + checkFlushException(); + } + + public String createBatchLabel() { + StringBuilder sb = new StringBuilder(); + if (! Strings.isNullOrEmpty(options.getLabelPrefix())) { + sb.append(options.getLabelPrefix()); + } + return sb.append(UUID.randomUUID().toString()) + .toString(); + } + + private void startAsyncFlushing() { + // start flush thread + Thread flushThread = new Thread(new Runnable(){ + public void run() { + while(true) { + try { + asyncFlush(); + } catch (Exception e) { + flushException = e; + } + } + } + }); + flushThread.setDaemon(true); + flushThread.start(); + } + + private void waitAsyncFlushingDone() throws InterruptedException { + // wait previous flushings + for (int i = 0; i <= options.getFlushQueueLength(); i++) { + flushQueue.put(new DorisWriterTuple("", 0l, null)); + } + checkFlushException(); + } + + private void asyncFlush() throws Exception { + DorisWriterTuple flushData = flushQueue.take(); + if (Strings.isNullOrEmpty(flushData.getLabel())) { + return; + } + stopScheduler(); + LOG.debug(String.format("Async stream load: rows[%d] bytes[%d] label[%s].", flushData.getRows().size(), flushData.getBytes(), flushData.getLabel())); + for (int i = 0; i <= options.getMaxRetries(); i++) { + try { + // flush to Doris with stream load + visitor.streamLoad(flushData); + LOG.info(String.format("Async stream load finished: label[%s].", flushData.getLabel())); + startScheduler(); + break; + } catch (Exception e) { + LOG.warn("Failed to flush batch data to Doris, retry times = {}", i, e); + if (i >= options.getMaxRetries()) { + throw new IOException(e); + } + if (e instanceof DorisStreamLoadExcetion && ((DorisStreamLoadExcetion)e).needReCreateLabel()) { + String newLabel = createBatchLabel(); + LOG.warn(String.format("Batch label changed from [%s] to [%s]", flushData.getLabel(), newLabel)); + flushData.setLabel(newLabel); + } + try { + Thread.sleep(1000l * Math.min(i + 1, 10)); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + throw new IOException("Unable to flush, interrupted while doing another attempt", e); + } + } + } + } + + private void checkFlushException() { + if (flushException != null) { + throw new RuntimeException("Writing records to Doris failed.", flushException); + } + } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterOptions.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterOptions.java new file mode 100644 index 00000000..b4395d2e --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterOptions.java @@ -0,0 +1,174 @@ +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class DorisWriterOptions implements Serializable { + + private static final long serialVersionUID = 1l; + private static final long KILO_BYTES_SCALE = 1024l; + private static final long MEGA_BYTES_SCALE = KILO_BYTES_SCALE * KILO_BYTES_SCALE; + private static final int MAX_RETRIES = 3; + private static final int BATCH_ROWS = 500000; + private static final long BATCH_BYTES = 90 * MEGA_BYTES_SCALE; + private static final long FLUSH_INTERVAL = 300000; + + private static final String KEY_LOAD_PROPS_FORMAT = "format"; + public enum StreamLoadFormat { + CSV, JSON; + } + + private static final String KEY_USERNAME = "username"; + private static final String KEY_PASSWORD = "password"; + private static final String KEY_DATABASE = "connection[0].database"; + private static final String KEY_TABLE = "connection[0].table[0]"; + private static final String KEY_COLUMN = "column"; + private static final String KEY_PRE_SQL = "preSql"; + private static final String KEY_POST_SQL = "postSql"; + private static final String KEY_JDBC_URL = "connection[0].jdbcUrl"; + private static final String KEY_LABEL_PREFIX = "labelPrefix"; + private static final String KEY_MAX_BATCH_ROWS = "maxBatchRows"; + private static final String KEY_MAX_BATCH_SIZE = "maxBatchSize"; + private static final String KEY_FLUSH_INTERVAL = "flushInterval"; + private static final String KEY_LOAD_URL = "loadUrl"; + private static final String KEY_FLUSH_QUEUE_LENGTH = "flushQueueLength"; + private static final String KEY_LOAD_PROPS = "loadProps"; + + private final Configuration options; + private List infoCchemaColumns; + private List userSetColumns; + private boolean isWildcardColumn; + + public DorisWriterOptions(Configuration options) { + this.options = options; + this.userSetColumns = options.getList(KEY_COLUMN, String.class).stream().map(str -> str.replace("`", "")).collect(Collectors.toList()); + if (1 == options.getList(KEY_COLUMN, String.class).size() && "*".trim().equals(options.getList(KEY_COLUMN, String.class).get(0))) { + this.isWildcardColumn = true; + } + } + + public void doPretreatment() { + validateRequired(); + validateStreamLoadUrl(); + } + + public String getJdbcUrl() { + return options.getString(KEY_JDBC_URL); + } + + public String getDatabase() { + return options.getString(KEY_DATABASE); + } + + public String getTable() { + return options.getString(KEY_TABLE); + } + + public String getUsername() { + return options.getString(KEY_USERNAME); + } + + public String getPassword() { + return options.getString(KEY_PASSWORD); + } + + public String getLabelPrefix() { + return options.getString(KEY_LABEL_PREFIX); + } + + public List getLoadUrlList() { + return options.getList(KEY_LOAD_URL, String.class); + } + + public List getColumns() { + if (isWildcardColumn) { + return this.infoCchemaColumns; + } + return this.userSetColumns; + } + + public boolean isWildcardColumn() { + return this.isWildcardColumn; + } + + public void setInfoCchemaColumns(List cols) { + this.infoCchemaColumns = cols; + } + + public List getPreSqlList() { + return options.getList(KEY_PRE_SQL, String.class); + } + + public List getPostSqlList() { + return options.getList(KEY_POST_SQL, String.class); + } + + public Map getLoadProps() { + return options.getMap(KEY_LOAD_PROPS); + } + + public int getMaxRetries() { + return MAX_RETRIES; + } + + public int getBatchRows() { + Integer rows = options.getInt(KEY_MAX_BATCH_ROWS); + return null == rows ? BATCH_ROWS : rows; + } + + public long getBatchSize() { + Long size = options.getLong(KEY_MAX_BATCH_SIZE); + return null == size ? BATCH_BYTES : size; + } + + public long getFlushInterval() { + Long interval = options.getLong(KEY_FLUSH_INTERVAL); + return null == interval ? FLUSH_INTERVAL : interval; + } + + public int getFlushQueueLength() { + Integer len = options.getInt(KEY_FLUSH_QUEUE_LENGTH); + return null == len ? 1 : len; + } + + public StreamLoadFormat getStreamLoadFormat() { + Map loadProps = getLoadProps(); + if (null == loadProps) { + return StreamLoadFormat.CSV; + } + if (loadProps.containsKey(KEY_LOAD_PROPS_FORMAT) + && StreamLoadFormat.JSON.name().equalsIgnoreCase(String.valueOf(loadProps.get(KEY_LOAD_PROPS_FORMAT)))) { + return StreamLoadFormat.JSON; + } + return StreamLoadFormat.CSV; + } + + private void validateStreamLoadUrl() { + List urlList = getLoadUrlList(); + for (String host : urlList) { + if (host.split(":").length < 2) { + throw DataXException.asDataXException(DBUtilErrorCode.CONF_ERROR, + "loadUrl的格式不正确,请输入 `fe_ip:fe_http_ip;fe_ip:fe_http_ip`。"); + } + } + } + + private void validateRequired() { + final String[] requiredOptionKeys = new String[]{ + KEY_USERNAME, + KEY_DATABASE, + KEY_TABLE, + KEY_COLUMN, + KEY_LOAD_URL + }; + for (String optionKey : requiredOptionKeys) { + options.getNecessaryValue(optionKey, DBUtilErrorCode.REQUIRED_VALUE); + } + } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterTuple.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterTuple.java new file mode 100644 index 00000000..0bc52daf --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterTuple.java @@ -0,0 +1,20 @@ +package com.alibaba.datax.plugin.writer.doriswriter; + +import java.util.List; + +public class DorisWriterTuple { + private String label; + private Long bytes; + private List rows; + + public DorisWriterTuple(String label,Long bytes,List rows){ + this.label = label; + this.rows = rows; + this.bytes = bytes; + } + + public String getLabel() { return label; } + public void setLabel(String label) { this.label = label; } + public Long getBytes() { return bytes; } + public List getRows() { return rows; } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/EscapeHandler.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/EscapeHandler.java deleted file mode 100644 index 91b0fbdb..00000000 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/EscapeHandler.java +++ /dev/null @@ -1,42 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package com.alibaba.datax.plugin.writer.doriswriter; - -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -/** - * Handler for escape in properties. - */ -public class EscapeHandler { - public static final String ESCAPE_DELIMITERS_FLAGS = "\\x"; - public static final Pattern ESCAPE_PATTERN = Pattern.compile("\\\\x([0-9|a-f|A-F]{2})"); - - public static String escapeString(String source) { - if (source.startsWith(ESCAPE_DELIMITERS_FLAGS)) { - Matcher m = ESCAPE_PATTERN.matcher(source); - StringBuffer buf = new StringBuffer(); - while (m.find()) { - m.appendReplacement(buf, String.format("%s", (char) Integer.parseInt(m.group(1), 16))); - } - m.appendTail(buf); - return buf.toString(); - } - return source; - } -} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java deleted file mode 100644 index a70f6650..00000000 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Key.java +++ /dev/null @@ -1,188 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package com.alibaba.datax.plugin.writer.doriswriter; - -import com.alibaba.datax.common.exception.DataXException; -import com.alibaba.datax.common.util.Configuration; -import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; -import com.google.common.base.Strings; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class Key implements Serializable { - public static final String FE_LOAD_URL = "loadUrl"; - public static final String BE_LOAD_URL = "beLoadUrl"; - public static final String JDBC_URL = "connection[0].jdbcUrl"; - - public static final String DATABASE = "connection[0].selectedDatabase"; - public static final String TABLE = "connection[0].table[0]"; - public static final String COLUMN = "column"; - - public static final String USERNAME = "username"; - public static final String PASSWORD = "password"; - - public static final String PRE_SQL = "preSql"; - public static final String POST_SQL = "postSql"; - - public static final String LOAD_PROPS = "loadProps"; - public static final String LOAD_PROPS_LINE_DELIMITER = "line_delimiter"; - public static final String LOAD_PROPS_COLUMN_SEPARATOR = "column_separator"; - - public static final String MAX_BATCH_ROWS = "maxBatchRows"; - public static final String BATCH_BYTE_SIZE = "maxBatchSize"; - public static final String MAX_RETRIES = "maxRetries"; - public static final String LABEL_PREFIX = "labelPrefix"; - public static final String FORMAT = "format"; - public static final String CONNECT_TIMEOUT = "connectTimeout"; - private final Configuration options; - - private static final long DEFAULT_MAX_BATCH_ROWS = 500000; - - private static final long DEFAULT_BATCH_BYTE_SIZE = 90 * 1024 * 1024; - private static final int DEFAULT_MAX_RETRIES = 0; - - private static final String DEFAULT_LABEL_PREFIX = "datax_doris_writer_"; - private static final String DEFAULT_COLUMN_SEPARATOR = "\\x01"; - private static final String DEFAULT_LINE_DELIMITER = "\\x02"; - public static final String DEFAULT_FORMAT_CSV = "csv"; - private static final String DEFAULT_TIME_ZONE = "+08:00"; - private static final int DEFAULT_CONNECT_TIMEOUT = -1; - - public Key(final Configuration options) { - this.options = options; - } - - public void doPretreatment() { - this.validateRequired(); - this.validateStreamLoadUrl(); - this.validateFormat(); - } - - public String getJdbcUrl() { - return this.options.getString(JDBC_URL); - } - - public String getDatabase() { - return this.options.getString(DATABASE); - } - - public String getTable() { - return this.options.getString(TABLE); - } - - public String getUsername() { - return this.options.getString(USERNAME); - } - - public String getPassword() { - return Strings.nullToEmpty(this.options.getString(PASSWORD)); - } - - public List getBeLoadUrlList() { - return this.options.getList(BE_LOAD_URL, String.class); - } - - public List getFeLoadUrlList() { - return this.options.getList(FE_LOAD_URL, String.class); - } - - public List getColumns() { - return this.options.getList(COLUMN, String.class); - } - - public List getPreSqlList() { - return this.options.getList(PRE_SQL, String.class); - } - - public List getPostSqlList() { - return this.options.getList(POST_SQL, String.class); - } - - public Map getLoadProps() { - return this.options.getMap(LOAD_PROPS, new HashMap<>()); - } - - public long getBatchRows() { - return this.options.getLong(MAX_BATCH_ROWS, DEFAULT_MAX_BATCH_ROWS); - } - - public long getBatchByteSize() { - return this.options.getLong(BATCH_BYTE_SIZE, DEFAULT_BATCH_BYTE_SIZE); - } - - public int getMaxRetries() { - return this.options.getInt(MAX_RETRIES, DEFAULT_MAX_RETRIES); - } - - public String getLabelPrefix() { - return this.options.getString(LABEL_PREFIX, DEFAULT_LABEL_PREFIX); - } - - public String getLineDelimiter() { - return getLoadProps().getOrDefault(LOAD_PROPS_LINE_DELIMITER, DEFAULT_LINE_DELIMITER).toString(); - } - - public String getFormat() { - return this.options.getString(FORMAT, DEFAULT_FORMAT_CSV); - } - - public String getColumnSeparator() { - return getLoadProps().getOrDefault(LOAD_PROPS_COLUMN_SEPARATOR, DEFAULT_COLUMN_SEPARATOR).toString(); - } - - public int getConnectTimeout() { - return this.options.getInt(CONNECT_TIMEOUT, DEFAULT_CONNECT_TIMEOUT); - } - - - private void validateStreamLoadUrl() { - List urlList = this.getBeLoadUrlList(); - if (urlList == null) { - urlList = this.getFeLoadUrlList(); - } - if (urlList == null || urlList.isEmpty()) { - throw DataXException.asDataXException(DBUtilErrorCode.CONF_ERROR, "Either beLoadUrl or feLoadUrl must be set"); - } - - for (final String host : urlList) { - if (host.split(":").length < 2) { - throw DataXException.asDataXException(DBUtilErrorCode.CONF_ERROR, - "Invalid load url format. IF use FE hosts, should be like: fe_host:fe_http_port." - + " If use BE hosts, should be like: be_host:be_webserver_port"); - } - } - } - - private void validateFormat() { - String format = this.getFormat(); - if (!Arrays.asList("csv", "json").contains(format.toLowerCase())) { - throw DataXException.asDataXException(DBUtilErrorCode.CONF_ERROR, "format only supports csv or json"); - } - } - - private void validateRequired() { - final String[] requiredOptionKeys = new String[]{JDBC_URL, USERNAME, DATABASE, TABLE, COLUMN}; - for (final String optionKey : requiredOptionKeys) { - this.options.getNecessaryValue(optionKey, DBUtilErrorCode.REQUIRED_VALUE); - } - } -} From fc4c162cffc1de23d03621dde1655a3fcbdf1015 Mon Sep 17 00:00:00 2001 From: "jiafeng.zhang" Date: Mon, 10 Oct 2022 10:51:53 +0800 Subject: [PATCH 12/13] Refactoring the doriswriter write plugin Refactoring the doriswriter write plugin --- doriswriter/doc/doriswriter.md | 43 ++--- ...imiterParser.java => DelimiterParser.java} | 2 +- ...aseSerializer.java => DorisBaseCodec.java} | 4 +- .../{DorisSerializer.java => DorisCodec.java} | 4 +- .../writer/doriswriter/DorisCodecFactory.java | 19 ++ ...sCsvSerializer.java => DorisCsvCodec.java} | 11 +- ...sonSerializer.java => DorisJsonCodec.java} | 8 +- .../doriswriter/DorisSerializerFactory.java | 19 -- ...itor.java => DorisStreamLoadObserver.java} | 50 ++--- .../plugin/writer/doriswriter/DorisUtil.java | 4 +- .../writer/doriswriter/DorisWriter.java | 29 ++- ...Excetion.java => DorisWriterExcetion.java} | 6 +- .../doriswriter/DorisWriterManager.java | 18 +- .../doriswriter/DorisWriterOptions.java | 174 ----------------- .../datax/plugin/writer/doriswriter/Keys.java | 177 ++++++++++++++++++ ...DorisWriterTuple.java => WriterTuple.java} | 4 +- 16 files changed, 283 insertions(+), 289 deletions(-) rename doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/{DorisDelimiterParser.java => DelimiterParser.java} (98%) rename doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/{DorisBaseSerializer.java => DorisBaseCodec.java} (88%) rename doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/{DorisSerializer.java => DorisCodec.java} (59%) create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodecFactory.java rename doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/{DorisCsvSerializer.java => DorisCsvCodec.java} (63%) rename doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/{DorisJsonSerializer.java => DorisJsonCodec.java} (72%) delete mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializerFactory.java rename doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/{DorisStreamLoadVisitor.java => DorisStreamLoadObserver.java} (81%) rename doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/{DorisStreamLoadExcetion.java => DorisWriterExcetion.java} (68%) delete mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterOptions.java create mode 100644 doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Keys.java rename doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/{DorisWriterTuple.java => WriterTuple.java} (81%) diff --git a/doriswriter/doc/doriswriter.md b/doriswriter/doc/doriswriter.md index 9f8f510c..1f0dff62 100644 --- a/doriswriter/doc/doriswriter.md +++ b/doriswriter/doc/doriswriter.md @@ -43,6 +43,7 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter "password": "xxxxxx", "postSql": ["select count(1) from all_employees_info"], "preSql": [], + "flushInterval":30000, "connection": [ { "jdbcUrl": "jdbc:mysql://172.16.0.13:9030/demo", @@ -77,7 +78,7 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter * **loadUrl** - - 描述:和 **beLoadUrl** 二选一。作为 Stream Load 的连接目标。格式为 "ip:port"。其中 IP 是 FE 节点 IP,port 是 FE 节点的 http_port。可以填写多个,doriswriter 将以轮询的方式访问。 + - 描述:作为 Stream Load 的连接目标。格式为 "ip:port"。其中 IP 是 FE 节点 IP,port 是 FE 节点的 http_port。可以填写多个,多个之间使用英文状态的分号隔开:`;`,doriswriter 将以轮询的方式访问。 - 必选:是 - 默认值:无 @@ -88,23 +89,21 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter - 默认值:无 * **password** - + - 描述:访问Doris数据库的密码 - 必选:否 - 默认值:空 -* **database** - +* **connection.selectedDatabase** - 描述:需要写入的Doris数据库名称。 - 必选:是 - 默认值:无 -* **table** - - - 描述:需要写入的Doris表名称。 +* **connection.table** + - 描述:需要写入的Doris表名称。 - 必选:是 - 默认值:无 - + * **column** - 描述:目的表**需要写入数据**的字段,这些字段将作为生成的 Json 数据的字段名。字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 @@ -144,32 +143,26 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter * **labelPrefix** - - 描述:每批次导入任务的 label 前缀。最终的 label 将有 `labelPrefix + UUID + 序号` 组成 + - 描述:每批次导入任务的 label 前缀。最终的 label 将有 `labelPrefix + UUID` 组成全局唯一的 label,确保数据不会重复导入 - 必选:否 - 默认值:`datax_doris_writer_` -* **format** - - - 描述:StreamLoad数据的组装格式,支持csv和json格式。csv默认的行分隔符是\x01,列分隔符是\x02。 - - 必选:否 - - 默认值:csv - * **loadProps** - - 描述:StreamLoad 的请求参数,详情参照StreamLoad介绍页面。 + - 描述:StreamLoad 的请求参数,详情参照StreamLoad介绍页面。[Stream load - Apache Doris](https://doris.apache.org/zh-CN/docs/data-operate/import/import-way/stream-load-manual) + + 这里包括导入的数据格式:format等,导入数据格式默认我们使用csv,支持JSON,具体可以参照下面类型转换部分,也可以参照上面Stream load 官方信息 + - 必选:否 + - 默认值:无 -* **connectTimeout** - - - 描述:StreamLoad单次请求的超时时间, 单位毫秒(ms)。 - - 必选:否 - - 默认值:-1 - ### 类型转换 默认传入的数据均会被转为字符串,并以`\t`作为列分隔符,`\n`作为行分隔符,组成`csv`文件进行StreamLoad导入操作。 -如需更改列分隔符, 则正确配置 `loadProps` 即可: + +默认是csv格式导入,如需更改列分隔符, 则正确配置 `loadProps` 即可: + ```json "loadProps": { "column_separator": "\\x01", @@ -183,4 +176,6 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter "format": "json", "strip_outer_array": true } -``` \ No newline at end of file +``` + +更多信息请参照 Doris 官网:[Stream load - Apache Doris](https://doris.apache.org/zh-CN/docs/data-operate/import/import-way/stream-load-manual) \ No newline at end of file diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisDelimiterParser.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DelimiterParser.java similarity index 98% rename from doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisDelimiterParser.java rename to doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DelimiterParser.java index 5c92af9f..e84bd7dd 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisDelimiterParser.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DelimiterParser.java @@ -4,7 +4,7 @@ import com.google.common.base.Strings; import java.io.StringWriter; -public class DorisDelimiterParser { +public class DelimiterParser { private static final String HEX_STRING = "0123456789ABCDEF"; diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseSerializer.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseCodec.java similarity index 88% rename from doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseSerializer.java rename to doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseCodec.java index c9aacc6e..ee7ded56 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseSerializer.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseCodec.java @@ -2,8 +2,8 @@ package com.alibaba.datax.plugin.writer.doriswriter; import com.alibaba.datax.common.element.Column; -public class DorisBaseSerializer { - protected String fieldConvertion( Column col) { +public class DorisBaseCodec { + protected String convertionField( Column col) { if (null == col.getRawData() || Column.Type.NULL == col.getType()) { return null; } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializer.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java similarity index 59% rename from doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializer.java rename to doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java index 58759ecf..a2437a1c 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializer.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java @@ -4,7 +4,7 @@ import com.alibaba.datax.common.element.Record; import java.io.Serializable; -public interface DorisSerializer extends Serializable { +public interface DorisCodec extends Serializable { - String serialize( Record row); + String codec( Record row); } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodecFactory.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodecFactory.java new file mode 100644 index 00000000..22c4b409 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodecFactory.java @@ -0,0 +1,19 @@ +package com.alibaba.datax.plugin.writer.doriswriter; + +import java.util.Map; + +public class DorisCodecFactory { + public DorisCodecFactory (){ + + } + public static DorisCodec createCodec( Keys writerOptions) { + if ( Keys.StreamLoadFormat.CSV.equals(writerOptions.getStreamLoadFormat())) { + Map props = writerOptions.getLoadProps(); + return new DorisCsvCodec (null == props || !props.containsKey("column_separator") ? null : String.valueOf(props.get("column_separator"))); + } + if ( Keys.StreamLoadFormat.JSON.equals(writerOptions.getStreamLoadFormat())) { + return new DorisJsonCodec (writerOptions.getColumns()); + } + throw new RuntimeException("Failed to create row serializer, unsupported `format` from stream load properties."); + } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvSerializer.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java similarity index 63% rename from doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvSerializer.java rename to doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java index 93c477d8..518aa304 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvSerializer.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java @@ -2,20 +2,21 @@ package com.alibaba.datax.plugin.writer.doriswriter; import com.alibaba.datax.common.element.Record; -public class DorisCsvSerializer extends DorisBaseSerializer implements DorisSerializer{ +public class DorisCsvCodec extends DorisBaseCodec implements DorisCodec { + private static final long serialVersionUID = 1L; private final String columnSeparator; - public DorisCsvSerializer(String sp) { - this.columnSeparator = DorisDelimiterParser.parse(sp, "\t"); + public DorisCsvCodec ( String sp) { + this.columnSeparator = DelimiterParser.parse(sp, "\t"); } @Override - public String serialize( Record row) { + public String codec( Record row) { StringBuilder sb = new StringBuilder(); for (int i = 0; i < row.getColumnNumber(); i++) { - String value = fieldConvertion(row.getColumn(i)); + String value = convertionField(row.getColumn(i)); sb.append(null == value ? "\\N" : value); if (i < row.getColumnNumber() - 1) { sb.append(columnSeparator); diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonSerializer.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java similarity index 72% rename from doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonSerializer.java rename to doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java index 315e472c..e6c05733 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonSerializer.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java @@ -7,25 +7,25 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -public class DorisJsonSerializer extends DorisBaseSerializer implements DorisSerializer{ +public class DorisJsonCodec extends DorisBaseCodec implements DorisCodec { private static final long serialVersionUID = 1L; private final List fieldNames; - public DorisJsonSerializer( List fieldNames) { + public DorisJsonCodec ( List fieldNames) { this.fieldNames = fieldNames; } @Override - public String serialize( Record row) { + public String codec( Record row) { if (null == fieldNames) { return ""; } Map rowMap = new HashMap<> (fieldNames.size()); int idx = 0; for (String fieldName : fieldNames) { - rowMap.put(fieldName, fieldConvertion(row.getColumn(idx))); + rowMap.put(fieldName, convertionField(row.getColumn(idx))); idx++; } return JSON.toJSONString(rowMap); diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializerFactory.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializerFactory.java deleted file mode 100644 index 8484e5b7..00000000 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisSerializerFactory.java +++ /dev/null @@ -1,19 +0,0 @@ -package com.alibaba.datax.plugin.writer.doriswriter; - -import java.util.Map; - -public class DorisSerializerFactory { - public DorisSerializerFactory(){ - - } - public static DorisSerializer createSerializer(DorisWriterOptions writerOptions) { - if (DorisWriterOptions.StreamLoadFormat.CSV.equals(writerOptions.getStreamLoadFormat())) { - Map props = writerOptions.getLoadProps(); - return new DorisCsvSerializer(null == props || !props.containsKey("column_separator") ? null : String.valueOf(props.get("column_separator"))); - } - if (DorisWriterOptions.StreamLoadFormat.JSON.equals(writerOptions.getStreamLoadFormat())) { - return new DorisJsonSerializer(writerOptions.getColumns()); - } - throw new RuntimeException("Failed to create row serializer, unsupported `format` from stream load properties."); - } -} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadVisitor.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadObserver.java similarity index 81% rename from doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadVisitor.java rename to doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadObserver.java index a18940cb..3e4db6cf 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadVisitor.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadObserver.java @@ -27,10 +27,10 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -public class DorisStreamLoadVisitor { - private static final Logger LOG = LoggerFactory.getLogger(DorisStreamLoadVisitor.class); +public class DorisStreamLoadObserver { + private static final Logger LOG = LoggerFactory.getLogger(DorisStreamLoadObserver.class); - private DorisWriterOptions options; + private Keys options; private long pos; private static final String RESULT_FAILED = "Fail"; @@ -42,14 +42,14 @@ public class DorisStreamLoadVisitor { private static final String RESULT_LABEL_UNKNOWN = "UNKNOWN"; - public DorisStreamLoadVisitor(DorisWriterOptions options){ + public DorisStreamLoadObserver ( Keys options){ this.options = options; } - public void streamLoad(DorisWriterTuple data) throws Exception { - String host = getAvailableHost(); + public void streamLoad(WriterTuple data) throws Exception { + String host = getLoadHost(); if(host == null){ - throw new IOException ("None of the host in `load_url` could be connected."); + throw new IOException ("load_url cannot be empty, or the host cannot connect.Please check your configuration."); } String loadUrl = new StringBuilder(host) .append("/api/") @@ -58,25 +58,25 @@ public class DorisStreamLoadVisitor { .append(options.getTable()) .append("/_stream_load") .toString(); - LOG.debug(String.format("Start to join batch data: rows[%d] bytes[%d] label[%s].", data.getRows().size(), data.getBytes(), data.getLabel())); - Map loadResult = doHttpPut(loadUrl, data.getLabel(), joinRows(data.getRows(), data.getBytes().intValue())); + LOG.info("Start to join batch data: rows[{}] bytes[{}] label[{}].", data.getRows().size(), data.getBytes(), data.getLabel()); + Map loadResult = put(loadUrl, data.getLabel(), addRows(data.getRows(), data.getBytes().intValue())); + LOG.info("StreamLoad response :{}",JSON.toJSONString(loadResult)); final String keyStatus = "Status"; if (null == loadResult || !loadResult.containsKey(keyStatus)) { throw new IOException("Unable to flush data to Doris: unknown result status."); } - LOG.debug(new StringBuilder("StreamLoad response:\n").append(JSON.toJSONString(loadResult)).toString()); + LOG.debug("StreamLoad response:{}",JSON.toJSONString(loadResult)); if (RESULT_FAILED.equals(loadResult.get(keyStatus))) { throw new IOException( new StringBuilder("Failed to flush data to Doris.\n").append(JSON.toJSONString(loadResult)).toString() ); } else if (RESULT_LABEL_EXISTED.equals(loadResult.get(keyStatus))) { - LOG.debug(new StringBuilder("StreamLoad response:\n").append(JSON.toJSONString(loadResult)).toString()); - // has to block-checking the state to get the final result - checkLabelState(host, data.getLabel()); + LOG.debug("StreamLoad response:{}",JSON.toJSONString(loadResult)); + checkStreamLoadState(host, data.getLabel()); } } - private void checkLabelState(String host, String label) throws IOException { + private void checkStreamLoadState(String host, String label) throws IOException { int idx = 0; while(true) { try { @@ -109,7 +109,7 @@ public class DorisStreamLoadVisitor { case RESULT_LABEL_PREPARE: continue; case RESULT_LABEL_ABORTED: - throw new DorisStreamLoadExcetion(String.format("Failed to flush data to Doris, Error " + + throw new DorisWriterExcetion (String.format("Failed to flush data to Doris, Error " + "label[%s] state[%s]\n", label, labelState), null, true); case RESULT_LABEL_UNKNOWN: default: @@ -121,10 +121,10 @@ public class DorisStreamLoadVisitor { } } - private byte[] joinRows(List rows, int totalBytes) { - if (DorisWriterOptions.StreamLoadFormat.CSV.equals(options.getStreamLoadFormat())) { + private byte[] addRows(List rows, int totalBytes) { + if (Keys.StreamLoadFormat.CSV.equals(options.getStreamLoadFormat())) { Map props = (options.getLoadProps() == null ? new HashMap<> () : options.getLoadProps()); - byte[] lineDelimiter = DorisDelimiterParser.parse((String)props.get("row_delimiter"), "\n").getBytes(StandardCharsets.UTF_8); + byte[] lineDelimiter = DelimiterParser.parse((String)props.get("row_delimiter"), "\n").getBytes(StandardCharsets.UTF_8); ByteBuffer bos = ByteBuffer.allocate(totalBytes + rows.size() * lineDelimiter.length); for (byte[] row : rows) { bos.put(row); @@ -133,7 +133,7 @@ public class DorisStreamLoadVisitor { return bos.array(); } - if (DorisWriterOptions.StreamLoadFormat.JSON.equals(options.getStreamLoadFormat())) { + if (Keys.StreamLoadFormat.JSON.equals(options.getStreamLoadFormat())) { ByteBuffer bos = ByteBuffer.allocate(totalBytes + (rows.isEmpty() ? 2 : rows.size() + 1)); bos.put("[".getBytes(StandardCharsets.UTF_8)); byte[] jsonDelimiter = ",".getBytes(StandardCharsets.UTF_8); @@ -150,7 +150,7 @@ public class DorisStreamLoadVisitor { } throw new RuntimeException("Failed to join rows data, unsupported `format` from stream load properties:"); } - private Map doHttpPut(String loadUrl, String label, byte[] data) throws IOException { + private Map put(String loadUrl, String label, byte[] data) throws IOException { LOG.info(String.format("Executing stream load to: '%s', size: '%s'", loadUrl, data.length)); final HttpClientBuilder httpClientBuilder = HttpClients.custom() .setRedirectStrategy(new DefaultRedirectStrategy () { @@ -162,7 +162,7 @@ public class DorisStreamLoadVisitor { try ( CloseableHttpClient httpclient = httpClientBuilder.build()) { HttpPut httpPut = new HttpPut(loadUrl); List cols = options.getColumns(); - if (null != cols && !cols.isEmpty() && DorisWriterOptions.StreamLoadFormat.CSV.equals(options.getStreamLoadFormat())) { + if (null != cols && !cols.isEmpty() && Keys.StreamLoadFormat.CSV.equals(options.getStreamLoadFormat())) { httpPut.setHeader("columns", String.join(",", cols.stream().map(f -> String.format("`%s`", f)).collect(Collectors.toList()))); } if (null != options.getLoadProps()) { @@ -205,19 +205,19 @@ public class DorisStreamLoadVisitor { return respEntity; } - private String getAvailableHost() { + private String getLoadHost() { List hostList = options.getLoadUrlList(); long tmp = pos + hostList.size(); for (; pos < tmp; pos++) { String host = new StringBuilder("http://").append(hostList.get((int) (pos % hostList.size()))).toString(); - if (tryHttpConnection(host)) { + if (checkConnection(host)) { return host; } } return null; } - private boolean tryHttpConnection(String host) { + private boolean checkConnection(String host) { try { URL url = new URL(host); HttpURLConnection co = (HttpURLConnection) url.openConnection(); @@ -226,7 +226,7 @@ public class DorisStreamLoadVisitor { co.disconnect(); return true; } catch (Exception e1) { - LOG.warn("Failed to connect to address:{}", host, e1); + e1.printStackTrace(); return false; } } diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisUtil.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisUtil.java index 7c7d9a92..5f5a6f34 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisUtil.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisUtil.java @@ -71,7 +71,7 @@ public class DorisUtil { } } - public static void preCheckPrePareSQL(DorisWriterOptions options) { + public static void preCheckPrePareSQL( Keys options) { String table = options.getTable(); List preSqls = options.getPreSqlList(); List renderedPreSqls = DorisUtil.renderPreOrPostSqls(preSqls, table); @@ -87,7 +87,7 @@ public class DorisUtil { } } - public static void preCheckPostSQL(DorisWriterOptions options) { + public static void preCheckPostSQL( Keys options) { String table = options.getTable(); List postSqls = options.getPostSqlList(); List renderedPostSqls = DorisUtil.renderPreOrPostSqls(postSqls, table); diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java index f16cd354..b44d5440 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java @@ -25,19 +25,12 @@ import com.alibaba.datax.common.util.Configuration; import com.alibaba.datax.plugin.rdbms.util.DBUtil; import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; import com.alibaba.datax.plugin.rdbms.util.DataBaseType; -import com.alibaba.datax.plugin.rdbms.util.RdbmsException; -import com.alibaba.datax.plugin.rdbms.writer.Constant; -import com.alibaba.druid.sql.parser.ParserException; -import com.google.common.base.Strings; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.sql.Connection; -import java.sql.Statement; import java.util.ArrayList; -import java.util.Collections; import java.util.List; -import java.util.UUID; /** * doris data writer @@ -48,12 +41,12 @@ public class DorisWriter extends Writer { private static final Logger LOG = LoggerFactory.getLogger(Job.class); private Configuration originalConfig = null; - private DorisWriterOptions options; + private Keys options; @Override public void init() { this.originalConfig = super.getPluginJobConf(); - options = new DorisWriterOptions(super.getPluginJobConf()); + options = new Keys (super.getPluginJobConf()); options.doPretreatment(); } @@ -92,11 +85,10 @@ public class DorisWriter extends Writer { String username = options.getUsername(); String password = options.getPassword(); String jdbcUrl = options.getJdbcUrl(); - LOG.info("userName :{},password:{},jdbcUrl:{}.", username,password,jdbcUrl); List renderedPostSqls = DorisUtil.renderPreOrPostSqls(options.getPostSqlList(), options.getTable()); if (null != renderedPostSqls && !renderedPostSqls.isEmpty()) { Connection conn = DBUtil.getConnection(DataBaseType.MySql, jdbcUrl, username, password); - LOG.info("Begin to execute preSqls:[{}]. context info:{}.", String.join(";", renderedPostSqls), jdbcUrl); + LOG.info("Start to execute preSqls:[{}]. context info:{}.", String.join(";", renderedPostSqls), jdbcUrl); DorisUtil.executeSqls(conn, renderedPostSqls); DBUtil.closeDBResources(null, null, conn); } @@ -110,19 +102,19 @@ public class DorisWriter extends Writer { public static class Task extends Writer.Task { private DorisWriterManager writerManager; - private DorisWriterOptions options; - private DorisSerializer rowSerializer; + private Keys options; + private DorisCodec rowCodec; @Override public void init() { - options = new DorisWriterOptions(super.getPluginJobConf()); + options = new Keys (super.getPluginJobConf()); if (options.isWildcardColumn()) { Connection conn = DBUtil.getConnection(DataBaseType.MySql, options.getJdbcUrl(), options.getUsername(), options.getPassword()); List columns = DorisUtil.getDorisTableColumns(conn, options.getDatabase(), options.getTable()); options.setInfoCchemaColumns(columns); } writerManager = new DorisWriterManager(options); - rowSerializer = DorisSerializerFactory.createSerializer(options); + rowCodec = DorisCodecFactory.createCodec(options); } @Override @@ -138,11 +130,14 @@ public class DorisWriter extends Writer { .asDataXException( DBUtilErrorCode.CONF_ERROR, String.format( - "列配置信息有错误. 因为您配置的任务中,源头读取字段数:%s 与 目的表要写入的字段数:%s 不相等. 请检查您的配置并作出修改.", + "There is an error in the column configuration information. " + + "This is because you have configured a task where the number of fields to be read from the source:%s " + + "is not equal to the number of fields to be written to the destination table:%s. " + + "Please check your configuration and make changes.", record.getColumnNumber(), options.getColumns().size())); } - writerManager.writeRecord(rowSerializer.serialize(record)); + writerManager.writeRecord(rowCodec.codec(record)); } } catch (Exception e) { throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, e); diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadExcetion.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterExcetion.java similarity index 68% rename from doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadExcetion.java rename to doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterExcetion.java index dc19b209..7797d79f 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadExcetion.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterExcetion.java @@ -3,17 +3,17 @@ package com.alibaba.datax.plugin.writer.doriswriter; import java.io.IOException; import java.util.Map; -public class DorisStreamLoadExcetion extends IOException { +public class DorisWriterExcetion extends IOException { private final Map response; private boolean reCreateLabel; - public DorisStreamLoadExcetion(String message, Map response) { + public DorisWriterExcetion ( String message, Map response) { super(message); this.response = response; } - public DorisStreamLoadExcetion(String message, Map response, boolean reCreateLabel) { + public DorisWriterExcetion ( String message, Map response, boolean reCreateLabel) { super(message); this.response = response; this.reCreateLabel = reCreateLabel; diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterManager.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterManager.java index df034076..f0ba6b52 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterManager.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterManager.java @@ -20,20 +20,20 @@ public class DorisWriterManager { private static final Logger LOG = LoggerFactory.getLogger(DorisWriterManager.class); - private final DorisStreamLoadVisitor visitor; - private final DorisWriterOptions options; + private final DorisStreamLoadObserver visitor; + private final Keys options; private final List buffer = new ArrayList<> (); private int batchCount = 0; private long batchSize = 0; private volatile boolean closed = false; private volatile Exception flushException; - private final LinkedBlockingDeque flushQueue; + private final LinkedBlockingDeque< WriterTuple > flushQueue; private ScheduledExecutorService scheduler; private ScheduledFuture scheduledFuture; - public DorisWriterManager(DorisWriterOptions options) { + public DorisWriterManager( Keys options) { this.options = options; - this.visitor = new DorisStreamLoadVisitor(options); + this.visitor = new DorisStreamLoadObserver (options); flushQueue = new LinkedBlockingDeque<>(options.getFlushQueueLength()); this.startScheduler(); this.startAsyncFlushing(); @@ -92,7 +92,7 @@ public class DorisWriterManager { } return; } - flushQueue.put(new DorisWriterTuple(label, batchSize, new ArrayList<>(buffer))); + flushQueue.put(new WriterTuple (label, batchSize, new ArrayList<>(buffer))); if (waitUtilDone) { // wait the last flush waitAsyncFlushingDone(); @@ -145,13 +145,13 @@ public class DorisWriterManager { private void waitAsyncFlushingDone() throws InterruptedException { // wait previous flushings for (int i = 0; i <= options.getFlushQueueLength(); i++) { - flushQueue.put(new DorisWriterTuple("", 0l, null)); + flushQueue.put(new WriterTuple ("", 0l, null)); } checkFlushException(); } private void asyncFlush() throws Exception { - DorisWriterTuple flushData = flushQueue.take(); + WriterTuple flushData = flushQueue.take(); if (Strings.isNullOrEmpty(flushData.getLabel())) { return; } @@ -169,7 +169,7 @@ public class DorisWriterManager { if (i >= options.getMaxRetries()) { throw new IOException(e); } - if (e instanceof DorisStreamLoadExcetion && ((DorisStreamLoadExcetion)e).needReCreateLabel()) { + if (e instanceof DorisWriterExcetion && (( DorisWriterExcetion )e).needReCreateLabel()) { String newLabel = createBatchLabel(); LOG.warn(String.format("Batch label changed from [%s] to [%s]", flushData.getLabel(), newLabel)); flushData.setLabel(newLabel); diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterOptions.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterOptions.java deleted file mode 100644 index b4395d2e..00000000 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterOptions.java +++ /dev/null @@ -1,174 +0,0 @@ -package com.alibaba.datax.plugin.writer.doriswriter; - -import com.alibaba.datax.common.exception.DataXException; -import com.alibaba.datax.common.util.Configuration; -import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; - -import java.io.Serializable; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -public class DorisWriterOptions implements Serializable { - - private static final long serialVersionUID = 1l; - private static final long KILO_BYTES_SCALE = 1024l; - private static final long MEGA_BYTES_SCALE = KILO_BYTES_SCALE * KILO_BYTES_SCALE; - private static final int MAX_RETRIES = 3; - private static final int BATCH_ROWS = 500000; - private static final long BATCH_BYTES = 90 * MEGA_BYTES_SCALE; - private static final long FLUSH_INTERVAL = 300000; - - private static final String KEY_LOAD_PROPS_FORMAT = "format"; - public enum StreamLoadFormat { - CSV, JSON; - } - - private static final String KEY_USERNAME = "username"; - private static final String KEY_PASSWORD = "password"; - private static final String KEY_DATABASE = "connection[0].database"; - private static final String KEY_TABLE = "connection[0].table[0]"; - private static final String KEY_COLUMN = "column"; - private static final String KEY_PRE_SQL = "preSql"; - private static final String KEY_POST_SQL = "postSql"; - private static final String KEY_JDBC_URL = "connection[0].jdbcUrl"; - private static final String KEY_LABEL_PREFIX = "labelPrefix"; - private static final String KEY_MAX_BATCH_ROWS = "maxBatchRows"; - private static final String KEY_MAX_BATCH_SIZE = "maxBatchSize"; - private static final String KEY_FLUSH_INTERVAL = "flushInterval"; - private static final String KEY_LOAD_URL = "loadUrl"; - private static final String KEY_FLUSH_QUEUE_LENGTH = "flushQueueLength"; - private static final String KEY_LOAD_PROPS = "loadProps"; - - private final Configuration options; - private List infoCchemaColumns; - private List userSetColumns; - private boolean isWildcardColumn; - - public DorisWriterOptions(Configuration options) { - this.options = options; - this.userSetColumns = options.getList(KEY_COLUMN, String.class).stream().map(str -> str.replace("`", "")).collect(Collectors.toList()); - if (1 == options.getList(KEY_COLUMN, String.class).size() && "*".trim().equals(options.getList(KEY_COLUMN, String.class).get(0))) { - this.isWildcardColumn = true; - } - } - - public void doPretreatment() { - validateRequired(); - validateStreamLoadUrl(); - } - - public String getJdbcUrl() { - return options.getString(KEY_JDBC_URL); - } - - public String getDatabase() { - return options.getString(KEY_DATABASE); - } - - public String getTable() { - return options.getString(KEY_TABLE); - } - - public String getUsername() { - return options.getString(KEY_USERNAME); - } - - public String getPassword() { - return options.getString(KEY_PASSWORD); - } - - public String getLabelPrefix() { - return options.getString(KEY_LABEL_PREFIX); - } - - public List getLoadUrlList() { - return options.getList(KEY_LOAD_URL, String.class); - } - - public List getColumns() { - if (isWildcardColumn) { - return this.infoCchemaColumns; - } - return this.userSetColumns; - } - - public boolean isWildcardColumn() { - return this.isWildcardColumn; - } - - public void setInfoCchemaColumns(List cols) { - this.infoCchemaColumns = cols; - } - - public List getPreSqlList() { - return options.getList(KEY_PRE_SQL, String.class); - } - - public List getPostSqlList() { - return options.getList(KEY_POST_SQL, String.class); - } - - public Map getLoadProps() { - return options.getMap(KEY_LOAD_PROPS); - } - - public int getMaxRetries() { - return MAX_RETRIES; - } - - public int getBatchRows() { - Integer rows = options.getInt(KEY_MAX_BATCH_ROWS); - return null == rows ? BATCH_ROWS : rows; - } - - public long getBatchSize() { - Long size = options.getLong(KEY_MAX_BATCH_SIZE); - return null == size ? BATCH_BYTES : size; - } - - public long getFlushInterval() { - Long interval = options.getLong(KEY_FLUSH_INTERVAL); - return null == interval ? FLUSH_INTERVAL : interval; - } - - public int getFlushQueueLength() { - Integer len = options.getInt(KEY_FLUSH_QUEUE_LENGTH); - return null == len ? 1 : len; - } - - public StreamLoadFormat getStreamLoadFormat() { - Map loadProps = getLoadProps(); - if (null == loadProps) { - return StreamLoadFormat.CSV; - } - if (loadProps.containsKey(KEY_LOAD_PROPS_FORMAT) - && StreamLoadFormat.JSON.name().equalsIgnoreCase(String.valueOf(loadProps.get(KEY_LOAD_PROPS_FORMAT)))) { - return StreamLoadFormat.JSON; - } - return StreamLoadFormat.CSV; - } - - private void validateStreamLoadUrl() { - List urlList = getLoadUrlList(); - for (String host : urlList) { - if (host.split(":").length < 2) { - throw DataXException.asDataXException(DBUtilErrorCode.CONF_ERROR, - "loadUrl的格式不正确,请输入 `fe_ip:fe_http_ip;fe_ip:fe_http_ip`。"); - } - } - } - - private void validateRequired() { - final String[] requiredOptionKeys = new String[]{ - KEY_USERNAME, - KEY_DATABASE, - KEY_TABLE, - KEY_COLUMN, - KEY_LOAD_URL - }; - for (String optionKey : requiredOptionKeys) { - options.getNecessaryValue(optionKey, DBUtilErrorCode.REQUIRED_VALUE); - } - } -} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Keys.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Keys.java new file mode 100644 index 00000000..01c0e3c6 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Keys.java @@ -0,0 +1,177 @@ +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class Keys implements Serializable { + + private static final long serialVersionUID = 1l; + private static final int MAX_RETRIES = 3; + private static final int BATCH_ROWS = 500000; + private static final long DEFAULT_FLUSH_INTERVAL = 30000; + + private static final String LOAD_PROPS_FORMAT = "format"; + public enum StreamLoadFormat { + CSV, JSON; + } + + private static final String USERNAME = "username"; + private static final String PASSWORD = "password"; + private static final String DATABASE = "connection[0].selectedDatabase"; + private static final String TABLE = "connection[0].table[0]"; + private static final String COLUMN = "column"; + private static final String PRE_SQL = "preSql"; + private static final String POST_SQL = "postSql"; + private static final String JDBC_URL = "connection[0].jdbcUrl"; + private static final String LABEL_PREFIX = "labelPrefix"; + private static final String MAX_BATCH_ROWS = "maxBatchRows"; + private static final String MAX_BATCH_SIZE = "maxBatchSize"; + private static final String FLUSH_INTERVAL = "flushInterval"; + private static final String LOAD_URL = "loadUrl"; + private static final String FLUSH_QUEUE_LENGTH = "flushQueueLength"; + private static final String LOAD_PROPS = "loadProps"; + + private static final String DEFAULT_LABEL_PREFIX = "datax_doris_writer_"; + + private static final long DEFAULT_MAX_BATCH_SIZE = 90 * 1024 * 1024; //default 90M + + private final Configuration options; + + private List infoSchemaColumns; + private List userSetColumns; + private boolean isWildcardColumn; + + public Keys ( Configuration options) { + this.options = options; + this.userSetColumns = options.getList(COLUMN, String.class).stream().map(str -> str.replace("`", "")).collect(Collectors.toList()); + if (1 == options.getList(COLUMN, String.class).size() && "*".trim().equals(options.getList(COLUMN, String.class).get(0))) { + this.isWildcardColumn = true; + } + } + + public void doPretreatment() { + validateRequired(); + validateStreamLoadUrl(); + } + + public String getJdbcUrl() { + return options.getString(JDBC_URL); + } + + public String getDatabase() { + return options.getString(DATABASE); + } + + public String getTable() { + return options.getString(TABLE); + } + + public String getUsername() { + return options.getString(USERNAME); + } + + public String getPassword() { + return options.getString(PASSWORD); + } + + public String getLabelPrefix() { + String label = options.getString(LABEL_PREFIX); + return null == label ? DEFAULT_LABEL_PREFIX : label; + } + + public List getLoadUrlList() { + return options.getList(LOAD_URL, String.class); + } + + public List getColumns() { + if (isWildcardColumn) { + return this.infoSchemaColumns; + } + return this.userSetColumns; + } + + public boolean isWildcardColumn() { + return this.isWildcardColumn; + } + + public void setInfoCchemaColumns(List cols) { + this.infoSchemaColumns = cols; + } + + public List getPreSqlList() { + return options.getList(PRE_SQL, String.class); + } + + public List getPostSqlList() { + return options.getList(POST_SQL, String.class); + } + + public Map getLoadProps() { + return options.getMap(LOAD_PROPS); + } + + public int getMaxRetries() { + return MAX_RETRIES; + } + + public int getBatchRows() { + Integer rows = options.getInt(MAX_BATCH_ROWS); + return null == rows ? BATCH_ROWS : rows; + } + + public long getBatchSize() { + Long size = options.getLong(MAX_BATCH_SIZE); + return null == size ? DEFAULT_MAX_BATCH_SIZE : size; + } + + public long getFlushInterval() { + Long interval = options.getLong(FLUSH_INTERVAL); + return null == interval ? DEFAULT_FLUSH_INTERVAL : interval; + } + + public int getFlushQueueLength() { + Integer len = options.getInt(FLUSH_QUEUE_LENGTH); + return null == len ? 1 : len; + } + + public StreamLoadFormat getStreamLoadFormat() { + Map loadProps = getLoadProps(); + if (null == loadProps) { + return StreamLoadFormat.CSV; + } + if (loadProps.containsKey(LOAD_PROPS_FORMAT) + && StreamLoadFormat.JSON.name().equalsIgnoreCase(String.valueOf(loadProps.get(LOAD_PROPS_FORMAT)))) { + return StreamLoadFormat.JSON; + } + return StreamLoadFormat.CSV; + } + + private void validateStreamLoadUrl() { + List urlList = getLoadUrlList(); + for (String host : urlList) { + if (host.split(":").length < 2) { + throw DataXException.asDataXException(DBUtilErrorCode.CONF_ERROR, + "The format of loadUrl is not correct, please enter:[`fe_ip:fe_http_ip;fe_ip:fe_http_ip`]."); + } + } + } + + private void validateRequired() { + final String[] requiredOptionKeys = new String[]{ + USERNAME, + DATABASE, + TABLE, + COLUMN, + LOAD_URL + }; + for (String optionKey : requiredOptionKeys) { + options.getNecessaryValue(optionKey, DBUtilErrorCode.REQUIRED_VALUE); + } + } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterTuple.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/WriterTuple.java similarity index 81% rename from doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterTuple.java rename to doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/WriterTuple.java index 0bc52daf..32e0b341 100644 --- a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterTuple.java +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/WriterTuple.java @@ -2,12 +2,12 @@ package com.alibaba.datax.plugin.writer.doriswriter; import java.util.List; -public class DorisWriterTuple { +public class WriterTuple { private String label; private Long bytes; private List rows; - public DorisWriterTuple(String label,Long bytes,List rows){ + public WriterTuple ( String label, Long bytes, List rows){ this.label = label; this.rows = rows; this.bytes = bytes; From 74a95011d4b2ebfb1d5b6d27a0359d4f7fd0e420 Mon Sep 17 00:00:00 2001 From: "jiafeng.zhang" Date: Mon, 10 Oct 2022 10:55:36 +0800 Subject: [PATCH 13/13] fix --- doriswriter/doc/doriswriter.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doriswriter/doc/doriswriter.md b/doriswriter/doc/doriswriter.md index 1f0dff62..973d4bcf 100644 --- a/doriswriter/doc/doriswriter.md +++ b/doriswriter/doc/doriswriter.md @@ -47,7 +47,7 @@ DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter "connection": [ { "jdbcUrl": "jdbc:mysql://172.16.0.13:9030/demo", - "database": "demo", + "selectedDatabase": "demo", "table": ["all_employees_info"] } ],