diff --git a/README.md b/README.md index 59795e25fa..b44a01f6e6 100644 --- a/README.md +++ b/README.md @@ -25,7 +25,7 @@ DataX本身作为数据同步框架,将不同数据源的同步抽象为从源 # Quick Start -##### Download [DataX下载地址](https://datax-opensource.oss-cn-hangzhou.aliyuncs.com/20220530/datax.tar.gz) +##### Download [DataX下载地址](https://datax-opensource.oss-cn-hangzhou.aliyuncs.com/202209/datax.tar.gz) ##### 请点击:[Quick Start](https://github.com/alibaba/DataX/blob/master/userGuid.md) @@ -44,6 +44,8 @@ DataX目前已经有了比较全面的插件体系,主流的RDBMS数据库、N | | SQLServer | √ | √ |[读](https://github.com/alibaba/DataX/blob/master/sqlserverreader/doc/sqlserverreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/sqlserverwriter/doc/sqlserverwriter.md)| | | PostgreSQL | √ | √ |[读](https://github.com/alibaba/DataX/blob/master/postgresqlreader/doc/postgresqlreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/postgresqlwriter/doc/postgresqlwriter.md)| | | DRDS | √ | √ |[读](https://github.com/alibaba/DataX/blob/master/drdsreader/doc/drdsreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/drdswriter/doc/drdswriter.md)| +| | Apache Doris | | √ |[写](https://github.com/alibaba/DataX/blob/master/doriswriter/doc/doriswriter.md)| +| | StarRocks | | √ |[写](https://github.com/alibaba/DataX/blob/master/starrockswriter/doc/starrockswriter.md)| | | 通用RDBMS(支持所有关系型数据库) | √ | √ |[读](https://github.com/alibaba/DataX/blob/master/rdbmsreader/doc/rdbmsreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/rdbmswriter/doc/rdbmswriter.md)| | 阿里云数仓数据存储 | ODPS | √ | √ |[读](https://github.com/alibaba/DataX/blob/master/odpsreader/doc/odpsreader.md) 、[写](https://github.com/alibaba/DataX/blob/master/odpswriter/doc/odpswriter.md)| | | ADS | | √ |[写](https://github.com/alibaba/DataX/blob/master/adswriter/doc/adswriter.md)| @@ -95,6 +97,9 @@ DataX目前已经有了比较全面的插件体系,主流的RDBMS数据库、N DataX 后续计划月度迭代更新,也欢迎感兴趣的同学提交 Pull requests,月度更新内容会介绍介绍如下。 +- [datax_v202209](https://github.com/alibaba/DataX/releases/tag/datax_v202209) + - 涉及通道能力更新(MaxCompute、Datahub、SLS等)、安全漏洞更新、通用打包更新等 + - [datax_v202205](https://github.com/alibaba/DataX/releases/tag/datax_v202205) - 涉及通道能力更新(MaxCompute、Hologres、OSS、Tdengine等)、安全漏洞更新、通用打包更新等 diff --git a/adswriter/src/main/java/com/alibaba/datax/plugin/writer/adswriter/odps/DataType.java b/adswriter/src/main/java/com/alibaba/datax/plugin/writer/adswriter/odps/DataType.java index 595b1dfd26..f625336e15 100644 --- a/adswriter/src/main/java/com/alibaba/datax/plugin/writer/adswriter/odps/DataType.java +++ b/adswriter/src/main/java/com/alibaba/datax/plugin/writer/adswriter/odps/DataType.java @@ -70,7 +70,7 @@ public static byte convertToDataType(String type) throws IllegalArgumentExceptio } else if ("datetime".equals(type)) { return DATETIME; } else { - throw new IllegalArgumentException("unkown type: " + type); + throw new IllegalArgumentException("unknown type: " + type); } } diff --git a/clickhousewriter/src/main/java/com/alibaba/datax/plugin/writer/clickhousewriter/ClickhouseWriter.java b/clickhousewriter/src/main/java/com/alibaba/datax/plugin/writer/clickhousewriter/ClickhouseWriter.java index b928d4218e..31ffdfec7f 100644 --- a/clickhousewriter/src/main/java/com/alibaba/datax/plugin/writer/clickhousewriter/ClickhouseWriter.java +++ b/clickhousewriter/src/main/java/com/alibaba/datax/plugin/writer/clickhousewriter/ClickhouseWriter.java @@ -68,7 +68,7 @@ public void init() { this.commonRdbmsWriterSlave = new CommonRdbmsWriter.Task(DATABASE_TYPE) { @Override - protected PreparedStatement fillPreparedStatementColumnType(PreparedStatement preparedStatement, int columnIndex, int columnSqltype, Column column) throws SQLException { + protected PreparedStatement fillPreparedStatementColumnType(PreparedStatement preparedStatement, int columnIndex, int columnSqltype, String typeName, Column column) throws SQLException { try { if (column.getRawData() == null) { preparedStatement.setNull(columnIndex + 1, columnSqltype); diff --git a/clickhousewriter/src/main/resources/plugin.json b/clickhousewriter/src/main/resources/plugin.json index ff1acf01bf..d70e2b1de5 100755 --- a/clickhousewriter/src/main/resources/plugin.json +++ b/clickhousewriter/src/main/resources/plugin.json @@ -2,5 +2,5 @@ "name": "clickhousewriter", "class": "com.alibaba.datax.plugin.writer.clickhousewriter.ClickhouseWriter", "description": "useScene: prod. mechanism: Jdbc connection using the database, execute insert sql.", - "developer": "jiye.tjy" + "developer": "alibaba" } \ No newline at end of file diff --git a/common/src/main/java/com/alibaba/datax/common/util/Configuration.java b/common/src/main/java/com/alibaba/datax/common/util/Configuration.java index f570dd00c2..cd88e84a1c 100755 --- a/common/src/main/java/com/alibaba/datax/common/util/Configuration.java +++ b/common/src/main/java/com/alibaba/datax/common/util/Configuration.java @@ -411,6 +411,15 @@ public List getList(final String path) { return list; } + public List getListWithJson(final String path, Class t) { + Object object = this.get(path, List.class); + if (null == object) { + return null; + } + + return JSON.parseArray(JSON.toJSONString(object),t); + } + /** * 根据用户提供的json path,寻址List对象,如果对象不存在,返回null */ diff --git a/common/src/main/java/com/alibaba/datax/common/util/StrUtil.java b/common/src/main/java/com/alibaba/datax/common/util/StrUtil.java index 82222b0d48..867a9516ab 100755 --- a/common/src/main/java/com/alibaba/datax/common/util/StrUtil.java +++ b/common/src/main/java/com/alibaba/datax/common/util/StrUtil.java @@ -3,6 +3,8 @@ import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.Validate; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; import java.text.DecimalFormat; import java.util.HashMap; import java.util.Map; @@ -82,4 +84,20 @@ public static String compressMiddle(String s, int headLength, int tailLength) { return s.substring(0, headLength) + "..." + s.substring(s.length() - tailLength); } + public static String getMd5(String plainText) { + try { + StringBuilder builder = new StringBuilder(); + for (byte b : MessageDigest.getInstance("MD5").digest(plainText.getBytes())) { + int i = b & 0xff; + if (i < 0x10) { + builder.append('0'); + } + builder.append(Integer.toHexString(i)); + } + return builder.toString(); + } catch (NoSuchAlgorithmException e) { + throw new RuntimeException(e); + } + } + } diff --git a/core/pom.xml b/core/pom.xml index 970f95a6d3..7685001b3e 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/core/src/main/java/com/alibaba/datax/core/transport/transformer/DigestTransformer.java b/core/src/main/java/com/alibaba/datax/core/transport/transformer/DigestTransformer.java new file mode 100644 index 0000000000..d2bf143147 --- /dev/null +++ b/core/src/main/java/com/alibaba/datax/core/transport/transformer/DigestTransformer.java @@ -0,0 +1,87 @@ +package com.alibaba.datax.core.transport.transformer; + +import com.alibaba.datax.common.element.Column; +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.element.StringColumn; +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.transformer.Transformer; + +import org.apache.commons.codec.digest.DigestUtils; +import org.apache.commons.lang.StringUtils; + +import java.util.Arrays; + +/** + * no comments. + * + * @author XuDaojie + * @since 2021-08-16 + */ +public class DigestTransformer extends Transformer { + + private static final String MD5 = "md5"; + private static final String SHA1 = "sha1"; + private static final String TO_UPPER_CASE = "toUpperCase"; + private static final String TO_LOWER_CASE = "toLowerCase"; + + public DigestTransformer() { + setTransformerName("dx_digest"); + } + + @Override + public Record evaluate(Record record, Object... paras) { + + int columnIndex; + String type; + String charType; + + try { + if (paras.length != 3) { + throw new RuntimeException("dx_digest paras length must be 3"); + } + + columnIndex = (Integer) paras[0]; + type = (String) paras[1]; + charType = (String) paras[2]; + + if (!StringUtils.equalsIgnoreCase(MD5, type) && !StringUtils.equalsIgnoreCase(SHA1, type)) { + throw new RuntimeException("dx_digest paras index 1 must be md5 or sha1"); + } + if (!StringUtils.equalsIgnoreCase(TO_UPPER_CASE, charType) && !StringUtils.equalsIgnoreCase(TO_LOWER_CASE, charType)) { + throw new RuntimeException("dx_digest paras index 2 must be toUpperCase or toLowerCase"); + } + } catch (Exception e) { + throw DataXException.asDataXException(TransformerErrorCode.TRANSFORMER_ILLEGAL_PARAMETER, "paras:" + Arrays.asList(paras) + " => " + e.getMessage()); + } + + Column column = record.getColumn(columnIndex); + + try { + String oriValue = column.asString(); + + // 如果字段为空,作为空字符串处理 + if (oriValue == null) { + oriValue = ""; + } + String newValue; + if (MD5.equals(type)) { + newValue = DigestUtils.md5Hex(oriValue); + } else { + newValue = DigestUtils.sha1Hex(oriValue); + } + + if (TO_UPPER_CASE.equals(charType)) { + newValue = newValue.toUpperCase(); + } else { + newValue = newValue.toLowerCase(); + } + + record.setColumn(columnIndex, new StringColumn(newValue)); + + } catch (Exception e) { + throw DataXException.asDataXException(TransformerErrorCode.TRANSFORMER_RUN_EXCEPTION, e.getMessage(), e); + } + return record; + } + +} diff --git a/core/src/main/java/com/alibaba/datax/core/transport/transformer/FilterTransformer.java b/core/src/main/java/com/alibaba/datax/core/transport/transformer/FilterTransformer.java index 8f6492fa11..a3251715d7 100644 --- a/core/src/main/java/com/alibaba/datax/core/transport/transformer/FilterTransformer.java +++ b/core/src/main/java/com/alibaba/datax/core/transport/transformer/FilterTransformer.java @@ -61,7 +61,7 @@ public Record evaluate(Record record, Object... paras) { } else if (code.equalsIgnoreCase("<=")) { return doLess(record, value, column, true); } else { - throw new RuntimeException("dx_filter can't suport code:" + code); + throw new RuntimeException("dx_filter can't support code:" + code); } } catch (Exception e) { throw DataXException.asDataXException(TransformerErrorCode.TRANSFORMER_RUN_EXCEPTION, e.getMessage(), e); diff --git a/core/src/main/java/com/alibaba/datax/core/transport/transformer/GroovyTransformerStaticUtil.java b/core/src/main/java/com/alibaba/datax/core/transport/transformer/GroovyTransformerStaticUtil.java index 4c872993ab..487a8be8f2 100644 --- a/core/src/main/java/com/alibaba/datax/core/transport/transformer/GroovyTransformerStaticUtil.java +++ b/core/src/main/java/com/alibaba/datax/core/transport/transformer/GroovyTransformerStaticUtil.java @@ -1,10 +1,18 @@ package com.alibaba.datax.core.transport.transformer; +import org.apache.commons.codec.digest.DigestUtils; + /** * GroovyTransformer的帮助类,供groovy代码使用,必须全是static的方法 * Created by liqiang on 16/3/4. */ public class GroovyTransformerStaticUtil { + public static String md5(final String data) { + return DigestUtils.md5Hex(data); + } + public static String sha1(final String data) { + return DigestUtils.sha1Hex(data); + } } diff --git a/core/src/main/java/com/alibaba/datax/core/transport/transformer/TransformerRegistry.java b/core/src/main/java/com/alibaba/datax/core/transport/transformer/TransformerRegistry.java index 96a0d98845..3c62515340 100644 --- a/core/src/main/java/com/alibaba/datax/core/transport/transformer/TransformerRegistry.java +++ b/core/src/main/java/com/alibaba/datax/core/transport/transformer/TransformerRegistry.java @@ -36,6 +36,7 @@ public class TransformerRegistry { registTransformer(new ReplaceTransformer()); registTransformer(new FilterTransformer()); registTransformer(new GroovyTransformer()); + registTransformer(new DigestTransformer()); } public static void loadTransformerFromLocalStorage() { diff --git a/core/src/main/job/job.json b/core/src/main/job/job.json index 582065929a..cc35387778 100755 --- a/core/src/main/job/job.json +++ b/core/src/main/job/job.json @@ -2,7 +2,7 @@ "job": { "setting": { "speed": { - "byte":10485760 + "channel":1 }, "errorLimit": { "record": 0, diff --git a/datahubreader/pom.xml b/datahubreader/pom.xml new file mode 100644 index 0000000000..c0022b44a9 --- /dev/null +++ b/datahubreader/pom.xml @@ -0,0 +1,79 @@ + + + + datax-all + com.alibaba.datax + 0.0.1-SNAPSHOT + + 4.0.0 + + datahubreader + + 0.0.1-SNAPSHOT + + + + com.alibaba.datax + datax-common + ${datax-project-version} + + + slf4j-log4j12 + org.slf4j + + + + + org.slf4j + slf4j-api + + + ch.qos.logback + logback-classic + + + com.aliyun.datahub + aliyun-sdk-datahub + 2.21.6-public + + + junit + junit + 4.12 + test + + + + + + + + maven-compiler-plugin + + ${jdk-version} + ${jdk-version} + ${project-sourceEncoding} + + + + + maven-assembly-plugin + + + src/main/assembly/package.xml + + datax + + + + dwzip + package + + single + + + + + + + diff --git a/datahubreader/src/main/assembly/package.xml b/datahubreader/src/main/assembly/package.xml new file mode 100644 index 0000000000..d14ea98192 --- /dev/null +++ b/datahubreader/src/main/assembly/package.xml @@ -0,0 +1,34 @@ + + + + dir + + false + + + src/main/resources + + plugin.json + + plugin/reader/datahubreader + + + target/ + + datahubreader-0.0.1-SNAPSHOT.jar + + plugin/reader/datahubreader + + + + + + false + plugin/reader/datahubreader/libs + runtime + + + diff --git a/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/Constant.java b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/Constant.java new file mode 100644 index 0000000000..bee3ccd797 --- /dev/null +++ b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/Constant.java @@ -0,0 +1,8 @@ +package com.alibaba.datax.plugin.reader.datahubreader; + +public class Constant { + + public static String DATETIME_FORMAT = "yyyyMMddHHmmss"; + public static String DATE_FORMAT = "yyyyMMdd"; + +} diff --git a/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubClientHelper.java b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubClientHelper.java new file mode 100644 index 0000000000..6f601fb45c --- /dev/null +++ b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubClientHelper.java @@ -0,0 +1,42 @@ +package com.alibaba.datax.plugin.reader.datahubreader; + +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.TypeReference; +import com.aliyun.datahub.client.DatahubClient; +import com.aliyun.datahub.client.DatahubClientBuilder; +import com.aliyun.datahub.client.auth.Account; +import com.aliyun.datahub.client.auth.AliyunAccount; +import com.aliyun.datahub.client.common.DatahubConfig; +import com.aliyun.datahub.client.http.HttpConfig; +import org.apache.commons.lang3.StringUtils; + +public class DatahubClientHelper { + public static DatahubClient getDatahubClient(Configuration jobConfig) { + String accessId = jobConfig.getNecessaryValue(Key.CONFIG_KEY_ACCESS_ID, + DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + String accessKey = jobConfig.getNecessaryValue(Key.CONFIG_KEY_ACCESS_KEY, + DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + String endpoint = jobConfig.getNecessaryValue(Key.CONFIG_KEY_ENDPOINT, + DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + Account account = new AliyunAccount(accessId, accessKey); + // 是否开启二进制传输,服务端2.12版本开始支持 + boolean enableBinary = jobConfig.getBool("enableBinary", false); + DatahubConfig datahubConfig = new DatahubConfig(endpoint, account, enableBinary); + // HttpConfig可不设置,不设置时采用默认值 + // 读写数据推荐打开网络传输 LZ4压缩 + HttpConfig httpConfig = null; + String httpConfigStr = jobConfig.getString("httpConfig"); + if (StringUtils.isNotBlank(httpConfigStr)) { + httpConfig = JSON.parseObject(httpConfigStr, new TypeReference() { + }); + } + + DatahubClientBuilder builder = DatahubClientBuilder.newBuilder().setDatahubConfig(datahubConfig); + if (null != httpConfig) { + builder.setHttpConfig(httpConfig); + } + DatahubClient datahubClient = builder.build(); + return datahubClient; + } +} diff --git a/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubReader.java b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubReader.java new file mode 100644 index 0000000000..4792ac39c9 --- /dev/null +++ b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubReader.java @@ -0,0 +1,292 @@ +package com.alibaba.datax.plugin.reader.datahubreader; + +import java.text.ParseException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +import com.aliyun.datahub.client.model.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.datax.common.element.Column; +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.element.StringColumn; +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.common.plugin.RecordSender; +import com.alibaba.datax.common.spi.Reader; +import com.alibaba.datax.common.util.Configuration; + + +import com.aliyun.datahub.client.DatahubClient; + + +public class DatahubReader extends Reader { + public static class Job extends Reader.Job { + private static final Logger LOG = LoggerFactory.getLogger(Job.class); + + private Configuration originalConfig; + + private Long beginTimestampMillis; + private Long endTimestampMillis; + + DatahubClient datahubClient; + + @Override + public void init() { + LOG.info("datahub reader job init begin ..."); + this.originalConfig = super.getPluginJobConf(); + validateParameter(originalConfig); + this.datahubClient = DatahubClientHelper.getDatahubClient(this.originalConfig); + LOG.info("datahub reader job init end."); + } + + private void validateParameter(Configuration conf){ + conf.getNecessaryValue(Key.ENDPOINT,DatahubReaderErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.ACCESSKEYID,DatahubReaderErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.ACCESSKEYSECRET,DatahubReaderErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.PROJECT,DatahubReaderErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.TOPIC,DatahubReaderErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.COLUMN,DatahubReaderErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.BEGINDATETIME,DatahubReaderErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.ENDDATETIME,DatahubReaderErrorCode.REQUIRE_VALUE); + + int batchSize = this.originalConfig.getInt(Key.BATCHSIZE, 1024); + if (batchSize > 10000) { + throw DataXException.asDataXException(DatahubReaderErrorCode.BAD_CONFIG_VALUE, + "Invalid batchSize[" + batchSize + "] value (0,10000]!"); + } + + String beginDateTime = this.originalConfig.getString(Key.BEGINDATETIME); + if (beginDateTime != null) { + try { + beginTimestampMillis = DatahubReaderUtils.getUnixTimeFromDateTime(beginDateTime); + } catch (ParseException e) { + throw DataXException.asDataXException(DatahubReaderErrorCode.BAD_CONFIG_VALUE, + "Invalid beginDateTime[" + beginDateTime + "], format [yyyyMMddHHmmss]!"); + } + } + + if (beginTimestampMillis != null && beginTimestampMillis <= 0) { + throw DataXException.asDataXException(DatahubReaderErrorCode.BAD_CONFIG_VALUE, + "Invalid beginTimestampMillis[" + beginTimestampMillis + "]!"); + } + + String endDateTime = this.originalConfig.getString(Key.ENDDATETIME); + if (endDateTime != null) { + try { + endTimestampMillis = DatahubReaderUtils.getUnixTimeFromDateTime(endDateTime); + } catch (ParseException e) { + throw DataXException.asDataXException(DatahubReaderErrorCode.BAD_CONFIG_VALUE, + "Invalid beginDateTime[" + endDateTime + "], format [yyyyMMddHHmmss]!"); + } + } + + if (endTimestampMillis != null && endTimestampMillis <= 0) { + throw DataXException.asDataXException(DatahubReaderErrorCode.BAD_CONFIG_VALUE, + "Invalid endTimestampMillis[" + endTimestampMillis + "]!"); + } + + if (beginTimestampMillis != null && endTimestampMillis != null + && endTimestampMillis <= beginTimestampMillis) { + throw DataXException.asDataXException(DatahubReaderErrorCode.BAD_CONFIG_VALUE, + "endTimestampMillis[" + endTimestampMillis + "] must bigger than beginTimestampMillis[" + beginTimestampMillis + "]!"); + } + } + + @Override + public void prepare() { + // create datahub client + String project = originalConfig.getNecessaryValue(Key.PROJECT, DatahubReaderErrorCode.REQUIRE_VALUE); + String topic = originalConfig.getNecessaryValue(Key.TOPIC, DatahubReaderErrorCode.REQUIRE_VALUE); + RecordType recordType = null; + try { + DatahubClient client = DatahubClientHelper.getDatahubClient(this.originalConfig); + GetTopicResult getTopicResult = client.getTopic(project, topic); + recordType = getTopicResult.getRecordType(); + } catch (Exception e) { + LOG.warn("get topic type error: {}", e.getMessage()); + } + if (null != recordType) { + if (recordType == RecordType.BLOB) { + throw DataXException.asDataXException(DatahubReaderErrorCode.BAD_CONFIG_VALUE, + "DatahubReader only support 'Tuple' RecordType now, but your RecordType is 'BLOB'"); + } + } + } + + @Override + public void destroy() { + } + + @Override + public List split(int adviceNumber) { + LOG.info("split() begin..."); + + List readerSplitConfigs = new ArrayList(); + + String project = this.originalConfig.getString(Key.PROJECT); + String topic = this.originalConfig.getString(Key.TOPIC); + + List shardEntrys = DatahubReaderUtils.getShardsWithRetry(this.datahubClient, project, topic); + if (shardEntrys == null || shardEntrys.isEmpty() || shardEntrys.size() == 0) { + throw DataXException.asDataXException(DatahubReaderErrorCode.BAD_CONFIG_VALUE, + "Project [" + project + "] Topic [" + topic + "] has no shards, please check !"); + } + + for (ShardEntry shardEntry : shardEntrys) { + Configuration splitedConfig = this.originalConfig.clone(); + splitedConfig.set(Key.SHARDID, shardEntry.getShardId()); + readerSplitConfigs.add(splitedConfig); + } + + LOG.info("split() ok and end..."); + return readerSplitConfigs; + } + + } + + public static class Task extends Reader.Task { + private static final Logger LOG = LoggerFactory.getLogger(Task.class); + + private Configuration taskConfig; + + private String accessId; + private String accessKey; + private String endpoint; + private String project; + private String topic; + private String shardId; + private Long beginTimestampMillis; + private Long endTimestampMillis; + private int batchSize; + private List columns; + private RecordSchema schema; + private String timeStampUnit; + + DatahubClient datahubClient; + + @Override + public void init() { + this.taskConfig = super.getPluginJobConf(); + + this.accessId = this.taskConfig.getString(Key.ACCESSKEYID); + this.accessKey = this.taskConfig.getString(Key.ACCESSKEYSECRET); + this.endpoint = this.taskConfig.getString(Key.ENDPOINT); + this.project = this.taskConfig.getString(Key.PROJECT); + this.topic = this.taskConfig.getString(Key.TOPIC); + this.shardId = this.taskConfig.getString(Key.SHARDID); + this.batchSize = this.taskConfig.getInt(Key.BATCHSIZE, 1024); + this.timeStampUnit = this.taskConfig.getString(Key.TIMESTAMP_UNIT, "MICROSECOND"); + try { + this.beginTimestampMillis = DatahubReaderUtils.getUnixTimeFromDateTime(this.taskConfig.getString(Key.BEGINDATETIME)); + } catch (ParseException e) { + } + + try { + this.endTimestampMillis = DatahubReaderUtils.getUnixTimeFromDateTime(this.taskConfig.getString(Key.ENDDATETIME)); + } catch (ParseException e) { + } + + this.columns = this.taskConfig.getList(Key.COLUMN, String.class); + + this.datahubClient = DatahubClientHelper.getDatahubClient(this.taskConfig); + + + this.schema = DatahubReaderUtils.getDatahubSchemaWithRetry(this.datahubClient, this.project, topic); + + LOG.info("init datahub reader task finished.project:{} topic:{} batchSize:{}", project, topic, batchSize); + } + + @Override + public void destroy() { + } + + @Override + public void startRead(RecordSender recordSender) { + LOG.info("read start"); + + String beginCursor = DatahubReaderUtils.getCursorWithRetry(this.datahubClient, this.project, + this.topic, this.shardId, this.beginTimestampMillis); + String endCursor = DatahubReaderUtils.getCursorWithRetry(this.datahubClient, this.project, + this.topic, this.shardId, this.endTimestampMillis); + + if (beginCursor == null) { + LOG.info("Shard:{} has no data!", this.shardId); + return; + } else if (endCursor == null) { + endCursor = DatahubReaderUtils.getLatestCursorWithRetry(this.datahubClient, this.project, + this.topic, this.shardId); + } + + String curCursor = beginCursor; + + boolean exit = false; + + while (true) { + + GetRecordsResult result = DatahubReaderUtils.getRecordsResultWithRetry(this.datahubClient, this.project, this.topic, + this.shardId, this.batchSize, curCursor, this.schema); + + List records = result.getRecords(); + if (records.size() > 0) { + for (RecordEntry record : records) { + if (record.getSystemTime() >= this.endTimestampMillis) { + exit = true; + break; + } + + HashMap dataMap = new HashMap(); + List fields = ((TupleRecordData) record.getRecordData()).getRecordSchema().getFields(); + for (int i = 0; i < fields.size(); i++) { + Field field = fields.get(i); + Column column = DatahubReaderUtils.getColumnFromField(record, field, this.timeStampUnit); + dataMap.put(field.getName(), column); + } + + Record dataxRecord = recordSender.createRecord(); + + if (null != this.columns && 1 == this.columns.size()) { + String columnsInStr = columns.get(0).toString(); + if ("\"*\"".equals(columnsInStr) || "*".equals(columnsInStr)) { + for (int i = 0; i < fields.size(); i++) { + dataxRecord.addColumn(dataMap.get(fields.get(i).getName())); + } + + } else { + if (dataMap.containsKey(columnsInStr)) { + dataxRecord.addColumn(dataMap.get(columnsInStr)); + } else { + dataxRecord.addColumn(new StringColumn(null)); + } + } + } else { + for (String col : this.columns) { + if (dataMap.containsKey(col)) { + dataxRecord.addColumn(dataMap.get(col)); + } else { + dataxRecord.addColumn(new StringColumn(null)); + } + } + } + + recordSender.sendToWriter(dataxRecord); + } + } else { + break; + } + + if (exit) { + break; + } + + curCursor = result.getNextCursor(); + } + + + LOG.info("end read datahub shard..."); + } + + } + +} diff --git a/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubReaderErrorCode.java b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubReaderErrorCode.java new file mode 100644 index 0000000000..949a66f0b8 --- /dev/null +++ b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubReaderErrorCode.java @@ -0,0 +1,35 @@ +package com.alibaba.datax.plugin.reader.datahubreader; + +import com.alibaba.datax.common.spi.ErrorCode; + +public enum DatahubReaderErrorCode implements ErrorCode { + BAD_CONFIG_VALUE("DatahubReader-00", "The value you configured is invalid."), + LOG_HUB_ERROR("DatahubReader-01","Datahub exception"), + REQUIRE_VALUE("DatahubReader-02","Missing parameters"), + EMPTY_LOGSTORE_VALUE("DatahubReader-03","There is no shard under this LogStore"); + + + private final String code; + private final String description; + + private DatahubReaderErrorCode(String code, String description) { + this.code = code; + this.description = description; + } + + @Override + public String getCode() { + return this.code; + } + + @Override + public String getDescription() { + return this.description; + } + + @Override + public String toString() { + return String.format("Code:[%s], Description:[%s]. ", this.code, + this.description); + } +} diff --git a/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubReaderUtils.java b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubReaderUtils.java new file mode 100644 index 0000000000..6c3455df67 --- /dev/null +++ b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubReaderUtils.java @@ -0,0 +1,200 @@ +package com.alibaba.datax.plugin.reader.datahubreader; + +import java.math.BigDecimal; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.List; +import java.util.concurrent.Callable; + +import com.alibaba.datax.common.element.*; +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.common.util.DataXCaseEnvUtil; +import com.alibaba.datax.common.util.RetryUtil; + +import com.aliyun.datahub.client.DatahubClient; +import com.aliyun.datahub.client.exception.InvalidParameterException; +import com.aliyun.datahub.client.model.*; + +public class DatahubReaderUtils { + + public static long getUnixTimeFromDateTime(String dateTime) throws ParseException { + try { + String format = Constant.DATETIME_FORMAT; + SimpleDateFormat simpleDateFormat = new SimpleDateFormat(format); + return simpleDateFormat.parse(dateTime).getTime(); + } catch (ParseException ignored) { + throw DataXException.asDataXException(DatahubReaderErrorCode.BAD_CONFIG_VALUE, + "Invalid DateTime[" + dateTime + "]!"); + } + } + + public static List getShardsWithRetry(final DatahubClient datahubClient, final String project, final String topic) { + + List shards = null; + try { + shards = RetryUtil.executeWithRetry(new Callable>() { + @Override + public List call() throws Exception { + ListShardResult listShardResult = datahubClient.listShard(project, topic); + return listShardResult.getShards(); + } + }, DataXCaseEnvUtil.getRetryTimes(7), DataXCaseEnvUtil.getRetryInterval(1000L), DataXCaseEnvUtil.getRetryExponential(true)); + + } catch (Exception e) { + throw DataXException.asDataXException(DatahubReaderErrorCode.BAD_CONFIG_VALUE, + "get Shards error, please check ! detail error messsage: " + e.toString()); + } + return shards; + } + + public static String getCursorWithRetry(final DatahubClient datahubClient, final String project, final String topic, + final String shardId, final long timestamp) { + + String cursor; + try { + cursor = RetryUtil.executeWithRetry(new Callable() { + @Override + public String call() throws Exception { + try { + return datahubClient.getCursor(project, topic, shardId, CursorType.SYSTEM_TIME, timestamp).getCursor(); + } catch (InvalidParameterException e) { + if (e.getErrorMessage().indexOf("Time in seek request is out of range") >= 0) { + return null; + } else { + throw e; + } + + } + } + }, DataXCaseEnvUtil.getRetryTimes(7), DataXCaseEnvUtil.getRetryInterval(1000L), DataXCaseEnvUtil.getRetryExponential(true)); + + } catch (Exception e) { + throw DataXException.asDataXException(DatahubReaderErrorCode.BAD_CONFIG_VALUE, + "get Cursor error, please check ! detail error messsage: " + e.toString()); + } + return cursor; + } + + public static String getLatestCursorWithRetry(final DatahubClient datahubClient, final String project, final String topic, + final String shardId) { + + String cursor; + try { + cursor = RetryUtil.executeWithRetry(new Callable() { + @Override + public String call() throws Exception { + return datahubClient.getCursor(project, topic, shardId, CursorType.LATEST).getCursor(); + } + }, DataXCaseEnvUtil.getRetryTimes(7), DataXCaseEnvUtil.getRetryInterval(1000L), DataXCaseEnvUtil.getRetryExponential(true)); + + } catch (Exception e) { + throw DataXException.asDataXException(DatahubReaderErrorCode.BAD_CONFIG_VALUE, + "get Cursor error, please check ! detail error messsage: " + e.toString()); + } + return cursor; + } + + public static RecordSchema getDatahubSchemaWithRetry(final DatahubClient datahubClient, final String project, final String topic) { + + RecordSchema schema; + try { + schema = RetryUtil.executeWithRetry(new Callable() { + @Override + public RecordSchema call() throws Exception { + return datahubClient.getTopic(project, topic).getRecordSchema(); + } + }, DataXCaseEnvUtil.getRetryTimes(7), DataXCaseEnvUtil.getRetryInterval(1000L), DataXCaseEnvUtil.getRetryExponential(true)); + + } catch (Exception e) { + throw DataXException.asDataXException(DatahubReaderErrorCode.BAD_CONFIG_VALUE, + "get Topic Schema error, please check ! detail error messsage: " + e.toString()); + } + return schema; + } + + public static GetRecordsResult getRecordsResultWithRetry(final DatahubClient datahubClient, final String project, + final String topic, final String shardId, final int batchSize, final String cursor, final RecordSchema schema) { + + GetRecordsResult result; + try { + result = RetryUtil.executeWithRetry(new Callable() { + @Override + public GetRecordsResult call() throws Exception { + return datahubClient.getRecords(project, topic, shardId, schema, cursor, batchSize); + } + }, DataXCaseEnvUtil.getRetryTimes(7), DataXCaseEnvUtil.getRetryInterval(1000L), DataXCaseEnvUtil.getRetryExponential(true)); + + } catch (Exception e) { + throw DataXException.asDataXException(DatahubReaderErrorCode.BAD_CONFIG_VALUE, + "get Record Result error, please check ! detail error messsage: " + e.toString()); + } + return result; + + } + + public static Column getColumnFromField(RecordEntry record, Field field, String timeStampUnit) { + Column col = null; + TupleRecordData o = (TupleRecordData) record.getRecordData(); + + switch (field.getType()) { + case SMALLINT: + Short shortValue = ((Short) o.getField(field.getName())); + col = new LongColumn(shortValue == null ? null: shortValue.longValue()); + break; + case INTEGER: + col = new LongColumn((Integer) o.getField(field.getName())); + break; + case BIGINT: { + col = new LongColumn((Long) o.getField(field.getName())); + break; + } + case TINYINT: { + Byte byteValue = ((Byte) o.getField(field.getName())); + col = new LongColumn(byteValue == null ? null : byteValue.longValue()); + break; + } + case BOOLEAN: { + col = new BoolColumn((Boolean) o.getField(field.getName())); + break; + } + case FLOAT: + col = new DoubleColumn((Float) o.getField(field.getName())); + break; + case DOUBLE: { + col = new DoubleColumn((Double) o.getField(field.getName())); + break; + } + case STRING: { + col = new StringColumn((String) o.getField(field.getName())); + break; + } + case DECIMAL: { + BigDecimal value = (BigDecimal) o.getField(field.getName()); + col = new DoubleColumn(value == null ? null : value.doubleValue()); + break; + } + case TIMESTAMP: { + Long value = (Long) o.getField(field.getName()); + + if ("MILLISECOND".equals(timeStampUnit)) { + // MILLISECOND, 13位精度,直接 new Date() + col = new DateColumn(value == null ? null : new Date(value)); + } + else if ("SECOND".equals(timeStampUnit)){ + col = new DateColumn(value == null ? null : new Date(value * 1000)); + } + else { + // 默认都是 MICROSECOND, 16位精度, 和之前的逻辑保持一致。 + col = new DateColumn(value == null ? null : new Date(value / 1000)); + } + break; + } + default: + throw new RuntimeException("Unknown column type: " + field.getType()); + } + + return col; + } + +} diff --git a/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubWriterErrorCode.java b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubWriterErrorCode.java new file mode 100644 index 0000000000..c8633ea8a8 --- /dev/null +++ b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/DatahubWriterErrorCode.java @@ -0,0 +1,37 @@ +package com.alibaba.datax.plugin.reader.datahubreader; + +import com.alibaba.datax.common.spi.ErrorCode; +import com.alibaba.datax.common.util.MessageSource; + +public enum DatahubWriterErrorCode implements ErrorCode { + MISSING_REQUIRED_VALUE("DatahubWriter-01", MessageSource.loadResourceBundle(DatahubWriterErrorCode.class).message("errorcode.missing_required_value")), + INVALID_CONFIG_VALUE("DatahubWriter-02", MessageSource.loadResourceBundle(DatahubWriterErrorCode.class).message("errorcode.invalid_config_value")), + GET_TOPOIC_INFO_FAIL("DatahubWriter-03", MessageSource.loadResourceBundle(DatahubWriterErrorCode.class).message("errorcode.get_topic_info_fail")), + WRITE_DATAHUB_FAIL("DatahubWriter-04", MessageSource.loadResourceBundle(DatahubWriterErrorCode.class).message("errorcode.write_datahub_fail")), + SCHEMA_NOT_MATCH("DatahubWriter-05", MessageSource.loadResourceBundle(DatahubWriterErrorCode.class).message("errorcode.schema_not_match")), + ; + + private final String code; + private final String description; + + private DatahubWriterErrorCode(String code, String description) { + this.code = code; + this.description = description; + } + + @Override + public String getCode() { + return this.code; + } + + @Override + public String getDescription() { + return this.description; + } + + @Override + public String toString() { + return String.format("Code:[%s], Description:[%s]. ", this.code, + this.description); + } +} \ No newline at end of file diff --git a/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/Key.java b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/Key.java new file mode 100644 index 0000000000..3cb84b4bc9 --- /dev/null +++ b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/Key.java @@ -0,0 +1,35 @@ +package com.alibaba.datax.plugin.reader.datahubreader; + +public final class Key { + + /** + * 此处声明插件用到的需要插件使用者提供的配置项 + */ + public static final String ENDPOINT = "endpoint"; + + public static final String ACCESSKEYID = "accessId"; + + public static final String ACCESSKEYSECRET = "accessKey"; + + public static final String PROJECT = "project"; + + public static final String TOPIC = "topic"; + + public static final String BEGINDATETIME = "beginDateTime"; + + public static final String ENDDATETIME = "endDateTime"; + + public static final String BATCHSIZE = "batchSize"; + + public static final String COLUMN = "column"; + + public static final String SHARDID = "shardId"; + + public static final String CONFIG_KEY_ENDPOINT = "endpoint"; + public static final String CONFIG_KEY_ACCESS_ID = "accessId"; + public static final String CONFIG_KEY_ACCESS_KEY = "accessKey"; + + + public static final String TIMESTAMP_UNIT = "timeStampUnit"; + +} \ No newline at end of file diff --git a/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings.properties b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings.properties new file mode 100644 index 0000000000..e85c8ab321 --- /dev/null +++ b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings.properties @@ -0,0 +1,5 @@ +errorcode.missing_required_value=\u60A8\u7F3A\u5931\u4E86\u5FC5\u987B\u586B\u5199\u7684\u53C2\u6570\u503C. +errorcode.invalid_config_value=\u60A8\u7684\u53C2\u6570\u914D\u7F6E\u9519\u8BEF. +errorcode.get_topic_info_fail=\u83B7\u53D6shard\u5217\u8868\u5931\u8D25. +errorcode.write_datahub_fail=\u5199\u6570\u636E\u5931\u8D25. +errorcode.schema_not_match=\u6570\u636E\u683C\u5F0F\u9519\u8BEF. diff --git a/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_en_US.properties b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_en_US.properties new file mode 100644 index 0000000000..31a291e6db --- /dev/null +++ b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_en_US.properties @@ -0,0 +1,5 @@ +errorcode.missing_required_value=\u60A8\u7F3A\u5931\u4E86\u5FC5\u987B\u586B\u5199\u7684\u53C2\u6570\u503C. +errorcode.invalid_config_value=\u60A8\u7684\u53C2\u6570\u914D\u7F6E\u9519\u8BEF. +errorcode.get_topic_info_fail=\u83B7\u53D6shard\u5217\u8868\u5931\u8D25. +errorcode.write_datahub_fail=\u5199\u6570\u636E\u5931\u8D25. +errorcode.schema_not_match=\u6570\u636E\u683C\u5F0F\u9519\u8BEF. \ No newline at end of file diff --git a/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_ja_JP.properties b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_ja_JP.properties new file mode 100644 index 0000000000..31a291e6db --- /dev/null +++ b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_ja_JP.properties @@ -0,0 +1,5 @@ +errorcode.missing_required_value=\u60A8\u7F3A\u5931\u4E86\u5FC5\u987B\u586B\u5199\u7684\u53C2\u6570\u503C. +errorcode.invalid_config_value=\u60A8\u7684\u53C2\u6570\u914D\u7F6E\u9519\u8BEF. +errorcode.get_topic_info_fail=\u83B7\u53D6shard\u5217\u8868\u5931\u8D25. +errorcode.write_datahub_fail=\u5199\u6570\u636E\u5931\u8D25. +errorcode.schema_not_match=\u6570\u636E\u683C\u5F0F\u9519\u8BEF. \ No newline at end of file diff --git a/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_zh_CN.properties b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_zh_CN.properties new file mode 100644 index 0000000000..31a291e6db --- /dev/null +++ b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_zh_CN.properties @@ -0,0 +1,5 @@ +errorcode.missing_required_value=\u60A8\u7F3A\u5931\u4E86\u5FC5\u987B\u586B\u5199\u7684\u53C2\u6570\u503C. +errorcode.invalid_config_value=\u60A8\u7684\u53C2\u6570\u914D\u7F6E\u9519\u8BEF. +errorcode.get_topic_info_fail=\u83B7\u53D6shard\u5217\u8868\u5931\u8D25. +errorcode.write_datahub_fail=\u5199\u6570\u636E\u5931\u8D25. +errorcode.schema_not_match=\u6570\u636E\u683C\u5F0F\u9519\u8BEF. \ No newline at end of file diff --git a/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_zh_HK.properties b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_zh_HK.properties new file mode 100644 index 0000000000..c6a3a0e01d --- /dev/null +++ b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_zh_HK.properties @@ -0,0 +1,9 @@ +errorcode.missing_required_value=\u60A8\u7F3A\u5931\u4E86\u5FC5\u987B\u586B\u5199\u7684\u53C2\u6570\u503C. +errorcode.invalid_config_value=\u60A8\u7684\u53C2\u6570\u914D\u7F6E\u9519\u8BEF. +errorcode.get_topic_info_fail=\u83B7\u53D6shard\u5217\u8868\u5931\u8D25. +errorcode.write_datahub_fail=\u5199\u6570\u636E\u5931\u8D25. +errorcode.schema_not_match=\u6570\u636E\u683C\u5F0F\u9519\u8BEF.errorcode.missing_required_value=您缺失了必須填寫的參數值. +errorcode.invalid_config_value=您的參數配寘錯誤. +errorcode.get_topic_info_fail=獲取shard清單失敗. +errorcode.write_datahub_fail=寫數據失敗. +errorcode.schema_not_match=數據格式錯誤. diff --git a/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_zh_TW.properties b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_zh_TW.properties new file mode 100644 index 0000000000..c6a3a0e01d --- /dev/null +++ b/datahubreader/src/main/java/com/alibaba/datax/plugin/reader/datahubreader/LocalStrings_zh_TW.properties @@ -0,0 +1,9 @@ +errorcode.missing_required_value=\u60A8\u7F3A\u5931\u4E86\u5FC5\u987B\u586B\u5199\u7684\u53C2\u6570\u503C. +errorcode.invalid_config_value=\u60A8\u7684\u53C2\u6570\u914D\u7F6E\u9519\u8BEF. +errorcode.get_topic_info_fail=\u83B7\u53D6shard\u5217\u8868\u5931\u8D25. +errorcode.write_datahub_fail=\u5199\u6570\u636E\u5931\u8D25. +errorcode.schema_not_match=\u6570\u636E\u683C\u5F0F\u9519\u8BEF.errorcode.missing_required_value=您缺失了必須填寫的參數值. +errorcode.invalid_config_value=您的參數配寘錯誤. +errorcode.get_topic_info_fail=獲取shard清單失敗. +errorcode.write_datahub_fail=寫數據失敗. +errorcode.schema_not_match=數據格式錯誤. diff --git a/datahubreader/src/main/resources/job_config_template.json b/datahubreader/src/main/resources/job_config_template.json new file mode 100644 index 0000000000..eaf89804c6 --- /dev/null +++ b/datahubreader/src/main/resources/job_config_template.json @@ -0,0 +1,14 @@ +{ + "name": "datahubreader", + "parameter": { + "endpoint":"", + "accessId": "", + "accessKey": "", + "project": "", + "topic": "", + "beginDateTime": "20180913121019", + "endDateTime": "20180913121119", + "batchSize": 1024, + "column": [] + } +} \ No newline at end of file diff --git a/datahubreader/src/main/resources/plugin.json b/datahubreader/src/main/resources/plugin.json new file mode 100644 index 0000000000..47b1c86b21 --- /dev/null +++ b/datahubreader/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "datahubreader", + "class": "com.alibaba.datax.plugin.reader.datahubreader.DatahubReader", + "description": "datahub reader", + "developer": "alibaba" +} \ No newline at end of file diff --git a/datahubwriter/pom.xml b/datahubwriter/pom.xml new file mode 100644 index 0000000000..1ee1fe9b0a --- /dev/null +++ b/datahubwriter/pom.xml @@ -0,0 +1,79 @@ + + + + datax-all + com.alibaba.datax + 0.0.1-SNAPSHOT + + 4.0.0 + + datahubwriter + + 0.0.1-SNAPSHOT + + + + com.alibaba.datax + datax-common + ${datax-project-version} + + + slf4j-log4j12 + org.slf4j + + + + + org.slf4j + slf4j-api + + + ch.qos.logback + logback-classic + + + com.aliyun.datahub + aliyun-sdk-datahub + 2.21.6-public + + + junit + junit + 4.12 + test + + + + + + + + maven-compiler-plugin + + ${jdk-version} + ${jdk-version} + ${project-sourceEncoding} + + + + + maven-assembly-plugin + + + src/main/assembly/package.xml + + datax + + + + dwzip + package + + single + + + + + + + diff --git a/datahubwriter/src/main/assembly/package.xml b/datahubwriter/src/main/assembly/package.xml new file mode 100644 index 0000000000..aaef9f997f --- /dev/null +++ b/datahubwriter/src/main/assembly/package.xml @@ -0,0 +1,34 @@ + + + + dir + + false + + + src/main/resources + + plugin.json + + plugin/writer/datahubwriter + + + target/ + + datahubwriter-0.0.1-SNAPSHOT.jar + + plugin/writer/datahubwriter + + + + + + false + plugin/writer/datahubwriter/libs + runtime + + + diff --git a/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/DatahubClientHelper.java b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/DatahubClientHelper.java new file mode 100644 index 0000000000..2d94212c1a --- /dev/null +++ b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/DatahubClientHelper.java @@ -0,0 +1,43 @@ +package com.alibaba.datax.plugin.writer.datahubwriter; + +import org.apache.commons.lang3.StringUtils; + +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.TypeReference; +import com.aliyun.datahub.client.DatahubClient; +import com.aliyun.datahub.client.DatahubClientBuilder; +import com.aliyun.datahub.client.auth.Account; +import com.aliyun.datahub.client.auth.AliyunAccount; +import com.aliyun.datahub.client.common.DatahubConfig; +import com.aliyun.datahub.client.http.HttpConfig; + +public class DatahubClientHelper { + public static DatahubClient getDatahubClient(Configuration jobConfig) { + String accessId = jobConfig.getNecessaryValue(Key.CONFIG_KEY_ACCESS_ID, + DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + String accessKey = jobConfig.getNecessaryValue(Key.CONFIG_KEY_ACCESS_KEY, + DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + String endpoint = jobConfig.getNecessaryValue(Key.CONFIG_KEY_ENDPOINT, + DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + Account account = new AliyunAccount(accessId, accessKey); + // 是否开启二进制传输,服务端2.12版本开始支持 + boolean enableBinary = jobConfig.getBool("enableBinary", false); + DatahubConfig datahubConfig = new DatahubConfig(endpoint, account, enableBinary); + // HttpConfig可不设置,不设置时采用默认值 + // 读写数据推荐打开网络传输 LZ4压缩 + HttpConfig httpConfig = null; + String httpConfigStr = jobConfig.getString("httpConfig"); + if (StringUtils.isNotBlank(httpConfigStr)) { + httpConfig = JSON.parseObject(httpConfigStr, new TypeReference() { + }); + } + + DatahubClientBuilder builder = DatahubClientBuilder.newBuilder().setDatahubConfig(datahubConfig); + if (null != httpConfig) { + builder.setHttpConfig(httpConfig); + } + DatahubClient datahubClient = builder.build(); + return datahubClient; + } +} diff --git a/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/DatahubWriter.java b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/DatahubWriter.java new file mode 100644 index 0000000000..f6dc110543 --- /dev/null +++ b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/DatahubWriter.java @@ -0,0 +1,355 @@ +package com.alibaba.datax.plugin.writer.datahubwriter; + +import com.alibaba.datax.common.element.Column; +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.common.util.DataXCaseEnvUtil; +import com.alibaba.datax.common.util.RetryUtil; +import com.alibaba.fastjson.JSON; +import com.aliyun.datahub.client.DatahubClient; +import com.aliyun.datahub.client.model.FieldType; +import com.aliyun.datahub.client.model.GetTopicResult; +import com.aliyun.datahub.client.model.ListShardResult; +import com.aliyun.datahub.client.model.PutErrorEntry; +import com.aliyun.datahub.client.model.PutRecordsResult; +import com.aliyun.datahub.client.model.RecordEntry; +import com.aliyun.datahub.client.model.RecordSchema; +import com.aliyun.datahub.client.model.RecordType; +import com.aliyun.datahub.client.model.ShardEntry; +import com.aliyun.datahub.client.model.ShardState; +import com.aliyun.datahub.client.model.TupleRecordData; + +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.concurrent.Callable; + +public class DatahubWriter extends Writer { + + /** + * Job 中的方法仅执行一次,Task 中方法会由框架启动多个 Task 线程并行执行。 + *

+ * 整个 Writer 执行流程是: + *

+     * Job类init-->prepare-->split
+     *
+     *                          Task类init-->prepare-->startWrite-->post-->destroy
+     *                          Task类init-->prepare-->startWrite-->post-->destroy
+     *
+     *                                                                            Job类post-->destroy
+     * 
+ */ + public static class Job extends Writer.Job { + private static final Logger LOG = LoggerFactory + .getLogger(Job.class); + + private Configuration jobConfig = null; + + @Override + public void init() { + this.jobConfig = super.getPluginJobConf(); + jobConfig.getNecessaryValue(Key.CONFIG_KEY_ENDPOINT, DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + jobConfig.getNecessaryValue(Key.CONFIG_KEY_ACCESS_ID, DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + jobConfig.getNecessaryValue(Key.CONFIG_KEY_ACCESS_KEY, DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + jobConfig.getNecessaryValue(Key.CONFIG_KEY_PROJECT, DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + jobConfig.getNecessaryValue(Key.CONFIG_KEY_TOPIC, DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + } + + @Override + public void prepare() { + String project = jobConfig.getNecessaryValue(Key.CONFIG_KEY_PROJECT, + DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + String topic = jobConfig.getNecessaryValue(Key.CONFIG_KEY_TOPIC, + DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + RecordType recordType = null; + DatahubClient client = DatahubClientHelper.getDatahubClient(this.jobConfig); + try { + GetTopicResult getTopicResult = client.getTopic(project, topic); + recordType = getTopicResult.getRecordType(); + } catch (Exception e) { + LOG.warn("get topic type error: {}", e.getMessage()); + } + if (null != recordType) { + if (recordType == RecordType.BLOB) { + throw DataXException.asDataXException(DatahubWriterErrorCode.WRITE_DATAHUB_FAIL, + "DatahubWriter only support 'Tuple' RecordType now, but your RecordType is 'BLOB'"); + } + } + } + + @Override + public List split(int mandatoryNumber) { + List configs = new ArrayList(); + for (int i = 0; i < mandatoryNumber; ++i) { + configs.add(jobConfig.clone()); + } + return configs; + } + + @Override + public void post() {} + + @Override + public void destroy() {} + + } + + public static class Task extends Writer.Task { + private static final Logger LOG = LoggerFactory + .getLogger(Task.class); + private static final List FATAL_ERRORS_DEFAULT = Arrays.asList( + "InvalidParameterM", + "MalformedRecord", + "INVALID_SHARDID", + "NoSuchTopic", + "NoSuchShard" + ); + + private Configuration taskConfig; + private DatahubClient client; + private String project; + private String topic; + private List shards; + private int maxCommitSize; + private int maxRetryCount; + private RecordSchema schema; + private long retryInterval; + private Random random; + private List column; + private List columnIndex; + private boolean enableColumnConfig; + private List fatalErrors; + + @Override + public void init() { + this.taskConfig = super.getPluginJobConf(); + project = taskConfig.getNecessaryValue(Key.CONFIG_KEY_PROJECT, DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + topic = taskConfig.getNecessaryValue(Key.CONFIG_KEY_TOPIC, DatahubWriterErrorCode.MISSING_REQUIRED_VALUE); + maxCommitSize = taskConfig.getInt(Key.CONFIG_KEY_MAX_COMMIT_SIZE, 1024*1024); + maxRetryCount = taskConfig.getInt(Key.CONFIG_KEY_MAX_RETRY_COUNT, 500); + this.retryInterval = taskConfig.getInt(Key.RETRY_INTERVAL, 650); + this.random = new Random(); + this.column = this.taskConfig.getList(Key.CONFIG_KEY_COLUMN, String.class); + // ["*"] + if (null != this.column && 1 == this.column.size()) { + if (StringUtils.equals("*", this.column.get(0))) { + this.column = null; + } + } + this.columnIndex = new ArrayList(); + // 留个开关保平安 + this.enableColumnConfig = this.taskConfig.getBool("enableColumnConfig", true); + this.fatalErrors = this.taskConfig.getList("fatalErrors", Task.FATAL_ERRORS_DEFAULT, String.class); + this.client = DatahubClientHelper.getDatahubClient(this.taskConfig); + } + + @Override + public void prepare() { + final String shardIdConfig = this.taskConfig.getString(Key.CONFIG_KEY_SHARD_ID); + this.shards = new ArrayList(); + try { + RetryUtil.executeWithRetry(new Callable() { + @Override + public Void call() throws Exception { + ListShardResult result = client.listShard(project, topic); + if (StringUtils.isNotBlank(shardIdConfig)) { + shards.add(shardIdConfig); + } else { + for (ShardEntry shard : result.getShards()) { + if (shard.getState() == ShardState.ACTIVE || shard.getState() == ShardState.OPENING) { + shards.add(shard.getShardId()); + } + } + } + schema = client.getTopic(project, topic).getRecordSchema(); + return null; + } + }, DataXCaseEnvUtil.getRetryTimes(5), DataXCaseEnvUtil.getRetryInterval(10000L), DataXCaseEnvUtil.getRetryExponential(false)); + } catch (Exception e) { + throw DataXException.asDataXException(DatahubWriterErrorCode.GET_TOPOIC_INFO_FAIL, + "get topic info failed", e); + } + LOG.info("datahub topic {} shard to write: {}", this.topic, JSON.toJSONString(this.shards)); + LOG.info("datahub topic {} has schema: {}", this.topic, JSON.toJSONString(this.schema)); + + // 根据 schmea 顺序 和用户配置的 column,计算写datahub的顺序关系,以支持列换序 + // 后续统一使用 columnIndex 的顺位关系写 datahub + int totalSize = this.schema.getFields().size(); + if (null != this.column && !this.column.isEmpty() && this.enableColumnConfig) { + for (String eachCol : this.column) { + int indexFound = -1; + for (int i = 0; i < totalSize; i++) { + // warn: 大小写ignore + if (StringUtils.equalsIgnoreCase(eachCol, this.schema.getField(i).getName())) { + indexFound = i; + break; + } + } + if (indexFound >= 0) { + this.columnIndex.add(indexFound); + } else { + throw DataXException.asDataXException(DatahubWriterErrorCode.SCHEMA_NOT_MATCH, + String.format("can not find column %s in datahub topic %s", eachCol, this.topic)); + } + } + } else { + for (int i = 0; i < totalSize; i++) { + this.columnIndex.add(i); + } + } + } + + @Override + public void startWrite(RecordReceiver recordReceiver) { + Record record; + List records = new ArrayList(); + String shardId = null; + if (1 == this.shards.size()) { + shardId = shards.get(0); + } else { + shardId = shards.get(this.random.nextInt(shards.size())); + } + int commitSize = 0; + try { + while ((record = recordReceiver.getFromReader()) != null) { + RecordEntry dhRecord = convertRecord(record, shardId); + if (dhRecord != null) { + records.add(dhRecord); + } + commitSize += record.getByteSize(); + if (commitSize >= maxCommitSize) { + commit(records); + records.clear(); + commitSize = 0; + if (1 == this.shards.size()) { + shardId = shards.get(0); + } else { + shardId = shards.get(this.random.nextInt(shards.size())); + } + } + } + if (commitSize > 0) { + commit(records); + } + } catch (Exception e) { + throw DataXException.asDataXException( + DatahubWriterErrorCode.WRITE_DATAHUB_FAIL, e); + } + } + + @Override + public void post() {} + + @Override + public void destroy() {} + + private void commit(List records) throws InterruptedException { + PutRecordsResult result = client.putRecords(project, topic, records); + if (result.getFailedRecordCount() > 0) { + for (int i = 0; i < maxRetryCount; ++i) { + boolean limitExceededMessagePrinted = false; + for (PutErrorEntry error : result.getPutErrorEntries()) { + // 如果是 LimitExceeded 这样打印日志,不能每行记录打印一次了 + if (StringUtils.equalsIgnoreCase("LimitExceeded", error.getErrorcode())) { + if (!limitExceededMessagePrinted) { + LOG.warn("write record error, request id: {}, error code: {}, error message: {}", + result.getRequestId(), error.getErrorcode(), error.getMessage()); + limitExceededMessagePrinted = true; + } + } else { + LOG.error("write record error, request id: {}, error code: {}, error message: {}", + result.getRequestId(), error.getErrorcode(), error.getMessage()); + } + if (this.fatalErrors.contains(error.getErrorcode())) { + throw DataXException.asDataXException( + DatahubWriterErrorCode.WRITE_DATAHUB_FAIL, + error.getMessage()); + } + } + + if (this.retryInterval >= 0) { + Thread.sleep(this.retryInterval); + } else { + Thread.sleep(new Random().nextInt(700) + 300); + } + + result = client.putRecords(project, topic, result.getFailedRecords()); + if (result.getFailedRecordCount() == 0) { + return; + } + } + throw DataXException.asDataXException( + DatahubWriterErrorCode.WRITE_DATAHUB_FAIL, + "write datahub failed"); + } + } + + private RecordEntry convertRecord(Record dxRecord, String shardId) { + try { + RecordEntry dhRecord = new RecordEntry(); + dhRecord.setShardId(shardId); + TupleRecordData data = new TupleRecordData(this.schema); + for (int i = 0; i < this.columnIndex.size(); ++i) { + int orderInSchema = this.columnIndex.get(i); + FieldType type = this.schema.getField(orderInSchema).getType(); + Column column = dxRecord.getColumn(i); + switch (type) { + case BIGINT: + data.setField(orderInSchema, column.asLong()); + break; + case DOUBLE: + data.setField(orderInSchema, column.asDouble()); + break; + case STRING: + data.setField(orderInSchema, column.asString()); + break; + case BOOLEAN: + data.setField(orderInSchema, column.asBoolean()); + break; + case TIMESTAMP: + if (null == column.asDate()) { + data.setField(orderInSchema, null); + } else { + data.setField(orderInSchema, column.asDate().getTime() * 1000); + } + break; + case DECIMAL: + // warn + data.setField(orderInSchema, column.asBigDecimal()); + break; + case INTEGER: + data.setField(orderInSchema, column.asLong()); + break; + case FLOAT: + data.setField(orderInSchema, column.asDouble()); + break; + case TINYINT: + data.setField(orderInSchema, column.asLong()); + break; + case SMALLINT: + data.setField(orderInSchema, column.asLong()); + break; + default: + throw DataXException.asDataXException( + DatahubWriterErrorCode.SCHEMA_NOT_MATCH, + String.format("does not support type: %s", type)); + } + } + dhRecord.setRecordData(data); + return dhRecord; + } catch (Exception e) { + super.getTaskPluginCollector().collectDirtyRecord(dxRecord, e, "convert recor failed"); + } + return null; + } + } + +} \ No newline at end of file diff --git a/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/DatahubWriterErrorCode.java b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/DatahubWriterErrorCode.java new file mode 100644 index 0000000000..ad03abd199 --- /dev/null +++ b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/DatahubWriterErrorCode.java @@ -0,0 +1,37 @@ +package com.alibaba.datax.plugin.writer.datahubwriter; + +import com.alibaba.datax.common.spi.ErrorCode; +import com.alibaba.datax.common.util.MessageSource; + +public enum DatahubWriterErrorCode implements ErrorCode { + MISSING_REQUIRED_VALUE("DatahubWriter-01", MessageSource.loadResourceBundle(DatahubWriterErrorCode.class).message("errorcode.missing_required_value")), + INVALID_CONFIG_VALUE("DatahubWriter-02", MessageSource.loadResourceBundle(DatahubWriterErrorCode.class).message("errorcode.invalid_config_value")), + GET_TOPOIC_INFO_FAIL("DatahubWriter-03", MessageSource.loadResourceBundle(DatahubWriterErrorCode.class).message("errorcode.get_topic_info_fail")), + WRITE_DATAHUB_FAIL("DatahubWriter-04", MessageSource.loadResourceBundle(DatahubWriterErrorCode.class).message("errorcode.write_datahub_fail")), + SCHEMA_NOT_MATCH("DatahubWriter-05", MessageSource.loadResourceBundle(DatahubWriterErrorCode.class).message("errorcode.schema_not_match")), + ; + + private final String code; + private final String description; + + private DatahubWriterErrorCode(String code, String description) { + this.code = code; + this.description = description; + } + + @Override + public String getCode() { + return this.code; + } + + @Override + public String getDescription() { + return this.description; + } + + @Override + public String toString() { + return String.format("Code:[%s], Description:[%s]. ", this.code, + this.description); + } +} \ No newline at end of file diff --git a/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/Key.java b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/Key.java new file mode 100644 index 0000000000..5f17923470 --- /dev/null +++ b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/Key.java @@ -0,0 +1,26 @@ +package com.alibaba.datax.plugin.writer.datahubwriter; + +public final class Key { + + /** + * 此处声明插件用到的需要插件使用者提供的配置项 + */ + public static final String CONFIG_KEY_ENDPOINT = "endpoint"; + public static final String CONFIG_KEY_ACCESS_ID = "accessId"; + public static final String CONFIG_KEY_ACCESS_KEY = "accessKey"; + public static final String CONFIG_KEY_PROJECT = "project"; + public static final String CONFIG_KEY_TOPIC = "topic"; + public static final String CONFIG_KEY_WRITE_MODE = "mode"; + public static final String CONFIG_KEY_SHARD_ID = "shardId"; + public static final String CONFIG_KEY_MAX_COMMIT_SIZE = "maxCommitSize"; + public static final String CONFIG_KEY_MAX_RETRY_COUNT = "maxRetryCount"; + + public static final String CONFIG_VALUE_SEQUENCE_MODE = "sequence"; + public static final String CONFIG_VALUE_RANDOM_MODE = "random"; + + public final static String MAX_RETRY_TIME = "maxRetryTime"; + + public final static String RETRY_INTERVAL = "retryInterval"; + + public final static String CONFIG_KEY_COLUMN = "column"; +} diff --git a/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings.properties b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings.properties new file mode 100644 index 0000000000..e85c8ab321 --- /dev/null +++ b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings.properties @@ -0,0 +1,5 @@ +errorcode.missing_required_value=\u60A8\u7F3A\u5931\u4E86\u5FC5\u987B\u586B\u5199\u7684\u53C2\u6570\u503C. +errorcode.invalid_config_value=\u60A8\u7684\u53C2\u6570\u914D\u7F6E\u9519\u8BEF. +errorcode.get_topic_info_fail=\u83B7\u53D6shard\u5217\u8868\u5931\u8D25. +errorcode.write_datahub_fail=\u5199\u6570\u636E\u5931\u8D25. +errorcode.schema_not_match=\u6570\u636E\u683C\u5F0F\u9519\u8BEF. diff --git a/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_en_US.properties b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_en_US.properties new file mode 100644 index 0000000000..31a291e6db --- /dev/null +++ b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_en_US.properties @@ -0,0 +1,5 @@ +errorcode.missing_required_value=\u60A8\u7F3A\u5931\u4E86\u5FC5\u987B\u586B\u5199\u7684\u53C2\u6570\u503C. +errorcode.invalid_config_value=\u60A8\u7684\u53C2\u6570\u914D\u7F6E\u9519\u8BEF. +errorcode.get_topic_info_fail=\u83B7\u53D6shard\u5217\u8868\u5931\u8D25. +errorcode.write_datahub_fail=\u5199\u6570\u636E\u5931\u8D25. +errorcode.schema_not_match=\u6570\u636E\u683C\u5F0F\u9519\u8BEF. \ No newline at end of file diff --git a/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_ja_JP.properties b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_ja_JP.properties new file mode 100644 index 0000000000..31a291e6db --- /dev/null +++ b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_ja_JP.properties @@ -0,0 +1,5 @@ +errorcode.missing_required_value=\u60A8\u7F3A\u5931\u4E86\u5FC5\u987B\u586B\u5199\u7684\u53C2\u6570\u503C. +errorcode.invalid_config_value=\u60A8\u7684\u53C2\u6570\u914D\u7F6E\u9519\u8BEF. +errorcode.get_topic_info_fail=\u83B7\u53D6shard\u5217\u8868\u5931\u8D25. +errorcode.write_datahub_fail=\u5199\u6570\u636E\u5931\u8D25. +errorcode.schema_not_match=\u6570\u636E\u683C\u5F0F\u9519\u8BEF. \ No newline at end of file diff --git a/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_zh_CN.properties b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_zh_CN.properties new file mode 100644 index 0000000000..31a291e6db --- /dev/null +++ b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_zh_CN.properties @@ -0,0 +1,5 @@ +errorcode.missing_required_value=\u60A8\u7F3A\u5931\u4E86\u5FC5\u987B\u586B\u5199\u7684\u53C2\u6570\u503C. +errorcode.invalid_config_value=\u60A8\u7684\u53C2\u6570\u914D\u7F6E\u9519\u8BEF. +errorcode.get_topic_info_fail=\u83B7\u53D6shard\u5217\u8868\u5931\u8D25. +errorcode.write_datahub_fail=\u5199\u6570\u636E\u5931\u8D25. +errorcode.schema_not_match=\u6570\u636E\u683C\u5F0F\u9519\u8BEF. \ No newline at end of file diff --git a/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_zh_HK.properties b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_zh_HK.properties new file mode 100644 index 0000000000..c6a3a0e01d --- /dev/null +++ b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_zh_HK.properties @@ -0,0 +1,9 @@ +errorcode.missing_required_value=\u60A8\u7F3A\u5931\u4E86\u5FC5\u987B\u586B\u5199\u7684\u53C2\u6570\u503C. +errorcode.invalid_config_value=\u60A8\u7684\u53C2\u6570\u914D\u7F6E\u9519\u8BEF. +errorcode.get_topic_info_fail=\u83B7\u53D6shard\u5217\u8868\u5931\u8D25. +errorcode.write_datahub_fail=\u5199\u6570\u636E\u5931\u8D25. +errorcode.schema_not_match=\u6570\u636E\u683C\u5F0F\u9519\u8BEF.errorcode.missing_required_value=您缺失了必須填寫的參數值. +errorcode.invalid_config_value=您的參數配寘錯誤. +errorcode.get_topic_info_fail=獲取shard清單失敗. +errorcode.write_datahub_fail=寫數據失敗. +errorcode.schema_not_match=數據格式錯誤. diff --git a/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_zh_TW.properties b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_zh_TW.properties new file mode 100644 index 0000000000..c6a3a0e01d --- /dev/null +++ b/datahubwriter/src/main/java/com/alibaba/datax/plugin/writer/datahubwriter/LocalStrings_zh_TW.properties @@ -0,0 +1,9 @@ +errorcode.missing_required_value=\u60A8\u7F3A\u5931\u4E86\u5FC5\u987B\u586B\u5199\u7684\u53C2\u6570\u503C. +errorcode.invalid_config_value=\u60A8\u7684\u53C2\u6570\u914D\u7F6E\u9519\u8BEF. +errorcode.get_topic_info_fail=\u83B7\u53D6shard\u5217\u8868\u5931\u8D25. +errorcode.write_datahub_fail=\u5199\u6570\u636E\u5931\u8D25. +errorcode.schema_not_match=\u6570\u636E\u683C\u5F0F\u9519\u8BEF.errorcode.missing_required_value=您缺失了必須填寫的參數值. +errorcode.invalid_config_value=您的參數配寘錯誤. +errorcode.get_topic_info_fail=獲取shard清單失敗. +errorcode.write_datahub_fail=寫數據失敗. +errorcode.schema_not_match=數據格式錯誤. diff --git a/datahubwriter/src/main/resources/job_config_template.json b/datahubwriter/src/main/resources/job_config_template.json new file mode 100644 index 0000000000..8b0b41aef8 --- /dev/null +++ b/datahubwriter/src/main/resources/job_config_template.json @@ -0,0 +1,14 @@ +{ + "name": "datahubwriter", + "parameter": { + "endpoint":"", + "accessId": "", + "accessKey": "", + "project": "", + "topic": "", + "mode": "random", + "shardId": "", + "maxCommitSize": 524288, + "maxRetryCount": 500 + } +} \ No newline at end of file diff --git a/datahubwriter/src/main/resources/plugin.json b/datahubwriter/src/main/resources/plugin.json new file mode 100644 index 0000000000..91c1729297 --- /dev/null +++ b/datahubwriter/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "datahubwriter", + "class": "com.alibaba.datax.plugin.writer.datahubwriter.DatahubWriter", + "description": "datahub writer", + "developer": "alibaba" +} \ No newline at end of file diff --git a/doriswriter/doc/doriswriter.md b/doriswriter/doc/doriswriter.md new file mode 100644 index 0000000000..ef5bbe3739 --- /dev/null +++ b/doriswriter/doc/doriswriter.md @@ -0,0 +1,181 @@ +# DorisWriter 插件文档 + +## 1 快速介绍 +DorisWriter支持将大批量数据写入Doris中。 + +## 2 实现原理 +DorisWriter 通过Doris原生支持Stream load方式导入数据, DorisWriter会将`reader`读取的数据进行缓存在内存中,拼接成Json文本,然后批量导入至Doris。 + +## 3 功能说明 + +### 3.1 配置样例 + +这里是一份从Stream读取数据后导入至Doris的配置文件。 + +``` +{ + "job": { + "content": [ + { + "reader": { + "name": "mysqlreader", + "parameter": { + "column": ["emp_no", "birth_date", "first_name","last_name","gender","hire_date"], + "connection": [ + { + "jdbcUrl": ["jdbc:mysql://localhost:3306/demo"], + "table": ["employees_1"] + } + ], + "username": "root", + "password": "xxxxx", + "where": "" + } + }, + "writer": { + "name": "doriswriter", + "parameter": { + "loadUrl": ["172.16.0.13:8030"], + "loadProps": { + }, + "column": ["emp_no", "birth_date", "first_name","last_name","gender","hire_date"], + "username": "root", + "password": "xxxxxx", + "postSql": ["select count(1) from all_employees_info"], + "preSql": [], + "flushInterval":30000, + "connection": [ + { + "jdbcUrl": "jdbc:mysql://172.16.0.13:9030/demo", + "selectedDatabase": "demo", + "table": ["all_employees_info"] + } + ], + "loadProps": { + "format": "json", + "strip_outer_array": true + } + } + } + } + ], + "setting": { + "speed": { + "channel": "1" + } + } + } +} +``` + +### 3.2 参数说明 + +* **jdbcUrl** + + - 描述:Doris 的 JDBC 连接串,用户执行 preSql 或 postSQL。 + - 必选:是 + - 默认值:无 + +* **loadUrl** + + - 描述:作为 Stream Load 的连接目标。格式为 "ip:port"。其中 IP 是 FE 节点 IP,port 是 FE 节点的 http_port。可以填写多个,多个之间使用英文状态的分号隔开:`;`,doriswriter 将以轮询的方式访问。 + - 必选:是 + - 默认值:无 + +* **username** + + - 描述:访问Doris数据库的用户名 + - 必选:是 + - 默认值:无 + +* **password** + + - 描述:访问Doris数据库的密码 + - 必选:否 + - 默认值:空 + +* **connection.selectedDatabase** + - 描述:需要写入的Doris数据库名称。 + - 必选:是 + - 默认值:无 + +* **connection.table** + - 描述:需要写入的Doris表名称。 + - 必选:是 + - 默认值:无 + +* **column** + + - 描述:目的表**需要写入数据**的字段,这些字段将作为生成的 Json 数据的字段名。字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 + - 必选:是 + - 默认值:否 + +* **preSql** + + - 描述:写入数据到目的表前,会先执行这里的标准语句。 + - 必选:否 + - 默认值:无 + +* **postSql** + + - 描述:写入数据到目的表后,会执行这里的标准语句。 + - 必选:否 + - 默认值:无 + + +* **maxBatchRows** + + - 描述:每批次导入数据的最大行数。和 **maxBatchSize** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 + - 必选:否 + - 默认值:500000 + +* **batchSize** + + - 描述:每批次导入数据的最大数据量。和 **maxBatchRows** 共同控制每批次的导入数量。每批次数据达到两个阈值之一,即开始导入这一批次的数据。 + - 必选:否 + - 默认值:104857600 + +* **maxRetries** + + - 描述:每批次导入数据失败后的重试次数。 + - 必选:否 + - 默认值:0 + +* **labelPrefix** + + - 描述:每批次导入任务的 label 前缀。最终的 label 将有 `labelPrefix + UUID` 组成全局唯一的 label,确保数据不会重复导入 + - 必选:否 + - 默认值:`datax_doris_writer_` + +* **loadProps** + + - 描述: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 官方信息 + + - 必选:否 + + - 默认值:无 + +### 类型转换 + +默认传入的数据均会被转为字符串,并以`\t`作为列分隔符,`\n`作为行分隔符,组成`csv`文件进行StreamLoad导入操作。 + +默认是csv格式导入,如需更改列分隔符, 则正确配置 `loadProps` 即可: + +```json +"loadProps": { + "column_separator": "\\x01", + "row_delimiter": "\\x02" +} +``` + +如需更改导入格式为`json`, 则正确配置 `loadProps` 即可: +```json +"loadProps": { + "format": "json", + "strip_outer_array": true +} +``` + +更多信息请参照 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/doc/mysql2doris.json b/doriswriter/doc/mysql2doris.json new file mode 100644 index 0000000000..6992a2be45 --- /dev/null +++ b/doriswriter/doc/mysql2doris.json @@ -0,0 +1,46 @@ +{ + "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": "", + "where": "" + } + }, + "writer": { + "name": "doriswriter", + "parameter": { + "loadUrl": ["192.168.1.1:8030"], + "loadProps": {}, + "database": "db1", + "column": ["k1", "k2", "k3"], + "username": "root", + "password": "", + "postSql": [], + "preSql": [], + "connection": [ + "jdbcUrl":"jdbc:mysql://192.168.1.1:9030/", + "table":["xxx"], + "selectedDatabase":"xxxx" + ] + } + } + } + ], + "setting": { + "speed": { + "channel": "1" + } + } + } +} diff --git a/doriswriter/pom.xml b/doriswriter/pom.xml new file mode 100644 index 0000000000..aa1e6ff080 --- /dev/null +++ b/doriswriter/pom.xml @@ -0,0 +1,99 @@ + + + + + datax-all + com.alibaba.datax + 0.0.1-SNAPSHOT + + 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 0000000000..715963328e --- /dev/null +++ b/doriswriter/src/main/assembly/package.xml @@ -0,0 +1,52 @@ + + + + + + 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/DelimiterParser.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DelimiterParser.java new file mode 100644 index 0000000000..e84bd7dd85 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DelimiterParser.java @@ -0,0 +1,54 @@ +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.google.common.base.Strings; + +import java.io.StringWriter; + +public class DelimiterParser { + + 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/DorisBaseCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseCodec.java new file mode 100644 index 0000000000..ee7ded5606 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisBaseCodec.java @@ -0,0 +1,23 @@ +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.element.Column; + +public class DorisBaseCodec { + protected String convertionField( 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 new file mode 100644 index 0000000000..a2437a1cbb --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCodec.java @@ -0,0 +1,10 @@ +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.element.Record; + +import java.io.Serializable; + +public interface DorisCodec extends Serializable { + + 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 0000000000..22c4b4099b --- /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/DorisCsvCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java new file mode 100644 index 0000000000..518aa3043f --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisCsvCodec.java @@ -0,0 +1,27 @@ +package com.alibaba.datax.plugin.writer.doriswriter; + +import com.alibaba.datax.common.element.Record; + +public class DorisCsvCodec extends DorisBaseCodec implements DorisCodec { + + private static final long serialVersionUID = 1L; + + private final String columnSeparator; + + public DorisCsvCodec ( String sp) { + this.columnSeparator = DelimiterParser.parse(sp, "\t"); + } + + @Override + public String codec( Record row) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < row.getColumnNumber(); i++) { + String value = convertionField(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/DorisJsonCodec.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java new file mode 100644 index 0000000000..e6c05733f2 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisJsonCodec.java @@ -0,0 +1,33 @@ +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 DorisJsonCodec extends DorisBaseCodec implements DorisCodec { + + private static final long serialVersionUID = 1L; + + private final List fieldNames; + + public DorisJsonCodec ( List fieldNames) { + this.fieldNames = fieldNames; + } + + @Override + 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, convertionField(row.getColumn(idx))); + idx++; + } + return JSON.toJSONString(rowMap); + } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadObserver.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadObserver.java new file mode 100644 index 0000000000..3e4db6cfcf --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisStreamLoadObserver.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 DorisStreamLoadObserver { + private static final Logger LOG = LoggerFactory.getLogger(DorisStreamLoadObserver.class); + + private Keys 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 DorisStreamLoadObserver ( Keys options){ + this.options = options; + } + + public void streamLoad(WriterTuple data) throws Exception { + String host = getLoadHost(); + if(host == null){ + throw new IOException ("load_url cannot be empty, or the host cannot connect.Please check your configuration."); + } + String loadUrl = new StringBuilder(host) + .append("/api/") + .append(options.getDatabase()) + .append("/") + .append(options.getTable()) + .append("/_stream_load") + .toString(); + 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("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("StreamLoad response:{}",JSON.toJSONString(loadResult)); + checkStreamLoadState(host, data.getLabel()); + } + } + + private void checkStreamLoadState(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 DorisWriterExcetion (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[] addRows(List rows, int totalBytes) { + if (Keys.StreamLoadFormat.CSV.equals(options.getStreamLoadFormat())) { + Map props = (options.getLoadProps() == null ? new HashMap<> () : options.getLoadProps()); + 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); + bos.put(lineDelimiter); + } + return bos.array(); + } + + 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); + 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 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 () { + @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() && 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()) { + 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 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 (checkConnection(host)) { + return host; + } + } + return null; + } + + private boolean checkConnection(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) { + 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 new file mode 100644 index 0000000000..5f5a6f3443 --- /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( Keys 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( Keys 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 new file mode 100644 index 0000000000..b44d544068 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriter.java @@ -0,0 +1,164 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES 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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.util.ArrayList; +import java.util.List; + +/** + * doris data writer + */ +public class DorisWriter extends Writer { + + public static class Job extends Writer.Job { + + private static final Logger LOG = LoggerFactory.getLogger(Job.class); + private Configuration originalConfig = null; + private Keys options; + + @Override + public void init() { + this.originalConfig = super.getPluginJobConf(); + options = new Keys (super.getPluginJobConf()); + options.doPretreatment(); + } + + @Override + public void preCheck(){ + this.init(); + DorisUtil.preCheckPrePareSQL(options); + DorisUtil.preCheckPostSQL(options); + } + + @Override + public void prepare() { + 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("Begin to execute preSqls:[{}]. context info:{}.", String.join(";", renderedPreSqls), jdbcUrl); + DorisUtil.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(originalConfig); + } + return configurations; + } + + @Override + public void post() { + String username = options.getUsername(); + String password = options.getPassword(); + String jdbcUrl = options.getJdbcUrl(); + List renderedPostSqls = DorisUtil.renderPreOrPostSqls(options.getPostSqlList(), options.getTable()); + if (null != renderedPostSqls && !renderedPostSqls.isEmpty()) { + Connection conn = DBUtil.getConnection(DataBaseType.MySql, jdbcUrl, username, password); + LOG.info("Start to execute preSqls:[{}]. context info:{}.", String.join(";", renderedPostSqls), jdbcUrl); + DorisUtil.executeSqls(conn, renderedPostSqls); + DBUtil.closeDBResources(null, null, conn); + } + } + + @Override + public void destroy() { + } + + } + + public static class Task extends Writer.Task { + private DorisWriterManager writerManager; + private Keys options; + private DorisCodec rowCodec; + + @Override + public void init() { + 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); + rowCodec = DorisCodecFactory.createCodec(options); + } + + @Override + public void prepare() { + } + + public void startWrite(RecordReceiver recordReceiver) { + try { + Record record; + while ((record = recordReceiver.getFromReader()) != null) { + if (record.getColumnNumber() != options.getColumns().size()) { + throw DataXException + .asDataXException( + DBUtilErrorCode.CONF_ERROR, + String.format( + "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(rowCodec.codec(record)); + } + } catch (Exception e) { + throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, e); + } + } + + @Override + public void post() { + try { + writerManager.close(); + } catch (Exception e) { + throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, e); + } + } + + @Override + public void destroy() {} + + @Override + public boolean supportFailOver(){ + return false; + } + } +} diff --git a/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterExcetion.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterExcetion.java new file mode 100644 index 0000000000..7797d79fdd --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterExcetion.java @@ -0,0 +1,29 @@ +package com.alibaba.datax.plugin.writer.doriswriter; + +import java.io.IOException; +import java.util.Map; + +public class DorisWriterExcetion extends IOException { + + private final Map response; + private boolean reCreateLabel; + + public DorisWriterExcetion ( String message, Map response) { + super(message); + this.response = response; + } + + public DorisWriterExcetion ( 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/DorisWriterManager.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/DorisWriterManager.java new file mode 100644 index 0000000000..f0ba6b5283 --- /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 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< WriterTuple > flushQueue; + private ScheduledExecutorService scheduler; + private ScheduledFuture scheduledFuture; + + public DorisWriterManager( Keys options) { + this.options = options; + this.visitor = new DorisStreamLoadObserver (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 WriterTuple (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 WriterTuple ("", 0l, null)); + } + checkFlushException(); + } + + private void asyncFlush() throws Exception { + WriterTuple 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 DorisWriterExcetion && (( DorisWriterExcetion )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/Keys.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/Keys.java new file mode 100644 index 0000000000..e460e76b83 --- /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 = "batchSize"; + 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/WriterTuple.java b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/WriterTuple.java new file mode 100644 index 0000000000..32e0b341b8 --- /dev/null +++ b/doriswriter/src/main/java/com/alibaba/datax/plugin/writer/doriswriter/WriterTuple.java @@ -0,0 +1,20 @@ +package com.alibaba.datax.plugin.writer.doriswriter; + +import java.util.List; + +public class WriterTuple { + private String label; + private Long bytes; + private List rows; + + public WriterTuple ( 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/resources/plugin.json b/doriswriter/src/main/resources/plugin.json new file mode 100644 index 0000000000..69dc31a26c --- /dev/null +++ b/doriswriter/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "doriswriter", + "class": "com.alibaba.datax.plugin.writer.doriswriter.DorisWriter", + "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 new file mode 100644 index 0000000000..0187e53965 --- /dev/null +++ b/doriswriter/src/main/resources/plugin_job_template.json @@ -0,0 +1,20 @@ +{ + "name": "doriswriter", + "parameter": { + "username": "", + "password": "", + "column": [], + "preSql": [], + "postSql": [], + "beLoadUrl": [], + "loadUrl": [], + "loadProps": {}, + "connection": [ + { + "jdbcUrl": "", + "selectedDatabase": "", + "table": [] + } + ] + } +} \ No newline at end of file diff --git a/elasticsearchwriter/pom.xml b/elasticsearchwriter/pom.xml index a60dbd88ce..8699c6e599 100644 --- a/elasticsearchwriter/pom.xml +++ b/elasticsearchwriter/pom.xml @@ -35,12 +35,12 @@ io.searchbox jest-common - 2.4.0 + 6.3.1 io.searchbox jest - 2.4.0 + 6.3.1 joda-time diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESClient.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESClient.java deleted file mode 100644 index 34bb7e5420..0000000000 --- a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESClient.java +++ /dev/null @@ -1,236 +0,0 @@ -package com.alibaba.datax.plugin.writer.elasticsearchwriter; - -import com.google.gson.Gson; -import com.google.gson.JsonElement; -import com.google.gson.JsonObject; -import com.google.gson.JsonParser; -import io.searchbox.action.Action; -import io.searchbox.client.JestClient; -import io.searchbox.client.JestClientFactory; -import io.searchbox.client.JestResult; -import io.searchbox.client.config.HttpClientConfig; -import io.searchbox.client.config.HttpClientConfig.Builder; -import io.searchbox.core.Bulk; -import io.searchbox.indices.CreateIndex; -import io.searchbox.indices.DeleteIndex; -import io.searchbox.indices.IndicesExists; -import io.searchbox.indices.aliases.*; -import io.searchbox.indices.mapping.PutMapping; -import org.apache.http.HttpHost; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -/** - * Created by xiongfeng.bxf on 17/2/8. - */ -public class ESClient { - private static final Logger log = LoggerFactory.getLogger(ESClient.class); - - private JestClient jestClient; - - public JestClient getClient() { - return jestClient; - } - - public void createClient(String endpoint, - String user, - String passwd, - boolean multiThread, - int readTimeout, - boolean compression, - boolean discovery) { - - JestClientFactory factory = new JestClientFactory(); - Builder httpClientConfig = new HttpClientConfig - .Builder(endpoint) - .setPreemptiveAuth(new HttpHost(endpoint)) - .multiThreaded(multiThread) - .connTimeout(30000) - .readTimeout(readTimeout) - .maxTotalConnection(200) - .requestCompressionEnabled(compression) - .discoveryEnabled(discovery) - .discoveryFrequency(5l, TimeUnit.MINUTES); - - if (!("".equals(user) || "".equals(passwd))) { - httpClientConfig.defaultCredentials(user, passwd); - } - - factory.setHttpClientConfig(httpClientConfig.build()); - - jestClient = factory.getObject(); - } - - public boolean indicesExists(String indexName) throws Exception { - boolean isIndicesExists = false; - JestResult rst = jestClient.execute(new IndicesExists.Builder(indexName).build()); - if (rst.isSucceeded()) { - isIndicesExists = true; - } else { - switch (rst.getResponseCode()) { - case 404: - isIndicesExists = false; - break; - case 401: - // 无权访问 - default: - log.warn(rst.getErrorMessage()); - break; - } - } - return isIndicesExists; - } - - public boolean deleteIndex(String indexName) throws Exception { - log.info("delete index " + indexName); - if (indicesExists(indexName)) { - JestResult rst = execute(new DeleteIndex.Builder(indexName).build()); - if (!rst.isSucceeded()) { - return false; - } - } else { - log.info("index cannot found, skip delete " + indexName); - } - return true; - } - - public boolean createIndex(String indexName, String typeName, - Object mappings, String settings, boolean dynamic) throws Exception { - JestResult rst = null; - if (!indicesExists(indexName)) { - log.info("create index " + indexName); - rst = jestClient.execute( - new CreateIndex.Builder(indexName) - .settings(settings) - .setParameter("master_timeout", "5m") - .build() - ); - //index_already_exists_exception - if (!rst.isSucceeded()) { - if (getStatus(rst) == 400) { - log.info(String.format("index [%s] already exists", indexName)); - return true; - } else { - log.error(rst.getErrorMessage()); - return false; - } - } else { - log.info(String.format("create [%s] index success", indexName)); - } - } - - int idx = 0; - while (idx < 5) { - if (indicesExists(indexName)) { - break; - } - Thread.sleep(2000); - idx ++; - } - if (idx >= 5) { - return false; - } - - if (dynamic) { - log.info("ignore mappings"); - return true; - } - log.info("create mappings for " + indexName + " " + mappings); - rst = jestClient.execute(new PutMapping.Builder(indexName, typeName, mappings) - .setParameter("master_timeout", "5m").build()); - if (!rst.isSucceeded()) { - if (getStatus(rst) == 400) { - log.info(String.format("index [%s] mappings already exists", indexName)); - } else { - log.error(rst.getErrorMessage()); - return false; - } - } else { - log.info(String.format("index [%s] put mappings success", indexName)); - } - return true; - } - - public JestResult execute(Action clientRequest) throws Exception { - JestResult rst = null; - rst = jestClient.execute(clientRequest); - if (!rst.isSucceeded()) { - //log.warn(rst.getErrorMessage()); - } - return rst; - } - - public Integer getStatus(JestResult rst) { - JsonObject jsonObject = rst.getJsonObject(); - if (jsonObject.has("status")) { - return jsonObject.get("status").getAsInt(); - } - return 600; - } - - public boolean isBulkResult(JestResult rst) { - JsonObject jsonObject = rst.getJsonObject(); - return jsonObject.has("items"); - } - - - public boolean alias(String indexname, String aliasname, boolean needClean) throws IOException { - GetAliases getAliases = new GetAliases.Builder().addIndex(aliasname).build(); - AliasMapping addAliasMapping = new AddAliasMapping.Builder(indexname, aliasname).build(); - JestResult rst = jestClient.execute(getAliases); - log.info(rst.getJsonString()); - List list = new ArrayList(); - if (rst.isSucceeded()) { - JsonParser jp = new JsonParser(); - JsonObject jo = (JsonObject)jp.parse(rst.getJsonString()); - for(Map.Entry entry : jo.entrySet()){ - String tindex = entry.getKey(); - if (indexname.equals(tindex)) { - continue; - } - AliasMapping m = new RemoveAliasMapping.Builder(tindex, aliasname).build(); - String s = new Gson().toJson(m.getData()); - log.info(s); - if (needClean) { - list.add(m); - } - } - } - - ModifyAliases modifyAliases = new ModifyAliases.Builder(addAliasMapping).addAlias(list).setParameter("master_timeout", "5m").build(); - rst = jestClient.execute(modifyAliases); - if (!rst.isSucceeded()) { - log.error(rst.getErrorMessage()); - return false; - } - return true; - } - - public JestResult bulkInsert(Bulk.Builder bulk, int trySize) throws Exception { - // es_rejected_execution_exception - // illegal_argument_exception - // cluster_block_exception - JestResult rst = null; - rst = jestClient.execute(bulk.build()); - if (!rst.isSucceeded()) { - log.warn(rst.getErrorMessage()); - } - return rst; - } - - /** - * 关闭JestClient客户端 - * - */ - public void closeJestClient() { - if (jestClient != null) { - jestClient.shutdownClient(); - } - } -} diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESColumn.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESColumn.java deleted file mode 100644 index 8990d77c21..0000000000 --- a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESColumn.java +++ /dev/null @@ -1,65 +0,0 @@ -package com.alibaba.datax.plugin.writer.elasticsearchwriter; - -/** - * Created by xiongfeng.bxf on 17/3/2. - */ -public class ESColumn { - - private String name;//: "appkey", - - private String type;//": "TEXT", - - private String timezone; - - private String format; - - private Boolean array; - - public void setName(String name) { - this.name = name; - } - - public void setType(String type) { - this.type = type; - } - - public void setTimeZone(String timezone) { - this.timezone = timezone; - } - - public void setFormat(String format) { - this.format = format; - } - - public String getName() { - return name; - } - - public String getType() { - return type; - } - - public String getTimezone() { - return timezone; - } - - public String getFormat() { - return format; - } - - public void setTimezone(String timezone) { - this.timezone = timezone; - } - - public Boolean isArray() { - return array; - } - - public void setArray(Boolean array) { - this.array = array; - } - - public Boolean getArray() { - return array; - } -} diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESWriter.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESWriter.java deleted file mode 100644 index eb0e9a8137..0000000000 --- a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESWriter.java +++ /dev/null @@ -1,460 +0,0 @@ -package com.alibaba.datax.plugin.writer.elasticsearchwriter; - -import com.alibaba.datax.common.element.Column; -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.common.util.RetryUtil; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.JSONObject; -import com.alibaba.fastjson.TypeReference; -import io.searchbox.client.JestResult; -import io.searchbox.core.Bulk; -import io.searchbox.core.BulkResult; -import io.searchbox.core.Index; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.net.URLEncoder; -import java.util.*; -import java.util.concurrent.Callable; - -public class ESWriter extends Writer { - private final static String WRITE_COLUMNS = "write_columns"; - - public static class Job extends Writer.Job { - private static final Logger log = LoggerFactory.getLogger(Job.class); - - private Configuration conf = null; - - @Override - public void init() { - this.conf = super.getPluginJobConf(); - } - - @Override - public void prepare() { - /** - * 注意:此方法仅执行一次。 - * 最佳实践:如果 Job 中有需要进行数据同步之前的处理,可以在此处完成,如果没有必要则可以直接去掉。 - */ - ESClient esClient = new ESClient(); - esClient.createClient(Key.getEndpoint(conf), - Key.getAccessID(conf), - Key.getAccessKey(conf), - false, - 300000, - false, - false); - - String indexName = Key.getIndexName(conf); - String typeName = Key.getTypeName(conf); - boolean dynamic = Key.getDynamic(conf); - String mappings = genMappings(typeName); - String settings = JSONObject.toJSONString( - Key.getSettings(conf) - ); - log.info(String.format("index:[%s], type:[%s], mappings:[%s]", indexName, typeName, mappings)); - - try { - boolean isIndicesExists = esClient.indicesExists(indexName); - if (Key.isCleanup(this.conf) && isIndicesExists) { - esClient.deleteIndex(indexName); - } - // 强制创建,内部自动忽略已存在的情况 - if (!esClient.createIndex(indexName, typeName, mappings, settings, dynamic)) { - throw new IOException("create index or mapping failed"); - } - } catch (Exception ex) { - throw DataXException.asDataXException(ESWriterErrorCode.ES_MAPPINGS, ex.toString()); - } - esClient.closeJestClient(); - } - - private String genMappings(String typeName) { - String mappings = null; - Map propMap = new HashMap(); - List columnList = new ArrayList(); - - List column = conf.getList("column"); - if (column != null) { - for (Object col : column) { - JSONObject jo = JSONObject.parseObject(col.toString()); - String colName = jo.getString("name"); - String colTypeStr = jo.getString("type"); - if (colTypeStr == null) { - throw DataXException.asDataXException(ESWriterErrorCode.BAD_CONFIG_VALUE, col.toString() + " column must have type"); - } - ESFieldType colType = ESFieldType.getESFieldType(colTypeStr); - if (colType == null) { - throw DataXException.asDataXException(ESWriterErrorCode.BAD_CONFIG_VALUE, col.toString() + " unsupported type"); - } - - ESColumn columnItem = new ESColumn(); - - if (colName.equals(Key.PRIMARY_KEY_COLUMN_NAME)) { - // 兼容已有版本 - colType = ESFieldType.ID; - colTypeStr = "id"; - } - - columnItem.setName(colName); - columnItem.setType(colTypeStr); - - if (colType == ESFieldType.ID) { - columnList.add(columnItem); - // 如果是id,则properties为空 - continue; - } - - Boolean array = jo.getBoolean("array"); - if (array != null) { - columnItem.setArray(array); - } - Map field = new HashMap(); - field.put("type", colTypeStr); - //https://www.elastic.co/guide/en/elasticsearch/reference/5.2/breaking_50_mapping_changes.html#_literal_index_literal_property - // https://www.elastic.co/guide/en/elasticsearch/guide/2.x/_deep_dive_on_doc_values.html#_disabling_doc_values - field.put("doc_values", jo.getBoolean("doc_values")); - field.put("ignore_above", jo.getInteger("ignore_above")); - field.put("index", jo.getBoolean("index")); - - switch (colType) { - case STRING: - // 兼容string类型,ES5之前版本 - break; - case KEYWORD: - // https://www.elastic.co/guide/en/elasticsearch/reference/current/tune-for-search-speed.html#_warm_up_global_ordinals - field.put("eager_global_ordinals", jo.getBoolean("eager_global_ordinals")); - case TEXT: - field.put("analyzer", jo.getString("analyzer")); - // 优化disk使用,也同步会提高index性能 - // https://www.elastic.co/guide/en/elasticsearch/reference/current/tune-for-disk-usage.html - field.put("norms", jo.getBoolean("norms")); - field.put("index_options", jo.getBoolean("index_options")); - break; - case DATE: - columnItem.setTimeZone(jo.getString("timezone")); - columnItem.setFormat(jo.getString("format")); - // 后面时间会处理为带时区的标准时间,所以不需要给ES指定格式 - /* - if (jo.getString("format") != null) { - field.put("format", jo.getString("format")); - } else { - //field.put("format", "strict_date_optional_time||epoch_millis||yyyy-MM-dd HH:mm:ss||yyyy-MM-dd"); - } - */ - break; - case GEO_SHAPE: - field.put("tree", jo.getString("tree")); - field.put("precision", jo.getString("precision")); - default: - break; - } - propMap.put(colName, field); - columnList.add(columnItem); - } - } - - conf.set(WRITE_COLUMNS, JSON.toJSONString(columnList)); - - log.info(JSON.toJSONString(columnList)); - - Map rootMappings = new HashMap(); - Map typeMappings = new HashMap(); - typeMappings.put("properties", propMap); - rootMappings.put(typeName, typeMappings); - - mappings = JSON.toJSONString(rootMappings); - - if (mappings == null || "".equals(mappings)) { - throw DataXException.asDataXException(ESWriterErrorCode.BAD_CONFIG_VALUE, "must have mappings"); - } - - return mappings; - } - - @Override - public List split(int mandatoryNumber) { - List configurations = new ArrayList(mandatoryNumber); - for (int i = 0; i < mandatoryNumber; i++) { - configurations.add(conf); - } - return configurations; - } - - @Override - public void post() { - ESClient esClient = new ESClient(); - esClient.createClient(Key.getEndpoint(conf), - Key.getAccessID(conf), - Key.getAccessKey(conf), - false, - 300000, - false, - false); - String alias = Key.getAlias(conf); - if (!"".equals(alias)) { - log.info(String.format("alias [%s] to [%s]", alias, Key.getIndexName(conf))); - try { - esClient.alias(Key.getIndexName(conf), alias, Key.isNeedCleanAlias(conf)); - } catch (IOException e) { - throw DataXException.asDataXException(ESWriterErrorCode.ES_ALIAS_MODIFY, e); - } - } - } - - @Override - public void destroy() { - - } - } - - public static class Task extends Writer.Task { - - private static final Logger log = LoggerFactory.getLogger(Job.class); - - private Configuration conf; - - - ESClient esClient = null; - private List typeList; - private List columnList; - - private int trySize; - private int batchSize; - private String index; - private String type; - private String splitter; - - @Override - public void init() { - this.conf = super.getPluginJobConf(); - index = Key.getIndexName(conf); - type = Key.getTypeName(conf); - - trySize = Key.getTrySize(conf); - batchSize = Key.getBatchSize(conf); - splitter = Key.getSplitter(conf); - columnList = JSON.parseObject(this.conf.getString(WRITE_COLUMNS), new TypeReference>() { - }); - - typeList = new ArrayList(); - - for (ESColumn col : columnList) { - typeList.add(ESFieldType.getESFieldType(col.getType())); - } - - esClient = new ESClient(); - } - - @Override - public void prepare() { - esClient.createClient(Key.getEndpoint(conf), - Key.getAccessID(conf), - Key.getAccessKey(conf), - Key.isMultiThread(conf), - Key.getTimeout(conf), - Key.isCompression(conf), - Key.isDiscovery(conf)); - } - - @Override - public void startWrite(RecordReceiver recordReceiver) { - List writerBuffer = new ArrayList(this.batchSize); - Record record = null; - long total = 0; - while ((record = recordReceiver.getFromReader()) != null) { - writerBuffer.add(record); - if (writerBuffer.size() >= this.batchSize) { - total += doBatchInsert(writerBuffer); - writerBuffer.clear(); - } - } - - if (!writerBuffer.isEmpty()) { - total += doBatchInsert(writerBuffer); - writerBuffer.clear(); - } - - String msg = String.format("task end, write size :%d", total); - getTaskPluginCollector().collectMessage("writesize", String.valueOf(total)); - log.info(msg); - esClient.closeJestClient(); - } - - private String getDateStr(ESColumn esColumn, Column column) { - DateTime date = null; - DateTimeZone dtz = DateTimeZone.getDefault(); - if (esColumn.getTimezone() != null) { - // 所有时区参考 http://www.joda.org/joda-time/timezones.html - dtz = DateTimeZone.forID(esColumn.getTimezone()); - } - if (column.getType() != Column.Type.DATE && esColumn.getFormat() != null) { - DateTimeFormatter formatter = DateTimeFormat.forPattern(esColumn.getFormat()); - date = formatter.withZone(dtz).parseDateTime(column.asString()); - return date.toString(); - } else if (column.getType() == Column.Type.DATE) { - date = new DateTime(column.asLong(), dtz); - return date.toString(); - } else { - return column.asString(); - } - } - - private long doBatchInsert(final List writerBuffer) { - Map data = null; - final Bulk.Builder bulkaction = new Bulk.Builder().defaultIndex(this.index).defaultType(this.type); - for (Record record : writerBuffer) { - data = new HashMap(); - String id = null; - for (int i = 0; i < record.getColumnNumber(); i++) { - Column column = record.getColumn(i); - String columnName = columnList.get(i).getName(); - ESFieldType columnType = typeList.get(i); - //如果是数组类型,那它传入的必是字符串类型 - if (columnList.get(i).isArray() != null && columnList.get(i).isArray()) { - String[] dataList = column.asString().split(splitter); - if (!columnType.equals(ESFieldType.DATE)) { - data.put(columnName, dataList); - } else { - for (int pos = 0; pos < dataList.length; pos++) { - dataList[pos] = getDateStr(columnList.get(i), column); - } - data.put(columnName, dataList); - } - } else { - switch (columnType) { - case ID: - if (id != null) { - id += record.getColumn(i).asString(); - } else { - id = record.getColumn(i).asString(); - } - break; - case DATE: - try { - String dateStr = getDateStr(columnList.get(i), column); - data.put(columnName, dateStr); - } catch (Exception e) { - getTaskPluginCollector().collectDirtyRecord(record, String.format("时间类型解析失败 [%s:%s] exception: %s", columnName, column.toString(), e.toString())); - } - break; - case KEYWORD: - case STRING: - case TEXT: - case IP: - case GEO_POINT: - data.put(columnName, column.asString()); - break; - case BOOLEAN: - data.put(columnName, column.asBoolean()); - break; - case BYTE: - case BINARY: - data.put(columnName, column.asBytes()); - break; - case LONG: - data.put(columnName, column.asLong()); - break; - case INTEGER: - data.put(columnName, column.asBigInteger()); - break; - case SHORT: - data.put(columnName, column.asBigInteger()); - break; - case FLOAT: - case DOUBLE: - data.put(columnName, column.asDouble()); - break; - case NESTED: - case OBJECT: - case GEO_SHAPE: - data.put(columnName, JSON.parse(column.asString())); - break; - default: - getTaskPluginCollector().collectDirtyRecord(record, "类型错误:不支持的类型:" + columnType + " " + columnName); - } - } - } - - if (id == null) { - //id = UUID.randomUUID().toString(); - bulkaction.addAction(new Index.Builder(data).build()); - } else { - bulkaction.addAction(new Index.Builder(data).id(id).build()); - } - } - - try { - return RetryUtil.executeWithRetry(new Callable() { - @Override - public Integer call() throws Exception { - JestResult jestResult = esClient.bulkInsert(bulkaction, 1); - if (jestResult.isSucceeded()) { - return writerBuffer.size(); - } - - String msg = String.format("response code: [%d] error :[%s]", jestResult.getResponseCode(), jestResult.getErrorMessage()); - log.warn(msg); - if (esClient.isBulkResult(jestResult)) { - BulkResult brst = (BulkResult) jestResult; - List failedItems = brst.getFailedItems(); - for (BulkResult.BulkResultItem item : failedItems) { - if (item.status != 400) { - // 400 BAD_REQUEST 如果非数据异常,请求异常,则不允许忽略 - throw DataXException.asDataXException(ESWriterErrorCode.ES_INDEX_INSERT, String.format("status:[%d], error: %s", item.status, item.error)); - } else { - // 如果用户选择不忽略解析错误,则抛异常,默认为忽略 - if (!Key.isIgnoreParseError(conf)) { - throw DataXException.asDataXException(ESWriterErrorCode.ES_INDEX_INSERT, String.format("status:[%d], error: %s, config not ignoreParseError so throw this error", item.status, item.error)); - } - } - } - - List items = brst.getItems(); - for (int idx = 0; idx < items.size(); ++idx) { - BulkResult.BulkResultItem item = items.get(idx); - if (item.error != null && !"".equals(item.error)) { - getTaskPluginCollector().collectDirtyRecord(writerBuffer.get(idx), String.format("status:[%d], error: %s", item.status, item.error)); - } - } - return writerBuffer.size() - brst.getFailedItems().size(); - } else { - Integer status = esClient.getStatus(jestResult); - switch (status) { - case 429: //TOO_MANY_REQUESTS - log.warn("server response too many requests, so auto reduce speed"); - break; - } - throw DataXException.asDataXException(ESWriterErrorCode.ES_INDEX_INSERT, jestResult.getErrorMessage()); - } - } - }, trySize, 60000L, true); - } catch (Exception e) { - if (Key.isIgnoreWriteError(this.conf)) { - log.warn(String.format("重试[%d]次写入失败,忽略该错误,继续写入!", trySize)); - } else { - throw DataXException.asDataXException(ESWriterErrorCode.ES_INDEX_INSERT, e); - } - } - return 0; - } - - @Override - public void post() { - } - - @Override - public void destroy() { - esClient.closeJestClient(); - } - } -} diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESWriterErrorCode.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESWriterErrorCode.java deleted file mode 100644 index 59dcbd0ae1..0000000000 --- a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESWriterErrorCode.java +++ /dev/null @@ -1,37 +0,0 @@ -package com.alibaba.datax.plugin.writer.elasticsearchwriter; - -import com.alibaba.datax.common.spi.ErrorCode; - -public enum ESWriterErrorCode implements ErrorCode { - BAD_CONFIG_VALUE("ESWriter-00", "您配置的值不合法."), - ES_INDEX_DELETE("ESWriter-01", "删除index错误."), - ES_INDEX_CREATE("ESWriter-02", "创建index错误."), - ES_MAPPINGS("ESWriter-03", "mappings错误."), - ES_INDEX_INSERT("ESWriter-04", "插入数据错误."), - ES_ALIAS_MODIFY("ESWriter-05", "别名修改错误."), - ; - - private final String code; - private final String description; - - ESWriterErrorCode(String code, String description) { - this.code = code; - this.description = description; - } - - @Override - public String getCode() { - return this.code; - } - - @Override - public String getDescription() { - return this.description; - } - - @Override - public String toString() { - return String.format("Code:[%s], Description:[%s]. ", this.code, - this.description); - } -} \ No newline at end of file diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchClient.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchClient.java new file mode 100644 index 0000000000..12ac3dd9d8 --- /dev/null +++ b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchClient.java @@ -0,0 +1,312 @@ +package com.alibaba.datax.plugin.writer.elasticsearchwriter; + +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.writer.elasticsearchwriter.jest.ClusterInfo; +import com.alibaba.datax.plugin.writer.elasticsearchwriter.jest.ClusterInfoResult; +import com.alibaba.datax.plugin.writer.elasticsearchwriter.jest.PutMapping7; +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.JSONObject; +import com.google.gson.Gson; +import com.google.gson.JsonElement; +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; +import io.searchbox.action.Action; +import io.searchbox.client.JestClient; +import io.searchbox.client.JestClientFactory; +import io.searchbox.client.JestResult; +import io.searchbox.client.config.HttpClientConfig; +import io.searchbox.client.config.HttpClientConfig.Builder; +import io.searchbox.core.Bulk; +import io.searchbox.indices.CreateIndex; +import io.searchbox.indices.DeleteIndex; +import io.searchbox.indices.IndicesExists; +import io.searchbox.indices.aliases.*; +import io.searchbox.indices.mapping.GetMapping; +import io.searchbox.indices.mapping.PutMapping; + +import io.searchbox.indices.settings.GetSettings; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Created by xiongfeng.bxf on 17/2/8. + */ +public class ElasticSearchClient { + private static final Logger LOGGER = LoggerFactory.getLogger(ElasticSearchClient.class); + + private JestClient jestClient; + private Configuration conf; + + public JestClient getClient() { + return jestClient; + } + + public ElasticSearchClient(Configuration conf) { + this.conf = conf; + String endpoint = Key.getEndpoint(conf); + String user = Key.getUsername(conf); + String passwd = Key.getPassword(conf); + boolean multiThread = Key.isMultiThread(conf); + int readTimeout = Key.getTimeout(conf); + boolean compression = Key.isCompression(conf); + boolean discovery = Key.isDiscovery(conf); + String discoveryFilter = Key.getDiscoveryFilter(conf); + int totalConnection = this.conf.getInt("maxTotalConnection", 200); + JestClientFactory factory = new JestClientFactory(); + Builder httpClientConfig = new HttpClientConfig + .Builder(endpoint) +// .setPreemptiveAuth(new HttpHost(endpoint)) + .multiThreaded(multiThread) + .connTimeout(readTimeout) + .readTimeout(readTimeout) + .maxTotalConnection(totalConnection) + .requestCompressionEnabled(compression) + .discoveryEnabled(discovery) + .discoveryFrequency(5L, TimeUnit.MINUTES) + .discoveryFilter(discoveryFilter); + if (!(StringUtils.isBlank(user) || StringUtils.isBlank(passwd))) { + // 匿名登录 + httpClientConfig.defaultCredentials(user, passwd); + } + factory.setHttpClientConfig(httpClientConfig.build()); + this.jestClient = factory.getObject(); + } + + public boolean indicesExists(String indexName) throws Exception { + boolean isIndicesExists = false; + JestResult rst = execute(new IndicesExists.Builder(indexName).build()); + if (rst.isSucceeded()) { + isIndicesExists = true; + } else { + LOGGER.warn("IndicesExists got ResponseCode: {} ErrorMessage: {}", rst.getResponseCode(), rst.getErrorMessage()); + switch (rst.getResponseCode()) { + case 404: + isIndicesExists = false; + break; + case 401: + // 无权访问 + default: + LOGGER.warn(rst.getErrorMessage()); + break; + } + } + return isIndicesExists; + } + + public boolean deleteIndex(String indexName) throws Exception { + LOGGER.info("delete index {}", indexName); + if (indicesExists(indexName)) { + JestResult rst = execute(new DeleteIndex.Builder(indexName).build()); + if (!rst.isSucceeded()) { + LOGGER.warn("DeleteIndex got ResponseCode: {}, ErrorMessage: {}", rst.getResponseCode(), rst.getErrorMessage()); + return false; + } else { + LOGGER.info("delete index {} success", indexName); + } + } else { + LOGGER.info("index cannot found, skip delete index {}", indexName); + } + return true; + } + + public boolean isGreaterOrEqualThan7() throws Exception { + try { + ClusterInfoResult result = execute(new ClusterInfo.Builder().build()); + LOGGER.info("ClusterInfoResult: {}", result.getJsonString()); + return result.isGreaterOrEqualThan7(); + }catch(Exception e) { + LOGGER.warn(e.getMessage()); + return false; + } + } + + /** + * 获取索引的settings + * @param indexName 索引名 + * @return 设置 + */ + public String getIndexSettings(String indexName) { + GetSettings.Builder builder = new GetSettings.Builder(); + builder.addIndex(indexName); + GetSettings getSettings = builder.build(); + try { + LOGGER.info("begin GetSettings for index: {}", indexName); + JestResult result = this.execute(getSettings); + return result.getJsonString(); + } catch (Exception e) { + String message = "GetSettings for index error: " + e.getMessage(); + LOGGER.warn(message, e); + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.ES_GET_SETTINGS, e.getMessage(), e); + } + } + + public boolean createIndexIfNotExists(String indexName, String typeName, + Object mappings, String settings, + boolean dynamic, boolean isGreaterOrEqualThan7) throws Exception { + JestResult rst; + if (!indicesExists(indexName)) { + LOGGER.info("create index {}", indexName); + rst = execute( + new CreateIndex.Builder(indexName) + .settings(settings) + .setParameter("master_timeout", Key.getMasterTimeout(this.conf)) + .build() + ); + //index_already_exists_exception + if (!rst.isSucceeded()) { + LOGGER.warn("CreateIndex got ResponseCode: {}, ErrorMessage: {}", rst.getResponseCode(), rst.getErrorMessage()); + if (getStatus(rst) == 400) { + LOGGER.info(String.format("index {} already exists", indexName)); + return true; + } else { + return false; + } + } else { + LOGGER.info("create {} index success", indexName); + } + } + + if (dynamic) { + LOGGER.info("dynamic is true, ignore mappings"); + return true; + } + LOGGER.info("create mappings for {} {}", indexName, mappings); + //如果大于7.x,mapping的PUT请求URI中不能带type,并且mapping设置中不能带有嵌套结构 + if (isGreaterOrEqualThan7) { + rst = execute(new PutMapping7.Builder(indexName, mappings). + setParameter("master_timeout", Key.getMasterTimeout(this.conf)).build()); + } else { + rst = execute(new PutMapping.Builder(indexName, typeName, mappings) + .setParameter("master_timeout", Key.getMasterTimeout(this.conf)).build()); + } + if (!rst.isSucceeded()) { + LOGGER.error("PutMapping got ResponseCode: {}, ErrorMessage: {}", rst.getResponseCode(), rst.getErrorMessage()); + return false; + } else { + LOGGER.info("index {} put mappings success", indexName); + } + return true; + } + + public T execute(Action clientRequest) throws IOException { + T rst = jestClient.execute(clientRequest); + if (!rst.isSucceeded()) { + LOGGER.warn(rst.getJsonString()); + } + return rst; + } + + public Integer getStatus(JestResult rst) { + JsonObject jsonObject = rst.getJsonObject(); + if (jsonObject.has("status")) { + return jsonObject.get("status").getAsInt(); + } + return 600; + } + + public boolean isBulkResult(JestResult rst) { + JsonObject jsonObject = rst.getJsonObject(); + return jsonObject.has("items"); + } + + + public boolean alias(String indexname, String aliasname, boolean needClean) throws IOException { + GetAliases getAliases = new GetAliases.Builder().addIndex(aliasname).build(); + AliasMapping addAliasMapping = new AddAliasMapping.Builder(indexname, aliasname).build(); + JestResult rst = null; + List list = new ArrayList(); + if (needClean) { + rst = execute(getAliases); + if (rst.isSucceeded()) { + JsonParser jp = new JsonParser(); + JsonObject jo = (JsonObject) jp.parse(rst.getJsonString()); + for (Map.Entry entry : jo.entrySet()) { + String tindex = entry.getKey(); + if (indexname.equals(tindex)) { + continue; + } + AliasMapping m = new RemoveAliasMapping.Builder(tindex, aliasname).build(); + String s = new Gson().toJson(m.getData()); + LOGGER.info(s); + list.add(m); + } + } + } + + ModifyAliases modifyAliases = new ModifyAliases.Builder(addAliasMapping).addAlias(list).setParameter("master_timeout", Key.getMasterTimeout(this.conf)).build(); + rst = execute(modifyAliases); + if (!rst.isSucceeded()) { + LOGGER.error(rst.getErrorMessage()); + throw new IOException(rst.getErrorMessage()); + } + return true; + } + + /** + * 获取index的mapping + */ + public String getIndexMapping(String indexName) { + GetMapping.Builder builder = new GetMapping.Builder(); + builder.addIndex(indexName); + GetMapping getMapping = builder.build(); + try { + LOGGER.info("begin GetMapping for index: {}", indexName); + JestResult result = this.execute(getMapping); + return result.getJsonString(); + } catch (Exception e) { + String message = "GetMapping for index error: " + e.getMessage(); + LOGGER.warn(message, e); + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.ES_MAPPINGS, e.getMessage(), e); + } + } + + public String getMappingForIndexType(String indexName, String typeName) { + String indexMapping = this.getIndexMapping(indexName); + JSONObject indexMappingInJson = JSON.parseObject(indexMapping); + List paths = Arrays.asList(indexName, "mappings"); + JSONObject properties = JsonPathUtil.getJsonObject(paths, indexMappingInJson); + JSONObject propertiesParent = properties; + if (StringUtils.isNotBlank(typeName) && properties.containsKey(typeName)) { + propertiesParent = (JSONObject) properties.get(typeName); + } + JSONObject mapping = (JSONObject) propertiesParent.get("properties"); + return JSON.toJSONString(mapping); + } + + public JestResult bulkInsert(Bulk.Builder bulk) throws Exception { + // es_rejected_execution_exception + // illegal_argument_exception + // cluster_block_exception + JestResult rst = null; + rst = execute(bulk.build()); + if (!rst.isSucceeded()) { + LOGGER.warn(rst.getErrorMessage()); + } + return rst; + } + + /** + * 关闭JestClient客户端 + * + */ + public void closeJestClient() { + if (jestClient != null) { + try { + // jestClient.shutdownClient(); + jestClient.close(); + } catch (IOException e) { + LOGGER.warn("ignore error: ", e.getMessage()); + } + + } + } +} diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchColumn.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchColumn.java new file mode 100644 index 0000000000..a27b15b211 --- /dev/null +++ b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchColumn.java @@ -0,0 +1,126 @@ +package com.alibaba.datax.plugin.writer.elasticsearchwriter; + +import java.util.List; + +/** + * Created by xiongfeng.bxf on 17/3/2. + */ +public class ElasticSearchColumn { + + private String name;//: "appkey", + + private String type;//": "TEXT", + + private String timezone; + + /** + * 源头数据格式化处理,datax做的事情 + */ + private String format; + + /** + * 目标端格式化,es原生支持的格式 + */ + private String dstFormat; + + private boolean array; + + /** + * 是否使用目标端(ES原生)数组类型 + * + * 默认是false + */ + private boolean dstArray = false; + + private boolean jsonArray; + + private boolean origin; + + private List combineFields; + + private String combineFieldsValueSeparator = "-"; + + public String getCombineFieldsValueSeparator() { + return combineFieldsValueSeparator; + } + + public void setCombineFieldsValueSeparator(String combineFieldsValueSeparator) { + this.combineFieldsValueSeparator = combineFieldsValueSeparator; + } + + public List getCombineFields() { + return combineFields; + } + + public void setCombineFields(List combineFields) { + this.combineFields = combineFields; + } + + public void setName(String name) { + this.name = name; + } + + public void setType(String type) { + this.type = type; + } + + public void setTimeZone(String timezone) { + this.timezone = timezone; + } + + public void setFormat(String format) { + this.format = format; + } + + public String getName() { + return name; + } + + public String getType() { + return type; + } + + public boolean isOrigin() { return origin; } + + public void setOrigin(boolean origin) { this.origin = origin; } + + public String getTimezone() { + return timezone; + } + + public String getFormat() { + return format; + } + + public void setTimezone(String timezone) { + this.timezone = timezone; + } + + public boolean isArray() { + return array; + } + + public void setArray(boolean array) { + this.array = array; + } + + public boolean isJsonArray() {return jsonArray;} + + public void setJsonArray(boolean jsonArray) {this.jsonArray = jsonArray;} + + public String getDstFormat() { + return dstFormat; + } + + public void setDstFormat(String dstFormat) { + this.dstFormat = dstFormat; + } + + public boolean isDstArray() { + return dstArray; + } + + public void setDstArray(boolean dstArray) { + this.dstArray = dstArray; + } +} diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESFieldType.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchFieldType.java similarity index 73% rename from elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESFieldType.java rename to elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchFieldType.java index 14b096891a..22c3ee6b53 100644 --- a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ESFieldType.java +++ b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchFieldType.java @@ -3,8 +3,11 @@ /** * Created by xiongfeng.bxf on 17/3/1. */ -public enum ESFieldType { +public enum ElasticSearchFieldType { ID, + PARENT, + ROUTING, + VERSION, STRING, TEXT, KEYWORD, @@ -24,20 +27,18 @@ public enum ESFieldType { DATE_RANGE, GEO_POINT, GEO_SHAPE, - IP, + IP_RANGE, COMPLETION, TOKEN_COUNT, - - ARRAY, OBJECT, NESTED; - public static ESFieldType getESFieldType(String type) { + public static ElasticSearchFieldType getESFieldType(String type) { if (type == null) { return null; } - for (ESFieldType f : ESFieldType.values()) { + for (ElasticSearchFieldType f : ElasticSearchFieldType.values()) { if (f.name().compareTo(type.toUpperCase()) == 0) { return f; } diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchWriter.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchWriter.java new file mode 100644 index 0000000000..6236e33370 --- /dev/null +++ b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchWriter.java @@ -0,0 +1,1119 @@ +package com.alibaba.datax.plugin.writer.elasticsearchwriter; + +import com.alibaba.datax.common.element.Column; +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.common.util.DataXCaseEnvUtil; +import com.alibaba.datax.common.util.RetryUtil; +import com.alibaba.datax.plugin.writer.elasticsearchwriter.Key.ActionType; +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import com.alibaba.fastjson.TypeReference; +import com.alibaba.fastjson.serializer.SerializerFeature; +import com.google.common.base.Joiner; +import io.searchbox.client.JestResult; +import io.searchbox.core.*; +import io.searchbox.params.Parameters; +import org.apache.commons.lang3.StringUtils; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.*; +import java.util.concurrent.Callable; + +public class ElasticSearchWriter extends Writer { + private final static String WRITE_COLUMNS = "write_columns"; + + public static class Job extends Writer.Job { + private static final Logger LOGGER = LoggerFactory.getLogger(Job.class); + + private Configuration conf = null; + int retryTimes = 3; + long sleepTimeInMilliSecond = 10000L; + + private String settingsCache; + + private void setSettings(String settings) { + this.settingsCache = JsonUtil.mergeJsonStr(settings, this.settingsCache); + } + + @Override + public void init() { + this.conf = super.getPluginJobConf(); + //LOGGER.info("conf:{}", conf); + this.retryTimes = this.conf.getInt("retryTimes", 3); + this.sleepTimeInMilliSecond = this.conf.getLong("sleepTimeInMilliSecond", 10000L); + } + + public List getIncludeSettings() { + return this.conf.getList("includeSettingKeys", Arrays.asList("number_of_shards", "number_of_replicas"), String.class); + } + + /** + * 从es中获取的原始settings转为需要的settings + * @param originSettings 原始settings + * @return settings + */ + private String convertSettings(String originSettings) { + if(StringUtils.isBlank(originSettings)) { + return null; + } + JSONObject jsonObject = JSON.parseObject(originSettings); + for(String key : jsonObject.keySet()) { + JSONObject settingsObj = jsonObject.getJSONObject(key); + if(settingsObj != null) { + JSONObject indexObj = settingsObj.getJSONObject("settings"); + JSONObject settings = indexObj.getJSONObject("index"); + JSONObject filterSettings = new JSONObject(); + if(settings != null) { + List includeSettings = getIncludeSettings(); + if(includeSettings != null && includeSettings.size() > 0) { + for(String includeSetting : includeSettings) { + Object fieldValue = settings.get(includeSetting); + if(fieldValue != null) { + filterSettings.put(includeSetting, fieldValue); + } + } + return filterSettings.toJSONString(); + } + } + } + } + return null; + } + + @Override + public void prepare() { + /** + * 注意:此方法仅执行一次。 + * 最佳实践:如果 Job 中有需要进行数据同步之前的处理,可以在此处完成,如果没有必要则可以直接去掉。 + * 对于7.x之后的es版本,取消了index设置type的逻辑,因此在prepare阶段,加入了判断是否为7.x及以上版本 + * 如果是7.x及以上版本,需要对于index的type做不同的处理 + * 详见 : https://www.elastic.co/guide/en/elasticsearch/reference/6.8/removal-of-types.html + */ + final ElasticSearchClient esClient = new ElasticSearchClient(this.conf); + final String indexName = Key.getIndexName(conf); + ActionType actionType = Key.getActionType(conf); + final String typeName = Key.getTypeName(conf); + final boolean dynamic = Key.getDynamic(conf); + final String dstDynamic = Key.getDstDynamic(conf); + final String newSettings = JSONObject.toJSONString(Key.getSettings(conf)); + LOGGER.info("conf settings:{}, settingsCache:{}", newSettings, this.settingsCache); + final Integer esVersion = Key.getESVersion(conf); + boolean hasId = this.hasID(); + this.conf.set("hasId", hasId); + if (ActionType.UPDATE.equals(actionType) && !hasId && !hasPrimaryKeyInfo()) { + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.UPDATE_WITH_ID, "Update mode must specify column type with id or primaryKeyInfo config"); + } + + try { + RetryUtil.executeWithRetry(() -> { + boolean isGreaterOrEqualThan7 = esClient.isGreaterOrEqualThan7(); + if (esVersion != null && esVersion >= 7) { + isGreaterOrEqualThan7 = true; + } + String mappings = genMappings(dstDynamic, typeName, isGreaterOrEqualThan7); + conf.set("isGreaterOrEqualThan7", isGreaterOrEqualThan7); + + + LOGGER.info(String.format("index:[%s], type:[%s], mappings:[%s]", indexName, typeName, mappings)); + boolean isIndicesExists = esClient.indicesExists(indexName); + if (isIndicesExists) { + try { + // 将原有的mapping打印出来,便于排查问题 + String oldMappings = esClient.getMappingForIndexType(indexName, typeName); + LOGGER.info("the mappings for old index is: {}", oldMappings); + } catch (Exception e) { + LOGGER.warn("warn message: {}", e.getMessage()); + } + } + + if (Key.isTruncate(conf) && isIndicesExists) { + // 备份老的索引中的settings到缓存 + try { + String oldOriginSettings = esClient.getIndexSettings(indexName); + if (StringUtils.isNotBlank(oldOriginSettings)) { + String includeSettings = convertSettings(oldOriginSettings); + LOGGER.info("merge1 settings:{}, settingsCache:{}, includeSettings:{}", + oldOriginSettings, + this.settingsCache, includeSettings); + this.setSettings(includeSettings); + } + } catch (Exception e) { + LOGGER.warn("get old settings fail, indexName:{}", indexName); + } + esClient.deleteIndex(indexName); + } + + // 更新缓存中的settings + this.setSettings(newSettings); + LOGGER.info("merge2 settings:{}, settingsCache:{}", newSettings, this.settingsCache); + // 强制创建,内部自动忽略已存在的情况 + if (!esClient.createIndexIfNotExists(indexName, typeName, mappings, this.settingsCache, dynamic, + isGreaterOrEqualThan7)) { + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.ES_MAPPINGS, ""); + } + + return true; + }, DataXCaseEnvUtil.getRetryTimes(this.retryTimes), DataXCaseEnvUtil.getRetryInterval(this.sleepTimeInMilliSecond), DataXCaseEnvUtil.getRetryExponential(false)); + } catch (Exception ex) { + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.ES_MAPPINGS, ex.getMessage(), ex); + } finally { + try { + esClient.closeJestClient(); + } catch (Exception e) { + LOGGER.warn("ignore close jest client error: {}", e.getMessage()); + } + } + } + + private boolean hasID() { + List column = conf.getList("column"); + if (column != null) { + for (Object col : column) { + JSONObject jo = JSONObject.parseObject(col.toString()); + String colTypeStr = jo.getString("type"); + ElasticSearchFieldType colType = ElasticSearchFieldType.getESFieldType(colTypeStr); + if (ElasticSearchFieldType.ID.equals(colType)) { + return true; + } + } + } + return false; + } + + private boolean hasPrimaryKeyInfo() { + PrimaryKeyInfo primaryKeyInfo = Key.getPrimaryKeyInfo(this.conf); + if (null != primaryKeyInfo && null != primaryKeyInfo.getColumn() && !primaryKeyInfo.getColumn().isEmpty()) { + return true; + } else { + return false; + } + } + + + private String genMappings(String dstDynamic, String typeName, boolean isGreaterOrEqualThan7) { + String mappings; + Map propMap = new HashMap(); + List columnList = new ArrayList(); + ElasticSearchColumn combineItem = null; + + List column = conf.getList("column"); + if (column != null) { + for (Object col : column) { + JSONObject jo = JSONObject.parseObject(col.toString()); + String colName = jo.getString("name"); + String colTypeStr = jo.getString("type"); + if (colTypeStr == null) { + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.BAD_CONFIG_VALUE, col.toString() + " column must have type"); + } + ElasticSearchFieldType colType = ElasticSearchFieldType.getESFieldType(colTypeStr); + if (colType == null) { + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.BAD_CONFIG_VALUE, col.toString() + " unsupported type"); + } + + ElasticSearchColumn columnItem = new ElasticSearchColumn(); + + if (Key.PRIMARY_KEY_COLUMN_NAME.equals(colName)) { + // 兼容已有版本 + colType = ElasticSearchFieldType.ID; + colTypeStr = "id"; + } + + columnItem.setName(colName); + columnItem.setType(colTypeStr); + + JSONArray combineFields = jo.getJSONArray("combineFields"); + if (combineFields != null && !combineFields.isEmpty() && ElasticSearchFieldType.ID.equals(ElasticSearchFieldType.getESFieldType(colTypeStr))) { + List fields = new ArrayList(); + for (Object item : combineFields) { + fields.add((String) item); + } + columnItem.setCombineFields(fields); + combineItem = columnItem; + } + + String combineFieldsValueSeparator = jo.getString("combineFieldsValueSeparator"); + if (StringUtils.isNotBlank(combineFieldsValueSeparator)) { + columnItem.setCombineFieldsValueSeparator(combineFieldsValueSeparator); + } + + // 如果是id,version,routing,不需要创建mapping + if (colType == ElasticSearchFieldType.ID || colType == ElasticSearchFieldType.VERSION || colType == ElasticSearchFieldType.ROUTING) { + columnList.add(columnItem); + continue; + } + + // 如果是组合id中的字段,不需要创建mapping + // 所以组合id的定义必须要在columns最前面 + if (combineItem != null && combineItem.getCombineFields().contains(colName)) { + columnList.add(columnItem); + continue; + } + columnItem.setDstArray(false); + Boolean array = jo.getBoolean("array"); + if (array != null) { + columnItem.setArray(array); + Boolean dstArray = jo.getBoolean("dstArray"); + if(dstArray!=null) { + columnItem.setDstArray(dstArray); + } + } else { + columnItem.setArray(false); + } + Boolean jsonArray = jo.getBoolean("json_array"); + if (jsonArray != null) { + columnItem.setJsonArray(jsonArray); + } else { + columnItem.setJsonArray(false); + } + Map field = new HashMap(); + field.put("type", colTypeStr); + //https://www.elastic.co/guide/en/elasticsearch/reference/5.2/breaking_50_mapping_changes.html#_literal_index_literal_property + // https://www.elastic.co/guide/en/elasticsearch/guide/2.x/_deep_dive_on_doc_values.html#_disabling_doc_values + field.put("doc_values", jo.getBoolean("doc_values")); + field.put("ignore_above", jo.getInteger("ignore_above")); + field.put("index", jo.getBoolean("index")); + switch (colType) { + case STRING: + // 兼容string类型,ES5之前版本 + break; + case KEYWORD: + // https://www.elastic.co/guide/en/elasticsearch/reference/current/tune-for-search-speed.html#_warm_up_global_ordinals + field.put("eager_global_ordinals", jo.getBoolean("eager_global_ordinals")); + break; + case TEXT: + field.put("analyzer", jo.getString("analyzer")); + // 优化disk使用,也同步会提高index性能 + // https://www.elastic.co/guide/en/elasticsearch/reference/current/tune-for-disk-usage.html + field.put("norms", jo.getBoolean("norms")); + field.put("index_options", jo.getBoolean("index_options")); + if(jo.getString("fields") != null) { + field.put("fields", jo.getJSONObject("fields")); + } + break; + case DATE: + if (Boolean.TRUE.equals(jo.getBoolean("origin"))) { + if (jo.getString("format") != null) { + field.put("format", jo.getString("format")); + } + // es原生format覆盖原先来的format + if (jo.getString("dstFormat") != null) { + field.put("format", jo.getString("dstFormat")); + } + if(jo.getBoolean("origin") != null) { + columnItem.setOrigin(jo.getBoolean("origin")); + } + } else { + columnItem.setTimeZone(jo.getString("timezone")); + columnItem.setFormat(jo.getString("format")); + } + break; + case GEO_SHAPE: + field.put("tree", jo.getString("tree")); + field.put("precision", jo.getString("precision")); + break; + case OBJECT: + case NESTED: + if (jo.getString("dynamic") != null) { + field.put("dynamic", jo.getString("dynamic")); + } + break; + default: + break; + } + if (jo.containsKey("other_params")) { + field.putAll(jo.getJSONObject("other_params")); + } + propMap.put(colName, field); + columnList.add(columnItem); + } + } + + long version = System.currentTimeMillis(); + LOGGER.info("unified version: {}", version); + conf.set("version", version); + conf.set(WRITE_COLUMNS, JSON.toJSONString(columnList)); + + LOGGER.info(JSON.toJSONString(columnList)); + + Map rootMappings = new HashMap(); + Map typeMappings = new HashMap(); + typeMappings.put("properties", propMap); + rootMappings.put(typeName, typeMappings); + + // 7.x以后版本取消了index中关于type的指定,所以mapping的格式只能支持 + // { + // "properties" : { + // "abc" : { + // "type" : "text" + // } + // } + // } + // properties 外不能再嵌套typeName + + if(StringUtils.isNotBlank(dstDynamic)) { + typeMappings.put("dynamic", dstDynamic); + } + if (isGreaterOrEqualThan7) { + mappings = JSON.toJSONString(typeMappings); + } else { + mappings = JSON.toJSONString(rootMappings); + } + if (StringUtils.isBlank(mappings)) { + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.BAD_CONFIG_VALUE, "must have mappings"); + } + + return mappings; + } + + @Override + public List split(int mandatoryNumber) { + List configurations = new ArrayList(mandatoryNumber); + for (int i = 0; i < mandatoryNumber; i++) { + configurations.add(this.conf.clone()); + } + return configurations; + } + + @Override + public void post() { + ElasticSearchClient esClient = new ElasticSearchClient(this.conf); + String alias = Key.getAlias(conf); + if (!"".equals(alias)) { + LOGGER.info(String.format("alias [%s] to [%s]", alias, Key.getIndexName(conf))); + try { + esClient.alias(Key.getIndexName(conf), alias, Key.isNeedCleanAlias(conf)); + } catch (IOException e) { + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.ES_ALIAS_MODIFY, e); + } + } + } + + @Override + public void destroy() { + + } + } + + public static class Task extends Writer.Task { + + private static final Logger LOGGER = LoggerFactory.getLogger(Job.class); + + private Configuration conf; + + + ElasticSearchClient esClient = null; + private List typeList; + private List columnList; + private List> deleteByConditions; + + private int trySize; + private long tryInterval; + private int batchSize; + private String index; + private String type; + private String splitter; + private ActionType actionType; + private ElasticSearchColumn combinedIdColumn; + private Map colNameToIndexMap; + private Map urlParams; + private boolean columnSizeChecked = false; + private boolean enableRedundantColumn = false; + private boolean enableWriteNull = true; + int retryTimes = 3; + long sleepTimeInMilliSecond = 10000L; + boolean isGreaterOrEqualThan7 = false; + private String fieldDelimiter; + private boolean hasId; + private PrimaryKeyInfo primaryKeyInfo; + private boolean hasPrimaryKeyInfo = false; + private List esPartitionColumn; + private boolean hasEsPartitionColumn = false; + + @Override + public void init() { + this.conf = super.getPluginJobConf(); + this.index = Key.getIndexName(conf); + this.type = Key.getTypeName(conf); + this.trySize = Key.getTrySize(conf); + this.tryInterval = Key.getTryInterval(conf); + this.batchSize = Key.getBatchSize(conf); + this.splitter = Key.getSplitter(conf); + this.actionType = Key.getActionType(conf); + this.urlParams = Key.getUrlParams(conf); + this.enableWriteNull = Key.isEnableNullUpdate(conf); + this.retryTimes = this.conf.getInt("retryTimes", 3); + this.sleepTimeInMilliSecond = this.conf.getLong("sleepTimeInMilliSecond", 10000L); + this.isGreaterOrEqualThan7 = this.conf.getBool("isGreaterOrEqualThan7", false); + this.parseDeleteCondition(conf); + this.columnList = JSON.parseObject(this.conf.getString(WRITE_COLUMNS), new TypeReference>() { + }); + LOGGER.info("columnList: {}", JSON.toJSONString(columnList)); + this.hasId = this.conf.getBool("hasId", false); + if (hasId) { + LOGGER.info("Task has id column, will use it to set _id property"); + } else { + LOGGER.info("Task will use elasticsearch auto generated _id property"); + } + this.fieldDelimiter = Key.getFieldDelimiter(this.conf); + this.enableRedundantColumn = this.conf.getBool("enableRedundantColumn", false); + this.typeList = new ArrayList(); + for (ElasticSearchColumn esColumn : this.columnList) { + this.typeList.add(ElasticSearchFieldType.getESFieldType(esColumn.getType())); + if (esColumn.getCombineFields() != null && esColumn.getCombineFields().size() > 0 + && ElasticSearchFieldType.getESFieldType(esColumn.getType()).equals(ElasticSearchFieldType.ID)) { + combinedIdColumn = esColumn; + } + } + this.primaryKeyInfo = Key.getPrimaryKeyInfo(this.conf); + this.esPartitionColumn = Key.getEsPartitionColumn(this.conf); + this.colNameToIndexMap = new HashMap(5); + this.handleMetaKeys(); + this.esClient = new ElasticSearchClient(this.conf); + } + + private void handleMetaKeys() { + if (null != this.primaryKeyInfo && null != this.primaryKeyInfo.getColumn() + && !this.primaryKeyInfo.getColumn().isEmpty()) { + this.hasPrimaryKeyInfo = true; + if (null == this.primaryKeyInfo.getFieldDelimiter()) { + if (null != this.fieldDelimiter) { + this.primaryKeyInfo.setFieldDelimiter(this.fieldDelimiter); + } else { + this.primaryKeyInfo.setFieldDelimiter(""); + } + } + + for (String eachPk : this.primaryKeyInfo.getColumn()) { + boolean foundKeyInColumn = false; + for (int i = 0; i < columnList.size(); i++) { + if (StringUtils.equals(eachPk, columnList.get(i).getName())) { + this.colNameToIndexMap.put(eachPk, i); + foundKeyInColumn = true; + break; + } + } + if (!foundKeyInColumn) { + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.RECORD_FIELD_NOT_FOUND, + "primaryKeyInfo has column not exists in column"); + } + } + } + + if (null != this.esPartitionColumn && !this.esPartitionColumn.isEmpty()) { + this.hasEsPartitionColumn = true; + for (PartitionColumn eachPartitionCol : this.esPartitionColumn) { + boolean foundKeyInColumn = false; + for (int i = 0; i < columnList.size(); i++) { + if (StringUtils.equals(eachPartitionCol.getName(), columnList.get(i).getName())) { + this.colNameToIndexMap.put(eachPartitionCol.getName(), i); + foundKeyInColumn = true; + break; + } + } + if (!foundKeyInColumn) { + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.RECORD_FIELD_NOT_FOUND, + "esPartitionColumn has column not exists in column"); + } + } + } + } + + private void parseDeleteCondition(Configuration conf) { + List> list = new ArrayList>(); + String config = Key.getDeleteBy(conf); + if (config != null) { + JSONArray array = JSON.parseArray(config); + for (Object obj : array) { + list.add((Map) obj); + } + deleteByConditions = list; + } + } + + + @Override + public void prepare() { + } + + /** + * 示例:{ + * "deleteBy" : [ + * {"product_status" : [-1,-2], "sub_status" : -3}, + * {"product_status" : -3} + * ] + * } + * + * 表示以下两类数据删除: + * 1. product_status为-1或-2并且sub_status为-3 + * 2. product_status为-3 + * + * 注意[{}]返回true + * @param record + * @return + */ + private boolean isDeleteRecord(Record record) { + if (deleteByConditions == null) { + return false; + } + + Map kv = new HashMap(); + for (int i = 0; i < record.getColumnNumber(); i++) { + Column column = record.getColumn(i); + String columnName = columnList.get(i).getName(); + kv.put(columnName, column.asString()); + } + + for (Map delCondition : deleteByConditions) { + if (meetAllCondition(kv, delCondition)) { + return true; + } + } + + return false; + } + + private boolean meetAllCondition(Map kv, Map delCondition) { + for (Map.Entry oneCondition : delCondition.entrySet()) { + if (!checkOneCondition(kv, oneCondition)) { + return false; + } + } + return true; + } + + private boolean checkOneCondition(Map kv, Map.Entry entry) { + Object value = kv.get(entry.getKey()); + if (entry.getValue() instanceof List) { + for (Object obj : (List) entry.getValue()) { + if (obj.toString().equals(value)) { + return true; + } + } + } else { + if (value != null && value.equals(entry.getValue().toString())) { + return true; + } + } + return false; + } + + @Override + public void startWrite(RecordReceiver recordReceiver) { + List writerBuffer = new ArrayList(this.batchSize); + Record record = null; + while ((record = recordReceiver.getFromReader()) != null) { + if (!columnSizeChecked) { + boolean isInvalid = true; + if (enableRedundantColumn) { + isInvalid = this.columnList.size() > record.getColumnNumber(); + } else { + isInvalid = this.columnList.size() != record.getColumnNumber(); + } + if (isInvalid) { + String message = String.format( + "column number not equal error, reader column size is %s, but the writer column size is %s", + record.getColumnNumber(), this.columnList.size()); + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.BAD_CONFIG_VALUE, message); + } + columnSizeChecked = true; + } + writerBuffer.add(record); + if (writerBuffer.size() >= this.batchSize) { + this.doBatchInsert(writerBuffer); + writerBuffer.clear(); + } + } + + if (!writerBuffer.isEmpty()) { + this.doBatchInsert(writerBuffer); + writerBuffer.clear(); + } + } + + private String getDateStr(ElasticSearchColumn esColumn, Column column) { + // 如果保持原样,就直接返回 + if (esColumn.isOrigin()) { + return column.asString(); + } + DateTime date = null; + DateTimeZone dtz = DateTimeZone.getDefault(); + if (esColumn.getTimezone() != null) { + // 所有时区参考 http://www.joda.org/joda-time/timezones.html + // TODO:创建一次多处复用 + dtz = DateTimeZone.forID(esColumn.getTimezone()); + } + if (column.getType() != Column.Type.DATE && esColumn.getFormat() != null) { + // TODO:创建一次多处复用 + DateTimeFormatter formatter = DateTimeFormat.forPattern(esColumn.getFormat()); + date = formatter.withZone(dtz).parseDateTime(column.asString()); + return date.toString(); + } else if (column.getType() == Column.Type.DATE) { + if (null == column.getRawData()) { + return null; + } else { + date = new DateTime(column.asLong(), dtz); + return date.toString(); + } + } else { + return column.asString(); + } + } + + private void doBatchInsert(final List writerBuffer) { + Map data = null; + Bulk.Builder bulkactionTmp = null; + int totalNumber = writerBuffer.size(); + int dirtyDataNumber = 0; + if (this.isGreaterOrEqualThan7) { + bulkactionTmp = new Bulk.Builder().defaultIndex(this.index); + } else { + bulkactionTmp = new Bulk.Builder().defaultIndex(this.index).defaultType(this.type); + } + final Bulk.Builder bulkaction = bulkactionTmp; + // 增加url的参数 + for (Map.Entry entry : urlParams.entrySet()) { + bulkaction.setParameter(entry.getKey(), entry.getValue()); + } + for (Record record : writerBuffer) { + data = new HashMap(); + String id = null; + String parent = null; + String routing = null; + String version = null; + String columnName = null; + Column column = null; + try { + for (int i = 0; i < record.getColumnNumber(); i++) { + column = record.getColumn(i); + columnName = columnList.get(i).getName(); + // 如果组合id不等于null,需要把相关的字段全部忽略 + if (combinedIdColumn != null) { + if (combinedIdColumn.getCombineFields().contains(columnName)) { + continue; + } + } + //如果是json数组,当成对象类型处理 + ElasticSearchFieldType columnType = columnList.get(i).isJsonArray() ? ElasticSearchFieldType.NESTED : typeList.get(i); + + Boolean dstArray = columnList.get(i).isDstArray(); + + //如果是数组类型,那它传入的是字符串类型,也有可能是null + if (columnList.get(i).isArray() && null != column.asString()) { + String[] dataList = column.asString().split(splitter); + if (!columnType.equals(ElasticSearchFieldType.DATE)) { + if (dstArray) { + try { + // 根据客户配置的类型,转换成相应的类型 + switch (columnType) { + case BYTE: + case KEYWORD: + case TEXT: + data.put(columnName, dataList); + break; + case SHORT: + case INTEGER: + if (StringUtils.isBlank(column.asString().trim())) { + data.put(columnName, null); + } else { + Integer[] intDataList = new Integer[dataList.length]; + for (int j = 0; j < dataList.length; j++) { + dataList[j] = dataList[j].trim(); + if (StringUtils.isNotBlank(dataList[j])) { + intDataList[j] = Integer.valueOf(dataList[j]); + } + } + data.put(columnName, intDataList); + } + break; + case LONG: + if (StringUtils.isBlank(column.asString().trim())) { + data.put(columnName, null); + } else { + Long[] longDataList = new Long[dataList.length]; + for (int j = 0; j < dataList.length; j++) { + dataList[j] = dataList[j].trim(); + if (StringUtils.isNotBlank(dataList[j])) { + longDataList[j] = Long.valueOf(dataList[j]); + } + } + data.put(columnName, longDataList); + } + break; + case FLOAT: + case DOUBLE: + if (StringUtils.isBlank(column.asString().trim())) { + data.put(columnName, null); + } else { + Double[] doubleDataList = new Double[dataList.length]; + for (int j = 0; j < dataList.length; j++) { + dataList[j] = dataList[j].trim(); + if (StringUtils.isNotBlank(dataList[j])) { + doubleDataList[j] = Double.valueOf(dataList[j]); + } + } + data.put(columnName, doubleDataList); + } + break; + default: + data.put(columnName, dataList); + break; + } + } catch (Exception e) { + LOGGER.info("脏数据,记录:{}", record.toString()); + continue; + } + } else { + data.put(columnName, dataList); + } + } else { + data.put(columnName, dataList); + } + } else { + // LOGGER.info("columnType: {} integer: {}", columnType, column.asString()); + switch (columnType) { + case ID: + if (id != null) { + id += record.getColumn(i).asString(); + } else { + id = record.getColumn(i).asString(); + } + break; + case PARENT: + if (parent != null) { + parent += record.getColumn(i).asString(); + } else { + parent = record.getColumn(i).asString(); + } + break; + case ROUTING: + if (routing != null) { + routing += record.getColumn(i).asString(); + } else { + routing = record.getColumn(i).asString(); + } + break; + + case VERSION: + if (version != null) { + version += record.getColumn(i).asString(); + } else { + version = record.getColumn(i).asString(); + } + break; + case DATE: + String dateStr = getDateStr(columnList.get(i), column); + data.put(columnName, dateStr); + break; + case KEYWORD: + case STRING: + case TEXT: + case IP: + case GEO_POINT: + case IP_RANGE: + data.put(columnName, column.asString()); + break; + case BOOLEAN: + data.put(columnName, column.asBoolean()); + break; + case BYTE: + case BINARY: + // json序列化不支持byte类型,es支持的binary类型,必须传入base64的格式 + data.put(columnName, column.asString()); + break; + case LONG: + data.put(columnName, column.asLong()); + break; + case INTEGER: + data.put(columnName, column.asLong()); + break; + case SHORT: + data.put(columnName, column.asLong()); + break; + case FLOAT: + case DOUBLE: + data.put(columnName, column.asDouble()); + break; + case GEO_SHAPE: + case DATE_RANGE: + case INTEGER_RANGE: + case FLOAT_RANGE: + case LONG_RANGE: + case DOUBLE_RANGE: + if (null == column.asString()) { + data.put(columnName, column.asString()); + } else { + data.put(columnName, JSON.parse(column.asString())); + } + break; + case NESTED: + case OBJECT: + if (null == column.asString()) { + data.put(columnName, column.asString()); + } else { + // 转json格式 + data.put(columnName, JSON.parse(column.asString())); + } + break; + default: + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.BAD_CONFIG_VALUE, String.format( + "Type error: unsupported type %s for column %s", columnType, columnName)); + } + } + } + + + if (this.hasPrimaryKeyInfo) { + List idData = new ArrayList(); + for (String eachCol : this.primaryKeyInfo.getColumn()) { + Column recordColumn = record.getColumn(this.colNameToIndexMap.get(eachCol)); + idData.add(recordColumn.asString()); + } + id = StringUtils.join(idData, this.primaryKeyInfo.getFieldDelimiter()); + } + if (this.hasEsPartitionColumn) { + List idData = new ArrayList(); + for (PartitionColumn eachCol : this.esPartitionColumn) { + Column recordColumn = record.getColumn(this.colNameToIndexMap.get(eachCol.getName())); + idData.add(recordColumn.asString()); + } + routing = StringUtils.join(idData, ""); + } + } catch (Exception e) { + // 脏数据 + super.getTaskPluginCollector().collectDirtyRecord(record, + String.format("parse error for column: %s errorMessage: %s", columnName, e.getMessage())); + dirtyDataNumber++; + // 处理下一个record + continue; + } + + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("id: {} routing: {} data: {}", id, routing, JSON.toJSONString(data)); + } + + + if (isDeleteRecord(record)) { + Delete.Builder builder = new Delete.Builder(id); + bulkaction.addAction(builder.build()); + } else { + // 使用用户自定义组合唯一键 + if (combinedIdColumn != null) { + try { + id = processIDCombineFields(record, combinedIdColumn); + // LOGGER.debug("id: {}", id); + } catch (Exception e) { + // 脏数据 + super.getTaskPluginCollector().collectDirtyRecord(record, + String.format("parse error for column: %s errorMessage: %s", columnName, e.getMessage())); + // 处理下一个record + dirtyDataNumber++; + continue; + } + } + switch (actionType) { + case INDEX: + // 先进行json序列化,jest client的gson序列化会把等号按照html序列化 + Index.Builder builder = null; + if (this.enableWriteNull) { + builder = new Index.Builder( + JSONObject.toJSONString(data, SerializerFeature.WriteMapNullValue, + SerializerFeature.QuoteFieldNames, SerializerFeature.SkipTransientField, + SerializerFeature.WriteEnumUsingToString, SerializerFeature.SortField)); + } else { + builder = new Index.Builder(JSONObject.toJSONString(data)); + } + if (id != null) { + builder.id(id); + } + if (parent != null) { + builder.setParameter(Parameters.PARENT, parent); + } + if (routing != null) { + builder.setParameter(Parameters.ROUTING, routing); + } + if (version != null) { + builder.setParameter(Parameters.VERSION, version); + builder.setParameter(Parameters.VERSION_TYPE, "external"); + } + bulkaction.addAction(builder.build()); + break; + case UPDATE: + // doc: https://www.cnblogs.com/crystaltu/articles/6992935.html + // doc: https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-update.html + Map updateDoc = new HashMap(); + updateDoc.put("doc", data); + updateDoc.put("doc_as_upsert", true); + Update.Builder update = null; + if (this.enableWriteNull) { + // write: {a:"1",b:null} + update = new Update.Builder( + JSONObject.toJSONString(updateDoc, SerializerFeature.WriteMapNullValue, + SerializerFeature.QuoteFieldNames, SerializerFeature.SkipTransientField, + SerializerFeature.WriteEnumUsingToString, SerializerFeature.SortField)); + // 在DEFAULT_GENERATE_FEATURE基础上,只增加了SerializerFeature.WRITE_MAP_NULL_FEATURES + } else { + // write: {"a":"1"} + update = new Update.Builder(JSONObject.toJSONString(updateDoc)); + } + if (id != null) { + update.id(id); + } + if (parent != null) { + update.setParameter(Parameters.PARENT, parent); + } + if (routing != null) { + update.setParameter(Parameters.ROUTING, routing); + } + // version type [EXTERNAL] is not supported by the update API + if (version != null) { + update.setParameter(Parameters.VERSION, version); + } + bulkaction.addAction(update.build()); + break; + default: + break; + } + } + } + + if (dirtyDataNumber >= totalNumber) { + // all batch is dirty data + LOGGER.warn("all this batch is dirty data, dirtyDataNumber: {} totalDataNumber: {}", dirtyDataNumber, + totalNumber); + return; + } + + BulkResult bulkResult = null; + try { + bulkResult = RetryUtil.executeWithRetry(new Callable() { + @Override + public BulkResult call() throws Exception { + JestResult jestResult = esClient.bulkInsert(bulkaction); + if (jestResult.isSucceeded()) { + return null; + } + String msg = String.format("response code: [%d] error :[%s]", jestResult.getResponseCode(), + jestResult.getErrorMessage()); + LOGGER.warn(msg); + if (esClient.isBulkResult(jestResult)) { + BulkResult brst = (BulkResult) jestResult; + List failedItems = brst.getFailedItems(); + for (BulkResult.BulkResultItem item : failedItems) { + if (item.status != 400) { + // 400 BAD_REQUEST 如果非数据异常,请求异常,则不允许忽略 + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.ES_INDEX_INSERT, + String.format("status:[%d], error: %s", item.status, item.error)); + } else { + // 如果用户选择不忽略解析错误,则抛异常,默认为忽略 + if (!Key.isIgnoreParseError(conf)) { + throw new NoReRunException(ElasticSearchWriterErrorCode.ES_INDEX_INSERT, + String.format( + "status:[%d], error: %s, config not ignoreParseError so throw this error", + item.status, item.error)); + } + } + } + return brst; + } else { + Integer status = esClient.getStatus(jestResult); + switch (status) { + case 429: // TOO_MANY_REQUESTS + LOGGER.warn("server response too many requests, so auto reduce speed"); + break; + default: + break; + } + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.ES_INDEX_INSERT, + jestResult.getErrorMessage()); + } + } + }, this.trySize, this.tryInterval, false, Arrays.asList(DataXException.class)); + } catch (Exception e) { + if (Key.isIgnoreWriteError(this.conf)) { + LOGGER.warn(String.format("Retry [%d] write failed, ignore the error, continue to write!", trySize)); + } else { + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.ES_INDEX_INSERT, e.getMessage(), e); + } + } + + if (null != bulkResult) { + List items = bulkResult.getItems(); + for (int idx = 0; idx < items.size(); ++idx) { + BulkResult.BulkResultItem item = items.get(idx); + if (item.error != null && !"".equals(item.error)) { + super.getTaskPluginCollector().collectDirtyRecord(writerBuffer.get(idx), + String.format("status:[%d], error: %s", item.status, item.error)); + } + } + } + } + + private int getRecordColumnIndex(Record record, String columnName) { + if (colNameToIndexMap.containsKey(columnName)) { + return colNameToIndexMap.get(columnName); + } + + List columns = new ArrayList(); + int index = -1; + for (int i=0; i 1) { + throw DataXException.asDataXException( + ElasticSearchWriterErrorCode.RECORD_FIELD_NOT_FOUND, + "record has multiple columns found by name: " + columnName); + } + + colNameToIndexMap.put(columnName, index); + return index; + } + + private String processIDCombineFields(Record record, ElasticSearchColumn esColumn) { + List values = new ArrayList(esColumn.getCombineFields().size()); + for (String field : esColumn.getCombineFields()) { + int colIndex = getRecordColumnIndex(record, field); + Column col = record.getColumnNumber() <= colIndex ? null : record.getColumn(colIndex); + if (col == null) { + throw DataXException.asDataXException(ElasticSearchWriterErrorCode.RECORD_FIELD_NOT_FOUND, field); + } + values.add(col.asString()); + } + return Joiner.on(esColumn.getCombineFieldsValueSeparator()).join(values); + } + + @Override + public void post() { + } + + @Override + public void destroy() { + try { + this.esClient.closeJestClient(); + } catch (Exception e) { + LOGGER.warn("ignore close jest client error: {}", e.getMessage()); + } + } + + } +} diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchWriterErrorCode.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchWriterErrorCode.java new file mode 100644 index 0000000000..c9b02532e3 --- /dev/null +++ b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/ElasticSearchWriterErrorCode.java @@ -0,0 +1,41 @@ +package com.alibaba.datax.plugin.writer.elasticsearchwriter; + +import com.alibaba.datax.common.spi.ErrorCode; + +public enum ElasticSearchWriterErrorCode implements ErrorCode { + BAD_CONFIG_VALUE("ESWriter-00", "The value you configured is not valid."), + ES_INDEX_DELETE("ESWriter-01", "Delete index error."), + ES_INDEX_CREATE("ESWriter-02", "Index creation error."), + ES_MAPPINGS("ESWriter-03", "The mappings error."), + ES_INDEX_INSERT("ESWriter-04", "Insert data error."), + ES_ALIAS_MODIFY("ESWriter-05", "Alias modification error."), + JSON_PARSE("ESWrite-06", "Json format parsing error"), + UPDATE_WITH_ID("ESWrite-07", "Update mode must specify column type with id"), + RECORD_FIELD_NOT_FOUND("ESWrite-08", "Field does not exist in the original table"), + ES_GET_SETTINGS("ESWriter-09", "get settings failed"); + ; + + private final String code; + private final String description; + + ElasticSearchWriterErrorCode(String code, String description) { + this.code = code; + this.description = description; + } + + @Override + public String getCode() { + return this.code; + } + + @Override + public String getDescription() { + return this.description; + } + + @Override + public String toString() { + return String.format("Code:[%s], Description:[%s]. ", this.code, + this.description); + } +} diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/JsonPathUtil.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/JsonPathUtil.java new file mode 100644 index 0000000000..49703435ef --- /dev/null +++ b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/JsonPathUtil.java @@ -0,0 +1,28 @@ +package com.alibaba.datax.plugin.writer.elasticsearchwriter; + +import java.util.List; + +import com.alibaba.fastjson.JSONObject; + +public class JsonPathUtil { + + public static JSONObject getJsonObject(List paths, JSONObject data) { + if (null == paths || paths.isEmpty()) { + return data; + } + + if (null == data) { + return null; + } + + JSONObject dataTmp = data; + for (String each : paths) { + if (null != dataTmp) { + dataTmp = dataTmp.getJSONObject(each); + } else { + return null; + } + } + return dataTmp; + } +} \ No newline at end of file diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/JsonUtil.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/JsonUtil.java new file mode 100644 index 0000000000..e73c87be34 --- /dev/null +++ b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/JsonUtil.java @@ -0,0 +1,54 @@ +package com.alibaba.datax.plugin.writer.elasticsearchwriter; + +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.JSONException; +import com.alibaba.fastjson.JSONObject; + +/** + * @author bozu + * @date 2021/01/06 + */ +public class JsonUtil { + + /** + * 合并两个json + * @param source 源json + * @param target 目标json + * @return 合并后的json + * @throws JSONException + */ + public static String mergeJsonStr(String source, String target) throws JSONException { + if(source == null) { + return target; + } + if(target == null) { + return source; + } + return JSON.toJSONString(deepMerge(JSON.parseObject(source), JSON.parseObject(target))); + } + + /** + * 深度合并两个json对象,将source的值,merge到target中 + * @param source 源json + * @param target 目标json + * @return 合并后的json + * @throws JSONException + */ + private static JSONObject deepMerge(JSONObject source, JSONObject target) throws JSONException { + for (String key: source.keySet()) { + Object value = source.get(key); + if (target.containsKey(key)) { + // existing value for "key" - recursively deep merge: + if (value instanceof JSONObject) { + JSONObject valueJson = (JSONObject)value; + deepMerge(valueJson, target.getJSONObject(key)); + } else { + target.put(key, value); + } + } else { + target.put(key, value); + } + } + return target; + } +} diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/Key.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/Key.java index 0f2d3f5c20..af19771136 100644 --- a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/Key.java +++ b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/Key.java @@ -1,9 +1,13 @@ package com.alibaba.datax.plugin.writer.elasticsearchwriter; import com.alibaba.datax.common.util.Configuration; +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.TypeReference; + import org.apache.commons.lang3.StringUtils; import java.util.HashMap; +import java.util.List; import java.util.Map; public final class Key { @@ -37,31 +41,35 @@ public static ActionType getActionType(Configuration conf) { public static String getEndpoint(Configuration conf) { - return conf.getNecessaryValue("endpoint", ESWriterErrorCode.BAD_CONFIG_VALUE); + return conf.getNecessaryValue("endpoint", ElasticSearchWriterErrorCode.BAD_CONFIG_VALUE); } - public static String getAccessID(Configuration conf) { - return conf.getString("accessId", ""); + public static String getUsername(Configuration conf) { + return conf.getString("username", conf.getString("accessId")); } - public static String getAccessKey(Configuration conf) { - return conf.getString("accessKey", ""); + public static String getPassword(Configuration conf) { + return conf.getString("password", conf.getString("accessKey")); } public static int getBatchSize(Configuration conf) { - return conf.getInt("batchSize", 1000); + return conf.getInt("batchSize", 1024); } public static int getTrySize(Configuration conf) { return conf.getInt("trySize", 30); } + public static long getTryInterval(Configuration conf) { + return conf.getLong("tryInterval", 60000L); + } + public static int getTimeout(Configuration conf) { return conf.getInt("timeout", 600000); } - public static boolean isCleanup(Configuration conf) { - return conf.getBool("cleanup", false); + public static boolean isTruncate(Configuration conf) { + return conf.getBool("truncate", conf.getBool("cleanup", false)); } public static boolean isDiscovery(Configuration conf) { @@ -69,7 +77,7 @@ public static boolean isDiscovery(Configuration conf) { } public static boolean isCompression(Configuration conf) { - return conf.getBool("compression", true); + return conf.getBool("compress", conf.getBool("compression", true)); } public static boolean isMultiThread(Configuration conf) { @@ -77,9 +85,17 @@ public static boolean isMultiThread(Configuration conf) { } public static String getIndexName(Configuration conf) { - return conf.getNecessaryValue("index", ESWriterErrorCode.BAD_CONFIG_VALUE); + return conf.getNecessaryValue("index", ElasticSearchWriterErrorCode.BAD_CONFIG_VALUE); } + public static String getDeleteBy(Configuration conf) { + return conf.getString("deleteBy"); + } + + + /** + * TODO: 在7.0开始,一个索引只能建一个Type为_doc + * */ public static String getTypeName(Configuration conf) { String indexType = conf.getString("indexType"); if(StringUtils.isBlank(indexType)){ @@ -128,4 +144,58 @@ public static String getSplitter(Configuration conf) { public static boolean getDynamic(Configuration conf) { return conf.getBool("dynamic", false); } + + public static String getDstDynamic(Configuration conf) { + return conf.getString("dstDynamic"); + } + + public static String getDiscoveryFilter(Configuration conf){ + return conf.getString("discoveryFilter","_all"); + } + + public static Boolean getVersioning(Configuration conf) { + return conf.getBool("versioning", false); + } + + public static Long getUnifiedVersion(Configuration conf) { + return conf.getLong("version", System.currentTimeMillis()); + } + + public static Map getUrlParams(Configuration conf) { + return conf.getMap("urlParams", new HashMap()); + } + + public static Integer getESVersion(Configuration conf) { + return conf.getInt("esVersion"); + } + + public static String getMasterTimeout(Configuration conf) { + return conf.getString("masterTimeout", "5m"); + } + + public static boolean isEnableNullUpdate(Configuration conf) { + return conf.getBool("enableWriteNull", true); + } + + public static String getFieldDelimiter(Configuration conf) { + return conf.getString("fieldDelimiter", ""); + } + + public static PrimaryKeyInfo getPrimaryKeyInfo(Configuration conf) { + String primaryKeyInfoString = conf.getString("primaryKeyInfo"); + if (StringUtils.isNotBlank(primaryKeyInfoString)) { + return JSON.parseObject(primaryKeyInfoString, new TypeReference() {}); + } else { + return null; + } + } + + public static List getEsPartitionColumn(Configuration conf) { + String esPartitionColumnString = conf.getString("esPartitionColumn"); + if (StringUtils.isNotBlank(esPartitionColumnString)) { + return JSON.parseObject(esPartitionColumnString, new TypeReference>() {}); + } else { + return null; + } + } } diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/NoReRunException.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/NoReRunException.java new file mode 100644 index 0000000000..52064e58d7 --- /dev/null +++ b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/NoReRunException.java @@ -0,0 +1,16 @@ +package com.alibaba.datax.plugin.writer.elasticsearchwriter; + +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.common.spi.ErrorCode; + +public class NoReRunException extends DataXException { + public NoReRunException(String errorMessage) { + super(errorMessage); + } + + public NoReRunException(ErrorCode errorCode, String errorMessage) { + super(errorCode, errorMessage); + } + + private static final long serialVersionUID = 1L; +} \ No newline at end of file diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/PartitionColumn.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/PartitionColumn.java new file mode 100644 index 0000000000..b99829b2cc --- /dev/null +++ b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/PartitionColumn.java @@ -0,0 +1,42 @@ +package com.alibaba.datax.plugin.writer.elasticsearchwriter; + +public class PartitionColumn { + private String name; + // like: DATA + private String metaType; + private String comment; + // like: VARCHAR + private String type; + + public String getName() { + return name; + } + + public String getMetaType() { + return metaType; + } + + public String getComment() { + return comment; + } + + public String getType() { + return type; + } + + public void setName(String name) { + this.name = name; + } + + public void setMetaType(String metaType) { + this.metaType = metaType; + } + + public void setComment(String comment) { + this.comment = comment; + } + + public void setType(String type) { + this.type = type; + } +} \ No newline at end of file diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/PrimaryKeyInfo.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/PrimaryKeyInfo.java new file mode 100644 index 0000000000..b5821f515e --- /dev/null +++ b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/PrimaryKeyInfo.java @@ -0,0 +1,47 @@ +package com.alibaba.datax.plugin.writer.elasticsearchwriter; + +import java.util.List; + +public class PrimaryKeyInfo { + + /** + * 主键类型:PrimaryKeyTypeEnum + * + * pk: 单个(业务)主键 specific: 联合主键 + */ + private String type; + + /** + * 用户定义的联合主键的连接符号 + */ + private String fieldDelimiter; + + /** + * 主键的列的名称 + */ + private List column; + + public String getType() { + return type; + } + + public String getFieldDelimiter() { + return fieldDelimiter; + } + + public List getColumn() { + return column; + } + + public void setType(String type) { + this.type = type; + } + + public void setFieldDelimiter(String fieldDelimiter) { + this.fieldDelimiter = fieldDelimiter; + } + + public void setColumn(List column) { + this.column = column; + } +} \ No newline at end of file diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/jest/ClusterInfo.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/jest/ClusterInfo.java new file mode 100644 index 0000000000..173bc9e264 --- /dev/null +++ b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/jest/ClusterInfo.java @@ -0,0 +1,35 @@ +package com.alibaba.datax.plugin.writer.elasticsearchwriter.jest; + +import com.google.gson.Gson; +import io.searchbox.action.AbstractAction; +import io.searchbox.client.config.ElasticsearchVersion; + +public class ClusterInfo extends AbstractAction { + @Override + protected String buildURI(ElasticsearchVersion elasticsearchVersion) { + return ""; + } + + @Override + public String getRestMethodName() { + return "GET"; + } + + @Override + public ClusterInfoResult createNewElasticSearchResult(String responseBody, int statusCode, String reasonPhrase, Gson gson) { + return createNewElasticSearchResult(new ClusterInfoResult(gson), responseBody, statusCode, reasonPhrase, gson); + } + + public static class Builder extends AbstractAction.Builder { + + public Builder() { + setHeader("accept", "application/json"); + setHeader("content-type", "application/json"); + } + + @Override + public ClusterInfo build() { + return new ClusterInfo(); + } + } +} diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/jest/ClusterInfoResult.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/jest/ClusterInfoResult.java new file mode 100644 index 0000000000..b4f49a37c0 --- /dev/null +++ b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/jest/ClusterInfoResult.java @@ -0,0 +1,49 @@ +package com.alibaba.datax.plugin.writer.elasticsearchwriter.jest; + +import com.google.gson.Gson; +import io.searchbox.client.JestResult; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class ClusterInfoResult extends JestResult { + + private static final Pattern FIRST_NUMBER = Pattern.compile("\\d"); + + private static final int SEVEN = 7; + + public ClusterInfoResult(Gson gson) { + super(gson); + } + + public ClusterInfoResult(JestResult source) { + super(source); + } + + /** + * 判断es集群的部署版本是否大于7.x + * 大于7.x的es对于Index的type有较大改动,需要做额外判定 + * 对于7.x与6.x版本的es都做过测试,返回符合预期;5.x以下版本直接try-catch后返回false,向下兼容 + * @return + */ + public Boolean isGreaterOrEqualThan7() throws Exception { + // 如果是没有权限,直接返回false,兼容老版本 + if (responseCode == 403) { + return false; + } + if (!isSucceeded) { + throw new Exception(getJsonString()); + } + try { + String version = jsonObject.getAsJsonObject("version").get("number").toString(); + Matcher matcher = FIRST_NUMBER.matcher(version); + matcher.find(); + String number = matcher.group(); + Integer versionNum = Integer.valueOf(number); + return versionNum >= SEVEN; + } catch (Exception e) { + //5.x 以下版本不做兼容测试,如果返回json格式解析失败,有可能是以下版本,所以认为不大于7.x + return false; + } + } +} diff --git a/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/jest/PutMapping7.java b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/jest/PutMapping7.java new file mode 100644 index 0000000000..c9f1d6be84 --- /dev/null +++ b/elasticsearchwriter/src/main/java/com/alibaba/datax/plugin/writer/elasticsearchwriter/jest/PutMapping7.java @@ -0,0 +1,39 @@ +package com.alibaba.datax.plugin.writer.elasticsearchwriter.jest; + +import io.searchbox.action.GenericResultAbstractAction; +import io.searchbox.client.config.ElasticsearchVersion; + +public class PutMapping7 extends GenericResultAbstractAction { + protected PutMapping7(PutMapping7.Builder builder) { + super(builder); + + this.indexName = builder.index; + this.payload = builder.source; + } + + @Override + protected String buildURI(ElasticsearchVersion elasticsearchVersion) { + return super.buildURI(elasticsearchVersion) + "/_mapping"; + } + + @Override + public String getRestMethodName() { + return "PUT"; + } + + public static class Builder extends GenericResultAbstractAction.Builder { + private String index; + private Object source; + + public Builder(String index, Object source) { + this.index = index; + this.source = source; + } + + @Override + public PutMapping7 build() { + return new PutMapping7(this); + } + } + +} diff --git a/elasticsearchwriter/src/main/resources/plugin.json b/elasticsearchwriter/src/main/resources/plugin.json index b6e6384bce..b39f1222d0 100644 --- a/elasticsearchwriter/src/main/resources/plugin.json +++ b/elasticsearchwriter/src/main/resources/plugin.json @@ -1,6 +1,6 @@ { "name": "elasticsearchwriter", - "class": "com.alibaba.datax.plugin.writer.elasticsearchwriter.ESWriter", + "class": "com.alibaba.datax.plugin.writer.elasticsearchwriter.ElasticSearchWriter", "description": "适用于: 生产环境. 原理: TODO", "developer": "alibaba" } \ No newline at end of file diff --git a/hbase20xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase20xsqlreader/HBase20SQLReaderHelper.java b/hbase20xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase20xsqlreader/HBase20SQLReaderHelper.java index 0edc993f81..11bbf734a4 100644 --- a/hbase20xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase20xsqlreader/HBase20SQLReaderHelper.java +++ b/hbase20xsqlreader/src/main/java/com/alibaba/datax/plugin/reader/hbase20xsqlreader/HBase20SQLReaderHelper.java @@ -175,7 +175,7 @@ public List doSplit(int adviceNumber) { if (querySql == null || querySql.isEmpty()) { // 如果splitPoints为空,则根据splitKey自动切分,不过这种切分方式无法保证数据均分,且只支持整形和字符型列 if (splitPoints == null || splitPoints.isEmpty()) { - LOG.info("Split accoring min and max value of splitColumn..."); + LOG.info("Split according min and max value of splitColumn..."); Pair minMaxPK = getPkRange(configuration); if (null == minMaxPK) { throw DataXException.asDataXException(HBase20xSQLReaderErrorCode.ILLEGAL_SPLIT_PK, @@ -208,7 +208,7 @@ public List doSplit(int adviceNumber) { } } else { - LOG.info("Split accoring splitPoints..."); + LOG.info("Split according splitPoints..."); // 根据指定splitPoints进行切分 rangeList = buildSplitRange(); } diff --git a/hbase20xsqlreader/src/main/resources/plugin.json b/hbase20xsqlreader/src/main/resources/plugin.json index 45856411e0..4a7b4edfd2 100644 --- a/hbase20xsqlreader/src/main/resources/plugin.json +++ b/hbase20xsqlreader/src/main/resources/plugin.json @@ -2,6 +2,6 @@ "name": "hbase20xsqlreader", "class": "com.alibaba.datax.plugin.reader.hbase20xsqlreader.HBase20xSQLReader", "description": "useScene: prod. mechanism: read data from phoenix through queryserver.", - "developer": "bake" + "developer": "alibaba" } diff --git a/hbase20xsqlwriter/src/main/resources/plugin.json b/hbase20xsqlwriter/src/main/resources/plugin.json index 91b7069f4a..93d3002a54 100755 --- a/hbase20xsqlwriter/src/main/resources/plugin.json +++ b/hbase20xsqlwriter/src/main/resources/plugin.json @@ -2,6 +2,6 @@ "name": "hbase20xsqlwriter", "class": "com.alibaba.datax.plugin.writer.hbase20xsqlwriter.HBase20xSQLWriter", "description": "useScene: prod. mechanism: use hbase sql UPSERT to put data, index tables will be updated too.", - "developer": "bake" + "developer": "alibaba" } diff --git a/hdfswriter/doc/hdfswriter.md b/hdfswriter/doc/hdfswriter.md index 028a544e63..1259b253a4 100644 --- a/hdfswriter/doc/hdfswriter.md +++ b/hdfswriter/doc/hdfswriter.md @@ -231,6 +231,7 @@ HdfsWriter提供向HDFS文件系统指定路径中写入TEXTFile文件和ORCFile * append,写入前不做任何处理,DataX hdfswriter直接使用filename写入,并保证文件名不冲突。 * nonConflict,如果目录下有fileName前缀的文件,直接报错。 + * truncate,如果目录下有fileName前缀的文件,先删除后写入。 * 必选:是
diff --git a/hologresjdbcwriter/src/main/resources/plugin.json b/hologresjdbcwriter/src/main/resources/plugin.json index d46f216b15..a9f939965e 100644 --- a/hologresjdbcwriter/src/main/resources/plugin.json +++ b/hologresjdbcwriter/src/main/resources/plugin.json @@ -1,6 +1,6 @@ { - "name": "hologreswriter", - "class": "com.alibaba.datax.plugin.writer.hologreswriter.HologresWriter", + "name": "hologresjdbcwriter", + "class": "com.alibaba.datax.plugin.writer.hologresjdbcwriter.HologresJdbcWriter", "description": "", "developer": "alibaba" -} +} \ No newline at end of file diff --git a/hologresjdbcwriter/src/main/resources/plugin_job_template.json b/hologresjdbcwriter/src/main/resources/plugin_job_template.json index 656971c3e8..f509ccc06b 100644 --- a/hologresjdbcwriter/src/main/resources/plugin_job_template.json +++ b/hologresjdbcwriter/src/main/resources/plugin_job_template.json @@ -1,5 +1,5 @@ { - "name": "hologreswriter", + "name": "hologresjdbcwriter", "parameter": { "url": "", "username": "", diff --git a/loghubreader/pom.xml b/loghubreader/pom.xml new file mode 100644 index 0000000000..b2f52f3da1 --- /dev/null +++ b/loghubreader/pom.xml @@ -0,0 +1,73 @@ + + + + datax-all + com.alibaba.datax + 0.0.1-SNAPSHOT + + 4.0.0 + + loghubreader + + 0.0.1-SNAPSHOT + + + + com.alibaba.datax + datax-common + ${datax-project-version} + + + slf4j-log4j12 + org.slf4j + + + + + org.slf4j + slf4j-api + + + ch.qos.logback + logback-classic + + + com.aliyun.openservices + aliyun-log + 0.6.22 + + + + + + + + maven-compiler-plugin + + ${jdk-version} + ${jdk-version} + ${project-sourceEncoding} + + + + + maven-assembly-plugin + + + src/main/assembly/package.xml + + datax + + + + dwzip + package + + single + + + + + + + diff --git a/loghubreader/src/main/assembly/package.xml b/loghubreader/src/main/assembly/package.xml new file mode 100644 index 0000000000..e1d8d73912 --- /dev/null +++ b/loghubreader/src/main/assembly/package.xml @@ -0,0 +1,34 @@ + + + + dir + + false + + + src/main/resources + + plugin.json + + plugin/reader/loghubreader + + + target/ + + loghubreader-0.0.1-SNAPSHOT.jar + + plugin/reader/loghubreader + + + + + + false + plugin/reader/loghubreader/libs + runtime + + + diff --git a/loghubreader/src/main/java/com/alibaba/datax/plugin/reader/loghubreader/Constant.java b/loghubreader/src/main/java/com/alibaba/datax/plugin/reader/loghubreader/Constant.java new file mode 100644 index 0000000000..fd9e88dc28 --- /dev/null +++ b/loghubreader/src/main/java/com/alibaba/datax/plugin/reader/loghubreader/Constant.java @@ -0,0 +1,26 @@ +package com.alibaba.datax.plugin.reader.loghubreader; + +public class Constant { + + public static String DATETIME_FORMAT = "yyyyMMddHHmmss"; + public static String DATE_FORMAT = "yyyyMMdd"; + + static String META_COL_SOURCE = "__source__"; + static String META_COL_TOPIC = "__topic__"; + static String META_COL_CATEGORY = "__category__"; + static String META_COL_MACHINEUUID = "__machineUUID__"; + static String META_COL_HOSTNAME = "__hostname__"; + static String META_COL_PATH = "__path__"; + static String META_COL_LOGTIME = "__logtime__"; + public static String META_COL_RECEIVE_TIME = "__receive_time__"; + + /** + * 除用户手动配置的列之外,其余数据列作为一个 json 读取到一列 + */ + static String COL_EXTRACT_OTHERS = "C__extract_others__"; + + /** + * 将所有元数据列作为一个 json 读取到一列 + */ + static String COL_EXTRACT_ALL_META = "C__extract_all_meta__"; +} diff --git a/loghubreader/src/main/java/com/alibaba/datax/plugin/reader/loghubreader/Key.java b/loghubreader/src/main/java/com/alibaba/datax/plugin/reader/loghubreader/Key.java new file mode 100644 index 0000000000..9067cc68fc --- /dev/null +++ b/loghubreader/src/main/java/com/alibaba/datax/plugin/reader/loghubreader/Key.java @@ -0,0 +1,38 @@ +package com.alibaba.datax.plugin.reader.loghubreader; + +public final class Key { + + /** + * 此处声明插件用到的需要插件使用者提供的配置项 + */ + public static final String ENDPOINT = "endpoint"; + + public static final String ACCESSKEYID = "accessId"; + + public static final String ACCESSKEYSECRET = "accessKey"; + + public static final String PROJECT = "project"; + + public static final String LOGSTORE = "logstore"; + + public static final String TOPIC = "topic"; + + public static final String COLUMN = "column"; + + public static final String BATCHSIZE = "batchSize"; + + public static final String BEGINTIMESTAMPMILLIS = "beginTimestampMillis"; + + public static final String ENDTIMESTAMPMILLIS = "endTimestampMillis"; + + public static final String BEGINDATETIME = "beginDateTime"; + + public static final String ENDDATETIME = "endDateTime"; + + public static final String TIMEFORMAT = "timeformat"; + + public static final String SOURCE = "source"; + + public static final String SHARD = "shard"; + +} diff --git a/loghubreader/src/main/java/com/alibaba/datax/plugin/reader/loghubreader/LogHubReader.java b/loghubreader/src/main/java/com/alibaba/datax/plugin/reader/loghubreader/LogHubReader.java new file mode 100644 index 0000000000..f25fbc6176 --- /dev/null +++ b/loghubreader/src/main/java/com/alibaba/datax/plugin/reader/loghubreader/LogHubReader.java @@ -0,0 +1,482 @@ +package com.alibaba.datax.plugin.reader.loghubreader; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.datax.common.element.StringColumn; +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.common.plugin.RecordSender; +import com.alibaba.datax.common.spi.Reader; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.common.util.DataXCaseEnvUtil; +import com.alibaba.datax.common.util.RetryUtil; +import com.alibaba.fastjson.JSONObject; +import com.aliyun.openservices.log.Client; +import com.aliyun.openservices.log.common.Consts.CursorMode; +import com.aliyun.openservices.log.common.*; +import com.aliyun.openservices.log.exception.LogException; +import com.aliyun.openservices.log.response.BatchGetLogResponse; +import com.aliyun.openservices.log.response.GetCursorResponse; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.*; +import java.util.concurrent.Callable; + +public class LogHubReader extends Reader { + public static class Job extends Reader.Job { + + private static final Logger LOG = LoggerFactory.getLogger(Job.class); + + private Client client; + private Configuration originalConfig; + + private Long beginTimestampMillis; + private Long endTimestampMillis; + + @Override + public void init() { + LOG.info("loghub reader job init begin ..."); + this.originalConfig = super.getPluginJobConf(); + validateParameter(originalConfig); + + String endPoint = this.originalConfig.getString(Key.ENDPOINT); + String accessKeyId = this.originalConfig.getString(Key.ACCESSKEYID); + String accessKeySecret = this.originalConfig.getString(Key.ACCESSKEYSECRET); + + client = new Client(endPoint, accessKeyId, accessKeySecret); + LOG.info("loghub reader job init end."); + } + + private void validateParameter(Configuration conf){ + conf.getNecessaryValue(Key.ENDPOINT,LogHubReaderErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.ACCESSKEYID,LogHubReaderErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.ACCESSKEYSECRET,LogHubReaderErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.PROJECT,LogHubReaderErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.LOGSTORE,LogHubReaderErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.COLUMN,LogHubReaderErrorCode.REQUIRE_VALUE); + + int batchSize = this.originalConfig.getInt(Key.BATCHSIZE); + if (batchSize > 1000) { + throw DataXException.asDataXException(LogHubReaderErrorCode.BAD_CONFIG_VALUE, + "Invalid batchSize[" + batchSize + "] value (0,1000]!"); + } + + beginTimestampMillis = this.originalConfig.getLong(Key.BEGINTIMESTAMPMILLIS); + String beginDateTime = this.originalConfig.getString(Key.BEGINDATETIME); + + if (beginDateTime != null) { + try { + beginTimestampMillis = getUnixTimeFromDateTime(beginDateTime); + } catch (ParseException e) { + throw DataXException.asDataXException(LogHubReaderErrorCode.BAD_CONFIG_VALUE, + "Invalid beginDateTime[" + beginDateTime + "], format [yyyyMMddHHmmss or yyyyMMdd]!"); + } + } + + if (beginTimestampMillis != null && beginTimestampMillis <= 0) { + throw DataXException.asDataXException(LogHubReaderErrorCode.BAD_CONFIG_VALUE, + "Invalid beginTimestampMillis[" + beginTimestampMillis + "]!"); + } + + endTimestampMillis = this.originalConfig.getLong(Key.ENDTIMESTAMPMILLIS); + String endDateTime = this.originalConfig.getString(Key.ENDDATETIME); + + if (endDateTime != null) { + try { + endTimestampMillis = getUnixTimeFromDateTime(endDateTime); + } catch (ParseException e) { + throw DataXException.asDataXException(LogHubReaderErrorCode.BAD_CONFIG_VALUE, + "Invalid beginDateTime[" + endDateTime + "], format [yyyyMMddHHmmss or yyyyMMdd]!"); + } + } + + if (endTimestampMillis != null && endTimestampMillis <= 0) { + throw DataXException.asDataXException(LogHubReaderErrorCode.BAD_CONFIG_VALUE, + "Invalid endTimestampMillis[" + endTimestampMillis + "]!"); + } + + if (beginTimestampMillis != null && endTimestampMillis != null + && endTimestampMillis <= beginTimestampMillis) { + throw DataXException.asDataXException(LogHubReaderErrorCode.BAD_CONFIG_VALUE, + "endTimestampMillis[" + endTimestampMillis + "] must bigger than beginTimestampMillis[" + beginTimestampMillis + "]!"); + } + } + + private long getUnixTimeFromDateTime(String dateTime) throws ParseException { + try { + String format = Constant.DATETIME_FORMAT; + SimpleDateFormat simpleDateFormat = new SimpleDateFormat(format); + return simpleDateFormat.parse(dateTime).getTime() / 1000; + } catch (ParseException ignored) { + throw DataXException.asDataXException(LogHubReaderErrorCode.BAD_CONFIG_VALUE, + "Invalid DateTime[" + dateTime + "]!"); + } + } + + @Override + public void prepare() { + } + + @Override + public List split(int adviceNumber) { + LOG.info("split() begin..."); + + List readerSplitConfigs = new ArrayList(); + + final String project = this.originalConfig.getString(Key.PROJECT); + final String logstore = this.originalConfig.getString(Key.LOGSTORE); + + List logStore = null; + try { + logStore = RetryUtil.executeWithRetry(new Callable>() { + @Override + public List call() throws Exception { + return client.ListShard(project, logstore).GetShards(); + } + }, DataXCaseEnvUtil.getRetryTimes(7), DataXCaseEnvUtil.getRetryInterval(1000L), DataXCaseEnvUtil.getRetryExponential(true)); + } catch (Exception e) { + throw DataXException.asDataXException(LogHubReaderErrorCode.BAD_CONFIG_VALUE, + "get LogStore[" + logstore + "] error, please check ! detail error messsage: " + e.toString()); + } + + if (logStore == null) { + throw DataXException.asDataXException(LogHubReaderErrorCode.BAD_CONFIG_VALUE, + "LogStore[" + logstore + "] isn't exists, please check !"); + } + + int splitNumber = logStore.size(); + if (0 == splitNumber) { + throw DataXException.asDataXException(LogHubReaderErrorCode.EMPTY_LOGSTORE_VALUE, + "LogStore[" + logstore + "] has 0 shard, please check !"); + } + + Collections.shuffle(logStore); + for (int i = 0; i < logStore.size(); i++) { + if (beginTimestampMillis != null && endTimestampMillis != null) { + try { + String beginCursor = getCursorWithRetry(client, project, logstore, logStore.get(i).GetShardId(), beginTimestampMillis).GetCursor(); + String endCursor = getCursorWithRetry(client, project, logstore, logStore.get(i).GetShardId(), endTimestampMillis).GetCursor(); + if (beginCursor.equals(endCursor)) { + if ((i == logStore.size() - 1) && (readerSplitConfigs.size() == 0)) { + + } else { + LOG.info("skip empty shard[" + logStore.get(i) + "]!"); + continue; + } + } + } catch (Exception e) { + LOG.error("Check Shard[" + logStore.get(i) + "] Error, please check !" + e.toString()); + throw DataXException.asDataXException(LogHubReaderErrorCode.LOG_HUB_ERROR, e); + } + } + Configuration splitedConfig = this.originalConfig.clone(); + splitedConfig.set(Key.SHARD, logStore.get(i).GetShardId()); + readerSplitConfigs.add(splitedConfig); + } + + if (splitNumber < adviceNumber) { + // LOG.info(MESSAGE_SOURCE.message("hdfsreader.12", + // splitNumber, adviceNumber, splitNumber, splitNumber)); + } + LOG.info("split() ok and end..."); + + return readerSplitConfigs; + } + + @Override + public void post() { + } + + @Override + public void destroy() { + } + + private GetCursorResponse getCursorWithRetry(final Client client, final String project, final String logstore, final int shard, final long fromTime) throws Exception { + return + RetryUtil.executeWithRetry(new Callable() { + @Override + public GetCursorResponse call() throws Exception { + LOG.info("loghug get cursor with project: {} logstore: {} shard: {} time: {}", project, logstore, shard, fromTime); + return client.GetCursor(project, logstore, shard, fromTime); + } + }, 7, 1000L, true); + } + + } + + public static class Task extends Reader.Task { + + private static final Logger LOG = LoggerFactory.getLogger(Task.class); + + private Configuration taskConfig; + private Client client; + private String endPoint; + private String accessKeyId; + private String accessKeySecret; + private String project; + private String logstore; + private long beginTimestampMillis; + private long endTimestampMillis; + private int batchSize; + private int shard; + private List columns; + + @Override + public void init() { + this.taskConfig = super.getPluginJobConf(); + + endPoint = this.taskConfig.getString(Key.ENDPOINT); + accessKeyId = this.taskConfig.getString(Key.ACCESSKEYID); + accessKeySecret = this.taskConfig.getString(Key.ACCESSKEYSECRET); + project = this.taskConfig.getString(Key.PROJECT); + logstore = this.taskConfig.getString(Key.LOGSTORE); + batchSize = this.taskConfig.getInt(Key.BATCHSIZE, 128); + + this.beginTimestampMillis = this.taskConfig.getLong(Key.BEGINTIMESTAMPMILLIS, -1); + String beginDateTime = this.taskConfig.getString(Key.BEGINDATETIME); + + if (beginDateTime != null) { + try { + beginTimestampMillis = getUnixTimeFromDateTime(beginDateTime); + } catch (ParseException e) { + } + } + + this.endTimestampMillis = this.taskConfig.getLong(Key.ENDTIMESTAMPMILLIS, -1); + String endDateTime = this.taskConfig.getString(Key.ENDDATETIME); + + if (endDateTime != null) { + try { + endTimestampMillis = getUnixTimeFromDateTime(endDateTime); + } catch (ParseException e) { + } + } + + columns = this.taskConfig.getList(Key.COLUMN, String.class); + + shard = this.taskConfig.getInt(Key.SHARD); + + client = new Client(endPoint, accessKeyId, accessKeySecret); + LOG.info("init loghub reader task finished.project:{} logstore:{} batchSize:{}", project, logstore, batchSize); + } + + @Override + public void prepare() { + } + + private long getUnixTimeFromDateTime(String dateTime) throws ParseException { + try { + String format = Constant.DATETIME_FORMAT; + SimpleDateFormat simpleDateFormat = new SimpleDateFormat(format); + return simpleDateFormat.parse(dateTime).getTime() / 1000; + } catch (ParseException ignored) { + } + String format = Constant.DATE_FORMAT; + SimpleDateFormat simpleDateFormat = new SimpleDateFormat(format); + return simpleDateFormat.parse(dateTime).getTime() / 1000; + } + + private GetCursorResponse getCursorWithRetry(final Client client, final String project, final String logstore, final int shard, final long fromTime) throws Exception { + return + RetryUtil.executeWithRetry(new Callable() { + @Override + public GetCursorResponse call() throws Exception { + LOG.info("loghug get cursor with project: {} logstore: {} shard: {} time: {}", project, logstore, shard, fromTime); + return client.GetCursor(project, logstore, shard, fromTime); + } + }, 7, 1000L, true); + } + + private GetCursorResponse getCursorWithRetry(final Client client, final String project, final String logstore, final int shard, final CursorMode mode) throws Exception { + return + RetryUtil.executeWithRetry(new Callable() { + @Override + public GetCursorResponse call() throws Exception { + LOG.info("loghug get cursor with project: {} logstore: {} shard: {} mode: {}", project, logstore, shard, mode); + return client.GetCursor(project, logstore, shard, mode); + } + }, 7, 1000L, true); + } + + private BatchGetLogResponse batchGetLogWithRetry(final Client client, final String project, final String logstore, final int shard, final int batchSize, + final String curCursor, final String endCursor) throws Exception { + return + RetryUtil.executeWithRetry(new Callable() { + @Override + public BatchGetLogResponse call() throws Exception { + return client.BatchGetLog(project, logstore, shard, batchSize, curCursor, endCursor); + } + }, 7, 1000L, true); + } + + @Override + public void startRead(RecordSender recordSender) { + LOG.info("read start"); + + try { + GetCursorResponse cursorRes; + if (this.beginTimestampMillis != -1) { + cursorRes = getCursorWithRetry(client, project, logstore, this.shard, beginTimestampMillis); + } else { + cursorRes = getCursorWithRetry(client, project, logstore, this.shard, CursorMode.BEGIN); + } + String beginCursor = cursorRes.GetCursor(); + + LOG.info("the begin cursor, loghub requestId: {} cursor: {}", cursorRes.GetRequestId(), cursorRes.GetCursor()); + + if (this.endTimestampMillis != -1) { + cursorRes = getCursorWithRetry(client, project, logstore, this.shard, endTimestampMillis); + } else { + cursorRes = getCursorWithRetry(client, project, logstore, this.shard, CursorMode.END); + } + String endCursor = cursorRes.GetCursor(); + LOG.info("the end cursor, loghub requestId: {} cursor: {}", cursorRes.GetRequestId(), cursorRes.GetCursor()); + + if (StringUtils.equals(beginCursor, endCursor)) { + LOG.info("beginCursor:{} equals endCursor:{}, end directly!", beginCursor, endCursor); + return; + } + + String currentCursor = null; + String nextCursor = beginCursor; + + HashMap metaMap = new HashMap(); + HashMap dataMap = new HashMap(); + JSONObject allMetaJson = new JSONObject(); + while (!StringUtils.equals(currentCursor, nextCursor)) { + currentCursor = nextCursor; + BatchGetLogResponse logDataRes = batchGetLogWithRetry(client, project, logstore, this.shard, this.batchSize, currentCursor, endCursor); + + List logGroups = logDataRes.GetLogGroups(); + + for(LogGroupData logGroup: logGroups) { + metaMap.clear(); + allMetaJson.clear(); + FastLogGroup flg = logGroup.GetFastLogGroup(); + + metaMap.put("C_Category", flg.getCategory()); + metaMap.put(Constant.META_COL_CATEGORY, flg.getCategory()); + allMetaJson.put(Constant.META_COL_CATEGORY, flg.getCategory()); + + metaMap.put("C_Source", flg.getSource()); + metaMap.put(Constant.META_COL_SOURCE, flg.getSource()); + allMetaJson.put(Constant.META_COL_SOURCE, flg.getSource()); + + metaMap.put("C_Topic", flg.getTopic()); + metaMap.put(Constant.META_COL_TOPIC, flg.getTopic()); + allMetaJson.put(Constant.META_COL_TOPIC, flg.getTopic()); + + metaMap.put("C_MachineUUID", flg.getMachineUUID()); + metaMap.put(Constant.META_COL_MACHINEUUID, flg.getMachineUUID()); + allMetaJson.put(Constant.META_COL_MACHINEUUID, flg.getMachineUUID()); + + for (int tagIdx = 0; tagIdx < flg.getLogTagsCount(); ++tagIdx) { + FastLogTag logtag = flg.getLogTags(tagIdx); + String tagKey = logtag.getKey(); + String tagValue = logtag.getValue(); + if (tagKey.equals(Constant.META_COL_HOSTNAME)) { + metaMap.put("C_HostName", logtag.getValue()); + } else if (tagKey.equals(Constant.META_COL_PATH)) { + metaMap.put("C_Path", logtag.getValue()); + } + metaMap.put(tagKey, tagValue); + allMetaJson.put(tagKey, tagValue); + } + + for (int lIdx = 0; lIdx < flg.getLogsCount(); ++lIdx) { + dataMap.clear(); + FastLog log = flg.getLogs(lIdx); + + String logTime = String.valueOf(log.getTime()); + metaMap.put("C_LogTime", logTime); + metaMap.put(Constant.META_COL_LOGTIME, logTime); + allMetaJson.put(Constant.META_COL_LOGTIME, logTime); + + for (int cIdx = 0; cIdx < log.getContentsCount(); ++cIdx) { + FastLogContent content = log.getContents(cIdx); + dataMap.put(content.getKey(), content.getValue()); + } + + Record record = recordSender.createRecord(); + + JSONObject extractOthers = new JSONObject(); + if(columns.contains(Constant.COL_EXTRACT_OTHERS)){ + List keyList = Arrays.asList(dataMap.keySet().toArray(new String[dataMap.keySet().size()])); + for (String otherKey:keyList) { + if (!columns.contains(otherKey)){ + extractOthers.put(otherKey,dataMap.get(otherKey)); + } + } + } + if (null != this.columns && 1 == this.columns.size()) { + String columnsInStr = columns.get(0).toString(); + if ("\"*\"".equals(columnsInStr) || "*".equals(columnsInStr)) { + List keyList = Arrays.asList(dataMap.keySet().toArray(new String[dataMap.keySet().size()])); + Collections.sort(keyList); + + for (String key : keyList) { + record.addColumn(new StringColumn(key + ":" + dataMap.get(key))); + } + } else { + if (dataMap.containsKey(columnsInStr)) { + record.addColumn(new StringColumn(dataMap.get(columnsInStr))); + } else if (metaMap.containsKey(columnsInStr)) { + record.addColumn(new StringColumn(metaMap.get(columnsInStr))); + } else if (Constant.COL_EXTRACT_OTHERS.equals(columnsInStr)){ + record.addColumn(new StringColumn(extractOthers.toJSONString())); + } else if (Constant.COL_EXTRACT_ALL_META.equals(columnsInStr)) { + record.addColumn(new StringColumn(allMetaJson.toJSONString())); + } + } + } else { + for (String col : this.columns) { + if (dataMap.containsKey(col)) { + record.addColumn(new StringColumn(dataMap.get(col))); + } else if (metaMap.containsKey(col)) { + record.addColumn(new StringColumn(metaMap.get(col))); + } else if (col != null && col.startsWith("'") && col.endsWith("'")){ + String constant = col.substring(1, col.length()-1); + record.addColumn(new StringColumn(constant)); + }else if (Constant.COL_EXTRACT_OTHERS.equals(col)){ + record.addColumn(new StringColumn(extractOthers.toJSONString())); + } else if (Constant.COL_EXTRACT_ALL_META.equals(col)) { + record.addColumn(new StringColumn(allMetaJson.toJSONString())); + } else { + record.addColumn(new StringColumn(null)); + } + } + } + + recordSender.sendToWriter(record); + } + } + + nextCursor = logDataRes.GetNextCursor(); + } + } catch (LogException e) { + if (e.GetErrorCode().equals("LogStoreNotExist")) { + LOG.info("logStore[" + logstore +"] Not Exits! detail error messsage: " + e.toString()); + } else { + LOG.error("read LogStore[" + logstore + "] error, please check ! detail error messsage: " + e.toString()); + throw DataXException.asDataXException(LogHubReaderErrorCode.LOG_HUB_ERROR, e); + } + + } catch (Exception e) { + LOG.error("read LogStore[" + logstore + "] error, please check ! detail error messsage: " + e.toString()); + throw DataXException.asDataXException(LogHubReaderErrorCode.LOG_HUB_ERROR, e); + } + + LOG.info("end read loghub shard..."); + } + + @Override + public void post() { + } + + @Override + public void destroy() { + } + } +} diff --git a/loghubreader/src/main/java/com/alibaba/datax/plugin/reader/loghubreader/LogHubReaderErrorCode.java b/loghubreader/src/main/java/com/alibaba/datax/plugin/reader/loghubreader/LogHubReaderErrorCode.java new file mode 100644 index 0000000000..d9ee4c8ee1 --- /dev/null +++ b/loghubreader/src/main/java/com/alibaba/datax/plugin/reader/loghubreader/LogHubReaderErrorCode.java @@ -0,0 +1,34 @@ +package com.alibaba.datax.plugin.reader.loghubreader; + +import com.alibaba.datax.common.spi.ErrorCode; + +public enum LogHubReaderErrorCode implements ErrorCode { + BAD_CONFIG_VALUE("LogHuReader-00", "The value you configured is invalid."), + LOG_HUB_ERROR("LogHubReader-01","LogHub access encounter exception"), + REQUIRE_VALUE("LogHubReader-02","Missing parameters"), + EMPTY_LOGSTORE_VALUE("LogHubReader-03","There is no shard in this LogStore"); + + private final String code; + private final String description; + + private LogHubReaderErrorCode(String code, String description) { + this.code = code; + this.description = description; + } + + @Override + public String getCode() { + return this.code; + } + + @Override + public String getDescription() { + return this.description; + } + + @Override + public String toString() { + return String.format("Code:[%s], Description:[%s]. ", this.code, + this.description); + } +} diff --git a/loghubreader/src/main/resources/plugin.json b/loghubreader/src/main/resources/plugin.json new file mode 100644 index 0000000000..31403dd63d --- /dev/null +++ b/loghubreader/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "loghubreader", + "class": "com.alibaba.datax.plugin.reader.loghubreader.LogHubReader", + "description": "适用于: 从SLS LogHub中读取数据", + "developer": "alibaba" +} \ No newline at end of file diff --git a/loghubreader/src/main/resources/plugin_job_template.json b/loghubreader/src/main/resources/plugin_job_template.json new file mode 100644 index 0000000000..4d536eb9fb --- /dev/null +++ b/loghubreader/src/main/resources/plugin_job_template.json @@ -0,0 +1,12 @@ +{ + "name": "loghubreader", + "parameter": { + "endpoint": "", + "accessId": "", + "accessKey": "", + "project": "", + "logstore": "", + "batchSize":1024, + "column": [] + } +} \ No newline at end of file diff --git a/loghubwriter/pom.xml b/loghubwriter/pom.xml new file mode 100644 index 0000000000..d43b7286d8 --- /dev/null +++ b/loghubwriter/pom.xml @@ -0,0 +1,73 @@ + + + + datax-all + com.alibaba.datax + 0.0.1-SNAPSHOT + + 4.0.0 + + loghubwriter + + 0.0.1-SNAPSHOT + + + + com.alibaba.datax + datax-common + ${datax-project-version} + + + slf4j-log4j12 + org.slf4j + + + + + org.slf4j + slf4j-api + + + ch.qos.logback + logback-classic + + + com.aliyun.openservices + aliyun-log + 0.6.12 + + + + + + + + maven-compiler-plugin + + ${jdk-version} + ${jdk-version} + ${project-sourceEncoding} + + + + + maven-assembly-plugin + + + src/main/assembly/package.xml + + datax + + + + dwzip + package + + single + + + + + + + diff --git a/loghubwriter/src/main/assembly/package.xml b/loghubwriter/src/main/assembly/package.xml new file mode 100644 index 0000000000..44d25a48a3 --- /dev/null +++ b/loghubwriter/src/main/assembly/package.xml @@ -0,0 +1,34 @@ + + + + dir + + false + + + src/main/resources + + plugin.json + + plugin/writer/loghubwriter + + + target/ + + loghubwriter-0.0.1-SNAPSHOT.jar + + plugin/writer/loghubwriter + + + + + + false + plugin/writer/loghubwriter/libs + runtime + + + diff --git a/loghubwriter/src/main/java/com/alibaba/datax/plugin/writer/loghubwriter/Key.java b/loghubwriter/src/main/java/com/alibaba/datax/plugin/writer/loghubwriter/Key.java new file mode 100644 index 0000000000..bdfe3fa537 --- /dev/null +++ b/loghubwriter/src/main/java/com/alibaba/datax/plugin/writer/loghubwriter/Key.java @@ -0,0 +1,35 @@ +package com.alibaba.datax.plugin.writer.loghubwriter; + +/** + * 配置关键字 + * @author + */ +public final class Key { + + /** + * 此处声明插件用到的需要插件使用者提供的配置项 + */ + public static final String ENDPOINT = "endpoint"; + + public static final String ACCESS_KEY_ID = "accessId"; + + public static final String ACCESS_KEY_SECRET = "accessKey"; + + public static final String PROJECT = "project"; + + public static final String LOG_STORE = "logstore"; + + public static final String TOPIC = "topic"; + + public static final String COLUMN = "column"; + + public static final String BATCH_SIZE = "batchSize"; + + public static final String TIME = "time"; + + public static final String TIME_FORMAT = "timeformat"; + + public static final String SOURCE = "source"; + + public static final String HASH_BY_KEY = "hashKey"; +} diff --git a/loghubwriter/src/main/java/com/alibaba/datax/plugin/writer/loghubwriter/LogHubWriter.java b/loghubwriter/src/main/java/com/alibaba/datax/plugin/writer/loghubwriter/LogHubWriter.java new file mode 100644 index 0000000000..bf60d08c0f --- /dev/null +++ b/loghubwriter/src/main/java/com/alibaba/datax/plugin/writer/loghubwriter/LogHubWriter.java @@ -0,0 +1,315 @@ +package com.alibaba.datax.plugin.writer.loghubwriter; + +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.common.util.RetryUtil; +import com.alibaba.datax.common.util.StrUtil; +import com.aliyun.openservices.log.Client; +import com.aliyun.openservices.log.common.LogItem; +import com.aliyun.openservices.log.common.Shard; +import com.aliyun.openservices.log.exception.LogException; +import com.aliyun.openservices.log.request.ListShardRequest; +import com.aliyun.openservices.log.request.PutLogsRequest; +import com.aliyun.openservices.log.response.ListShardResponse; +import com.aliyun.openservices.log.response.PutLogsResponse; + +import org.apache.commons.codec.digest.Md5Crypt; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import sun.security.provider.MD5; + +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; + +/** + * SLS 写插件 + * @author + */ +public class LogHubWriter extends Writer { + + public static class Job extends Writer.Job { + private static final Logger LOG = LoggerFactory.getLogger(Job.class); + + private Configuration jobConfig = null; + + @Override + public void init() { + info(LOG, "loghub writer job init begin ..."); + this.jobConfig = super.getPluginJobConf(); + validateParameter(jobConfig); + info(LOG, "loghub writer job init end."); + } + + private void validateParameter(Configuration conf){ + conf.getNecessaryValue(Key.ENDPOINT,LogHubWriterErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.ACCESS_KEY_ID,LogHubWriterErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.ACCESS_KEY_SECRET,LogHubWriterErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.PROJECT,LogHubWriterErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.LOG_STORE,LogHubWriterErrorCode.REQUIRE_VALUE); + conf.getNecessaryValue(Key.COLUMN,LogHubWriterErrorCode.REQUIRE_VALUE); + } + + @Override + public List split(int mandatoryNumber) { + info(LOG, "split begin..."); + List configurationList = new ArrayList(); + for (int i = 0; i < mandatoryNumber; i++) { + configurationList.add(this.jobConfig.clone()); + } + info(LOG, "split end..."); + return configurationList; + } + + @Override + public void post() { + } + + @Override + public void destroy() { + } + } + + public static class Task extends Writer.Task { + private static final Logger LOG = LoggerFactory.getLogger(Task.class); + private Configuration taskConfig; + private com.aliyun.openservices.log.Client logHubClient; + private String logStore; + private String topic; + private String project; + private List columnList; + private int batchSize; + private String timeCol; + private String timeFormat; + private String source; + private boolean isHashKey; + private List shards; + public void init() { + this.taskConfig = super.getPluginJobConf(); + String endpoint = taskConfig.getString(Key.ENDPOINT); + String accessKeyId = taskConfig.getString(Key.ACCESS_KEY_ID); + String accessKeySecret = taskConfig.getString(Key.ACCESS_KEY_SECRET); + project = taskConfig.getString(Key.PROJECT); + logStore = taskConfig.getString(Key.LOG_STORE); + topic = taskConfig.getString(Key.TOPIC,""); + columnList = taskConfig.getList(Key.COLUMN,String.class); + batchSize = taskConfig.getInt(Key.BATCH_SIZE,1024); + timeCol = taskConfig.getString(Key.TIME,""); + timeFormat = taskConfig.getString(Key.TIME_FORMAT,""); + source = taskConfig.getString(Key.SOURCE,""); + isHashKey = taskConfig.getBool(Key.HASH_BY_KEY,false); + logHubClient = new Client(endpoint, accessKeyId, accessKeySecret); + if (isHashKey) { + listShard(); + info(LOG, "init loghub writer with hash key mode."); + } + if (LOG.isInfoEnabled()) { + LOG.info("init loghub writer task finished.project:{} logstore:{} topic:{} batchSize:{}",project,logStore,topic,batchSize); + } + } + + /** + * 获取通道的分片信息 + */ + private void listShard() { + try { + ListShardResponse response = logHubClient.ListShard(new ListShardRequest(project,logStore)); + shards = response.GetShards(); + if (LOG.isInfoEnabled()) { + LOG.info("Get shard count:{}", shards.size()); + } + } catch (LogException e) { + info(LOG, "Get shard failed!"); + throw new RuntimeException("Get shard failed!", e); + } + } + + @Override + public void prepare() { + } + + private int getTime(String v) { + try { + if ("bigint".equalsIgnoreCase(timeFormat)) { + return Integer.valueOf(v); + } + + DateFormat sdf = new SimpleDateFormat(timeFormat); + Date date = sdf.parse(v); + return (int)(date.getTime()/1000); + } catch (Exception e) { + LOG.warn("Format time failed!", e); + } + return (int)(((new Date())).getTime()/1000); + } + + @Override + public void startWrite(RecordReceiver recordReceiver) { + info(LOG, "start to write....................."); + // 按照shared做hash处理 + if (isHashKey) { + processDataWithHashKey(recordReceiver); + } else { + processDataWithoutHashKey(recordReceiver); + } + info(LOG, "finish to write........."); + } + + private void processDataWithHashKey(RecordReceiver receiver) { + Record record; + Map> logMap = new HashMap>(shards.size()); + int count = 0; + try { + while ((record = receiver.getFromReader()) != null) { + LogItem logItem = new LogItem(); + if (record.getColumnNumber() != columnList.size()) { + this.getTaskPluginCollector().collectDirtyRecord(record, "column not match"); + } + + String id = ""; + for (int i = 0; i < record.getColumnNumber(); i++) { + String colName = columnList.get(i); + String colValue = record.getColumn(i).asString(); + if (colName.endsWith("_id")) { + id = colValue; + } + + logItem.PushBack(colName, colValue); + if (colName.equals(timeCol)) { + logItem.SetTime(getTime(colValue)); + } + } + + String hashKey = getShardHashKey(StrUtil.getMd5(id), shards); + if (!logMap.containsKey(hashKey)) { + info(LOG, "Hash key:" + hashKey); + logMap.put(hashKey, new ArrayList()); + } + logMap.get(hashKey).add(logItem); + + if (logMap.get(hashKey).size() % batchSize == 0) { + PutLogsRequest request = new PutLogsRequest(project, logStore, topic, source, logMap.get(hashKey), hashKey); + PutLogsResponse response = putLog(request); + count += logMap.get(hashKey).size(); + if (LOG.isDebugEnabled()) { + LOG.debug("record count:{}, request id:{}", logMap.get(hashKey).size(), response.GetRequestId()); + } + logMap.get(hashKey).clear(); + } + } + + for (Map.Entry> entry : logMap.entrySet()) { + if (!entry.getValue().isEmpty()) { + // 将剩余的数据发送 + PutLogsRequest request = new PutLogsRequest(project, logStore, topic, source, entry.getValue(), entry.getKey()); + PutLogsResponse response = putLog(request); + count += entry.getValue().size(); + if (LOG.isDebugEnabled()) { + LOG.debug("record count:{}, request id:{}", entry.getValue().size(), response.GetRequestId()); + } + entry.getValue().clear(); + } + } + LOG.info("{} records have been sent", count); + } catch (LogException ex) { + throw DataXException.asDataXException(LogHubWriterErrorCode.LOG_HUB_ERROR, ex.getMessage(), ex); + } catch (Exception e) { + throw DataXException.asDataXException(LogHubWriterErrorCode.LOG_HUB_ERROR, e.getMessage(), e); + } + } + + private void processDataWithoutHashKey(RecordReceiver receiver) { + Record record; + ArrayList logGroup = new ArrayList(); + int count = 0; + try { + while ((record = receiver.getFromReader()) != null) { + LogItem logItem = new LogItem(); + if(record.getColumnNumber() != columnList.size()){ + this.getTaskPluginCollector().collectDirtyRecord(record,"column not match"); + } + for (int i = 0; i < record.getColumnNumber(); i++) { + String colName = columnList.get(i); + String colValue = record.getColumn(i).asString(); + logItem.PushBack(colName, colValue); + if(colName.equals(timeCol)){ + logItem.SetTime(getTime(colValue)); + } + } + + logGroup.add(logItem); + count++; + if (count % batchSize == 0) { + PutLogsRequest request = new PutLogsRequest(project, logStore, topic, source, logGroup); + PutLogsResponse response = putLog(request); + logGroup.clear(); + if (LOG.isDebugEnabled()) { + LOG.debug("record count:{}, request id:{}", count, response.GetRequestId()); + } + } + } + if (!logGroup.isEmpty()) { + //将剩余的数据发送 + PutLogsRequest request = new PutLogsRequest(project, logStore, topic, source, logGroup); + PutLogsResponse response = putLog(request); + logGroup.clear(); + if (LOG.isDebugEnabled()) { + LOG.debug("record count:{}, request id:{}", count, response.GetRequestId()); + } + } + LOG.info("{} records have been sent", count); + } catch (LogException ex) { + throw DataXException.asDataXException(LogHubWriterErrorCode.LOG_HUB_ERROR, ex.getMessage(), ex); + } catch (Exception e) { + throw DataXException.asDataXException(LogHubWriterErrorCode.LOG_HUB_ERROR, e.getMessage(), e); + } + } + + private PutLogsResponse putLog(final PutLogsRequest request) throws Exception{ + final Client client = this.logHubClient; + + return RetryUtil.executeWithRetry(new Callable() { + public PutLogsResponse call() throws LogException{ + return client.PutLogs(request); + } + }, 3, 1000L, false); + } + + private String getShardHashKey(String hashKey, List shards) { + for (Shard shard : shards) { + if (hashKey.compareTo(shard.getExclusiveEndKey()) < 0 && hashKey.compareTo(shard.getInclusiveBeginKey()) >= 0) { + return shard.getInclusiveBeginKey(); + } + } + return shards.get(0).getInclusiveBeginKey(); + } + + @Override + public void post() { + } + + @Override + public void destroy() { + } + } + + /** + * 日志打印控制 + * + * @param logger + * @param message + */ + public static void info(Logger logger, String message) { + if (logger.isInfoEnabled()) { + logger.info(message); + } + } +} diff --git a/loghubwriter/src/main/java/com/alibaba/datax/plugin/writer/loghubwriter/LogHubWriterErrorCode.java b/loghubwriter/src/main/java/com/alibaba/datax/plugin/writer/loghubwriter/LogHubWriterErrorCode.java new file mode 100644 index 0000000000..98c5e16f88 --- /dev/null +++ b/loghubwriter/src/main/java/com/alibaba/datax/plugin/writer/loghubwriter/LogHubWriterErrorCode.java @@ -0,0 +1,33 @@ +package com.alibaba.datax.plugin.writer.loghubwriter; + +import com.alibaba.datax.common.spi.ErrorCode; + +public enum LogHubWriterErrorCode implements ErrorCode { + BAD_CONFIG_VALUE("LogHubWriter-00", "The value you configured is invalid."), + LOG_HUB_ERROR("LogHubWriter-01","LogHub access encounter exception"), + REQUIRE_VALUE("LogHubWriter-02","Missing parameters"); + + private final String code; + private final String description; + + private LogHubWriterErrorCode(String code, String description) { + this.code = code; + this.description = description; + } + + @Override + public String getCode() { + return this.code; + } + + @Override + public String getDescription() { + return this.description; + } + + @Override + public String toString() { + return String.format("Code:[%s], Description:[%s]. ", this.code, + this.description); + } +} \ No newline at end of file diff --git a/loghubwriter/src/main/resources/plugin.json b/loghubwriter/src/main/resources/plugin.json new file mode 100644 index 0000000000..2a913b1437 --- /dev/null +++ b/loghubwriter/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "loghubwriter", + "class": "com.alibaba.datax.plugin.writer.loghubwriter.LogHubWriter", + "description": "适用于: 将数据导入到SLS LogHub中", + "developer": "alibaba" +} \ No newline at end of file diff --git a/loghubwriter/src/main/resources/plugin_job_template.json b/loghubwriter/src/main/resources/plugin_job_template.json new file mode 100644 index 0000000000..ac0d3b2a04 --- /dev/null +++ b/loghubwriter/src/main/resources/plugin_job_template.json @@ -0,0 +1,13 @@ +{ + "name": "loghubwriter", + "parameter": { + "endpoint": "", + "accessId": "", + "accessKey": "", + "project": "", + "logstore": "", + "topic": "", + "batchSize":1024, + "column": [] + } +} \ No newline at end of file diff --git a/mysqlreader/doc/mysqlreader.md b/mysqlreader/doc/mysqlreader.md index 24589579bd..bae4bce0f6 100644 --- a/mysqlreader/doc/mysqlreader.md +++ b/mysqlreader/doc/mysqlreader.md @@ -197,9 +197,9 @@ MysqlReader插件实现了从Mysql读取数据。在底层实现上,MysqlReade * **querySql** - * 描述:在有些业务场景下,where这一配置项不足以描述所筛选的条件,用户可以通过该配置型来自定义筛选SQL。当用户配置了这一项之后,DataX系统就会忽略column这些配置型,直接使用这个配置项的内容对数据进行筛选,例如需要进行多表join后同步数据,使用select a,b from table_a join table_b on table_a.id = table_b.id
+ * 描述:在有些业务场景下,where这一配置项不足以描述所筛选的条件,用户可以通过该配置型来自定义筛选SQL。当用户配置了这一项之后,DataX系统就会忽略table,column这些配置型,直接使用这个配置项的内容对数据进行筛选,例如需要进行多表join后同步数据,使用select a,b from table_a join table_b on table_a.id = table_b.id
- `当用户配置querySql时,MysqlReader直接忽略column、where条件的配置`,querySql优先级大于column、where选项。querySql和table不能同时存在 + `当用户配置querySql时,MysqlReader直接忽略table、column、where条件的配置`,querySql优先级大于table、column、where选项。 * 必选:否
diff --git a/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/ext/ReaderJob.java b/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/ext/ReaderJob.java index 291dc785cf..642c99fe12 100644 --- a/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/ext/ReaderJob.java +++ b/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/ext/ReaderJob.java @@ -1,6 +1,5 @@ package com.alibaba.datax.plugin.reader.oceanbasev10reader.ext; -import java.util.Arrays; import java.util.List; import com.alibaba.datax.common.constant.CommonConstant; @@ -27,7 +26,7 @@ public ReaderJob() { public void init(Configuration originalConfig) { //将config中的column和table中的关键字进行转义 List columns = originalConfig.getList(Key.COLUMN, String.class); - ObReaderUtils.escapeDatabaseKeywords(columns); + ObReaderUtils.escapeDatabaseKeyword(columns); originalConfig.set(Key.COLUMN, columns); List conns = originalConfig.getList(Constant.CONN_MARK, JSONObject.class); @@ -38,7 +37,7 @@ public void init(Configuration originalConfig) { // tables will be null when querySql is configured if (tables != null) { - ObReaderUtils.escapeDatabaseKeywords(tables); + ObReaderUtils.escapeDatabaseKeyword(tables); originalConfig.set(String.format("%s[%d].%s", Constant.CONN_MARK, i, Key.TABLE), tables); } @@ -79,7 +78,8 @@ private String getObRegionName(String jdbcUrl) { final String obJdbcDelimiter = com.alibaba.datax.plugin.rdbms.writer.Constant.OB10_SPLIT_STRING; if (jdbcUrl.startsWith(obJdbcDelimiter)) { String[] ss = jdbcUrl.split(obJdbcDelimiter); - if (ss.length >= 2) { + int elementCount = 2; + if (ss.length >= elementCount) { String tenant = ss[1].trim(); String[] sss = tenant.split(":"); return sss[0]; diff --git a/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/ObReaderUtils.java b/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/ObReaderUtils.java index 6356b97b13..06d5310879 100644 --- a/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/ObReaderUtils.java +++ b/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/ObReaderUtils.java @@ -37,12 +37,15 @@ public class ObReaderUtils { public static final DataBaseType databaseType = DataBaseType.OceanBase; + private static final String TABLE_SCHEMA_DELIMITER = "."; + + private static final Pattern JDBC_PATTERN = Pattern.compile("jdbc:(oceanbase|mysql)://([\\w\\.-]+:\\d+)/([\\w\\.-]+)"); private static Set keywordsFromString2HashSet(final String keywords) { return new HashSet(Arrays.asList(keywords.split(","))); } - public static String escapeDatabaseKeywords(String keyword) { + public static String escapeDatabaseKeyword(String keyword) { if (databaseKeywords == null) { if (isOracleMode(compatibleMode)) { databaseKeywords = keywordsFromString2HashSet(ORACLE_KEYWORDS); @@ -57,10 +60,10 @@ public static String escapeDatabaseKeywords(String keyword) { return keyword; } - public static void escapeDatabaseKeywords(List ids) { + public static void escapeDatabaseKeyword(List ids) { if (ids != null && ids.size() > 0) { for (int i = 0; i < ids.size(); i++) { - ids.set(i, escapeDatabaseKeywords(ids.get(i))); + ids.set(i, escapeDatabaseKeyword(ids.get(i))); } } } @@ -144,7 +147,7 @@ private static String[] getPkColumns(Connection conn, TaskContext context) { if (isOracleMode(context.getCompatibleMode())) { tableName = tableName.toUpperCase(); String schema; - if (tableName.contains(".")) { + if (tableName.contains(TABLE_SCHEMA_DELIMITER)) { schema = String.format("'%s'", tableName.substring(0, tableName.indexOf("."))); tableName = tableName.substring(tableName.indexOf(".") + 1); } else { @@ -170,7 +173,7 @@ private static String[] getPkColumns(Connection conn, TaskContext context) { while (rs.next()) { hasPk = true; String columnName = rs.getString("Column_name"); - columnName = escapeDatabaseKeywords(columnName); + columnName = escapeDatabaseKeyword(columnName); if (!realIndex.contains(columnName)) { realIndex.add(columnName); } @@ -462,7 +465,7 @@ private static Map> getAllIndex(Connection conn, String tab if (isOracleMode(compatibleMode)) { String schema; tableName = tableName.toUpperCase(); - if (tableName.contains(".")) { + if (tableName.contains(TABLE_SCHEMA_DELIMITER)) { schema = String.format("'%s'", tableName.substring(0, tableName.indexOf("."))); tableName = tableName.substring(tableName.indexOf(".") + 1); } else { @@ -513,7 +516,7 @@ private static Map> getAllIndex(Connection conn, String tab Iterator>> iterator = allIndex.entrySet().iterator(); while (iterator.hasNext()) { Map.Entry> entry = iterator.next(); - if (entry.getKey().equals("PRIMARY")) { + if ("PRIMARY".equals(entry.getKey())) { continue; } @@ -770,9 +773,7 @@ public static boolean isOracleMode(String mode) { } public static String getDbNameFromJdbcUrl(String jdbcUrl) { - final Pattern pattern = Pattern.compile("jdbc:(oceanbase|mysql)://([\\w\\.-]+:\\d+)/([\\w\\.-]+)"); - - Matcher matcher = pattern.matcher(jdbcUrl); + Matcher matcher = JDBC_PATTERN.matcher(jdbcUrl); if (matcher.find()) { return matcher.group(3); } else { @@ -814,18 +815,52 @@ public static int compareObVersion(String version1, String version2) { if (version1 == null || version2 == null) { throw new RuntimeException("can not compare null version"); } + ObVersion v1 = new ObVersion(version1); + ObVersion v2 = new ObVersion(version2); + return v1.compareTo(v2); + } - String[] ver1Part = version1.split("\\."); - String[] ver2Part = version2.split("\\."); - for (int i = 0; i < ver1Part.length; i++) { - int v1 = Integer.parseInt(ver1Part[i]), v2 = Integer.parseInt(ver2Part[i]); - if (v1 > v2) { - return 1; - } else if (v1 < v2) { - return -1; + /** + * + * @param conn + * @param sql + * @return + */ + public static List getResultsFromSql(Connection conn, String sql) { + List list = new ArrayList(); + Statement stmt = null; + ResultSet rs = null; + + LOG.info("executing sql: " + sql); + + try { + stmt = conn.createStatement(); + rs = stmt.executeQuery(sql); + while (rs.next()) { + list.add(rs.getString(1)); } + } catch (Exception e) { + LOG.error("error when executing sql: " + e.getMessage()); + } finally { + DBUtil.closeDBResources(rs, stmt, null); + } + + return list; + } + + /** + * get obversion, try ob_version first, and then try version if failed + * @param conn + * @return + */ + public static ObVersion getObVersion(Connection conn) { + List results = getResultsFromSql(conn, "select ob_version()"); + if (results.size() == 0) { + results = getResultsFromSql(conn, "select version()"); } + ObVersion obVersion = new ObVersion(results.get(0)); - return 0; + LOG.info("obVersion: " + obVersion); + return obVersion; } } diff --git a/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/ObVersion.java b/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/ObVersion.java new file mode 100644 index 0000000000..7bb6559e57 --- /dev/null +++ b/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/ObVersion.java @@ -0,0 +1,86 @@ +package com.alibaba.datax.plugin.reader.oceanbasev10reader.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @author johnrobbet + */ +public class ObVersion implements Comparable { + + private static final Logger LOG = LoggerFactory.getLogger(ObVersion.class); + + private int majorVersion; + private int minorVersion; + private int releaseNumber; + private int patchNumber; + + public static final ObVersion V2276 = valueOf("2.2.76"); + private static final ObVersion DEFAULT_VERSION = + valueOf(System.getProperty("defaultObVersion","3.2.3.0")); + + private static final int VERSION_PART_COUNT = 4; + + public ObVersion(String version) { + try { + String[] versionParts = version.split("\\."); + majorVersion = Integer.valueOf(versionParts[0]); + minorVersion = Integer.valueOf(versionParts[1]); + releaseNumber = Integer.valueOf(versionParts[2]); + int tempPatchNum = 0; + if (versionParts.length == VERSION_PART_COUNT) { + try { + tempPatchNum = Integer.valueOf(versionParts[3]); + } catch (Exception e) { + LOG.warn("fail to parse ob version: " + e.getMessage()); + } + } + patchNumber = tempPatchNum; + } catch (Exception ex) { + LOG.warn("fail to get ob version, using default {} {}", + DEFAULT_VERSION, ex.getMessage()); + majorVersion = DEFAULT_VERSION.majorVersion; + minorVersion = DEFAULT_VERSION.minorVersion; + releaseNumber = DEFAULT_VERSION.releaseNumber; + patchNumber = DEFAULT_VERSION.patchNumber; + } + } + + public static ObVersion valueOf(String version) { + return new ObVersion(version); + } + + @Override + public int compareTo(ObVersion o) { + if (this.majorVersion > o.majorVersion) { + return 1; + } else if (this.majorVersion < o.majorVersion) { + return -1; + } + + if (this.minorVersion > o.minorVersion) { + return 1; + } else if (this.minorVersion < o.minorVersion) { + return -1; + } + + if (this.releaseNumber > o.releaseNumber) { + return 1; + } else if (this.releaseNumber < o.releaseNumber) { + return -1; + } + + if (this.patchNumber > o.patchNumber) { + return 1; + } else if (this.patchNumber < o.patchNumber) { + return -1; + } + + return 0; + } + + @Override + public String toString() { + return String.format("%d.%d.%d.%d", majorVersion, minorVersion, releaseNumber, patchNumber); + } +} diff --git a/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/PartType.java b/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/PartType.java index be190755e6..05c23d6ff9 100644 --- a/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/PartType.java +++ b/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/PartType.java @@ -5,8 +5,13 @@ */ public enum PartType { + // Non partitioned table NONPARTITION("NONPARTITION"), + + // Partitioned table PARTITION("PARTITION"), + + // Subpartitioned table SUBPARTITION("SUBPARTITION"); private String typeString; diff --git a/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/PartitionSplitUtil.java b/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/PartitionSplitUtil.java index 3bf2320acc..bc3d455c91 100644 --- a/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/PartitionSplitUtil.java +++ b/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/PartitionSplitUtil.java @@ -3,7 +3,6 @@ import com.alibaba.datax.common.util.Configuration; import com.alibaba.datax.plugin.rdbms.reader.Constant; import com.alibaba.datax.plugin.rdbms.reader.Key; -import com.alibaba.datax.plugin.rdbms.reader.util.HintUtil; import com.alibaba.datax.plugin.rdbms.util.DBUtil; import com.alibaba.datax.plugin.rdbms.util.DataBaseType; import com.alibaba.datax.plugin.reader.oceanbasev10reader.ext.ObReaderKey; @@ -11,8 +10,6 @@ import org.slf4j.LoggerFactory; import java.sql.Connection; -import java.sql.ResultSet; -import java.sql.Statement; import java.util.ArrayList; import java.util.List; @@ -22,12 +19,76 @@ public class PartitionSplitUtil { private static final Logger LOG = LoggerFactory.getLogger(PartitionSplitUtil.class); + private static final String ORACLE_GET_SUBPART_TEMPLATE = + "select subpartition_name " + + "from dba_tab_subpartitions " + + "where table_name = '%s' and table_owner = '%s'"; + + private static final String ORACLE_GET_PART_TEMPLATE = + "select partition_name " + + "from dba_tab_partitions " + + "where table_name = '%s' and table_owner = '%s'"; + + private static final String MYSQL_GET_PART_TEMPLATE = + "select p.part_name " + + "from oceanbase.__all_part p, oceanbase.%s t, oceanbase.__all_database d " + + "where p.table_id = t.table_id " + + "and d.database_id = t.database_id " + + "and d.database_name = '%s' " + + "and t.table_name = '%s'"; + + private static final String MYSQL_GET_SUBPART_TEMPLATE = + "select p.sub_part_name " + + "from oceanbase.__all_sub_part p, oceanbase.%s t, oceanbase.__all_database d " + + "where p.table_id = t.table_id " + + "and d.database_id = t.database_id " + + "and d.database_name = '%s' " + + "and t.table_name = '%s'"; + + /** + * get partition info from data dictionary in ob oracle mode + * @param config + * @param tableName + * @return + */ + public static PartInfo getObOraclePartInfoBySQL(Configuration config, String tableName) { + PartInfo partInfo; + DataBaseType dbType = ObReaderUtils.databaseType; + String jdbcUrl = config.getString(Key.JDBC_URL); + String username = config.getString(Key.USERNAME); + String password = config.getString(Key.PASSWORD); + String dbname = ObReaderUtils.getDbNameFromJdbcUrl(jdbcUrl).toUpperCase(); + Connection conn = DBUtil.getConnection(dbType, jdbcUrl, username, password); + tableName = tableName.toUpperCase(); + + // check if the table has subpartitions or not + String getSubPartSql = String.format(ORACLE_GET_SUBPART_TEMPLATE, tableName, dbname); + List partList = ObReaderUtils.getResultsFromSql(conn, getSubPartSql); + if (partList != null && partList.size() > 0) { + partInfo = new PartInfo(PartType.SUBPARTITION); + partInfo.addPart(partList); + return partInfo; + } + + String getPartSql = String.format(ORACLE_GET_PART_TEMPLATE, tableName, dbname); + partList = ObReaderUtils.getResultsFromSql(conn, getPartSql); + if (partList != null && partList.size() > 0) { + partInfo = new PartInfo(PartType.PARTITION); + partInfo.addPart(partList); + return partInfo; + } + + // table is not partitioned + partInfo = new PartInfo(PartType.NONPARTITION); + return partInfo; + } + public static List splitByPartition (Configuration configuration) { List allSlices = new ArrayList<>(); - List conns = configuration.getList(Constant.CONN_MARK, Object.class); - for (int i = 0, len = conns.size(); i < len; i++) { + List connections = configuration.getList(Constant.CONN_MARK, Object.class); + for (int i = 0, len = connections.size(); i < len; i++) { Configuration sliceConfig = configuration.clone(); - Configuration connConf = Configuration.from(conns.get(i).toString()); + Configuration connConf = Configuration.from(connections.get(i).toString()); String jdbcUrl = connConf.getString(Key.JDBC_URL); sliceConfig.set(Key.JDBC_URL, jdbcUrl); sliceConfig.remove(Constant.CONN_MARK); @@ -64,7 +125,7 @@ private static List splitSinglePartitionTable(Configuration confi slices.add(slice); } } else { - LOG.info("fail to get table part info or table is not partitioned, proceed as non-partitioned table."); + LOG.info("table is not partitioned."); Configuration slice = configuration.clone(); slice.set(Key.QUERY_SQL, ObReaderUtils.buildQuerySql(weakRead, column, table, where)); @@ -74,7 +135,16 @@ private static List splitSinglePartitionTable(Configuration confi return slices; } - private static PartInfo getObPartInfoBySQL(Configuration config, String table) { + public static PartInfo getObPartInfoBySQL(Configuration config, String table) { + boolean isOracleMode = config.getString(ObReaderKey.OB_COMPATIBILITY_MODE).equals("ORACLE"); + if (isOracleMode) { + return getObOraclePartInfoBySQL(config, table); + } else { + return getObMySQLPartInfoBySQL(config, table); + } + } + + public static PartInfo getObMySQLPartInfoBySQL(Configuration config, String table) { PartInfo partInfo = new PartInfo(PartType.NONPARTITION); List partList; Connection conn = null; @@ -86,45 +156,22 @@ private static PartInfo getObPartInfoBySQL(Configuration config, String table) { String allTable = "__all_table"; conn = DBUtil.getConnection(DataBaseType.OceanBase, jdbcUrl, username, password); - String obVersion = getResultsFromSql(conn, "select version()").get(0); - - LOG.info("obVersion: " + obVersion); - - if (ObReaderUtils.compareObVersion("2.2.76", obVersion) < 0) { + ObVersion obVersion = ObReaderUtils.getObVersion(conn); + if (obVersion.compareTo(ObVersion.V2276) >= 0) { allTable = "__all_table_v2"; } - String queryPart = String.format( - "select p.part_name " + - "from oceanbase.__all_part p, oceanbase.%s t, oceanbase.__all_database d " + - "where p.table_id = t.table_id " + - "and d.database_id = t.database_id " + - "and d.database_name = '%s' " + - "and t.table_name = '%s'", allTable, dbname, table); - String querySubPart = String.format( - "select p.sub_part_name " + - "from oceanbase.__all_sub_part p, oceanbase.%s t, oceanbase.__all_database d " + - "where p.table_id = t.table_id " + - "and d.database_id = t.database_id " + - "and d.database_name = '%s' " + - "and t.table_name = '%s'", allTable, dbname, table); - if (config.getString(ObReaderKey.OB_COMPATIBILITY_MODE).equals("ORACLE")) { - queryPart = String.format( - "select partition_name from all_tab_partitions where TABLE_OWNER = '%s' and table_name = '%s'", - dbname.toUpperCase(), table.toUpperCase()); - querySubPart = String.format( - "select subpartition_name from all_tab_subpartitions where TABLE_OWNER = '%s' and table_name = '%s'", - dbname.toUpperCase(), table.toUpperCase()); - } + String querySubPart = String.format(MYSQL_GET_SUBPART_TEMPLATE, allTable, dbname, table); PartType partType = PartType.SUBPARTITION; // try subpartition first - partList = getResultsFromSql(conn, querySubPart); + partList = ObReaderUtils.getResultsFromSql(conn, querySubPart); // if table is not sub-partitioned, the try partition if (partList.isEmpty()) { - partList = getResultsFromSql(conn, queryPart); + String queryPart = String.format(MYSQL_GET_PART_TEMPLATE, allTable, dbname, table); + partList = ObReaderUtils.getResultsFromSql(conn, queryPart); partType = PartType.PARTITION; } @@ -140,26 +187,4 @@ private static PartInfo getObPartInfoBySQL(Configuration config, String table) { return partInfo; } - - private static List getResultsFromSql(Connection conn, String sql) { - List list = new ArrayList(); - Statement stmt = null; - ResultSet rs = null; - - LOG.info("executing sql: " + sql); - - try { - stmt = conn.createStatement(); - rs = stmt.executeQuery(sql); - while (rs.next()) { - list.add(rs.getString(1)); - } - } catch (Exception e) { - LOG.error("error when executing sql: " + e.getMessage()); - } finally { - DBUtil.closeDBResources(rs, stmt, null); - } - - return list; - } } diff --git a/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/TaskContext.java b/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/TaskContext.java index d482232ae6..17655a52e4 100644 --- a/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/TaskContext.java +++ b/oceanbasev10reader/src/main/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/TaskContext.java @@ -19,6 +19,15 @@ public class TaskContext { private boolean weakRead = true; private String userSavePoint; private String compatibleMode = ObReaderUtils.OB_COMPATIBLE_MODE_MYSQL; + + public String getPartitionName() { + return partitionName; + } + + public void setPartitionName(String partitionName) { + this.partitionName = partitionName; + } + private String partitionName; // 断点续读的保存点 @@ -165,12 +174,4 @@ public String getCompatibleMode() { public void setCompatibleMode(String compatibleMode) { this.compatibleMode = compatibleMode; } - - public String getPartitionName() { - return partitionName; - } - - public void setPartitionName(String partitionName) { - this.partitionName = partitionName; - } } diff --git a/oceanbasev10reader/src/test/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/ObReaderUtilsTest.java b/oceanbasev10reader/src/test/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/ObReaderUtilsTest.java index bc38776761..3596659502 100644 --- a/oceanbasev10reader/src/test/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/ObReaderUtilsTest.java +++ b/oceanbasev10reader/src/test/java/com/alibaba/datax/plugin/reader/oceanbasev10reader/util/ObReaderUtilsTest.java @@ -18,5 +18,7 @@ public void compareObVersionTest() { assert ObReaderUtils.compareObVersion("2.2.70", "2.2.50") == 1; assert ObReaderUtils.compareObVersion("2.2.70", "3.1.2") == -1; assert ObReaderUtils.compareObVersion("3.1.2", "3.1.2") == 0; + assert ObReaderUtils.compareObVersion("3.2.3.0", "3.2.3.0") == 0; + assert ObReaderUtils.compareObVersion("3.2.3.0-CE", "3.2.3.0") == 0; } } diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/OceanBaseV10Writer.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/OceanBaseV10Writer.java index ede2eb01f7..62656843a3 100644 --- a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/OceanBaseV10Writer.java +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/OceanBaseV10Writer.java @@ -61,7 +61,7 @@ public void init() { checkCompatibleMode(originalConfig); //将config中的column和table中的关键字进行转义 List columns = originalConfig.getList(Key.COLUMN, String.class); - ObWriterUtils.escapeDatabaseKeywords(columns); + ObWriterUtils.escapeDatabaseKeyword(columns); originalConfig.set(Key.COLUMN, columns); List conns = originalConfig.getList(Constant.CONN_MARK, JSONObject.class); @@ -69,7 +69,7 @@ public void init() { JSONObject conn = conns.get(i); Configuration connConfig = Configuration.from(conn.toString()); List tables = connConfig.getList(Key.TABLE, String.class); - ObWriterUtils.escapeDatabaseKeywords(tables); + ObWriterUtils.escapeDatabaseKeyword(tables); originalConfig.set(String.format("%s[%d].%s", Constant.CONN_MARK, i, Key.TABLE), tables); } this.commonJob = new CommonRdbmsWriter.Job(DATABASE_TYPE); diff --git a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/util/ObWriterUtils.java b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/util/ObWriterUtils.java index ff1648a11f..edc4b23657 100644 --- a/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/util/ObWriterUtils.java +++ b/oceanbasev10writer/src/main/java/com/alibaba/datax/plugin/writer/oceanbasev10writer/util/ObWriterUtils.java @@ -25,7 +25,7 @@ private static Set keywordsFromString2HashSet(final String keywords) { return new HashSet(Arrays.asList(keywords.split(","))); } - public static String escapeDatabaseKeywords(String keyword) { + public static String escapeDatabaseKeyword(String keyword) { if (databaseKeywords == null) { if (isOracleMode()) { databaseKeywords = keywordsFromString2HashSet(ORACLE_KEYWORDS); @@ -40,9 +40,9 @@ public static String escapeDatabaseKeywords(String keyword) { return keyword; } - public static void escapeDatabaseKeywords(List keywords) { + public static void escapeDatabaseKeyword(List keywords) { for (int i = 0; i < keywords.size(); i++) { - keywords.set(i, escapeDatabaseKeywords(keywords.get(i))); + keywords.set(i, escapeDatabaseKeyword(keywords.get(i))); } } public static Boolean isEscapeMode(String keyword){ @@ -159,7 +159,7 @@ private static Map> getAllUniqueIndex(Connection conn, Stri while (rs.next()) { String keyName = rs.getString("Key_name"); String columnName = rs.getString("Column_name"); - columnName=escapeDatabaseKeywords(columnName); + columnName= escapeDatabaseKeyword(columnName); if(!ObWriterUtils.isEscapeMode(columnName)){ columnName = columnName.toUpperCase(); } diff --git a/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/ColumnType.java b/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/ColumnType.java index eb674a7f67..1c771d3e69 100644 --- a/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/ColumnType.java +++ b/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/ColumnType.java @@ -3,20 +3,6 @@ public enum ColumnType { PARTITION, NORMAL, CONSTANT, UNKNOWN, ; - @Override - public String toString() { - switch (this) { - case PARTITION: - return "partition"; - case NORMAL: - return "normal"; - case CONSTANT: - return "constant"; - default: - return "unknown"; - } - } - public static ColumnType asColumnType(String columnTypeString) { if ("partition".equals(columnTypeString)) { return PARTITION; diff --git a/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/InternalColumnInfo.java b/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/InternalColumnInfo.java new file mode 100644 index 0000000000..b5a15f1d76 --- /dev/null +++ b/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/InternalColumnInfo.java @@ -0,0 +1,24 @@ +package com.alibaba.datax.plugin.reader.odpsreader; + +public class InternalColumnInfo { + + private String columnName; + + private ColumnType columnType; + + public String getColumnName() { + return columnName; + } + + public void setColumnName(String columnName) { + this.columnName = columnName; + } + + public ColumnType getColumnType() { + return columnType; + } + + public void setColumnType(ColumnType columnType) { + this.columnType = columnType; + } +} diff --git a/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/OdpsReader.java b/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/OdpsReader.java index 8cb7ba31a6..d7ea6b1cb2 100755 --- a/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/OdpsReader.java +++ b/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/OdpsReader.java @@ -15,8 +15,6 @@ import com.aliyun.odps.tunnel.TableTunnel.DownloadSession; import com.aliyun.odps.type.TypeInfo; import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.tuple.MutablePair; -import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -311,7 +309,7 @@ private void dealColumn(Table table) { */ List allPartitionColumns = this.originalConfig.getList( Constant.PARTITION_COLUMNS, String.class); - List> parsedColumns = OdpsUtil + List parsedColumns = OdpsUtil .parseColumns(allNormalColumns, allPartitionColumns, userConfiguredColumns); @@ -320,13 +318,15 @@ private void dealColumn(Table table) { StringBuilder sb = new StringBuilder(); sb.append("[ "); for (int i = 0, len = parsedColumns.size(); i < len; i++) { - Pair pair = parsedColumns.get(i); - sb.append(String.format(" %s : %s", pair.getLeft(), - pair.getRight())); + InternalColumnInfo pair = parsedColumns.get(i); + sb.append(String.format(" %s : %s", pair.getColumnName(), + pair.getColumnType())); if (i != len - 1) { sb.append(","); } } + + sb.append(" ]"); LOG.info("parsed column details: {} .", sb.toString()); } @@ -500,22 +500,11 @@ public void startRead(RecordSender recordSender) { } try { - List parsedColumnsTmp = this.readerSliceConf - .getListConfiguration(Constant.PARSED_COLUMNS); - List> parsedColumns = new ArrayList>(); - for (int i = 0; i < parsedColumnsTmp.size(); i++) { - Configuration eachColumnConfig = parsedColumnsTmp.get(i); - String columnName = eachColumnConfig.getString("left"); - ColumnType columnType = ColumnType - .asColumnType(eachColumnConfig.getString("right")); - parsedColumns.add(new MutablePair( - columnName, columnType)); - - } + List parsedColumns = this.readerSliceConf.getListWithJson(Constant.PARSED_COLUMNS, + InternalColumnInfo.class); ReaderProxy readerProxy = new ReaderProxy(recordSender, downloadSession, columnTypeMap, parsedColumns, partition, this.isPartitionedTable, start, count, this.isCompress, this.readerSliceConf); - readerProxy.doRead(); } catch (Exception e) { throw DataXException.asDataXException(OdpsReaderErrorCode.READ_DATA_FAIL, diff --git a/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/ReaderProxy.java b/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/ReaderProxy.java index 31d0d605d6..1d56d19153 100755 --- a/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/ReaderProxy.java +++ b/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/ReaderProxy.java @@ -17,7 +17,6 @@ import com.aliyun.odps.type.TypeInfo; import org.apache.commons.codec.binary.Base64; import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,7 +33,7 @@ public class ReaderProxy { private RecordSender recordSender; private TableTunnel.DownloadSession downloadSession; private Map columnTypeMap; - private List> parsedColumns; + private List parsedColumns; private String partition; private boolean isPartitionTable; @@ -71,7 +70,7 @@ private Calendar initCalendar(Configuration config) { public ReaderProxy(RecordSender recordSender, TableTunnel.DownloadSession downloadSession, Map columnTypeMap, - List> parsedColumns, String partition, + List parsedColumns, String partition, boolean isPartitionTable, long start, long count, boolean isCompress, Configuration taskConfig) { this.recordSender = recordSender; this.downloadSession = downloadSession; @@ -136,9 +135,9 @@ public void doRead() { // warn: for PARTITION||NORMAL columnTypeMap's key // sets(columnName) is big than parsedColumns's left // sets(columnName), always contain - for (Pair pair : this.parsedColumns) { - String columnName = pair.getLeft(); - switch (pair.getRight()) { + for (InternalColumnInfo pair : this.parsedColumns) { + String columnName = pair.getColumnName(); + switch (pair.getColumnType()) { case PARTITION: String partitionColumnValue = this .getPartitionColumnValue(partitionMap, diff --git a/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/util/OdpsUtil.java b/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/util/OdpsUtil.java index 0103a3832c..f2ad8e0f97 100755 --- a/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/util/OdpsUtil.java +++ b/odpsreader/src/main/java/com/alibaba/datax/plugin/reader/odpsreader/util/OdpsUtil.java @@ -7,6 +7,7 @@ import com.alibaba.datax.common.util.RetryUtil; import com.alibaba.datax.plugin.reader.odpsreader.ColumnType; import com.alibaba.datax.plugin.reader.odpsreader.Constant; +import com.alibaba.datax.plugin.reader.odpsreader.InternalColumnInfo; import com.alibaba.datax.plugin.reader.odpsreader.Key; import com.alibaba.datax.plugin.reader.odpsreader.OdpsReaderErrorCode; import com.aliyun.odps.*; @@ -215,19 +216,18 @@ public static UserConfiguredPartitionClassification classifyUserConfiguredPartit return userConfiguredPartitionClassification; } - public static List> parseColumns( + public static List parseColumns( List allNormalColumns, List allPartitionColumns, List userConfiguredColumns) { - List> parsededColumns = new ArrayList>(); + List parsededColumns = new ArrayList(); // warn: upper & lower case for (String column : userConfiguredColumns) { - MutablePair pair = new MutablePair(); - + InternalColumnInfo pair = new InternalColumnInfo(); // if constant column if (OdpsUtil.checkIfConstantColumn(column)) { // remove first and last ' - pair.setLeft(column.substring(1, column.length() - 1)); - pair.setRight(ColumnType.CONSTANT); + pair.setColumnName(column.substring(1, column.length() - 1)); + pair.setColumnType(ColumnType.CONSTANT); parsededColumns.add(pair); continue; } @@ -236,8 +236,8 @@ public static List> parseColumns( // repeated in partitioning columns int index = OdpsUtil.indexOfIgnoreCase(allNormalColumns, column); if (0 <= index) { - pair.setLeft(allNormalColumns.get(index)); - pair.setRight(ColumnType.NORMAL); + pair.setColumnName(allNormalColumns.get(index)); + pair.setColumnType(ColumnType.NORMAL); parsededColumns.add(pair); continue; } @@ -245,8 +245,8 @@ public static List> parseColumns( // if partition column index = OdpsUtil.indexOfIgnoreCase(allPartitionColumns, column); if (0 <= index) { - pair.setLeft(allPartitionColumns.get(index)); - pair.setRight(ColumnType.PARTITION); + pair.setColumnName(allPartitionColumns.get(index)); + pair.setColumnType(ColumnType.PARTITION); parsededColumns.add(pair); continue; } @@ -431,13 +431,13 @@ public static void throwDataXExceptionWhenReloadTable(Exception e, String tableN MESSAGE_SOURCE.message("odpsutil.12", tableName), e); } - public static List getNormalColumns(List> parsedColumns, + public static List getNormalColumns(List parsedColumns, Map columnTypeMap) { List userConfigNormalColumns = new ArrayList(); Set columnNameSet = new HashSet(); - for (Pair columnInfo : parsedColumns) { - if (columnInfo.getValue() == ColumnType.NORMAL) { - String columnName = columnInfo.getKey(); + for (InternalColumnInfo columnInfo : parsedColumns) { + if (columnInfo.getColumnType() == ColumnType.NORMAL) { + String columnName = columnInfo.getColumnName(); if (!columnNameSet.contains(columnName)) { Column column = new Column(columnName, columnTypeMap.get(columnName)); userConfigNormalColumns.add(column); diff --git a/opentsdbreader/pom.xml b/opentsdbreader/pom.xml index f2263726a5..83d7c42447 100644 --- a/opentsdbreader/pom.xml +++ b/opentsdbreader/pom.xml @@ -24,9 +24,6 @@ 4.5 2.4 - - 1.2.28 - 2.3.2 @@ -94,7 +91,6 @@ com.alibaba fastjson - ${fastjson.version} diff --git a/opentsdbreader/src/main/resources/plugin.json b/opentsdbreader/src/main/resources/plugin.json index 692a9853f8..5c9cbed980 100755 --- a/opentsdbreader/src/main/resources/plugin.json +++ b/opentsdbreader/src/main/resources/plugin.json @@ -6,5 +6,5 @@ "mechanism": "根据时间和 metric 直连底层 HBase 存储,从而 Scan 出符合条件的数据点", "warn": "指定起止时间会自动忽略分钟和秒,转为整点时刻,例如 2019-4-18 的 [3:35, 4:55) 会被转为 [3:00, 4:00)" }, - "developer": "Benedict Jin" + "developer": "alibaba" } diff --git a/oraclereader/pom.xml b/oraclereader/pom.xml index ae8e06fa44..d60e5ebf6f 100755 --- a/oraclereader/pom.xml +++ b/oraclereader/pom.xml @@ -44,8 +44,6 @@ com.oracle ojdbc6 11.2.0.3 - system - ${basedir}/src/main/lib/ojdbc6-11.2.0.3.jar diff --git a/oraclereader/src/main/assembly/package.xml b/oraclereader/src/main/assembly/package.xml index a0c9fd1c70..a954a30d5e 100755 --- a/oraclereader/src/main/assembly/package.xml +++ b/oraclereader/src/main/assembly/package.xml @@ -15,13 +15,6 @@ plugin_job_template.json plugin/reader/oraclereader - - - src/main/lib - - ojdbc6-11.2.0.3.jar - - plugin/reader/oraclereader/libs target/ diff --git a/oraclereader/src/main/lib/ojdbc6-11.2.0.3.jar b/oraclereader/src/main/lib/ojdbc6-11.2.0.3.jar deleted file mode 100644 index 01da074d5a..0000000000 Binary files a/oraclereader/src/main/lib/ojdbc6-11.2.0.3.jar and /dev/null differ diff --git a/oraclewriter/pom.xml b/oraclewriter/pom.xml index 95b78cafa3..1e8d027432 100755 --- a/oraclewriter/pom.xml +++ b/oraclewriter/pom.xml @@ -42,8 +42,6 @@ com.oracle ojdbc6 11.2.0.3 - system - ${basedir}/src/main/lib/ojdbc6-11.2.0.3.jar diff --git a/oraclewriter/src/main/assembly/package.xml b/oraclewriter/src/main/assembly/package.xml index 09a25d1a2e..9dab0c8e18 100755 --- a/oraclewriter/src/main/assembly/package.xml +++ b/oraclewriter/src/main/assembly/package.xml @@ -16,13 +16,6 @@ plugin/writer/oraclewriter - - src/main/lib - - ojdbc6-11.2.0.3.jar - - plugin/writer/oraclewriter/libs - target/ diff --git a/oraclewriter/src/main/lib/ojdbc6-11.2.0.3.jar b/oraclewriter/src/main/lib/ojdbc6-11.2.0.3.jar deleted file mode 100644 index 01da074d5a..0000000000 Binary files a/oraclewriter/src/main/lib/ojdbc6-11.2.0.3.jar and /dev/null differ diff --git a/oscarwriter/src/main/resources/plugin.json b/oscarwriter/src/main/resources/plugin.json index f1a99fec0b..43adfbfeab 100644 --- a/oscarwriter/src/main/resources/plugin.json +++ b/oscarwriter/src/main/resources/plugin.json @@ -2,5 +2,5 @@ "name": "oscarwriter", "class": "com.alibaba.datax.plugin.writer.oscarwriter.OscarWriter", "description": "useScene: prod. mechanism: Jdbc connection using the database, execute insert sql. warn: The more you know about the database, the less problems you encounter.", - "developer": "linjiayu" + "developer": "alibaba" } \ No newline at end of file diff --git a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/Common.java b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/Common.java index 7bb3f52ea1..fb8c7feb6c 100644 --- a/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/Common.java +++ b/otsreader/src/main/java/com/alibaba/datax/plugin/reader/otsreader/utils/Common.java @@ -119,7 +119,7 @@ public static Record parseRowToLine(Row row, List columns, Record lin case BOOLEAN: line.addColumn(new BoolColumn(v.asBoolean())); break; case BINARY: line.addColumn(new BytesColumn(v.asBinary())); break; default: - throw new IllegalArgumentException("Unsuporrt tranform the type: " + col.getValue().getType() + "."); + throw new IllegalArgumentException("Unsupported transform the type: " + col.getValue().getType() + "."); } } } diff --git a/otsstreamreader/tools/tablestore_streamreader_console.py b/otsstreamreader/tools/tablestore_streamreader_console.py deleted file mode 100644 index f9379d727c..0000000000 --- a/otsstreamreader/tools/tablestore_streamreader_console.py +++ /dev/null @@ -1,179 +0,0 @@ -#!/bin/usr/env python -#-*- coding: utf-8 -*- - -from optparse import OptionParser -import sys -import json -import tabulate -import zlib -from ots2 import * - -class ConsoleConfig: - def __init__(self, config_file): - f = open(config_file, 'r') - config = json.loads(f.read()) - self.endpoint = str(config['endpoint']) - self.accessid = str(config['accessId']) - self.accesskey = str(config['accessKey']) - self.instance_name = str(config['instanceName']) - self.status_table = str(config['statusTable']) - - self.ots = OTSClient(self.endpoint, self.accessid, self.accesskey, self.instance_name) - -def describe_job(config, options): - ''' - 1. get job's description - 2. get all job's checkpoints and check if it is done - ''' - if not options.stream_id: - print "Error: Should set the stream id using '-s' or '--streamid'." - sys.exit(-1) - - if not options.timestamp: - print "Error: Should set the timestamp using '-t' or '--timestamp'." - sys.exit(-1) - - pk = [('StreamId', options.stream_id), ('StatusType', 'DataxJobDesc'), ('StatusValue', '%16d' % int(options.timestamp))] - consumed, pk, attrs, next_token = config.ots.get_row(config.status_table, pk, [], None, 1) - if not attrs: - print 'Stream job is not found.' - sys.exit(-1) - - job_detail = parse_job_detail(attrs) - print '----------JobDescriptions----------' - print json.dumps(job_detail, indent=2) - print '-----------------------------------' - - stream_checkpoints = _list_checkpoints(config, options.stream_id, int(options.timestamp)) - - cps_headers = ['ShardId', 'SendRecordCount', 'Checkpoint', 'SkipCount', 'Version'] - table_content = [] - for cp in stream_checkpoints: - table_content.append([cp['ShardId'], cp['SendRecordCount'], cp['Checkpoint'], cp['SkipCount'], cp['Version']]) - - print tabulate.tabulate(table_content, headers=cps_headers) - - # check if stream job has finished - finished = True - if len(job_detail['ShardIds']) != len(stream_checkpoints): - finished = False - - for cp in stream_checkpoints: - if cp['Version'] != job_detail['Version']: - finished = False - - print '----------JobSummary----------' - print 'ShardsCount:', len(job_detail['ShardIds']) - print 'CheckPointsCount:', len(stream_checkpoints) - print 'JobStatus:', 'Finished' if finished else 'NotFinished' - print '------------------------------' - -def _list_checkpoints(config, stream_id, timestamp): - start_pk = [('StreamId', stream_id), ('StatusType', 'CheckpointForDataxReader'), ('StatusValue', '%16d' % timestamp)] - end_pk = [('StreamId', stream_id), ('StatusType', 'CheckpointForDataxReader'), ('StatusValue', '%16d' % (timestamp + 1))] - - consumed_counter = CapacityUnit(0, 0) - columns_to_get = [] - checkpoints = [] - range_iter = config.ots.xget_range( - config.status_table, Direction.FORWARD, - start_pk, end_pk, - consumed_counter, columns_to_get, 100, - column_filter=None, max_version=1 - ) - - rows = [] - for (primary_key, attrs) in range_iter: - checkpoint = {} - for attr in attrs: - checkpoint[attr[0]] = attr[1] - - if not checkpoint.has_key('SendRecordCount'): - checkpoint['SendRecordCount'] = 0 - checkpoint['ShardId'] = primary_key[2][1].split('\t')[1] - checkpoints.append(checkpoint) - - return checkpoints - -def list_job(config, options): - ''' - Two options: - 1. list all jobs of stream - 2. list all jobs and all streams - ''' - consumed_counter = CapacityUnit(0, 0) - - if options.stream_id: - start_pk = [('StreamId', options.stream_id), ('StatusType', INF_MIN), ('StatusValue', INF_MIN)] - end_pk = [('StreamId', options.stream_id), ('StatusType', INF_MAX), ('StatusValue', INF_MAX)] - else: - start_pk = [('StreamId', INF_MIN), ('StatusType', INF_MIN), ('StatusValue', INF_MIN)] - end_pk = [('StreamId', INF_MAX), ('StatusType', INF_MAX), ('StatusValue', INF_MAX)] - - columns_to_get = [] - range_iter = config.ots.xget_range( - config.status_table, Direction.FORWARD, - start_pk, end_pk, - consumed_counter, columns_to_get, None, - column_filter=None, max_version=1 - ) - - rows = [] - for (primary_key, attrs) in range_iter: - if primary_key[1][1] == 'DataxJobDesc': - job_detail = parse_job_detail(attrs) - rows.append([job_detail['TableName'], job_detail['JobStreamId'], job_detail['EndTime'], job_detail['StartTime'], job_detail['EndTime'], job_detail['Version']]) - - headers = ['TableName', 'JobStreamId', 'Timestamp', 'StartTime', 'EndTime', 'Version'] - print tabulate.tabulate(rows, headers=headers) - -def parse_job_detail(attrs): - job_details = {} - shard_ids_content = '' - for attr in attrs: - if attr[0].startswith('ShardIds_'): - shard_ids_content += attr[1] - else: - job_details[attr[0]] = attr[1] - - shard_ids = json.loads(zlib.decompress(shard_ids_content)) - - if not job_details.has_key('Version'): - job_details['Version'] = '' - - if not job_details.has_key('SkipCount'): - job_details['SkipCount'] = 0 - job_details['ShardIds'] = shard_ids - - return job_details - -def parse_time(value): - try: - return int(value) - except Exception,e: - return int(time.mktime(time.strptime(value, '%Y-%m-%d %H:%M:%S'))) - -if __name__ == '__main__': - parser = OptionParser() - parser.add_option('-c', '--config', dest='config_file', help='path of config file', metavar='tablestore_streamreader_config.json') - parser.add_option('-a', '--action', dest='action', help='the action to do', choices = ['describe_job', 'list_job'], metavar='') - parser.add_option('-t', '--timestamp', dest='timestamp', help='the timestamp', metavar='') - parser.add_option('-s', '--streamid', dest='stream_id', help='the id of stream', metavar='') - parser.add_option('-d', '--shardid', dest='shard_id', help='the id of shard', metavar='') - - options, args = parser.parse_args() - - if not options.config_file: - print "Error: Should set the path of config file using '-c' or '--config'." - sys.exit(-1) - - if not options.action: - print "Error: Should set the action using '-a' or '--action'." - sys.exit(-1) - - console_config = ConsoleConfig(options.config_file) - if options.action == 'list_job': - list_job(console_config, options) - elif options.action == 'describe_job': - describe_job(console_config, options) - diff --git a/otsstreamreader/tools/tabulate.py b/otsstreamreader/tools/tabulate.py deleted file mode 100644 index 2444dcbfc7..0000000000 --- a/otsstreamreader/tools/tabulate.py +++ /dev/null @@ -1,1237 +0,0 @@ -# -*- coding: utf-8 -*- - -"""Pretty-print tabular data.""" - -from __future__ import print_function -from __future__ import unicode_literals -from collections import namedtuple, Iterable -from platform import python_version_tuple -import re - - -if python_version_tuple()[0] < "3": - from itertools import izip_longest - from functools import partial - _none_type = type(None) - _int_type = int - _long_type = long - _float_type = float - _text_type = unicode - _binary_type = str - - def _is_file(f): - return isinstance(f, file) - -else: - from itertools import zip_longest as izip_longest - from functools import reduce, partial - _none_type = type(None) - _int_type = int - _long_type = int - _float_type = float - _text_type = str - _binary_type = bytes - - import io - def _is_file(f): - return isinstance(f, io.IOBase) - -try: - import wcwidth # optional wide-character (CJK) support -except ImportError: - wcwidth = None - - -__all__ = ["tabulate", "tabulate_formats", "simple_separated_format"] -__version__ = "0.7.6-dev" - - -# minimum extra space in headers -MIN_PADDING = 2 - -# if True, enable wide-character (CJK) support -WIDE_CHARS_MODE = wcwidth is not None - - -Line = namedtuple("Line", ["begin", "hline", "sep", "end"]) - - -DataRow = namedtuple("DataRow", ["begin", "sep", "end"]) - - -# A table structure is suppposed to be: -# -# --- lineabove --------- -# headerrow -# --- linebelowheader --- -# datarow -# --- linebewteenrows --- -# ... (more datarows) ... -# --- linebewteenrows --- -# last datarow -# --- linebelow --------- -# -# TableFormat's line* elements can be -# -# - either None, if the element is not used, -# - or a Line tuple, -# - or a function: [col_widths], [col_alignments] -> string. -# -# TableFormat's *row elements can be -# -# - either None, if the element is not used, -# - or a DataRow tuple, -# - or a function: [cell_values], [col_widths], [col_alignments] -> string. -# -# padding (an integer) is the amount of white space around data values. -# -# with_header_hide: -# -# - either None, to display all table elements unconditionally, -# - or a list of elements not to be displayed if the table has column headers. -# -TableFormat = namedtuple("TableFormat", ["lineabove", "linebelowheader", - "linebetweenrows", "linebelow", - "headerrow", "datarow", - "padding", "with_header_hide"]) - - -def _pipe_segment_with_colons(align, colwidth): - """Return a segment of a horizontal line with optional colons which - indicate column's alignment (as in `pipe` output format).""" - w = colwidth - if align in ["right", "decimal"]: - return ('-' * (w - 1)) + ":" - elif align == "center": - return ":" + ('-' * (w - 2)) + ":" - elif align == "left": - return ":" + ('-' * (w - 1)) - else: - return '-' * w - - -def _pipe_line_with_colons(colwidths, colaligns): - """Return a horizontal line with optional colons to indicate column's - alignment (as in `pipe` output format).""" - segments = [_pipe_segment_with_colons(a, w) for a, w in zip(colaligns, colwidths)] - return "|" + "|".join(segments) + "|" - - -def _mediawiki_row_with_attrs(separator, cell_values, colwidths, colaligns): - alignment = { "left": '', - "right": 'align="right"| ', - "center": 'align="center"| ', - "decimal": 'align="right"| ' } - # hard-coded padding _around_ align attribute and value together - # rather than padding parameter which affects only the value - values_with_attrs = [' ' + alignment.get(a, '') + c + ' ' - for c, a in zip(cell_values, colaligns)] - colsep = separator*2 - return (separator + colsep.join(values_with_attrs)).rstrip() - - -def _textile_row_with_attrs(cell_values, colwidths, colaligns): - cell_values[0] += ' ' - alignment = { "left": "<.", "right": ">.", "center": "=.", "decimal": ">." } - values = (alignment.get(a, '') + v for a, v in zip(colaligns, cell_values)) - return '|' + '|'.join(values) + '|' - - -def _html_begin_table_without_header(colwidths_ignore, colaligns_ignore): - # this table header will be suppressed if there is a header row - return "\n".join(["", ""]) - - -def _html_row_with_attrs(celltag, cell_values, colwidths, colaligns): - alignment = { "left": '', - "right": ' style="text-align: right;"', - "center": ' style="text-align: center;"', - "decimal": ' style="text-align: right;"' } - values_with_attrs = ["<{0}{1}>{2}".format(celltag, alignment.get(a, ''), c) - for c, a in zip(cell_values, colaligns)] - rowhtml = "" + "".join(values_with_attrs).rstrip() + "" - if celltag == "th": # it's a header row, create a new table header - rowhtml = "\n".join(["
", - "", - rowhtml, - "", - ""]) - return rowhtml - -def _moin_row_with_attrs(celltag, cell_values, colwidths, colaligns, header=''): - alignment = { "left": '', - "right": '', - "center": '', - "decimal": '' } - values_with_attrs = ["{0}{1} {2} ".format(celltag, - alignment.get(a, ''), - header+c+header) - for c, a in zip(cell_values, colaligns)] - return "".join(values_with_attrs)+"||" - -def _latex_line_begin_tabular(colwidths, colaligns, booktabs=False): - alignment = { "left": "l", "right": "r", "center": "c", "decimal": "r" } - tabular_columns_fmt = "".join([alignment.get(a, "l") for a in colaligns]) - return "\n".join(["\\begin{tabular}{" + tabular_columns_fmt + "}", - "\\toprule" if booktabs else "\hline"]) - -LATEX_ESCAPE_RULES = {r"&": r"\&", r"%": r"\%", r"$": r"\$", r"#": r"\#", - r"_": r"\_", r"^": r"\^{}", r"{": r"\{", r"}": r"\}", - r"~": r"\textasciitilde{}", "\\": r"\textbackslash{}", - r"<": r"\ensuremath{<}", r">": r"\ensuremath{>}"} - - -def _latex_row(cell_values, colwidths, colaligns): - def escape_char(c): - return LATEX_ESCAPE_RULES.get(c, c) - escaped_values = ["".join(map(escape_char, cell)) for cell in cell_values] - rowfmt = DataRow("", "&", "\\\\") - return _build_simple_row(escaped_values, rowfmt) - - -_table_formats = {"simple": - TableFormat(lineabove=Line("", "-", " ", ""), - linebelowheader=Line("", "-", " ", ""), - linebetweenrows=None, - linebelow=Line("", "-", " ", ""), - headerrow=DataRow("", " ", ""), - datarow=DataRow("", " ", ""), - padding=0, - with_header_hide=["lineabove", "linebelow"]), - "plain": - TableFormat(lineabove=None, linebelowheader=None, - linebetweenrows=None, linebelow=None, - headerrow=DataRow("", " ", ""), - datarow=DataRow("", " ", ""), - padding=0, with_header_hide=None), - "grid": - TableFormat(lineabove=Line("+", "-", "+", "+"), - linebelowheader=Line("+", "=", "+", "+"), - linebetweenrows=Line("+", "-", "+", "+"), - linebelow=Line("+", "-", "+", "+"), - headerrow=DataRow("|", "|", "|"), - datarow=DataRow("|", "|", "|"), - padding=1, with_header_hide=None), - "fancy_grid": - TableFormat(lineabove=Line("╒", "═", "╤", "╕"), - linebelowheader=Line("╞", "═", "╪", "╡"), - linebetweenrows=Line("├", "─", "┼", "┤"), - linebelow=Line("╘", "═", "╧", "╛"), - headerrow=DataRow("│", "│", "│"), - datarow=DataRow("│", "│", "│"), - padding=1, with_header_hide=None), - "pipe": - TableFormat(lineabove=_pipe_line_with_colons, - linebelowheader=_pipe_line_with_colons, - linebetweenrows=None, - linebelow=None, - headerrow=DataRow("|", "|", "|"), - datarow=DataRow("|", "|", "|"), - padding=1, - with_header_hide=["lineabove"]), - "orgtbl": - TableFormat(lineabove=None, - linebelowheader=Line("|", "-", "+", "|"), - linebetweenrows=None, - linebelow=None, - headerrow=DataRow("|", "|", "|"), - datarow=DataRow("|", "|", "|"), - padding=1, with_header_hide=None), - "jira": - TableFormat(lineabove=None, - linebelowheader=None, - linebetweenrows=None, - linebelow=None, - headerrow=DataRow("||", "||", "||"), - datarow=DataRow("|", "|", "|"), - padding=1, with_header_hide=None), - "psql": - TableFormat(lineabove=Line("+", "-", "+", "+"), - linebelowheader=Line("|", "-", "+", "|"), - linebetweenrows=None, - linebelow=Line("+", "-", "+", "+"), - headerrow=DataRow("|", "|", "|"), - datarow=DataRow("|", "|", "|"), - padding=1, with_header_hide=None), - "rst": - TableFormat(lineabove=Line("", "=", " ", ""), - linebelowheader=Line("", "=", " ", ""), - linebetweenrows=None, - linebelow=Line("", "=", " ", ""), - headerrow=DataRow("", " ", ""), - datarow=DataRow("", " ", ""), - padding=0, with_header_hide=None), - "mediawiki": - TableFormat(lineabove=Line("{| class=\"wikitable\" style=\"text-align: left;\"", - "", "", "\n|+ \n|-"), - linebelowheader=Line("|-", "", "", ""), - linebetweenrows=Line("|-", "", "", ""), - linebelow=Line("|}", "", "", ""), - headerrow=partial(_mediawiki_row_with_attrs, "!"), - datarow=partial(_mediawiki_row_with_attrs, "|"), - padding=0, with_header_hide=None), - "moinmoin": - TableFormat(lineabove=None, - linebelowheader=None, - linebetweenrows=None, - linebelow=None, - headerrow=partial(_moin_row_with_attrs,"||",header="'''"), - datarow=partial(_moin_row_with_attrs,"||"), - padding=1, with_header_hide=None), - "html": - TableFormat(lineabove=_html_begin_table_without_header, - linebelowheader="", - linebetweenrows=None, - linebelow=Line("\n
", "", "", ""), - headerrow=partial(_html_row_with_attrs, "th"), - datarow=partial(_html_row_with_attrs, "td"), - padding=0, with_header_hide=["lineabove"]), - "latex": - TableFormat(lineabove=_latex_line_begin_tabular, - linebelowheader=Line("\\hline", "", "", ""), - linebetweenrows=None, - linebelow=Line("\\hline\n\\end{tabular}", "", "", ""), - headerrow=_latex_row, - datarow=_latex_row, - padding=1, with_header_hide=None), - "latex_booktabs": - TableFormat(lineabove=partial(_latex_line_begin_tabular, booktabs=True), - linebelowheader=Line("\\midrule", "", "", ""), - linebetweenrows=None, - linebelow=Line("\\bottomrule\n\\end{tabular}", "", "", ""), - headerrow=_latex_row, - datarow=_latex_row, - padding=1, with_header_hide=None), - "tsv": - TableFormat(lineabove=None, linebelowheader=None, - linebetweenrows=None, linebelow=None, - headerrow=DataRow("", "\t", ""), - datarow=DataRow("", "\t", ""), - padding=0, with_header_hide=None), - "textile": - TableFormat(lineabove=None, linebelowheader=None, - linebetweenrows=None, linebelow=None, - headerrow=DataRow("|_. ", "|_.", "|"), - datarow=_textile_row_with_attrs, - padding=1, with_header_hide=None)} - - -tabulate_formats = list(sorted(_table_formats.keys())) - - -_invisible_codes = re.compile(r"\x1b\[\d*m|\x1b\[\d*\;\d*\;\d*m") # ANSI color codes -_invisible_codes_bytes = re.compile(b"\x1b\[\d*m|\x1b\[\d*\;\d*\;\d*m") # ANSI color codes - - -def simple_separated_format(separator): - """Construct a simple TableFormat with columns separated by a separator. - - >>> tsv = simple_separated_format("\\t") ; \ - tabulate([["foo", 1], ["spam", 23]], tablefmt=tsv) == 'foo \\t 1\\nspam\\t23' - True - - """ - return TableFormat(None, None, None, None, - headerrow=DataRow('', separator, ''), - datarow=DataRow('', separator, ''), - padding=0, with_header_hide=None) - - -def _isconvertible(conv, string): - try: - n = conv(string) - return True - except (ValueError, TypeError): - return False - - -def _isnumber(string): - """ - >>> _isnumber("123.45") - True - >>> _isnumber("123") - True - >>> _isnumber("spam") - False - """ - return _isconvertible(float, string) - - -def _isint(string, inttype=int): - """ - >>> _isint("123") - True - >>> _isint("123.45") - False - """ - return type(string) is inttype or\ - (isinstance(string, _binary_type) or isinstance(string, _text_type))\ - and\ - _isconvertible(inttype, string) - - -def _type(string, has_invisible=True): - """The least generic type (type(None), int, float, str, unicode). - - >>> _type(None) is type(None) - True - >>> _type("foo") is type("") - True - >>> _type("1") is type(1) - True - >>> _type('\x1b[31m42\x1b[0m') is type(42) - True - >>> _type('\x1b[31m42\x1b[0m') is type(42) - True - - """ - - if has_invisible and \ - (isinstance(string, _text_type) or isinstance(string, _binary_type)): - string = _strip_invisible(string) - - if string is None: - return _none_type - elif hasattr(string, "isoformat"): # datetime.datetime, date, and time - return _text_type - elif _isint(string): - return int - elif _isint(string, _long_type): - return int - elif _isnumber(string): - return float - elif isinstance(string, _binary_type): - return _binary_type - else: - return _text_type - - -def _afterpoint(string): - """Symbols after a decimal point, -1 if the string lacks the decimal point. - - >>> _afterpoint("123.45") - 2 - >>> _afterpoint("1001") - -1 - >>> _afterpoint("eggs") - -1 - >>> _afterpoint("123e45") - 2 - - """ - if _isnumber(string): - if _isint(string): - return -1 - else: - pos = string.rfind(".") - pos = string.lower().rfind("e") if pos < 0 else pos - if pos >= 0: - return len(string) - pos - 1 - else: - return -1 # no point - else: - return -1 # not a number - - -def _padleft(width, s): - """Flush right. - - >>> _padleft(6, '\u044f\u0439\u0446\u0430') == ' \u044f\u0439\u0446\u0430' - True - - """ - fmt = "{0:>%ds}" % width - return fmt.format(s) - - -def _padright(width, s): - """Flush left. - - >>> _padright(6, '\u044f\u0439\u0446\u0430') == '\u044f\u0439\u0446\u0430 ' - True - - """ - fmt = "{0:<%ds}" % width - return fmt.format(s) - - -def _padboth(width, s): - """Center string. - - >>> _padboth(6, '\u044f\u0439\u0446\u0430') == ' \u044f\u0439\u0446\u0430 ' - True - - """ - fmt = "{0:^%ds}" % width - return fmt.format(s) - - -def _strip_invisible(s): - "Remove invisible ANSI color codes." - if isinstance(s, _text_type): - return re.sub(_invisible_codes, "", s) - else: # a bytestring - return re.sub(_invisible_codes_bytes, "", s) - - -def _visible_width(s): - """Visible width of a printed string. ANSI color codes are removed. - - >>> _visible_width('\x1b[31mhello\x1b[0m'), _visible_width("world") - (5, 5) - - """ - # optional wide-character support - if wcwidth is not None and WIDE_CHARS_MODE: - len_fn = wcwidth.wcswidth - else: - len_fn = len - if isinstance(s, _text_type) or isinstance(s, _binary_type): - return len_fn(_strip_invisible(s)) - else: - return len_fn(_text_type(s)) - - -def _align_column(strings, alignment, minwidth=0, has_invisible=True): - """[string] -> [padded_string] - - >>> list(map(str,_align_column(["12.345", "-1234.5", "1.23", "1234.5", "1e+234", "1.0e234"], "decimal"))) - [' 12.345 ', '-1234.5 ', ' 1.23 ', ' 1234.5 ', ' 1e+234 ', ' 1.0e234'] - - >>> list(map(str,_align_column(['123.4', '56.7890'], None))) - ['123.4', '56.7890'] - - """ - if alignment == "right": - strings = [s.strip() for s in strings] - padfn = _padleft - elif alignment == "center": - strings = [s.strip() for s in strings] - padfn = _padboth - elif alignment == "decimal": - if has_invisible: - decimals = [_afterpoint(_strip_invisible(s)) for s in strings] - else: - decimals = [_afterpoint(s) for s in strings] - maxdecimals = max(decimals) - strings = [s + (maxdecimals - decs) * " " - for s, decs in zip(strings, decimals)] - padfn = _padleft - elif not alignment: - return strings - else: - strings = [s.strip() for s in strings] - padfn = _padright - - enable_widechars = wcwidth is not None and WIDE_CHARS_MODE - if has_invisible: - width_fn = _visible_width - elif enable_widechars: # optional wide-character support if available - width_fn = wcwidth.wcswidth - else: - width_fn = len - - s_lens = list(map(len, strings)) - s_widths = list(map(width_fn, strings)) - maxwidth = max(max(s_widths), minwidth) - if not enable_widechars and not has_invisible: - padded_strings = [padfn(maxwidth, s) for s in strings] - else: - # enable wide-character width corrections - visible_widths = [maxwidth - (w - l) for w, l in zip(s_widths, s_lens)] - # wcswidth and _visible_width don't count invisible characters; - # padfn doesn't need to apply another correction - padded_strings = [padfn(w, s) for s, w in zip(strings, visible_widths)] - return padded_strings - - -def _more_generic(type1, type2): - types = { _none_type: 0, int: 1, float: 2, _binary_type: 3, _text_type: 4 } - invtypes = { 4: _text_type, 3: _binary_type, 2: float, 1: int, 0: _none_type } - moregeneric = max(types.get(type1, 4), types.get(type2, 4)) - return invtypes[moregeneric] - - -def _column_type(strings, has_invisible=True): - """The least generic type all column values are convertible to. - - >>> _column_type(["1", "2"]) is _int_type - True - >>> _column_type(["1", "2.3"]) is _float_type - True - >>> _column_type(["1", "2.3", "four"]) is _text_type - True - >>> _column_type(["four", '\u043f\u044f\u0442\u044c']) is _text_type - True - >>> _column_type([None, "brux"]) is _text_type - True - >>> _column_type([1, 2, None]) is _int_type - True - >>> import datetime as dt - >>> _column_type([dt.datetime(1991,2,19), dt.time(17,35)]) is _text_type - True - - """ - types = [_type(s, has_invisible) for s in strings ] - return reduce(_more_generic, types, int) - - -def _format(val, valtype, floatfmt, missingval="", has_invisible=True): - """Format a value accoding to its type. - - Unicode is supported: - - >>> hrow = ['\u0431\u0443\u043a\u0432\u0430', '\u0446\u0438\u0444\u0440\u0430'] ; \ - tbl = [['\u0430\u0437', 2], ['\u0431\u0443\u043a\u0438', 4]] ; \ - good_result = '\\u0431\\u0443\\u043a\\u0432\\u0430 \\u0446\\u0438\\u0444\\u0440\\u0430\\n------- -------\\n\\u0430\\u0437 2\\n\\u0431\\u0443\\u043a\\u0438 4' ; \ - tabulate(tbl, headers=hrow) == good_result - True - - """ - if val is None: - return missingval - - if valtype in [int, _text_type]: - return "{0}".format(val) - elif valtype is _binary_type: - try: - return _text_type(val, "ascii") - except TypeError: - return _text_type(val) - elif valtype is float: - is_a_colored_number = has_invisible and isinstance(val, (_text_type, _binary_type)) - if is_a_colored_number: - raw_val = _strip_invisible(val) - formatted_val = format(float(raw_val), floatfmt) - return val.replace(raw_val, formatted_val) - else: - return format(float(val), floatfmt) - else: - return "{0}".format(val) - - -def _align_header(header, alignment, width, visible_width): - "Pad string header to width chars given known visible_width of the header." - width += len(header) - visible_width - if alignment == "left": - return _padright(width, header) - elif alignment == "center": - return _padboth(width, header) - elif not alignment: - return "{0}".format(header) - else: - return _padleft(width, header) - - -def _prepend_row_index(rows, index): - """Add a left-most index column.""" - if index is None or index is False: - return rows - if len(index) != len(rows): - print('index=', index) - print('rows=', rows) - raise ValueError('index must be as long as the number of data rows') - rows = [[v]+list(row) for v,row in zip(index, rows)] - return rows - - -def _bool(val): - "A wrapper around standard bool() which doesn't throw on NumPy arrays" - try: - return bool(val) - except ValueError: # val is likely to be a numpy array with many elements - return False - - -def _normalize_tabular_data(tabular_data, headers, showindex="default"): - """Transform a supported data type to a list of lists, and a list of headers. - - Supported tabular data types: - - * list-of-lists or another iterable of iterables - - * list of named tuples (usually used with headers="keys") - - * list of dicts (usually used with headers="keys") - - * list of OrderedDicts (usually used with headers="keys") - - * 2D NumPy arrays - - * NumPy record arrays (usually used with headers="keys") - - * dict of iterables (usually used with headers="keys") - - * pandas.DataFrame (usually used with headers="keys") - - The first row can be used as headers if headers="firstrow", - column indices can be used as headers if headers="keys". - - If showindex="default", show row indices of the pandas.DataFrame. - If showindex="always", show row indices for all types of data. - If showindex="never", don't show row indices for all types of data. - If showindex is an iterable, show its values as row indices. - - """ - - try: - bool(headers) - is_headers2bool_broken = False - except ValueError: # numpy.ndarray, pandas.core.index.Index, ... - is_headers2bool_broken = True - headers = list(headers) - - index = None - if hasattr(tabular_data, "keys") and hasattr(tabular_data, "values"): - # dict-like and pandas.DataFrame? - if hasattr(tabular_data.values, "__call__"): - # likely a conventional dict - keys = tabular_data.keys() - rows = list(izip_longest(*tabular_data.values())) # columns have to be transposed - elif hasattr(tabular_data, "index"): - # values is a property, has .index => it's likely a pandas.DataFrame (pandas 0.11.0) - keys = tabular_data.keys() - vals = tabular_data.values # values matrix doesn't need to be transposed - # for DataFrames add an index per default - index = list(tabular_data.index) - rows = [list(row) for row in vals] - else: - raise ValueError("tabular data doesn't appear to be a dict or a DataFrame") - - if headers == "keys": - headers = list(map(_text_type,keys)) # headers should be strings - - else: # it's a usual an iterable of iterables, or a NumPy array - rows = list(tabular_data) - - if (headers == "keys" and - hasattr(tabular_data, "dtype") and - getattr(tabular_data.dtype, "names")): - # numpy record array - headers = tabular_data.dtype.names - elif (headers == "keys" - and len(rows) > 0 - and isinstance(rows[0], tuple) - and hasattr(rows[0], "_fields")): - # namedtuple - headers = list(map(_text_type, rows[0]._fields)) - elif (len(rows) > 0 - and isinstance(rows[0], dict)): - # dict or OrderedDict - uniq_keys = set() # implements hashed lookup - keys = [] # storage for set - if headers == "firstrow": - firstdict = rows[0] if len(rows) > 0 else {} - keys.extend(firstdict.keys()) - uniq_keys.update(keys) - rows = rows[1:] - for row in rows: - for k in row.keys(): - #Save unique items in input order - if k not in uniq_keys: - keys.append(k) - uniq_keys.add(k) - if headers == 'keys': - headers = keys - elif isinstance(headers, dict): - # a dict of headers for a list of dicts - headers = [headers.get(k, k) for k in keys] - headers = list(map(_text_type, headers)) - elif headers == "firstrow": - if len(rows) > 0: - headers = [firstdict.get(k, k) for k in keys] - headers = list(map(_text_type, headers)) - else: - headers = [] - elif headers: - raise ValueError('headers for a list of dicts is not a dict or a keyword') - rows = [[row.get(k) for k in keys] for row in rows] - - elif headers == "keys" and len(rows) > 0: - # keys are column indices - headers = list(map(_text_type, range(len(rows[0])))) - - # take headers from the first row if necessary - if headers == "firstrow" and len(rows) > 0: - if index is not None: - headers = [index[0]] + list(rows[0]) - index = index[1:] - else: - headers = rows[0] - headers = list(map(_text_type, headers)) # headers should be strings - rows = rows[1:] - - headers = list(map(_text_type,headers)) - rows = list(map(list,rows)) - - # add or remove an index column - showindex_is_a_str = type(showindex) in [_text_type, _binary_type] - if showindex == "default" and index is not None: - rows = _prepend_row_index(rows, index) - elif isinstance(showindex, Iterable) and not showindex_is_a_str: - rows = _prepend_row_index(rows, list(showindex)) - elif showindex == "always" or (_bool(showindex) and not showindex_is_a_str): - if index is None: - index = list(range(len(rows))) - rows = _prepend_row_index(rows, index) - elif showindex == "never" or (not _bool(showindex) and not showindex_is_a_str): - pass - - # pad with empty headers for initial columns if necessary - if headers and len(rows) > 0: - nhs = len(headers) - ncols = len(rows[0]) - if nhs < ncols: - headers = [""]*(ncols - nhs) + headers - - return rows, headers - - -def tabulate(tabular_data, headers=(), tablefmt="simple", - floatfmt="g", numalign="decimal", stralign="left", - missingval="", showindex="default"): - """Format a fixed width table for pretty printing. - - >>> print(tabulate([[1, 2.34], [-56, "8.999"], ["2", "10001"]])) - --- --------- - 1 2.34 - -56 8.999 - 2 10001 - --- --------- - - The first required argument (`tabular_data`) can be a - list-of-lists (or another iterable of iterables), a list of named - tuples, a dictionary of iterables, an iterable of dictionaries, - a two-dimensional NumPy array, NumPy record array, or a Pandas' - dataframe. - - - Table headers - ------------- - - To print nice column headers, supply the second argument (`headers`): - - - `headers` can be an explicit list of column headers - - if `headers="firstrow"`, then the first row of data is used - - if `headers="keys"`, then dictionary keys or column indices are used - - Otherwise a headerless table is produced. - - If the number of headers is less than the number of columns, they - are supposed to be names of the last columns. This is consistent - with the plain-text format of R and Pandas' dataframes. - - >>> print(tabulate([["sex","age"],["Alice","F",24],["Bob","M",19]], - ... headers="firstrow")) - sex age - ----- ----- ----- - Alice F 24 - Bob M 19 - - By default, pandas.DataFrame data have an additional column called - row index. To add a similar column to all other types of data, - use `showindex="always"` or `showindex=True`. To suppress row indices - for all types of data, pass `showindex="never" or `showindex=False`. - To add a custom row index column, pass `showindex=some_iterable`. - - >>> print(tabulate([["F",24],["M",19]], showindex="always")) - - - -- - 0 F 24 - 1 M 19 - - - -- - - - Column alignment - ---------------- - - `tabulate` tries to detect column types automatically, and aligns - the values properly. By default it aligns decimal points of the - numbers (or flushes integer numbers to the right), and flushes - everything else to the left. Possible column alignments - (`numalign`, `stralign`) are: "right", "center", "left", "decimal" - (only for `numalign`), and None (to disable alignment). - - - Table formats - ------------- - - `floatfmt` is a format specification used for columns which - contain numeric data with a decimal point. - - `None` values are replaced with a `missingval` string: - - >>> print(tabulate([["spam", 1, None], - ... ["eggs", 42, 3.14], - ... ["other", None, 2.7]], missingval="?")) - ----- -- ---- - spam 1 ? - eggs 42 3.14 - other ? 2.7 - ----- -- ---- - - Various plain-text table formats (`tablefmt`) are supported: - 'plain', 'simple', 'grid', 'pipe', 'orgtbl', 'rst', 'mediawiki', - 'latex', and 'latex_booktabs'. Variable `tabulate_formats` contains the list of - currently supported formats. - - "plain" format doesn't use any pseudographics to draw tables, - it separates columns with a double space: - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], - ... ["strings", "numbers"], "plain")) - strings numbers - spam 41.9999 - eggs 451 - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], tablefmt="plain")) - spam 41.9999 - eggs 451 - - "simple" format is like Pandoc simple_tables: - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], - ... ["strings", "numbers"], "simple")) - strings numbers - --------- --------- - spam 41.9999 - eggs 451 - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], tablefmt="simple")) - ---- -------- - spam 41.9999 - eggs 451 - ---- -------- - - "grid" is similar to tables produced by Emacs table.el package or - Pandoc grid_tables: - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], - ... ["strings", "numbers"], "grid")) - +-----------+-----------+ - | strings | numbers | - +===========+===========+ - | spam | 41.9999 | - +-----------+-----------+ - | eggs | 451 | - +-----------+-----------+ - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], tablefmt="grid")) - +------+----------+ - | spam | 41.9999 | - +------+----------+ - | eggs | 451 | - +------+----------+ - - "fancy_grid" draws a grid using box-drawing characters: - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], - ... ["strings", "numbers"], "fancy_grid")) - ╒═══════════╤═══════════╕ - │ strings │ numbers │ - ╞═══════════╪═══════════╡ - │ spam │ 41.9999 │ - ├───────────┼───────────┤ - │ eggs │ 451 │ - ╘═══════════╧═══════════╛ - - "pipe" is like tables in PHP Markdown Extra extension or Pandoc - pipe_tables: - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], - ... ["strings", "numbers"], "pipe")) - | strings | numbers | - |:----------|----------:| - | spam | 41.9999 | - | eggs | 451 | - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], tablefmt="pipe")) - |:-----|---------:| - | spam | 41.9999 | - | eggs | 451 | - - "orgtbl" is like tables in Emacs org-mode and orgtbl-mode. They - are slightly different from "pipe" format by not using colons to - define column alignment, and using a "+" sign to indicate line - intersections: - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], - ... ["strings", "numbers"], "orgtbl")) - | strings | numbers | - |-----------+-----------| - | spam | 41.9999 | - | eggs | 451 | - - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], tablefmt="orgtbl")) - | spam | 41.9999 | - | eggs | 451 | - - "rst" is like a simple table format from reStructuredText; please - note that reStructuredText accepts also "grid" tables: - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], - ... ["strings", "numbers"], "rst")) - ========= ========= - strings numbers - ========= ========= - spam 41.9999 - eggs 451 - ========= ========= - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], tablefmt="rst")) - ==== ======== - spam 41.9999 - eggs 451 - ==== ======== - - "mediawiki" produces a table markup used in Wikipedia and on other - MediaWiki-based sites: - - >>> print(tabulate([["strings", "numbers"], ["spam", 41.9999], ["eggs", "451.0"]], - ... headers="firstrow", tablefmt="mediawiki")) - {| class="wikitable" style="text-align: left;" - |+ - |- - ! strings !! align="right"| numbers - |- - | spam || align="right"| 41.9999 - |- - | eggs || align="right"| 451 - |} - - "html" produces HTML markup: - - >>> print(tabulate([["strings", "numbers"], ["spam", 41.9999], ["eggs", "451.0"]], - ... headers="firstrow", tablefmt="html")) - - - - - - - - -
strings numbers
spam 41.9999
eggs 451
- - "latex" produces a tabular environment of LaTeX document markup: - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], tablefmt="latex")) - \\begin{tabular}{lr} - \\hline - spam & 41.9999 \\\\ - eggs & 451 \\\\ - \\hline - \\end{tabular} - - "latex_booktabs" produces a tabular environment of LaTeX document markup - using the booktabs.sty package: - - >>> print(tabulate([["spam", 41.9999], ["eggs", "451.0"]], tablefmt="latex_booktabs")) - \\begin{tabular}{lr} - \\toprule - spam & 41.9999 \\\\ - eggs & 451 \\\\ - \\bottomrule - \end{tabular} - """ - if tabular_data is None: - tabular_data = [] - list_of_lists, headers = _normalize_tabular_data( - tabular_data, headers, showindex=showindex) - - # optimization: look for ANSI control codes once, - # enable smart width functions only if a control code is found - plain_text = '\n'.join(['\t'.join(map(_text_type, headers))] + \ - ['\t'.join(map(_text_type, row)) for row in list_of_lists]) - - has_invisible = re.search(_invisible_codes, plain_text) - enable_widechars = wcwidth is not None and WIDE_CHARS_MODE - if has_invisible: - width_fn = _visible_width - elif enable_widechars: # optional wide-character support if available - width_fn = wcwidth.wcswidth - else: - width_fn = len - - # format rows and columns, convert numeric values to strings - cols = list(zip(*list_of_lists)) - coltypes = list(map(_column_type, cols)) - cols = [[_format(v, ct, floatfmt, missingval, has_invisible) for v in c] - for c,ct in zip(cols, coltypes)] - - # align columns - aligns = [numalign if ct in [int,float] else stralign for ct in coltypes] - minwidths = [width_fn(h) + MIN_PADDING for h in headers] if headers else [0]*len(cols) - cols = [_align_column(c, a, minw, has_invisible) - for c, a, minw in zip(cols, aligns, minwidths)] - - if headers: - # align headers and add headers - t_cols = cols or [['']] * len(headers) - t_aligns = aligns or [stralign] * len(headers) - minwidths = [max(minw, width_fn(c[0])) for minw, c in zip(minwidths, t_cols)] - headers = [_align_header(h, a, minw, width_fn(h)) - for h, a, minw in zip(headers, t_aligns, minwidths)] - rows = list(zip(*cols)) - else: - minwidths = [width_fn(c[0]) for c in cols] - rows = list(zip(*cols)) - - if not isinstance(tablefmt, TableFormat): - tablefmt = _table_formats.get(tablefmt, _table_formats["simple"]) - - return _format_table(tablefmt, headers, rows, minwidths, aligns) - - -def _build_simple_row(padded_cells, rowfmt): - "Format row according to DataRow format without padding." - begin, sep, end = rowfmt - return (begin + sep.join(padded_cells) + end).rstrip() - - -def _build_row(padded_cells, colwidths, colaligns, rowfmt): - "Return a string which represents a row of data cells." - if not rowfmt: - return None - if hasattr(rowfmt, "__call__"): - return rowfmt(padded_cells, colwidths, colaligns) - else: - return _build_simple_row(padded_cells, rowfmt) - - -def _build_line(colwidths, colaligns, linefmt): - "Return a string which represents a horizontal line." - if not linefmt: - return None - if hasattr(linefmt, "__call__"): - return linefmt(colwidths, colaligns) - else: - begin, fill, sep, end = linefmt - cells = [fill*w for w in colwidths] - return _build_simple_row(cells, (begin, sep, end)) - - -def _pad_row(cells, padding): - if cells: - pad = " "*padding - padded_cells = [pad + cell + pad for cell in cells] - return padded_cells - else: - return cells - - -def _format_table(fmt, headers, rows, colwidths, colaligns): - """Produce a plain-text representation of the table.""" - lines = [] - hidden = fmt.with_header_hide if (headers and fmt.with_header_hide) else [] - pad = fmt.padding - headerrow = fmt.headerrow - - padded_widths = [(w + 2*pad) for w in colwidths] - padded_headers = _pad_row(headers, pad) - padded_rows = [_pad_row(row, pad) for row in rows] - - if fmt.lineabove and "lineabove" not in hidden: - lines.append(_build_line(padded_widths, colaligns, fmt.lineabove)) - - if padded_headers: - lines.append(_build_row(padded_headers, padded_widths, colaligns, headerrow)) - if fmt.linebelowheader and "linebelowheader" not in hidden: - lines.append(_build_line(padded_widths, colaligns, fmt.linebelowheader)) - - if padded_rows and fmt.linebetweenrows and "linebetweenrows" not in hidden: - # initial rows with a line below - for row in padded_rows[:-1]: - lines.append(_build_row(row, padded_widths, colaligns, fmt.datarow)) - lines.append(_build_line(padded_widths, colaligns, fmt.linebetweenrows)) - # the last row without a line below - lines.append(_build_row(padded_rows[-1], padded_widths, colaligns, fmt.datarow)) - else: - for row in padded_rows: - lines.append(_build_row(row, padded_widths, colaligns, fmt.datarow)) - - if fmt.linebelow and "linebelow" not in hidden: - lines.append(_build_line(padded_widths, colaligns, fmt.linebelow)) - - return "\n".join(lines) - - -def _main(): - """\ - Usage: tabulate [options] [FILE ...] - - Pretty-print tabular data. - See also https://bitbucket.org/astanin/python-tabulate - - FILE a filename of the file with tabular data; - if "-" or missing, read data from stdin. - - Options: - - -h, --help show this message - -1, --header use the first row of data as a table header - -o FILE, --output FILE print table to FILE (default: stdout) - -s REGEXP, --sep REGEXP use a custom column separator (default: whitespace) - -F FPFMT, --float FPFMT floating point number format (default: g) - -f FMT, --format FMT set output table format; supported formats: - plain, simple, grid, fancy_grid, pipe, orgtbl, - rst, mediawiki, html, latex, latex_booktabs, tsv - (default: simple) - """ - import getopt - import sys - import textwrap - usage = textwrap.dedent(_main.__doc__) - try: - opts, args = getopt.getopt(sys.argv[1:], - "h1o:s:F:f:", - ["help", "header", "output", "sep=", "float=", "format="]) - except getopt.GetoptError as e: - print(e) - print(usage) - sys.exit(2) - headers = [] - floatfmt = "g" - tablefmt = "simple" - sep = r"\s+" - outfile = "-" - for opt, value in opts: - if opt in ["-1", "--header"]: - headers = "firstrow" - elif opt in ["-o", "--output"]: - outfile = value - elif opt in ["-F", "--float"]: - floatfmt = value - elif opt in ["-f", "--format"]: - if value not in tabulate_formats: - print("%s is not a supported table format" % value) - print(usage) - sys.exit(3) - tablefmt = value - elif opt in ["-s", "--sep"]: - sep = value - elif opt in ["-h", "--help"]: - print(usage) - sys.exit(0) - files = [sys.stdin] if not args else args - with (sys.stdout if outfile == "-" else open(outfile, "w")) as out: - for f in files: - if f == "-": - f = sys.stdin - if _is_file(f): - _pprint_file(f, headers=headers, tablefmt=tablefmt, - sep=sep, floatfmt=floatfmt, file=out) - else: - with open(f) as fobj: - _pprint_file(fobj, headers=headers, tablefmt=tablefmt, - sep=sep, floatfmt=floatfmt, file=out) - - -def _pprint_file(fobject, headers, tablefmt, sep, floatfmt, file): - rows = fobject.readlines() - table = [re.split(sep, r.rstrip()) for r in rows if r.strip()] - print(tabulate(table, headers, tablefmt, floatfmt=floatfmt), file=file) - - -if __name__ == "__main__": - _main() \ No newline at end of file diff --git a/package.xml b/package.xml index 456f780c86..96b52c3002 100755 --- a/package.xml +++ b/package.xml @@ -60,13 +60,6 @@
datax
- - db2reader/target/datax/ - - **/*.* - - datax - postgresqlreader/target/datax/ @@ -103,13 +96,13 @@ datax - - otsstreamreader/target/datax/ - - **/*.* - - datax - + + otsstreamreader/target/datax/ + + **/*.* + + datax + txtfilereader/target/datax/ @@ -215,6 +208,27 @@ datax + + datahubreader/target/datax/ + + **/*.* + + datax + + + loghubreader/target/datax/ + + **/*.* + + datax + + + starrocksreader/target/datax/ + + **/*.* + + datax + @@ -231,6 +245,13 @@ datax + + starrockswriter/target/datax/ + + **/*.* + + datax + drdswriter/target/datax/ @@ -245,6 +266,13 @@ datax + + doriswriter/target/datax/ + + **/*.* + + datax + txtfilewriter/target/datax/ @@ -441,5 +469,19 @@ datax + + datahubwriter/target/datax/ + + **/*.* + + datax + + + loghubwriter/target/datax/ + + **/*.* + + datax + diff --git a/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/util/DataBaseType.java b/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/util/DataBaseType.java index 205919fef1..6a35b5fd7f 100755 --- a/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/util/DataBaseType.java +++ b/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/util/DataBaseType.java @@ -22,7 +22,8 @@ public enum DataBaseType { ClickHouse("clickhouse", "ru.yandex.clickhouse.ClickHouseDriver"), KingbaseES("kingbasees", "com.kingbase8.Driver"), Oscar("oscar", "com.oscar.Driver"), - OceanBase("oceanbase", "com.alipay.oceanbase.jdbc.Driver"); + OceanBase("oceanbase", "com.alipay.oceanbase.jdbc.Driver"), + StarRocks("starrocks", "com.mysql.jdbc.Driver"); private String typeName; diff --git a/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/writer/CommonRdbmsWriter.java b/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/writer/CommonRdbmsWriter.java index 27b88f4443..bec3c683f6 100755 --- a/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/writer/CommonRdbmsWriter.java +++ b/plugin-rdbms-util/src/main/java/com/alibaba/datax/plugin/rdbms/writer/CommonRdbmsWriter.java @@ -409,11 +409,6 @@ protected PreparedStatement fillPreparedStatement(PreparedStatement preparedStat return preparedStatement; } - protected PreparedStatement fillPreparedStatementColumnType(PreparedStatement preparedStatement, int columnIndex, - int columnSqltype, Column column) throws SQLException { - return fillPreparedStatementColumnType(preparedStatement, columnIndex, columnSqltype, null, column); - } - protected PreparedStatement fillPreparedStatementColumnType(PreparedStatement preparedStatement, int columnIndex, int columnSqltype, String typeName, Column column) throws SQLException { java.util.Date utilDate; @@ -524,7 +519,7 @@ protected PreparedStatement fillPreparedStatementColumnType(PreparedStatement pr break; case Types.BOOLEAN: - preparedStatement.setString(columnIndex + 1, column.asString()); + preparedStatement.setBoolean(columnIndex + 1, column.asBoolean()); break; // warn: bit(1) -> Types.BIT 可使用setBoolean diff --git a/pom.xml b/pom.xml index 1d298eb634..6f9383d116 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ 3.3.2 1.10 1.2 - 1.2.49 + 1.1.46.sec10 16.0.1 3.7.2.1-SNAPSHOT @@ -76,10 +76,13 @@ gdbreader tsdbreader opentsdbreader - + loghubreader + datahubreader + starrocksreader mysqlwriter + starrockswriter drdswriter oraclewriter sqlserverwriter @@ -87,8 +90,6 @@ kingbaseeswriter adswriter oceanbasev10writer - cassandrawriter - clickhousewriter adbpgwriter hologresjdbcwriter rdbmswriter @@ -114,10 +115,16 @@ tsdbwriter gdbwriter oscarwriter + loghubwriter + datahubwriter + cassandrawriter + clickhousewriter + doriswriter plugin-rdbms-util plugin-unstructured-storage-util + @@ -247,6 +254,14 @@ + + + src/main/java + + **/*.properties + + + maven-assembly-plugin diff --git a/rdbmsreader/pom.xml b/rdbmsreader/pom.xml index e79d86f7fb..d75c38402a 100755 --- a/rdbmsreader/pom.xml +++ b/rdbmsreader/pom.xml @@ -25,12 +25,13 @@ + - com.dm - dm - system - ${basedir}/src/main/libs/Dm7JdbcDriver16.jar + com.dameng + Dm7JdbcDriver17 + 7.6.0.142 + com.sybase jconn3 @@ -38,13 +39,20 @@ system ${basedir}/src/main/libs/jconn3-1.0.0-SNAPSHOT.jar - + + ppas ppas 16 system ${basedir}/src/main/libs/edb-jdbc16.jar + + + com.ibm.db2.jcc + db2jcc + db2jcc4 + org.slf4j @@ -97,13 +105,4 @@ - - - - com.dm - dm - 16 - - - diff --git a/rdbmsreader/src/main/libs/Dm7JdbcDriver16.jar b/rdbmsreader/src/main/libs/Dm7JdbcDriver16.jar deleted file mode 100755 index 30740dcd2c..0000000000 Binary files a/rdbmsreader/src/main/libs/Dm7JdbcDriver16.jar and /dev/null differ diff --git a/rdbmsreader/src/main/libs/db2jcc4.jar b/rdbmsreader/src/main/libs/db2jcc4.jar deleted file mode 100755 index fc53cfd94b..0000000000 Binary files a/rdbmsreader/src/main/libs/db2jcc4.jar and /dev/null differ diff --git a/rdbmsreader/src/main/resources/plugin.json b/rdbmsreader/src/main/resources/plugin.json index d344dd8602..f79a6ace8f 100755 --- a/rdbmsreader/src/main/resources/plugin.json +++ b/rdbmsreader/src/main/resources/plugin.json @@ -3,5 +3,5 @@ "class": "com.alibaba.datax.plugin.reader.rdbmsreader.RdbmsReader", "description": "useScene: prod. mechanism: Jdbc connection using the database, execute select sql, retrieve data from the ResultSet. warn: The more you know about the database, the less problems you encounter.", "developer": "alibaba", - "drivers":["dm.jdbc.driver.DmDriver", "com.sybase.jdbc3.jdbc.SybDriver", "com.edb.Driver"] + "drivers":["dm.jdbc.driver.DmDriver", "com.sybase.jdbc3.jdbc.SybDriver", "com.edb.Driver", "com.ibm.db2.jcc.DB2Driver"] } diff --git a/rdbmswriter/pom.xml b/rdbmswriter/pom.xml index 194619609e..a74838b750 100755 --- a/rdbmswriter/pom.xml +++ b/rdbmswriter/pom.xml @@ -25,27 +25,34 @@ + - com.dm - dm + com.dameng + Dm7JdbcDriver17 + 7.6.0.142 + + + + com.sybase + jconn3 + 1.0.0-SNAPSHOT + system + ${basedir}/src/main/libs/jconn3-1.0.0-SNAPSHOT.jar + + + + ppas + ppas 16 system - ${basedir}/src/main/libs/Dm7JdbcDriver16.jar + ${basedir}/src/main/libs/edb-jdbc16.jar + - com.sybase - jconn3 - 1.0.0-SNAPSHOT - system - ${basedir}/src/main/libs/jconn3-1.0.0-SNAPSHOT.jar - - - ppas - ppas - 16 - system - ${basedir}/src/main/libs/edb-jdbc16.jar - + com.ibm.db2.jcc + db2jcc + db2jcc4 + org.slf4j diff --git a/rdbmswriter/src/main/java/com/alibaba/datax/plugin/reader/rdbmswriter/SubCommonRdbmsWriter.java b/rdbmswriter/src/main/java/com/alibaba/datax/plugin/reader/rdbmswriter/SubCommonRdbmsWriter.java index f1fbc552ef..88e50f1124 100755 --- a/rdbmswriter/src/main/java/com/alibaba/datax/plugin/reader/rdbmswriter/SubCommonRdbmsWriter.java +++ b/rdbmswriter/src/main/java/com/alibaba/datax/plugin/reader/rdbmswriter/SubCommonRdbmsWriter.java @@ -29,7 +29,7 @@ public Task(DataBaseType dataBaseType) { @Override protected PreparedStatement fillPreparedStatementColumnType( PreparedStatement preparedStatement, int columnIndex, - int columnSqltype, Column column) throws SQLException { + int columnSqltype, String typeName, Column column) throws SQLException { java.util.Date utilDate; try { switch (columnSqltype) { diff --git a/rdbmswriter/src/main/libs/Dm7JdbcDriver16.jar b/rdbmswriter/src/main/libs/Dm7JdbcDriver16.jar deleted file mode 100755 index 30740dcd2c..0000000000 Binary files a/rdbmswriter/src/main/libs/Dm7JdbcDriver16.jar and /dev/null differ diff --git a/rdbmswriter/src/main/libs/db2jcc4.jar b/rdbmswriter/src/main/libs/db2jcc4.jar deleted file mode 100755 index fc53cfd94b..0000000000 Binary files a/rdbmswriter/src/main/libs/db2jcc4.jar and /dev/null differ diff --git a/rdbmswriter/src/main/resources/plugin.json b/rdbmswriter/src/main/resources/plugin.json index fa771af294..bf32140a9c 100755 --- a/rdbmswriter/src/main/resources/plugin.json +++ b/rdbmswriter/src/main/resources/plugin.json @@ -3,5 +3,5 @@ "class": "com.alibaba.datax.plugin.reader.rdbmswriter.RdbmsWriter", "description": "useScene: prod. mechanism: Jdbc connection using the database, execute select sql, retrieve data from the ResultSet. warn: The more you know about the database, the less problems you encounter.", "developer": "alibaba", - "drivers":["dm.jdbc.driver.DmDriver", "com.sybase.jdbc3.jdbc.SybDriver", "com.edb.Driver"] + "drivers":["dm.jdbc.driver.DmDriver", "com.sybase.jdbc3.jdbc.SybDriver", "com.edb.Driver", "com.ibm.db2.jcc.DB2Driver"] } diff --git a/sqlserverreader/pom.xml b/sqlserverreader/pom.xml index 5372a0577b..326f1ce51d 100755 --- a/sqlserverreader/pom.xml +++ b/sqlserverreader/pom.xml @@ -31,10 +31,7 @@ com.microsoft.sqlserver sqljdbc4 4.0 - system - ${basedir}/src/main/lib/sqljdbc4-4.0.jar - com.alibaba.datax plugin-rdbms-util diff --git a/sqlserverreader/src/main/assembly/package.xml b/sqlserverreader/src/main/assembly/package.xml index 55fbdc0b9c..6180fbc0a9 100755 --- a/sqlserverreader/src/main/assembly/package.xml +++ b/sqlserverreader/src/main/assembly/package.xml @@ -16,13 +16,6 @@ plugin/reader/sqlserverreader - - src/main/lib - - sqljdbc4-4.0.jar - - plugin/reader/sqlserverreader/libs - target/ diff --git a/sqlserverreader/src/main/lib/sqljdbc4-4.0.jar b/sqlserverreader/src/main/lib/sqljdbc4-4.0.jar deleted file mode 100644 index d6b7f6daf4..0000000000 Binary files a/sqlserverreader/src/main/lib/sqljdbc4-4.0.jar and /dev/null differ diff --git a/sqlserverwriter/pom.xml b/sqlserverwriter/pom.xml index d2b1eea1f8..6f52c14c96 100644 --- a/sqlserverwriter/pom.xml +++ b/sqlserverwriter/pom.xml @@ -35,8 +35,6 @@ com.microsoft.sqlserver sqljdbc4 4.0 - system - ${basedir}/src/main/lib/sqljdbc4-4.0.jar com.alibaba.datax diff --git a/sqlserverwriter/src/main/assembly/package.xml b/sqlserverwriter/src/main/assembly/package.xml index 761dffcd12..f8f262987a 100755 --- a/sqlserverwriter/src/main/assembly/package.xml +++ b/sqlserverwriter/src/main/assembly/package.xml @@ -16,13 +16,6 @@ plugin/writer/sqlserverwriter - - src/main/lib - - sqljdbc4-4.0.jar - - plugin/writer/sqlserverwriter/libs - target/ diff --git a/sqlserverwriter/src/main/lib/sqljdbc4-4.0.jar b/sqlserverwriter/src/main/lib/sqljdbc4-4.0.jar deleted file mode 100644 index d6b7f6daf4..0000000000 Binary files a/sqlserverwriter/src/main/lib/sqljdbc4-4.0.jar and /dev/null differ diff --git a/starrocksreader/pom.xml b/starrocksreader/pom.xml new file mode 100644 index 0000000000..a8b049ea9c --- /dev/null +++ b/starrocksreader/pom.xml @@ -0,0 +1,95 @@ + + + + datax-all + com.alibaba.datax + 0.0.1-SNAPSHOT + + 4.0.0 + starrocksreader + starrocksreader + jar + + + 8 + 8 + + + + + 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 + 5.1.46 + + + + + + + + src/main/java + + **/*.properties + + + + + + + maven-compiler-plugin + + ${jdk-version} + ${jdk-version} + ${project-sourceEncoding} + + + + + maven-assembly-plugin + + + src/main/assembly/package.xml + + datax + + + + dwzip + package + + single + + + + + + + \ No newline at end of file diff --git a/starrocksreader/src/main/assembly/package.xml b/starrocksreader/src/main/assembly/package.xml new file mode 100644 index 0000000000..c126c107a8 --- /dev/null +++ b/starrocksreader/src/main/assembly/package.xml @@ -0,0 +1,35 @@ + + + + dir + + false + + + src/main/resources + + plugin.json + plugin_job_template.json + + plugin/reader/starrocksreader + + + target/ + + starrocksreader-0.0.1-SNAPSHOT.jar + + plugin/reader/starrocksreader + + + + + + false + plugin/reader/starrocksreader/libs + runtime + + + diff --git a/starrocksreader/src/main/java/com/alibaba/datax/plugin/reader/starrocksreader/StarRocksReader.java b/starrocksreader/src/main/java/com/alibaba/datax/plugin/reader/starrocksreader/StarRocksReader.java new file mode 100644 index 0000000000..d4bf343714 --- /dev/null +++ b/starrocksreader/src/main/java/com/alibaba/datax/plugin/reader/starrocksreader/StarRocksReader.java @@ -0,0 +1,116 @@ +package com.alibaba.datax.plugin.reader.starrocksreader; + +import java.util.List; + +import com.alibaba.datax.common.plugin.RecordSender; +import com.alibaba.datax.common.spi.Reader; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.rdbms.reader.CommonRdbmsReader; +import com.alibaba.datax.plugin.rdbms.reader.Constant; +import com.alibaba.datax.plugin.rdbms.reader.Key; +import com.alibaba.datax.plugin.rdbms.util.DataBaseType; + +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class StarRocksReader extends Reader { + + private static final DataBaseType DATABASE_TYPE = DataBaseType.StarRocks; + + public static class Job extends Reader.Job { + private static final Logger LOG = LoggerFactory + .getLogger(Job.class); + + private Configuration originalConfig = null; + private CommonRdbmsReader.Job commonRdbmsReaderJob; + + @Override + public void init() { + this.originalConfig = super.getPluginJobConf(); + int fetchSize = this.originalConfig.getInt(Constant.FETCH_SIZE, + Integer.MIN_VALUE); + this.originalConfig.set(Constant.FETCH_SIZE, fetchSize); + + this.commonRdbmsReaderJob = new CommonRdbmsReader.Job(DATABASE_TYPE); + this.commonRdbmsReaderJob.init(this.originalConfig); + } + + @Override + public void preCheck(){ + init(); + this.commonRdbmsReaderJob.preCheck(this.originalConfig,DATABASE_TYPE); + + } + + @Override + public void prepare() { + } + + @Override + public List split(int adviceNumber) { + LOG.info("split() begin..."); + List splitResult = this.commonRdbmsReaderJob.split(this.originalConfig, adviceNumber); + /** + * 在日志中告知用户,为什么实际datax切分跑的channel数会小于用户配置的channel数 + */ + if(splitResult.size() < adviceNumber){ + // 如果用户没有配置切分主键splitPk + if(StringUtils.isBlank(this.originalConfig.getString(Key.SPLIT_PK, null))){ + LOG.info("User has not configured splitPk."); + }else{ + // 用户配置了切分主键,但是切分主键可能重复太多,或者要同步的表的记录太少,无法切分成adviceNumber个task + LOG.info("User has configured splitPk. But the number of task finally split is smaller than that user has configured. " + + "The possible reasons are: 1) too many repeated splitPk values, 2) too few records."); + } + } + LOG.info("split() ok and end..."); + return splitResult; + } + + @Override + public void post() { + this.commonRdbmsReaderJob.post(this.originalConfig); + } + + @Override + public void destroy() { + this.commonRdbmsReaderJob.destroy(this.originalConfig); + } + + } + + public static class Task extends Reader.Task { + + private Configuration readerSliceConfig; + private CommonRdbmsReader.Task commonRdbmsReaderTask; + + @Override + public void init() { + this.readerSliceConfig = super.getPluginJobConf(); + this.commonRdbmsReaderTask = new CommonRdbmsReader.Task(DATABASE_TYPE, super.getTaskGroupId(), super.getTaskId()); + this.commonRdbmsReaderTask.init(this.readerSliceConfig); + + } + + @Override + public void startRead(RecordSender recordSender) { + int fetchSize = this.readerSliceConfig.getInt(Constant.FETCH_SIZE); + + this.commonRdbmsReaderTask.startRead(this.readerSliceConfig, recordSender, + super.getTaskPluginCollector(), fetchSize); + } + + @Override + public void post() { + this.commonRdbmsReaderTask.post(this.readerSliceConfig); + } + + @Override + public void destroy() { + this.commonRdbmsReaderTask.destroy(this.readerSliceConfig); + } + + } +} diff --git a/starrocksreader/src/main/resources/plugin.json b/starrocksreader/src/main/resources/plugin.json new file mode 100644 index 0000000000..b0d6e0395e --- /dev/null +++ b/starrocksreader/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "starrocksreader", + "class": "com.alibaba.datax.plugin.reader.starrocksreader.StarRocksReader", + "description": "useScene: prod. mechanism: Jdbc connection using the database, execute select sql, retrieve data from the ResultSet. warn: The more you know about the database, the less problems you encounter.", + "developer": "alibaba" +} \ No newline at end of file diff --git a/starrockswriter/doc/starrockswriter.md b/starrockswriter/doc/starrockswriter.md new file mode 100644 index 0000000000..ba94e6af53 --- /dev/null +++ b/starrockswriter/doc/starrockswriter.md @@ -0,0 +1,222 @@ +# DataX StarRocksWriter + + +--- + + +## 1 快速介绍 + +StarRocksWriter 插件实现了写入数据到 StarRocks 主库的目的表的功能。在底层实现上, StarRocksWriter 通过Streamload以csv格式导入数据至StarRocks。 + + +## 2 实现原理 + + StarRocksWriter 通过Streamload以csv格式导入数据至StarRocks, 内部将`reader`读取的数据进行缓存后批量导入至StarRocks,以提高写入性能。 + + +## 3 功能说明 + +### 3.1 配置样例 + +* 这里使用一份从内存Mysql读取数据后导入至StarRocks。 + +```json +{ + "job": { + "setting": { + "speed": { + "channel": 1 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + } + }, + "content": [ + { + "reader": { + "name": "mysqlreader", + "parameter": { + "username": "xxxx", + "password": "xxxx", + "column": [ "k1", "k2", "v1", "v2" ], + "connection": [ + { + "table": [ "table1", "table2" ], + "jdbcUrl": [ + "jdbc:mysql://127.0.0.1:3306/datax_test1" + ] + }, + { + "table": [ "table3", "table4" ], + "jdbcUrl": [ + "jdbc:mysql://127.0.0.1:3306/datax_test2" + ] + } + ] + } + }, + "writer": { + "name": "starrockswriter", + "parameter": { + "username": "xxxx", + "password": "xxxx", + "column": ["k1", "k2", "v1", "v2"], + "preSql": [], + "postSql": [], + "connection": [ + { + "table": ["xxx"], + "jdbcUrl": "jdbc:mysql://172.28.17.100:9030/", + "selectedDatabase": "xxxx" + } + ], + "loadUrl": ["172.28.17.100:8030", "172.28.17.100:8030"], + "loadProps": {} + } + } + } + ] + } +} + +``` + + +### 3.2 参数说明 + +* **username** + + * 描述:StarRocks数据库的用户名
+ + * 必选:是
+ + * 默认值:无
+ +* **password** + + * 描述:StarRocks数据库的密码
+ + * 必选:是
+ + * 默认值:无
+ +* **selectedDatabase** + + * 描述:StarRocks表的数据库名称。 + + * 必选:是
+ + * 默认值:无
+ +* **table** + + * 描述:StarRocks表的表名称。 + + * 必选:是
+ + * 默认值:无
+ +* **loadUrl** + + * 描述:StarRocks FE的地址用于Streamload,可以为多个fe地址,`fe_ip:fe_http_port`。 + + * 必选:是
+ + * 默认值:无
+ +* **column** + + * 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 + + **column配置项必须指定,不能留空!** + + 注意:我们强烈不推荐你这样配置,因为当你目的表字段个数、类型等有改动时,你的任务可能运行不正确或者失败 + + * 必选:是
+ + * 默认值:否
+ +* **preSql** + + * 描述:写入数据到目的表前,会先执行这里的标准语句。
+ + * 必选:否
+ + * 默认值:无
+ +* **postSql** + + * 描述:写入数据到目的表后,会执行这里的标准语句。
+ + * 必选:否
+ + * 默认值:无
+ +* **jdbcUrl** + + * 描述:目的数据库的 JDBC 连接信息,用于执行`preSql`及`postSql`。
+ + * 必选:否
+ + * 默认值:无
+ +* **maxBatchRows** + + * 描述:单次StreamLoad导入的最大行数
+ + * 必选:否
+ + * 默认值:500000 (50W)
+ +* **maxBatchSize** + + * 描述:单次StreamLoad导入的最大字节数。
+ + * 必选:否
+ + * 默认值:104857600 (100M) + +* **flushInterval** + + * 描述:上一次StreamLoad结束至下一次开始的时间间隔(单位:ms)。
+ + * 必选:否
+ + * 默认值:300000 (ms) + +* **loadProps** + + * 描述:StreamLoad 的请求参数,详情参照StreamLoad介绍页面。
+ + * 必选:否
+ + * 默认值:无
+ + +### 3.3 类型转换 + +默认传入的数据均会被转为字符串,并以`\t`作为列分隔符,`\n`作为行分隔符,组成`csv`文件进行StreamLoad导入操作。 +如需更改列分隔符, 则正确配置 `loadProps` 即可: +```json +"loadProps": { + "column_separator": "\\x01", + "row_delimiter": "\\x02" +} +``` + +如需更改导入格式为`json`, 则正确配置 `loadProps` 即可: +```json +"loadProps": { + "format": "json", + "strip_outer_array": true +} +``` + +## 4 性能报告 + + +## 5 约束限制 + + +## FAQ diff --git a/starrockswriter/pom.xml b/starrockswriter/pom.xml new file mode 100755 index 0000000000..9fb9b14748 --- /dev/null +++ b/starrockswriter/pom.xml @@ -0,0 +1,161 @@ + + 4.0.0 + + com.alibaba.datax + datax-all + 0.0.1-SNAPSHOT + + starrockswriter + starrockswriter + 1.1.0 + 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} + + + commons-codec + commons-codec + 1.9 + + + org.apache.commons + commons-lang3 + 3.12.0 + + + commons-logging + commons-logging + 1.1.1 + + + org.apache.httpcomponents + httpcore + 4.4.6 + + + org.apache.httpcomponents + httpclient + 4.5.3 + + + com.alibaba + fastjson + 1.2.75 + + + mysql + mysql-connector-java + 5.1.46 + + + + + + + + maven-compiler-plugin + + ${jdk-version} + ${jdk-version} + ${project-sourceEncoding} + + + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + + package + + shade + + + true + + + com.alibaba.fastjson + com.starrocks.shade.com.alibaba.fastjson + + + org.apache.http + com.starrocks.shade.org.apache.http + + + org.apache.commons + com.starrocks.shade.org.apache.commons + + + + + org.apache.commons:commons-lang3 + commons-codec:commons-codec + commons-logging:* + org.apache.httpcomponents:httpclient + org.apache.httpcomponents:httpcore + com.alibaba:fastjson + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-assembly-plugin + + + src/main/assembly/package.xml + + datax + + + + dwzip + package + + single + + + + + + + diff --git a/starrockswriter/src/main/assembly/package.xml b/starrockswriter/src/main/assembly/package.xml new file mode 100755 index 0000000000..c63845b471 --- /dev/null +++ b/starrockswriter/src/main/assembly/package.xml @@ -0,0 +1,35 @@ + + + + dir + + false + + + src/main/resources + + plugin.json + plugin_job_template.json + + plugin/writer/starrockswriter + + + target/ + + starrockswriter-1.1.0.jar + + plugin/writer/starrockswriter + + + + + + false + plugin/writer/starrockswriter/libs + runtime + + + diff --git a/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/StarRocksWriter.java b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/StarRocksWriter.java new file mode 100755 index 0000000000..75b2df3ab7 --- /dev/null +++ b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/StarRocksWriter.java @@ -0,0 +1,151 @@ +package com.starrocks.connector.datax.plugin.writer.starrockswriter; + +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.starrocks.connector.datax.plugin.writer.starrockswriter.manager.StarRocksWriterManager; +import com.starrocks.connector.datax.plugin.writer.starrockswriter.row.StarRocksISerializer; +import com.starrocks.connector.datax.plugin.writer.starrockswriter.row.StarRocksSerializerFactory; +import com.starrocks.connector.datax.plugin.writer.starrockswriter.util.StarRocksWriterUtil; + +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.util.ArrayList; +import java.util.List; + +public class StarRocksWriter extends Writer { + + public static class Job extends Writer.Job { + + private static final Logger LOG = LoggerFactory.getLogger(Job.class); + private Configuration originalConfig = null; + private StarRocksWriterOptions options; + + @Override + public void init() { + this.originalConfig = super.getPluginJobConf(); + String selectedDatabase = super.getPluginJobConf().getString(StarRocksWriterOptions.KEY_SELECTED_DATABASE); + if(StringUtils.isBlank(this.originalConfig.getString(StarRocksWriterOptions.KEY_DATABASE)) && StringUtils.isNotBlank(selectedDatabase)){ + this.originalConfig.set(StarRocksWriterOptions.KEY_DATABASE, selectedDatabase); + } + options = new StarRocksWriterOptions(super.getPluginJobConf()); + options.doPretreatment(); + } + + @Override + public void preCheck(){ + this.init(); + StarRocksWriterUtil.preCheckPrePareSQL(options); + StarRocksWriterUtil.preCheckPostSQL(options); + } + + @Override + public void prepare() { + String username = options.getUsername(); + String password = options.getPassword(); + String jdbcUrl = options.getJdbcUrl(); + List renderedPreSqls = StarRocksWriterUtil.renderPreOrPostSqls(options.getPreSqlList(), options.getTable()); + if (null != renderedPreSqls && !renderedPreSqls.isEmpty()) { + Connection conn = DBUtil.getConnection(DataBaseType.MySql, jdbcUrl, username, password); + LOG.info("Begin to execute preSqls:[{}]. context info:{}.", String.join(";", renderedPreSqls), jdbcUrl); + StarRocksWriterUtil.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(originalConfig); + } + return configurations; + } + + @Override + public void post() { + String username = options.getUsername(); + String password = options.getPassword(); + String jdbcUrl = options.getJdbcUrl(); + List renderedPostSqls = StarRocksWriterUtil.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); + StarRocksWriterUtil.executeSqls(conn, renderedPostSqls); + DBUtil.closeDBResources(null, null, conn); + } + } + + @Override + public void destroy() { + } + + } + + public static class Task extends Writer.Task { + private StarRocksWriterManager writerManager; + private StarRocksWriterOptions options; + private StarRocksISerializer rowSerializer; + + @Override + public void init() { + options = new StarRocksWriterOptions(super.getPluginJobConf()); + if (options.isWildcardColumn()) { + Connection conn = DBUtil.getConnection(DataBaseType.MySql, options.getJdbcUrl(), options.getUsername(), options.getPassword()); + List columns = StarRocksWriterUtil.getStarRocksColumns(conn, options.getDatabase(), options.getTable()); + options.setInfoCchemaColumns(columns); + } + writerManager = new StarRocksWriterManager(options); + rowSerializer = StarRocksSerializerFactory.createSerializer(options); + } + + @Override + public void prepare() { + } + + public void startWrite(RecordReceiver recordReceiver) { + try { + Record record; + while ((record = recordReceiver.getFromReader()) != null) { + if (record.getColumnNumber() != options.getColumns().size()) { + throw DataXException + .asDataXException( + DBUtilErrorCode.CONF_ERROR, + String.format( + "Column configuration error. The number of reader columns %d and the number of writer columns %d are not equal.", + record.getColumnNumber(), + options.getColumns().size())); + } + writerManager.writeRecord(rowSerializer.serialize(record)); + } + } catch (Exception e) { + throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, e); + } + } + + @Override + public void post() { + try { + writerManager.close(); + } catch (Exception e) { + throw DataXException.asDataXException(DBUtilErrorCode.WRITE_DATA_ERROR, e); + } + } + + @Override + public void destroy() {} + + @Override + public boolean supportFailOver(){ + return false; + } + } +} diff --git a/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/StarRocksWriterOptions.java b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/StarRocksWriterOptions.java new file mode 100644 index 0000000000..5c6ddacde5 --- /dev/null +++ b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/StarRocksWriterOptions.java @@ -0,0 +1,199 @@ +package com.starrocks.connector.datax.plugin.writer.starrockswriter; + +import java.io.Serializable; + +import com.alibaba.datax.common.exception.DataXException; +import com.alibaba.datax.common.util.Configuration; +import com.alibaba.datax.plugin.rdbms.util.DBUtilErrorCode; +import org.apache.commons.lang3.StringUtils; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class StarRocksWriterOptions 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 = 1; + private static final int BATCH_ROWS = 500000; + private static final long BATCH_BYTES = 5 * MEGA_BYTES_SCALE; + private static final long FLUSH_INTERVAL = 300000; + + private static final String KEY_LOAD_PROPS_FORMAT = "format"; + public enum StreamLoadFormat { + CSV, JSON; + } + + public static final String KEY_USERNAME = "username"; + public static final String KEY_PASSWORD = "password"; + public static final String KEY_DATABASE = "database"; + public static final String KEY_SELECTED_DATABASE = "selectedDatabase"; + public static final String KEY_TABLE = "table"; + public static final String KEY_COLUMN = "column"; + public static final String KEY_PRE_SQL = "preSql"; + public static final String KEY_POST_SQL = "postSql"; + public static final String KEY_JDBC_URL = "jdbcUrl"; + public static final String KEY_LABEL_PREFIX = "labelPrefix"; + public static final String KEY_MAX_BATCH_ROWS = "maxBatchRows"; + public static final String KEY_MAX_BATCH_SIZE = "maxBatchSize"; + public static final String KEY_FLUSH_INTERVAL = "flushInterval"; + public static final String KEY_LOAD_URL = "loadUrl"; + public static final String KEY_FLUSH_QUEUE_LENGTH = "flushQueueLength"; + public static final String KEY_LOAD_PROPS = "loadProps"; + public static final String CONNECTION_JDBC_URL = "connection[0].jdbcUrl"; + public static final String CONNECTION_TABLE_NAME = "connection[0].table[0]"; + public static final String CONNECTION_SELECTED_DATABASE = "connection[0].selectedDatabase"; + + private final Configuration options; + private List infoCchemaColumns; + private List userSetColumns; + private boolean isWildcardColumn; + + public StarRocksWriterOptions(Configuration options) { + this.options = options; + // database + String database = this.options.getString(CONNECTION_SELECTED_DATABASE); + if (StringUtils.isBlank(database)) { + database = this.options.getString(KEY_SELECTED_DATABASE); + } + if (StringUtils.isNotBlank(database)) { + this.options.set(KEY_DATABASE, database); + } + // jdbcUrl + String jdbcUrl = this.options.getString(CONNECTION_JDBC_URL); + if (StringUtils.isNotBlank(jdbcUrl)) { + this.options.set(KEY_JDBC_URL, jdbcUrl); + } + // table + String table = this.options.getString(CONNECTION_TABLE_NAME); + if (StringUtils.isNotBlank(table)) { + this.options.set(KEY_TABLE, table); + } + // column + 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, + "The format of loadUrl is illegal, please input `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/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/manager/StarRocksFlushTuple.java b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/manager/StarRocksFlushTuple.java new file mode 100644 index 0000000000..5c939f9b08 --- /dev/null +++ b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/manager/StarRocksFlushTuple.java @@ -0,0 +1,21 @@ +package com.starrocks.connector.datax.plugin.writer.starrockswriter.manager; + +import java.util.List; + +public class StarRocksFlushTuple { + + private String label; + private Long bytes; + private List rows; + + public StarRocksFlushTuple(String label, Long bytes, List rows) { + this.label = label; + this.bytes = bytes; + this.rows = rows; + } + + public String getLabel() { return label; } + public void setLabel(String label) { this.label = label; } + public Long getBytes() { return bytes; } + public List getRows() { return rows; } +} \ No newline at end of file diff --git a/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/manager/StarRocksStreamLoadFailedException.java b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/manager/StarRocksStreamLoadFailedException.java new file mode 100644 index 0000000000..4eb4704859 --- /dev/null +++ b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/manager/StarRocksStreamLoadFailedException.java @@ -0,0 +1,33 @@ +package com.starrocks.connector.datax.plugin.writer.starrockswriter.manager; + +import java.io.IOException; +import java.util.Map; + + +public class StarRocksStreamLoadFailedException extends IOException { + + static final long serialVersionUID = 1L; + + private final Map response; + private boolean reCreateLabel; + + public StarRocksStreamLoadFailedException(String message, Map response) { + super(message); + this.response = response; + } + + public StarRocksStreamLoadFailedException(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/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/manager/StarRocksStreamLoadVisitor.java b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/manager/StarRocksStreamLoadVisitor.java new file mode 100644 index 0000000000..6bbbba1fcc --- /dev/null +++ b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/manager/StarRocksStreamLoadVisitor.java @@ -0,0 +1,304 @@ +package com.starrocks.connector.datax.plugin.writer.starrockswriter.manager; + +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.URL; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +import com.alibaba.fastjson.JSON; +import com.starrocks.connector.datax.plugin.writer.starrockswriter.StarRocksWriterOptions; +import com.starrocks.connector.datax.plugin.writer.starrockswriter.row.StarRocksDelimiterParser; + +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.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + + +public class StarRocksStreamLoadVisitor { + + private static final Logger LOG = LoggerFactory.getLogger(StarRocksStreamLoadVisitor.class); + + private final StarRocksWriterOptions writerOptions; + 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 StarRocksStreamLoadVisitor(StarRocksWriterOptions writerOptions) { + this.writerOptions = writerOptions; + } + + public void doStreamLoad(StarRocksFlushTuple flushData) throws IOException { + String host = getAvailableHost(); + if (null == host) { + throw new IOException("None of the host in `load_url` could be connected."); + } + String loadUrl = new StringBuilder(host) + .append("/api/") + .append(writerOptions.getDatabase()) + .append("/") + .append(writerOptions.getTable()) + .append("/_stream_load") + .toString(); + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("Start to join batch data: rows[%d] bytes[%d] label[%s].", flushData.getRows().size(), flushData.getBytes(), flushData.getLabel())); + } + Map loadResult = doHttpPut(loadUrl, flushData.getLabel(), joinRows(flushData.getRows(), flushData.getBytes().intValue())); + final String keyStatus = "Status"; + if (null == loadResult || !loadResult.containsKey(keyStatus)) { + LOG.error("unknown result status. {}", loadResult); + throw new IOException("Unable to flush data to StarRocks: unknown result status. " + loadResult); + } + if (LOG.isDebugEnabled()) { + LOG.debug(new StringBuilder("StreamLoad response:\n").append(JSON.toJSONString(loadResult)).toString()); + } + if (RESULT_FAILED.equals(loadResult.get(keyStatus))) { + StringBuilder errorBuilder = new StringBuilder("Failed to flush data to StarRocks.\n"); + if (loadResult.containsKey("Message")) { + errorBuilder.append(loadResult.get("Message")); + errorBuilder.append('\n'); + } + if (loadResult.containsKey("ErrorURL")) { + LOG.error("StreamLoad response: {}", loadResult); + try { + errorBuilder.append(doHttpGet(loadResult.get("ErrorURL").toString())); + errorBuilder.append('\n'); + } catch (IOException e) { + LOG.warn("Get Error URL failed. {} ", loadResult.get("ErrorURL"), e); + } + } else { + errorBuilder.append(JSON.toJSONString(loadResult)); + errorBuilder.append('\n'); + } + throw new IOException(errorBuilder.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, flushData.getLabel()); + } + } + + private String getAvailableHost() { + List hostList = writerOptions.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(1000); + co.connect(); + co.disconnect(); + return true; + } catch (Exception e1) { + LOG.warn("Failed to connect to address:{}", host, e1); + return false; + } + } + + private byte[] joinRows(List rows, int totalBytes) { + if (StarRocksWriterOptions.StreamLoadFormat.CSV.equals(writerOptions.getStreamLoadFormat())) { + Map props = (writerOptions.getLoadProps() == null ? new HashMap<>() : writerOptions.getLoadProps()); + byte[] lineDelimiter = StarRocksDelimiterParser.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 (StarRocksWriterOptions.StreamLoadFormat.JSON.equals(writerOptions.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:"); + } + + @SuppressWarnings("unchecked") + 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(writerOptions.getDatabase()).append("/get_load_state?label=").append(label).toString()); + httpGet.setHeader("Authorization", getBasicAuthHeader(writerOptions.getUsername(), writerOptions.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 StarRocks, 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 StarRocks, 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 StarRocksStreamLoadFailedException(String.format("Failed to flush data to StarRocks, 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 StarRocks, Error " + + "label[%s] state[%s]\n", label, labelState), null); + } + } + } + } + } + + @SuppressWarnings("unchecked") + 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 = writerOptions.getColumns(); + if (null != cols && !cols.isEmpty() && StarRocksWriterOptions.StreamLoadFormat.CSV.equals(writerOptions.getStreamLoadFormat())) { + httpPut.setHeader("columns", String.join(",", cols.stream().map(f -> String.format("`%s`", f)).collect(Collectors.toList()))); + } + if (null != writerOptions.getLoadProps()) { + for (Map.Entry entry : writerOptions.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(writerOptions.getUsername(), writerOptions.getPassword())); + httpPut.setEntity(new ByteArrayEntity(data)); + httpPut.setConfig(RequestConfig.custom().setRedirectsEnabled(true).build()); + try (CloseableHttpResponse resp = httpclient.execute(httpPut)) { + int code = resp.getStatusLine().getStatusCode(); + if (200 != code) { + String errorText; + try { + HttpEntity respEntity = resp.getEntity(); + errorText = EntityUtils.toString(respEntity); + } catch (Exception err) { + errorText = "find errorText failed: " + err.getMessage(); + } + LOG.warn("Request failed with code:{}, err:{}", code, errorText); + Map errorMap = new HashMap<>(); + errorMap.put("Status", "Fail"); + errorMap.put("Message", errorText); + return errorMap; + } + 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(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 doHttpGet(String getUrl) throws IOException { + LOG.info("Executing GET from {}.", getUrl); + try (CloseableHttpClient httpclient = buildHttpClient()) { + HttpGet httpGet = new HttpGet(getUrl); + try (CloseableHttpResponse resp = httpclient.execute(httpGet)) { + HttpEntity respEntity = resp.getEntity(); + if (null == respEntity) { + LOG.warn("Request failed with empty response."); + return null; + } + return EntityUtils.toString(respEntity); + } + } + } + + private CloseableHttpClient buildHttpClient(){ + final HttpClientBuilder httpClientBuilder = HttpClients.custom() + .setRedirectStrategy(new DefaultRedirectStrategy() { + @Override + protected boolean isRedirectable(String method) { + return true; + } + }); + return httpClientBuilder.build(); + } + +} diff --git a/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/manager/StarRocksWriterManager.java b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/manager/StarRocksWriterManager.java new file mode 100644 index 0000000000..a0cb1f8bd3 --- /dev/null +++ b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/manager/StarRocksWriterManager.java @@ -0,0 +1,203 @@ +package com.starrocks.connector.datax.plugin.writer.starrockswriter.manager; + +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; + +import com.google.common.base.Strings; +import com.starrocks.connector.datax.plugin.writer.starrockswriter.StarRocksWriterOptions; + +public class StarRocksWriterManager { + + private static final Logger LOG = LoggerFactory.getLogger(StarRocksWriterManager.class); + + private final StarRocksStreamLoadVisitor starrocksStreamLoadVisitor; + private final StarRocksWriterOptions writerOptions; + + 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 StarRocksWriterManager(StarRocksWriterOptions writerOptions) { + this.writerOptions = writerOptions; + this.starrocksStreamLoadVisitor = new StarRocksStreamLoadVisitor(writerOptions); + flushQueue = new LinkedBlockingDeque<>(writerOptions.getFlushQueueLength()); + this.startScheduler(); + this.startAsyncFlushing(); + } + + public void startScheduler() { + stopScheduler(); + this.scheduler = Executors.newScheduledThreadPool(1, new BasicThreadFactory.Builder().namingPattern("starrocks-interval-flush").daemon(true).build()); + this.scheduledFuture = this.scheduler.schedule(() -> { + synchronized (StarRocksWriterManager.this) { + if (!closed) { + try { + String label = createBatchLabel(); + LOG.info(String.format("StarRocks interval Sinking triggered: label[%s].", label)); + if (batchCount == 0) { + startScheduler(); + } + flush(label, false); + } catch (Exception e) { + flushException = e; + } + } + } + }, writerOptions.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 >= writerOptions.getBatchRows() || batchSize >= writerOptions.getBatchSize()) { + String label = createBatchLabel(); + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("StarRocks buffer Sinking triggered: rows[%d] label[%s].", batchCount, label)); + } + flush(label, false); + } + } catch (Exception e) { + throw new IOException("Writing records to StarRocks failed.", e); + } + } + + public synchronized void flush(String label, boolean waitUtilDone) throws Exception { + checkFlushException(); + if (batchCount == 0) { + if (waitUtilDone) { + waitAsyncFlushingDone(); + } + return; + } + flushQueue.put(new StarRocksFlushTuple(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) { + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("StarRocks Sink is about to close: label[%s].", label)); + } + } + flush(label, true); + } catch (Exception e) { + throw new RuntimeException("Writing records to StarRocks failed.", e); + } + } + checkFlushException(); + } + + public String createBatchLabel() { + StringBuilder sb = new StringBuilder(); + if (!Strings.isNullOrEmpty(writerOptions.getLabelPrefix())) { + sb.append(writerOptions.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 <= writerOptions.getFlushQueueLength(); i++) { + flushQueue.put(new StarRocksFlushTuple("", 0l, null)); + } + checkFlushException(); + } + + private void asyncFlush() throws Exception { + StarRocksFlushTuple flushData = flushQueue.take(); + if (Strings.isNullOrEmpty(flushData.getLabel())) { + return; + } + stopScheduler(); + if (LOG.isDebugEnabled()) { + 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 <= writerOptions.getMaxRetries(); i++) { + try { + // flush to StarRocks with stream load + starrocksStreamLoadVisitor.doStreamLoad(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 StarRocks, retry times = {}", i, e); + if (i >= writerOptions.getMaxRetries()) { + throw new IOException(e); + } + if (e instanceof StarRocksStreamLoadFailedException && ((StarRocksStreamLoadFailedException)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 StarRocks failed.", flushException); + } + } +} diff --git a/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksBaseSerializer.java b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksBaseSerializer.java new file mode 100644 index 0000000000..a7ad499d55 --- /dev/null +++ b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksBaseSerializer.java @@ -0,0 +1,26 @@ +package com.starrocks.connector.datax.plugin.writer.starrockswriter.row; + +import com.alibaba.datax.common.element.Column; +import com.alibaba.datax.common.element.Column.Type; + +public class StarRocksBaseSerializer { + + protected String fieldConvertion(Column col) { + if (null == col.getRawData() || Type.NULL == col.getType()) { + return null; + } + if (Type.BOOL == col.getType()) { + return String.valueOf(col.asLong()); + } + if (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(); + } + +} \ No newline at end of file diff --git a/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksCsvSerializer.java b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksCsvSerializer.java new file mode 100644 index 0000000000..1366d57097 --- /dev/null +++ b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksCsvSerializer.java @@ -0,0 +1,32 @@ +package com.starrocks.connector.datax.plugin.writer.starrockswriter.row; + +import java.io.StringWriter; + +import com.alibaba.datax.common.element.Record; + +import com.google.common.base.Strings; + +public class StarRocksCsvSerializer extends StarRocksBaseSerializer implements StarRocksISerializer { + + private static final long serialVersionUID = 1L; + + private final String columnSeparator; + + public StarRocksCsvSerializer(String sp) { + this.columnSeparator = StarRocksDelimiterParser.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/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksDelimiterParser.java b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksDelimiterParser.java new file mode 100644 index 0000000000..04301e0f13 --- /dev/null +++ b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksDelimiterParser.java @@ -0,0 +1,55 @@ +package com.starrocks.connector.datax.plugin.writer.starrockswriter.row; + +import java.io.StringWriter; + +import com.google.common.base.Strings; + +public class StarRocksDelimiterParser { + + 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/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksISerializer.java b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksISerializer.java new file mode 100644 index 0000000000..7bcb89739c --- /dev/null +++ b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksISerializer.java @@ -0,0 +1,11 @@ +package com.starrocks.connector.datax.plugin.writer.starrockswriter.row; + +import java.io.Serializable; + +import com.alibaba.datax.common.element.Record; + +public interface StarRocksISerializer extends Serializable { + + String serialize(Record row); + +} diff --git a/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksJsonSerializer.java b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksJsonSerializer.java new file mode 100644 index 0000000000..60faa1be63 --- /dev/null +++ b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksJsonSerializer.java @@ -0,0 +1,34 @@ +package com.starrocks.connector.datax.plugin.writer.starrockswriter.row; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import com.alibaba.datax.common.element.Record; +import com.alibaba.fastjson.JSON; + +public class StarRocksJsonSerializer extends StarRocksBaseSerializer implements StarRocksISerializer { + + private static final long serialVersionUID = 1L; + + private final List fieldNames; + + public StarRocksJsonSerializer(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/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksSerializerFactory.java b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksSerializerFactory.java new file mode 100644 index 0000000000..85f446cd93 --- /dev/null +++ b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/row/StarRocksSerializerFactory.java @@ -0,0 +1,22 @@ +package com.starrocks.connector.datax.plugin.writer.starrockswriter.row; + +import java.util.Map; + +import com.starrocks.connector.datax.plugin.writer.starrockswriter.StarRocksWriterOptions; + +public class StarRocksSerializerFactory { + + private StarRocksSerializerFactory() {} + + public static StarRocksISerializer createSerializer(StarRocksWriterOptions writerOptions) { + if (StarRocksWriterOptions.StreamLoadFormat.CSV.equals(writerOptions.getStreamLoadFormat())) { + Map props = writerOptions.getLoadProps(); + return new StarRocksCsvSerializer(null == props || !props.containsKey("column_separator") ? null : String.valueOf(props.get("column_separator"))); + } + if (StarRocksWriterOptions.StreamLoadFormat.JSON.equals(writerOptions.getStreamLoadFormat())) { + return new StarRocksJsonSerializer(writerOptions.getColumns()); + } + throw new RuntimeException("Failed to create row serializer, unsupported `format` from stream load properties."); + } + +} diff --git a/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/util/StarRocksWriterUtil.java b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/util/StarRocksWriterUtil.java new file mode 100755 index 0000000000..8de4ad60be --- /dev/null +++ b/starrockswriter/src/main/java/com/starrocks/connector/datax/plugin/writer/starrockswriter/util/StarRocksWriterUtil.java @@ -0,0 +1,102 @@ +package com.starrocks.connector.datax.plugin.writer.starrockswriter.util; + +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.starrocks.connector.datax.plugin.writer.starrockswriter.StarRocksWriterOptions; +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.*; + +public final class StarRocksWriterUtil { + private static final Logger LOG = LoggerFactory.getLogger(StarRocksWriterUtil.class); + + private StarRocksWriterUtil() {} + + public static List getStarRocksColumns(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(StarRocksWriterOptions options) { + String table = options.getTable(); + List preSqls = options.getPreSqlList(); + List renderedPreSqls = StarRocksWriterUtil.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(StarRocksWriterOptions options) { + String table = options.getTable(); + List postSqls = options.getPostSqlList(); + List renderedPostSqls = StarRocksWriterUtil.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/starrockswriter/src/main/resources/plugin.json b/starrockswriter/src/main/resources/plugin.json new file mode 100755 index 0000000000..8edec1e01e --- /dev/null +++ b/starrockswriter/src/main/resources/plugin.json @@ -0,0 +1,6 @@ +{ + "name": "starrockswriter", + "class": "com.starrocks.connector.datax.plugin.writer.starrockswriter.StarRocksWriter", + "description": "useScene: prod. mechanism: StarRocksStreamLoad. warn: The more you know about the database, the less problems you encounter.", + "developer": "starrocks" +} \ No newline at end of file diff --git a/starrockswriter/src/main/resources/plugin_job_template.json b/starrockswriter/src/main/resources/plugin_job_template.json new file mode 100644 index 0000000000..06c075bcb2 --- /dev/null +++ b/starrockswriter/src/main/resources/plugin_job_template.json @@ -0,0 +1,18 @@ +{ + "name": "starrockswriter", + "parameter": { + "username": "", + "password": "", + "column": [], + "preSql": [], + "postSql": [], + "loadUrl": [], + "connection": [ + { + "jdbcUrl": "", + "selectedDatabase": "", + "table": [] + } + ] + } +} \ No newline at end of file diff --git a/transformer/doc/transformer.md b/transformer/doc/transformer.md index 247ab39b4d..260c0fb689 100644 --- a/transformer/doc/transformer.md +++ b/transformer/doc/transformer.md @@ -59,7 +59,17 @@ dx_replace(1,"5","10","****") column 1的value为“dataxTest”=>"data****" dx_filter(1,"like","dataTest") dx_filter(1,">=","10") ``` -5. dx_groovy +5. dx_digest +* 参数:3个 + * 第一个参数:字段编号,对应record中第几个字段。 + * 第二个参数:hash类型,md5、sha1 + * 第三个参数:hash值大小写 toUpperCase(大写)、toLowerCase(小写) +* 返回: 返回指定类型的hashHex,如果字段为空,则转为空字符串,再返回对应hashHex +* 举例: +``` +dx_digest(1,"md5","toUpperCase"), column 1的值为 xyzzzzz => 9CDFFC4FA4E45A99DB8BBCD762ACFFA2 +``` +6. dx_groovy * 参数。 * 第一个参数: groovy code * 第二个参数(列表或者为空):extraPackage @@ -67,7 +77,9 @@ dx_filter(1,">=","10") * dx_groovy只能调用一次。不能多次调用。 * groovy code中支持java.lang, java.util的包,可直接引用的对象有record,以及element下的各种column(BoolColumn.class,BytesColumn.class,DateColumn.class,DoubleColumn.class,LongColumn.class,StringColumn.class)。不支持其他包,如果用户有需要用到其他包,可设置extraPackage,注意extraPackage不支持第三方jar包。 * groovy code中,返回更新过的Record(比如record.setColumn(columnIndex, new StringColumn(newValue));),或者null。返回null表示过滤此行。 - * 用户可以直接调用静态的Util方式(GroovyTransformerStaticUtil),目前GroovyTransformerStaticUtil的方法列表 (按需补充): + * 用户可以直接调用静态的Util方式(GroovyTransformerStaticUtil),目前GroovyTransformerStaticUtil的方法列表: + * md5(String):String + * sha1(String):String * 举例: ``` groovy 实现的subStr: @@ -109,7 +121,7 @@ String code3 = "Column column = record.getColumn(1);\n" + ``` ## Job定义 -* 本例中,配置3个UDF。 +* 本例中,配置4个UDF。 ``` { @@ -176,6 +188,14 @@ String code3 = "Column column = record.getColumn(1);\n" + "paras":["3","4","****"] } }, + { + "name": "dx_digest", + "parameter": + { + "columnIndex":3, + "paras":["md5", "toLowerCase"] + } + }, { "name": "dx_groovy", "parameter": diff --git a/tsdbreader/pom.xml b/tsdbreader/pom.xml index 0f99023427..d707fe4150 100644 --- a/tsdbreader/pom.xml +++ b/tsdbreader/pom.xml @@ -24,9 +24,6 @@ 4.5 2.4 - - 1.2.28 - 4.13.1 @@ -91,7 +88,6 @@ com.alibaba fastjson - ${fastjson.version} diff --git a/tsdbreader/src/main/resources/plugin.json b/tsdbreader/src/main/resources/plugin.json index f2dbb1f075..3b10d228b1 100755 --- a/tsdbreader/src/main/resources/plugin.json +++ b/tsdbreader/src/main/resources/plugin.json @@ -6,5 +6,5 @@ "mechanism": "通过 /api/query 接口查询出符合条件的数据点", "warn": "指定起止时间会自动忽略分钟和秒,转为整点时刻,例如 2019-4-18 的 [3:35, 4:55) 会被转为 [3:00, 4:00)" }, - "developer": "Benedict Jin" + "developer": "alibaba" } diff --git a/tsdbwriter/pom.xml b/tsdbwriter/pom.xml index 6f2bac5244..72215651f8 100644 --- a/tsdbwriter/pom.xml +++ b/tsdbwriter/pom.xml @@ -24,9 +24,6 @@ 4.5 2.4 - - 1.2.28 - 4.13.1 @@ -88,7 +85,6 @@ com.alibaba fastjson - ${fastjson.version} diff --git a/tsdbwriter/src/main/resources/plugin.json b/tsdbwriter/src/main/resources/plugin.json index 78c8273f6e..26f927c2bd 100755 --- a/tsdbwriter/src/main/resources/plugin.json +++ b/tsdbwriter/src/main/resources/plugin.json @@ -6,5 +6,5 @@ "mechanism": "调用 TSDB 的 /api/put 接口,实现数据点的写入", "warn": "" }, - "developer": "Benedict Jin" + "developer": "alibaba" }